From aaf38ddfe53bbb67fad4456ee1068d18b9b891b5 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Mon, 27 Jun 2016 11:24:13 -0700 Subject: [PATCH 1/8] BEAM-261 Apex runner PoC --- runners/apex/pom.xml | 226 +++++++++ .../runners/apex/ApexPipelineOptions.java | 60 +++ .../runners/apex/ApexPipelineTranslator.java | 134 ++++++ .../apache/beam/runners/apex/ApexRunner.java | 171 +++++++ .../beam/runners/apex/ApexRunnerResult.java | 85 ++++ .../beam/runners/apex/TestApexRunner.java | 56 +++ .../translators/CreateValuesTranslator.java | 49 ++ .../FlattenPCollectionTranslator.java | 52 +++ .../translators/GroupByKeyTranslator.java | 41 ++ .../translators/ParDoBoundTranslator.java | 43 ++ .../translators/ReadUnboundedTranslator.java | 42 ++ .../apex/translators/TransformTranslator.java | 31 ++ .../apex/translators/TranslationContext.java | 143 ++++++ .../functions/ApexGroupByKeyOperator.java | 427 ++++++++++++++++++ .../functions/ApexParDoOperator.java | 177 ++++++++ .../io/ApexReadUnboundedInputOperator.java | 125 +++++ .../apex/translators/io/ValuesSource.java | 152 +++++++ .../translators/utils/ApexStreamTuple.java | 191 ++++++++ .../utils/CoderAdapterStreamCodec.java | 73 +++ .../utils/NoOpSideInputReader.java | 47 ++ .../translators/utils/NoOpStepContext.java | 73 +++ .../utils/SerializablePipelineOptions.java | 61 +++ .../apex/examples/StreamingWordCountTest.java | 120 +++++ .../apex/examples/UnboundedTextSource.java | 144 ++++++ .../FlattenPCollectionTranslatorTest.java | 97 ++++ .../translators/GroupByKeyTranslatorTest.java | 248 ++++++++++ .../translators/ParDoBoundTranslatorTest.java | 164 +++++++ .../ReadUnboundTranslatorTest.java | 130 ++++++ .../translators/utils/CollectionSource.java | 137 ++++++ .../utils/PipelineOptionsTest.java | 82 ++++ .../apex/src/test/resources/log4j.properties | 33 ++ runners/pom.xml | 1 + 32 files changed, 3615 insertions(+) create mode 100644 runners/apex/pom.xml create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java create mode 100644 runners/apex/src/test/resources/log4j.properties diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml new file mode 100644 index 000000000000..bb08b3ca0ae6 --- /dev/null +++ b/runners/apex/pom.xml @@ -0,0 +1,226 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-runners-parent + 0.3.0-incubating-SNAPSHOT + ../pom.xml + + + beam-runners-apex_3.4.0 + + Apache Beam :: Runners :: Apex + + jar + + + 3.4.0 + 3.4.0 + true + + -Xmx2048m + + + + + + org.apache.apex + apex-common + ${apex.core.version} + + + org.apache.apex + malhar-library + ${apex.malhar.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.apex + apex-engine + ${apex.core.version} + test + + + + + org.apache.beam + beam-sdks-java-core + + + org.slf4j + slf4j-jdk14 + + + + + + org.apache.beam + beam-runners-core-java + + + org.slf4j + slf4j-jdk14 + + + + + + + com.google.code.findbugs + annotations + + + + + + org.hamcrest + hamcrest-all + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + + + + org.apache.beam + beam-sdks-java-core + tests + test + + + org.slf4j + slf4j-jdk14 + + + + + + + + + + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${surefire.args} + + + + runnable-on-service-tests + integration-test + + test + + + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:beam-sdks-java-core + + + + [ + "--runner=org.apache.beam.runners.apex.TestApexRunner", + "--streaming=true" + ] + + + ${skipIntegrationTests} + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + analyze-only + + + + org.apache.apex:apex-api:jar:3.4.0 + org.apache.commons:commons-lang3::3.1 + com.esotericsoftware.kryo:kryo::2.24.0 + com.datatorrent:netlet::1.2.1 + org.slf4j:slf4j-api:jar:1.7.14 + org.apache.hadoop:hadoop-common:jar:2.2.0 + joda-time:joda-time:jar:2.4 + com.google.guava:guava:jar:19.0 + + + + + + + + + + diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java new file mode 100644 index 000000000000..f70d24cc8cea --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * Options that configure the Apex pipeline. + */ +public interface ApexPipelineOptions extends PipelineOptions, java.io.Serializable { + + @Description("set unique application name for Apex runner") + void setApplicationName(String name); + + String getApplicationName(); + + @Description("set parallelism for Apex runner") + void setParallelism(int parallelism); + + @Default.Integer(1) + int getParallelism(); + + @Description("execute the pipeline with embedded cluster") + void setEmbeddedExecution(boolean embedded); + + @Default.Boolean(true) + boolean isEmbeddedExecution(); + + @Description("configure embedded execution with debug friendly options") + void setEmbeddedExecutionDebugMode(boolean embeddedDebug); + + @Default.Boolean(true) + boolean isEmbeddedExecutionDebugMode(); + + @Description("how long the client should wait for the pipeline to run") + void setRunMillis(long runMillis); + + @Default.Long(0) + long getRunMillis(); + +} + diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java new file mode 100644 index 000000000000..8ea7139808f9 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex; + +import org.apache.beam.runners.apex.translators.CreateValuesTranslator; +import org.apache.beam.runners.apex.translators.FlattenPCollectionTranslator; +import org.apache.beam.runners.apex.translators.GroupByKeyTranslator; +import org.apache.beam.runners.apex.translators.ParDoBoundTranslator; +import org.apache.beam.runners.apex.translators.ReadUnboundedTranslator; +import org.apache.beam.runners.apex.translators.TransformTranslator; +import org.apache.beam.runners.apex.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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** + * {@link ApexPipelineTranslator} translates {@link Pipeline} objects + * into Apex logical plan {@link DAG}. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { + + private static final Logger LOG = LoggerFactory.getLogger( + ApexPipelineTranslator.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(GroupByKey.class, new GroupByKeyTranslator()); + registerTransformTranslator(Flatten.FlattenPCollectionList.class, + new FlattenPCollectionTranslator()); + registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); + } + + public ApexPipelineTranslator(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/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java new file mode 100644 index 000000000000..87c8f97a4417 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunner.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.apex; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.beam.runners.apex.translators.TranslationContext; +import org.apache.beam.runners.core.UnboundedReadFromBoundedSource; +import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.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.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.hadoop.conf.Configuration; + +import com.datatorrent.api.Context.DAGContext; +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; +import com.datatorrent.api.StreamingApplication; +import com.google.common.base.Throwables; + +/** + * A {@link PipelineRunner} that translates the + * pipeline to an Apex DAG and executes it on an Apex cluster. + *

+ * Currently execution is always in embedded mode, + * launch on Hadoop cluster will be added in subsequent iteration. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class ApexRunner extends PipelineRunner { + + private final ApexPipelineOptions options; + + public ApexRunner(ApexPipelineOptions options) { + this.options = options; + } + + public static ApexRunner fromOptions(PipelineOptions options) { + return new ApexRunner((ApexPipelineOptions) options); + } + + @Override + public OutputT apply( + PTransform transform, InputT input) { + if (Window.Bound.class.equals(transform.getClass())) { + return (OutputT) ((PCollection) input).apply( + new AssignWindowsAndSetStrategy((Window.Bound) transform)); + } else if (Create.Values.class.equals(transform.getClass())) { + return (OutputT) PCollection + .createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED); + } else if (Read.Bounded.class.equals(transform.getClass())) { + return (OutputT) ((PBegin) input).apply(new UnboundedReadFromBoundedSource<>(((Read.Bounded)transform).getSource())); + } else { + return super.apply(transform, input); + } + } + + @Override + public ApexRunnerResult run(Pipeline pipeline) { + + final TranslationContext translationContext = new TranslationContext(options); + ApexPipelineTranslator translator = new ApexPipelineTranslator(translationContext); + translator.translate(pipeline); + + StreamingApplication apexApp = new StreamingApplication() + { + @Override + public void populateDAG(DAG dag, Configuration conf) + { + dag.setAttribute(DAGContext.APPLICATION_NAME, options.getApplicationName()); + translationContext.populateDAG(dag); + } + }; + + checkArgument(options.isEmbeddedExecution(), "only embedded execution is supported at this time"); + LocalMode lma = LocalMode.newInstance(); + Configuration conf = new Configuration(false); + try { + lma.prepareDAG(apexApp, conf); + LocalMode.Controller lc = lma.getController(); + if (options.isEmbeddedExecutionDebugMode()) { + // turns off timeout checking for operator progress + lc.setHeartbeatMonitoringEnabled(false); + } + if (options.getRunMillis() > 0) { + lc.run(options.getRunMillis()); + } else { + lc.runAsync(); + } + return new ApexRunnerResult(lma.getDAG(), lc); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * copied from DirectPipelineRunner. + * used to replace Window.Bound till equivalent function is added in Apex + */ + private static class AssignWindowsAndSetStrategy + extends PTransform, PCollection> { + + private final Window.Bound wrapped; + + public AssignWindowsAndSetStrategy(Window.Bound wrapped) { + this.wrapped = wrapped; + } + + @Override + public PCollection apply(PCollection input) { + WindowingStrategy outputStrategy = + wrapped.getOutputStrategyInternal(input.getWindowingStrategy()); + + WindowFn windowFn = + (WindowFn) outputStrategy.getWindowFn(); + + // If the Window.Bound transform only changed parts other than the WindowFn, then + // we skip AssignWindows even though it should be harmless in a perfect world. + // The world is not perfect, and a GBK may have set it to InvalidWindows to forcibly + // crash if another GBK is performed without explicitly setting the WindowFn. So we skip + // AssignWindows in this case. + if (wrapped.getWindowFn() == null) { + return input.apply("Identity", ParDo.of(new IdentityFn())) + .setWindowingStrategyInternal(outputStrategy); + } else { + return input + .apply("AssignWindows", new AssignWindows<>(windowFn)) + .setWindowingStrategyInternal(outputStrategy); + } + } + } + + private static class IdentityFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.element()); + } + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java new file mode 100644 index 000000000000..f28c8dc5d42b --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.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.runners.apex; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; + +import java.io.IOException; + +import org.apache.beam.sdk.AggregatorRetrievalException; +import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.transforms.Aggregator; +import org.joda.time.Duration; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; + +/** + * Result of executing a {@link Pipeline} with Apex in embedded mode. + */ +public class ApexRunnerResult implements PipelineResult { + private final DAG apexDAG; + private final LocalMode.Controller ctrl; + private State state = State.UNKNOWN; + + public ApexRunnerResult(DAG dag, LocalMode.Controller ctrl) { + this.apexDAG = dag; + this.ctrl = ctrl; + } + + @Override + public State getState() { + return state; + } + + @Override + public AggregatorValues getAggregatorValues(Aggregator aggregator) + throws AggregatorRetrievalException { + return null; + } + + @Override + public State cancel() throws IOException + { + ctrl.shutdown(); + state = State.CANCELLED; + return state; + } + + @Override + public State waitUntilFinish(Duration duration) throws IOException, InterruptedException + { + throw new UnsupportedOperationException(); + } + + @Override + public State waitUntilFinish() throws IOException, InterruptedException + { + throw new UnsupportedOperationException(); + } + + /** + * Return the DAG executed by the pipeline. + * @return + */ + public DAG getApexDAG() { + return apexDAG; + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java new file mode 100644 index 000000000000..45c143e32074 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.apex; + +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; + + +public class TestApexRunner extends PipelineRunner { + + private ApexRunner delegate; + + private TestApexRunner(ApexPipelineOptions options) { + options.setEmbeddedExecution(true); + //options.setEmbeddedExecutionDebugMode(false); + options.setRunMillis(20000); + this.delegate = ApexRunner.fromOptions(options); + } + + public static TestApexRunner fromOptions(PipelineOptions options) { + ApexPipelineOptions apexOptions = PipelineOptionsValidator.validate(ApexPipelineOptions.class, options); + return new TestApexRunner(apexOptions); + } + + @Override + public + OutputT apply(PTransform transform, InputT input) { + return delegate.apply(transform, input); + } + + @Override + public ApexRunnerResult run(Pipeline pipeline) { + return delegate.run(pipeline); + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java new file mode 100644 index 000000000000..387b19f5e39a --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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.apex.translators; + +import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.runners.apex.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 com.google.common.base.Throwables; + + +/** + * Wraps elements from Create.Values into an {@link UnboundedSource}. + * mainly used for test + */ +public class CreateValuesTranslator implements TransformTranslator> { + private static final long serialVersionUID = 1451000241832745629L; + + @Override + public void translate(Create.Values transform, TranslationContext context) { + try { + UnboundedSource unboundedSource = new ValuesSource<>(transform.getElements(), + transform.getDefaultOutputCoder(context.getInput())); + ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>(unboundedSource, + context.getPipelineOptions()); + context.addOperator(operator, operator.output); + } catch (CannotProvideCoderException e) { + Throwables.propagate(e); + } + } +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java new file mode 100644 index 000000000000..f22814936a94 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex.translators; + +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; + +import com.datatorrent.lib.stream.StreamMerger; + +/** + * Flatten.FlattenPCollectionList translation to Apex operator. + * TODO: support more than two streams + */ +public class FlattenPCollectionTranslator implements + TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { + StreamMerger operator = null; + PCollectionList collections = context.getInput(); + if (collections.size() > 2) { + throw new UnsupportedOperationException("Currently supports only 2 collections: " + transform); + } + for (PCollection collection : collections.getAll()) { + if (null == operator) { + operator = new StreamMerger(); + context.addStream(collection, operator.data1); + } else { + context.addStream(collection, operator.data2); + } + } + context.addOperator(operator, operator.out); + } +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.java new file mode 100644 index 000000000000..43c82a90745b --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.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.apex.translators; + +import org.apache.beam.runners.apex.translators.functions.ApexGroupByKeyOperator; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@link GroupByKey} translation to Apex operator. + */ +public class GroupByKeyTranslator implements TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(GroupByKey transform, TranslationContext context) { + + PCollection> input = context.getInput(); + ApexGroupByKeyOperator group = new ApexGroupByKeyOperator<>(context.getPipelineOptions(), input); + context.addOperator(group, group.output); + context.addStream(input, group.input); + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java new file mode 100644 index 000000000000..a9582349df7a --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex.translators; + +import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; +import org.apache.beam.runners.apex.translators.utils.NoOpSideInputReader; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + +/** + * {@link ParDo.Bound} is translated to Apex operator that wraps the {@link DoFn} + */ +public class ParDoBoundTranslator implements + TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(ParDo.Bound transform, TranslationContext context) { + OldDoFn doFn = transform.getFn(); + PCollection output = context.getOutput(); + ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), + doFn, output.getWindowingStrategy(), new NoOpSideInputReader()); + context.addOperator(operator, operator.output); + context.addStream(context.getInput(), operator.input); + } +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java new file mode 100644 index 000000000000..b53e4ddee721 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.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.apex.translators; + +import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.UnboundedSource; + +import com.datatorrent.api.InputOperator; + +/** + * {@link Read.Unbounded} is translated to Apex {@link InputOperator} + * that wraps {@link UnboundedSource}. + */ +public class ReadUnboundedTranslator implements TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(Read.Unbounded transform, TranslationContext context) { + UnboundedSource unboundedSource = transform.getSource(); + ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>( + unboundedSource, context.getPipelineOptions()); + context.addOperator(operator, operator.output); + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java new file mode 100644 index 000000000000..1a99885d1f9f --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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.apex.translators; + + +import org.apache.beam.sdk.transforms.PTransform; + +import java.io.Serializable; + +/** + * translates {@link PTransform} to Apex functions. + */ +public interface TransformTranslator> extends Serializable { + void translate(T transform, TranslationContext context); +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java new file mode 100644 index 000000000000..92afd583be9e --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.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.apex.translators; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; +import org.apache.beam.runners.apex.translators.utils.CoderAdapterStreamCodec; +import org.apache.beam.sdk.coders.Coder; +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.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; + +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; +import com.datatorrent.api.Context.PortContext; +import com.datatorrent.api.Operator.InputPort; +import com.datatorrent.api.Operator.OutputPort; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Maintains context data for {@link TransformTranslator}s. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class TranslationContext { + + private final ApexPipelineOptions pipelineOptions; + private AppliedPTransform currentTransform; + private final Map, List>>> streams = new HashMap<>(); + private final Map operators = new HashMap<>(); + + public TranslationContext(ApexPipelineOptions pipelineOptions) { + this.pipelineOptions = pipelineOptions; + } + + public void setCurrentTransform(TransformTreeNode treeNode) { + this.currentTransform = AppliedPTransform.of(treeNode.getFullName(), + treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform()); + } + + public ApexPipelineOptions getPipelineOptions() { + return pipelineOptions; + } + + public InputT getInput() { + return (InputT) getCurrentTransform().getInput(); + } + + public OutputT getOutput() { + return (OutputT) getCurrentTransform().getOutput(); + } + + private AppliedPTransform getCurrentTransform() { + checkArgument(currentTransform != null, "current transform not set"); + return currentTransform; + } + + public void addOperator(Operator operator, OutputPort port) { + // Apex DAG requires a unique operator name + // use the transform's name and make it unique + String name = getCurrentTransform().getFullName(); + for (int i=1; this.operators.containsKey(name); name = getCurrentTransform().getFullName() + i++); + this.operators.put(name, operator); + PCollection output = getOutput(); + this.streams.put(output, (Pair)new ImmutablePair<>(port, new ArrayList<>())); + } + + /** + * Add operator that is internal to a transformation. + * @param output + * @param operator + * @param port + * @param name + */ + public PInput addInternalOperator(Operator operator, OutputPort port, String name, Coder coder) { + checkArgument(this.operators.get(name) == null, "duplicate operator " + name); + this.operators.put(name, operator); + PCollection input = getInput(); + PCollection output = PCollection.createPrimitiveOutputInternal(input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + output.setCoder(coder); + this.streams.put(output, (Pair)new ImmutablePair<>(port, new ArrayList<>())); + return output; + } + + public void addStream(PInput input, InputPort inputPort) { + Pair, List>> stream = this.streams.get(input); + checkArgument(stream != null, "no upstream operator defined"); + stream.getRight().add(inputPort); + } + + public void populateDAG(DAG dag) { + for (Map.Entry nameAndOperator : this.operators.entrySet()) { + dag.addOperator(nameAndOperator.getKey(), nameAndOperator.getValue()); + } + int streamIndex = 0; + for (Map.Entry, List>>> streamEntry : this.streams.entrySet()) { + List> sinksList = streamEntry.getValue().getRight(); + InputPort[] sinks = sinksList.toArray(new InputPort[sinksList.size()]); + if (sinks.length > 0) { + dag.addStream("stream"+streamIndex++, streamEntry.getValue().getLeft(), sinks); + for (InputPort port : sinks) { + PCollection pc = streamEntry.getKey(); + Coder coder = pc.getCoder(); + if (pc.getWindowingStrategy() != null) { + coder = FullWindowedValueCoder.of(pc.getCoder(), + pc.getWindowingStrategy().getWindowFn().windowCoder() + ); + } + Coder wrapperCoder = ApexStreamTuple.ApexStreamTupleCoder.of(coder); + CoderAdapterStreamCodec streamCodec = new CoderAdapterStreamCodec(wrapperCoder); + dag.setInputPortAttribute(port, PortContext.STREAM_CODEC, streamCodec); + } + } + } + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java new file mode 100644 index 000000000000..4608c92249a4 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java @@ -0,0 +1,427 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.translators.functions; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; +import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; +import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.KeyedWorkItem; +import org.apache.beam.sdk.util.KeyedWorkItems; +import org.apache.beam.sdk.util.SystemReduceFn; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingInternals; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.util.state.InMemoryStateInternals; +import org.apache.beam.sdk.util.state.StateInternals; +import org.apache.beam.sdk.util.state.StateInternalsFactory; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Instant; + +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.Operator; +import com.datatorrent.api.StreamCodec; +import com.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + +/** + * Apex operator for Beam {@link GroupByKey}. + * This operator expects the input stream already partitioned by K, + * which is determined by the {@link StreamCodec} on the input port. + * + * @param + * @param + */ +public class ApexGroupByKeyOperator implements Operator +{ + @Bind(JavaSerializer.class) + private WindowingStrategy windowingStrategy; + @Bind(JavaSerializer.class) + private Coder valueCoder; + + @Bind(JavaSerializer.class) + private final SerializablePipelineOptions serializedOptions; + @Bind(JavaSerializer.class) + private Map> perKeyStateInternals = new HashMap<>(); + private Map> activeTimers = new HashMap<>(); + + private transient ProcessContext context; + private transient OldDoFn, KV>> fn; + private transient ApexTimerInternals timerInternals = new ApexTimerInternals(); + private Instant inputWatermark = new Instant(0); + + public final transient DefaultInputPort>>> input = new DefaultInputPort>>>() + { + @Override + public void process(ApexStreamTuple>> t) + { + //System.out.println("\n***RECEIVED: " +t); + try { + if (t instanceof ApexStreamTuple.WatermarkTuple) { + ApexStreamTuple.WatermarkTuple mark = (ApexStreamTuple.WatermarkTuple)t; + processWatermark(mark); + output.emit(ApexStreamTuple.WatermarkTuple.>>>of(mark.getTimestamp())); + return; + } + processElement(t.getValue()); + } catch (Exception e) { + Throwables.propagate(e); + } + } + }; + + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort>>>> output = new DefaultOutputPort<>(); + + @SuppressWarnings("unchecked") + public ApexGroupByKeyOperator(ApexPipelineOptions pipelineOptions, PCollection> input) + { + Preconditions.checkNotNull(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); + this.windowingStrategy = (WindowingStrategy)input.getWindowingStrategy(); + this.valueCoder = ((KvCoder)input.getCoder()).getValueCoder(); + } + + @SuppressWarnings("unused") // for Kryo + private ApexGroupByKeyOperator() + { + this.serializedOptions = null; + } + + @Override + public void beginWindow(long l) + { + } + + @Override + public void endWindow() + { + } + + @Override + public void setup(OperatorContext context) + { + StateInternalsFactory stateInternalsFactory = new GroupByKeyStateInternalsFactory(); + this.fn = GroupAlsoByWindowViaWindowSetDoFn.create(this.windowingStrategy, stateInternalsFactory, + SystemReduceFn.buffering(this.valueCoder)); + this.context = new ProcessContext(fn, this.timerInternals); + } + + @Override + public void teardown() + { + } + + /** + * Returns the list of timers that are ready to fire. These are the timers + * that are registered to be triggered at a time before the current watermark. + * We keep these timers in a Set, so that they are deduplicated, as the same + * timer can be registered multiple times. + */ + private Multimap getTimersReadyToProcess(long currentWatermark) { + + // we keep the timers to return in a different list and launch them later + // because we cannot prevent a trigger from registering another trigger, + // which would lead to concurrent modification exception. + Multimap toFire = HashMultimap.create(); + + Iterator>> it = activeTimers.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry> keyWithTimers = it.next(); + + Iterator timerIt = keyWithTimers.getValue().iterator(); + while (timerIt.hasNext()) { + TimerInternals.TimerData timerData = timerIt.next(); + if (timerData.getTimestamp().isBefore(currentWatermark)) { + toFire.put(keyWithTimers.getKey(), timerData); + timerIt.remove(); + } + } + + if (keyWithTimers.getValue().isEmpty()) { + it.remove(); + } + } + return toFire; + } + + private void processElement(WindowedValue> windowedValue) throws Exception { + final KV kv = windowedValue.getValue(); + final WindowedValue updatedWindowedValue = WindowedValue.of(kv.getValue(), + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPane()); + + KeyedWorkItem kwi = KeyedWorkItems.elementsWorkItem( + kv.getKey(), + Collections.singletonList(updatedWindowedValue)); + + context.setElement(kwi, getStateInternalsForKey(kwi.key())); + fn.processElement(context); + } + + private StateInternals getStateInternalsForKey(K key) { + StateInternals stateInternals = perKeyStateInternals.get(key); + if (stateInternals == null) { + //Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); + //OutputTimeFn outputTimeFn = this.windowingStrategy.getOutputTimeFn(); + stateInternals = InMemoryStateInternals.forKey(key); + perKeyStateInternals.put(key, stateInternals); + } + return stateInternals; + } + + private void registerActiveTimer(K key, TimerInternals.TimerData timer) { + Set timersForKey = activeTimers.get(key); + if (timersForKey == null) { + timersForKey = new HashSet<>(); + } + timersForKey.add(timer); + activeTimers.put(key, timersForKey); + } + + private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { + Set timersForKey = activeTimers.get(key); + if (timersForKey != null) { + timersForKey.remove(timer); + if (timersForKey.isEmpty()) { + activeTimers.remove(key); + } else { + activeTimers.put(key, timersForKey); + } + } + } + + private void processWatermark(ApexStreamTuple.WatermarkTuple mark) throws Exception { + this.inputWatermark = new Instant(mark.getTimestamp()); + Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); + if (!timers.isEmpty()) { + for (K key : timers.keySet()) { + KeyedWorkItem kwi = KeyedWorkItems.timersWorkItem(key, timers.get(key)); + context.setElement(kwi, getStateInternalsForKey(kwi.key())); + fn.processElement(context); + } + } + } + + private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn, ?, KeyedWorkItem>.ProcessContext { + + private final ApexTimerInternals timerInternals; + private StateInternals stateInternals; + private KeyedWorkItem element; + + public ProcessContext(OldDoFn, KV>> function, + ApexTimerInternals timerInternals) { + function.super(); + this.timerInternals = Preconditions.checkNotNull(timerInternals); + } + + public void setElement(KeyedWorkItem element, StateInternals stateForKey) { + this.element = element; + this.stateInternals = stateForKey; + } + + @Override + public KeyedWorkItem element() { + return this.element; + } + + @Override + public Instant timestamp() { + throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems."); + } + + @Override + public PipelineOptions getPipelineOptions() { + return serializedOptions.get(); + } + + @Override + public void output(KV> output) { + throw new UnsupportedOperationException( + "output() is not available when processing KeyedWorkItems."); + } + + @Override + public void outputWithTimestamp(KV> output, Instant timestamp) { + throw new UnsupportedOperationException( + "outputWithTimestamp() is not available when processing KeyedWorkItems."); + } + + @Override + public PaneInfo pane() { + throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems."); + } + + @Override + public BoundedWindow window() { + throw new UnsupportedOperationException( + "window() is not available when processing KeyedWorkItems."); + } + + @Override + public WindowingInternals, KV>> windowingInternals() { + return new WindowingInternals, KV>>() { + + @Override + public StateInternals stateInternals() { + return stateInternals; + } + + @Override + public void outputWindowedValue(KV> output, Instant timestamp, Collection windows, PaneInfo pane) { + System.out.println("\n***EMITTING: " + output + ", timestamp=" + timestamp); + ApexGroupByKeyOperator.this.output.emit(ApexStreamTuple.DataTuple.of(WindowedValue.of(output, timestamp, windows, pane))); + } + + @Override + public TimerInternals timerInternals() { + return timerInternals; + } + + @Override + public Collection windows() { + throw new UnsupportedOperationException("windows() is not available in Streaming mode."); + } + + @Override + public PaneInfo pane() { + throw new UnsupportedOperationException("pane() is not available in Streaming mode."); + } + + @Override + public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + throw new RuntimeException("writePCollectionViewData() not available in Streaming mode."); + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + throw new RuntimeException("sideInput() is not available in Streaming mode."); + } + }; + } + + @Override + public T sideInput(PCollectionView view) { + throw new RuntimeException("sideInput() is not supported in Streaming mode."); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + // ignore the side output, this can happen when a user does not register + // side outputs but then outputs using a freshly created TupleTag. + throw new RuntimeException("sideOutput() is not available when grouping by window."); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + sideOutput(tag, output); + } + + @Override + protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + throw new UnsupportedOperationException(); + } + } + + /** + * An implementation of Beam's {@link TimerInternals}. + * + */ + public class ApexTimerInternals implements TimerInternals { + + @Override + public void setTimer(TimerData timerKey) + { + registerActiveTimer(context.element().key(), timerKey); + } + + @Override + public void deleteTimer(TimerData timerKey) + { + unregisterActiveTimer(context.element().key(), timerKey); + } + + @Override + public Instant currentProcessingTime() + { + return Instant.now(); + } + + @Override + public Instant currentSynchronizedProcessingTime() + { + // TODO Auto-generated method stub + return null; + } + + @Override + public Instant currentInputWatermarkTime() + { + return inputWatermark; + } + + @Override + public Instant currentOutputWatermarkTime() + { + // TODO Auto-generated method stub + return null; + } + + } + + private class GroupByKeyStateInternalsFactory implements StateInternalsFactory, Serializable + { + @Override + public StateInternals stateInternalsForKey(K key) + { + return getStateInternalsForKey(key); + } + } + +} \ No newline at end of file diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java new file mode 100644 index 000000000000..8005832dc786 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.translators.functions; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; +import org.apache.beam.runners.apex.translators.utils.NoOpStepContext; +import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; +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.ExecutionContext; +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.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; +import com.datatorrent.common.util.BaseOperator; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.esotericsoftware.kryo.serializers.JavaSerializer; + +/** + * Apex operator for Beam {@link DoFn}. + */ +public class ApexParDoOperator extends BaseOperator implements OutputManager { + + private transient final TupleTag mainTag = new TupleTag(); + private transient DoFnRunner doFnRunner; + + @Bind(JavaSerializer.class) + private final SerializablePipelineOptions pipelineOptions; + @Bind(JavaSerializer.class) + private final OldDoFn doFn; + @Bind(JavaSerializer.class) + private final WindowingStrategy windowingStrategy; + @Bind(JavaSerializer.class) + private final SideInputReader sideInputReader; + + public ApexParDoOperator( + ApexPipelineOptions pipelineOptions, + OldDoFn doFn, + WindowingStrategy windowingStrategy, + SideInputReader sideInputReader) { + this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions); + this.doFn = doFn; + this.windowingStrategy = windowingStrategy; + this.sideInputReader = sideInputReader; + } + + @SuppressWarnings("unused") // for Kryo + private ApexParDoOperator() { + this(null, null, null, null); + } + + public final transient DefaultInputPort>> input = new DefaultInputPort>>() + { + @Override + public void process(ApexStreamTuple> t) + { + if (t instanceof ApexStreamTuple.WatermarkTuple) { + output.emit(t); + } else { + System.out.println("\n" + Thread.currentThread().getName() + "\n" + t.getValue() + "\n"); + doFnRunner.processElement(t.getValue()); + } + } + }; + + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort> output = new DefaultOutputPort<>(); + + @Override + public void output(TupleTag tag, WindowedValue tuple) + { + output.emit(ApexStreamTuple.DataTuple.of(tuple)); + } + + @Override + public void setup(OperatorContext context) + { + this.doFnRunner = DoFnRunners.simpleRunner(pipelineOptions.get(), + doFn, + sideInputReader, + this, + mainTag, + TupleTagList.empty().getAll(), + new NoOpStepContext(), + new NoOpAggregatorFactory(), + windowingStrategy + ); + } + + @Override + public void beginWindow(long windowId) + { + doFnRunner.startBundle(); + /* + Collection> aggregators = AggregatorRetriever.getAggregators(doFn); + if (!aggregators.isEmpty()) { + System.out.println("\n" + Thread.currentThread().getName() + "\n" +AggregatorRetriever.getAggregators(doFn) + "\n"); + } + */ + } + + @Override + public void endWindow() + { + doFnRunner.finishBundle(); + } + + /** + * TODO: Placeholder for aggregation, to be implemented for embedded and cluster mode. + * It is called from {@link org.apache.beam.sdk.util.SimpleDoFnRunner}. + */ + public class NoOpAggregatorFactory implements AggregatorFactory { + + private NoOpAggregatorFactory() { + } + + @Override + public Aggregator createAggregatorForDoFn( + Class fnClass, ExecutionContext.StepContext step, + String name, CombineFn combine) { + return new Aggregator() { + + @Override + public void addValue(InputT value) + { + } + + @Override + public String getName() + { + // TODO Auto-generated method stub + return null; + } + + @Override + public CombineFn getCombineFn() + { + // TODO Auto-generated method stub + return null; + } + + }; + } + } + + + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java new file mode 100644 index 000000000000..39114feb849b --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.translators.io; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple.DataTuple; +import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; +import org.apache.beam.sdk.io.UnboundedSource; +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.joda.time.Instant; + +import com.datatorrent.api.Context.OperatorContext; +import com.google.common.base.Throwables; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.InputOperator; +import com.datatorrent.common.util.BaseOperator; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.esotericsoftware.kryo.serializers.JavaSerializer; + +import java.io.IOException; + +/** + * Apex input operator that wraps Beam UnboundedSource. + */ +public class ApexReadUnboundedInputOperator + implements InputOperator { + + @Bind(JavaSerializer.class) + private final SerializablePipelineOptions pipelineOptions; + @Bind(JavaSerializer.class) + private final UnboundedSource source; + private transient UnboundedSource.UnboundedReader reader; + private transient boolean available = false; + public final transient DefaultOutputPort>> output = new DefaultOutputPort<>(); + + public ApexReadUnboundedInputOperator(UnboundedSource source, ApexPipelineOptions options) { + this.pipelineOptions = new SerializablePipelineOptions(options); + this.source = source; + } + + @SuppressWarnings("unused") // for Kryo + private ApexReadUnboundedInputOperator() { + this.pipelineOptions = null; this.source = null; + } + + @Override + public void beginWindow(long windowId) + { + Instant mark = reader.getWatermark(); + output.emit(ApexStreamTuple.WatermarkTuple.>of(mark.getMillis())); + if (!available && source instanceof ValuesSource) { + // if it's a Create transformation and the input was consumed, + // terminate the stream (allows tests to finish faster) + BaseOperator.shutdown(); + } + } + + @Override + public void endWindow() + { + } + + @Override + public void setup(OperatorContext context) + { + try { + reader = source.createReader(this.pipelineOptions.get(), null); + available = reader.start(); + } catch (IOException e) { + Throwables.propagate(e); + } + } + + @Override + public void teardown() + { + try { + if (reader != null) { + reader.close(); + } + } catch (IOException e) { + Throwables.propagate(e); + } + } + + @Override + public void emitTuples() + { + try { + if (!available) { + available = reader.advance(); + } + if (available) { + OutputT data = reader.getCurrent(); + Instant timestamp = reader.getCurrentTimestamp(); + available = reader.advance(); + output.emit(DataTuple.of(WindowedValue.of( + data, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); + } + } catch (Exception e) { + Throwables.propagate(e); + } + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java new file mode 100644 index 000000000000..2c4b298c56ee --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.translators.io; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + +import com.google.common.base.Throwables; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * unbounded source that reads from a Java {@link Iterable}. + */ +public class ValuesSource extends UnboundedSource { + private static final long serialVersionUID = 1L; + + private final byte[] codedValues; + private final IterableCoder iterableCoder; + + public ValuesSource(Iterable values, Coder coder) { + this.iterableCoder = IterableCoder.of(coder); + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + try { + iterableCoder.encode(values, bos, Context.OUTER); + } catch (IOException ex) { + Throwables.propagate(ex); + } + this.codedValues = bos.toByteArray(); + } + + @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) { + ByteArrayInputStream bis = new ByteArrayInputStream(codedValues); + try { + Iterable values = this.iterableCoder.decode(bis, Context.OUTER); + return new ValuesReader<>(values, this); + } catch (IOException ex) { + throw Throwables.propagate(ex); + } + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() { + } + + @Override + public Coder getDefaultOutputCoder() { + return iterableCoder.getElemCoder(); + } + + private static class ValuesReader extends UnboundedReader { + + private final Iterable values; + private final UnboundedSource source; + private transient Iterator iterator; + private T current; + + public ValuesReader(Iterable values, UnboundedSource source) { + this.values = values; + this.source = source; + } + + @Override + public boolean start() throws IOException { + if (null == iterator) { + iterator = values.iterator(); + } + return advance(); + } + + @Override + public boolean advance() throws IOException { + if (iterator.hasNext()) { + current = 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/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java new file mode 100644 index 000000000000..efb69eef8ed3 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.translators.utils; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +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.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StandardCoder; + +public interface ApexStreamTuple +{ + /** + * Gets the value of the tuple + * + * @return + */ + T getValue(); + + /** + * Plain tuple class + * + * @param + */ + class DataTuple implements ApexStreamTuple + { + private T value; + + public static DataTuple of(T value) { + return new DataTuple<>(value); + } + + private DataTuple(T value) + { + this.value = value; + } + + @Override + public T getValue() + { + return value; + } + + public void setValue(T value) + { + this.value = value; + } + + @Override + public String toString() + { + return value.toString(); + } + + } + + /** + * Tuple that includes a timestamp + * + * @param + */ + class TimestampedTuple extends DataTuple + { + private long timestamp; + + public TimestampedTuple(long timestamp, T value) + { + super(value); + this.timestamp = timestamp; + } + + public long getTimestamp() + { + return timestamp; + } + + public void setTimestamp(long timestamp) + { + this.timestamp = timestamp; + } + } + + /** + * Tuple that represents a watermark + * + * @param + */ + class WatermarkTuple extends TimestampedTuple + { + public static WatermarkTuple of(long timestamp) { + return new WatermarkTuple<>(timestamp); + } + + protected WatermarkTuple(long timestamp) + { + super(timestamp, null); + } + + @Override + public String toString() + { + return "[Watermark " + getTimestamp() + "]"; + } + } + + /** + * Coder for {@link ApexStreamTuple}. + */ + public static class ApexStreamTupleCoder extends StandardCoder> { + private static final long serialVersionUID = 1L; + final Coder valueCoder; + + public static ApexStreamTupleCoder of(Coder valueCoder) { + return new ApexStreamTupleCoder<>(valueCoder); + } + + protected ApexStreamTupleCoder(Coder valueCoder) { + this.valueCoder = checkNotNull(valueCoder); + } + + @Override + public void encode(ApexStreamTuple value, OutputStream outStream, Context context) + throws CoderException, IOException { + if (value instanceof WatermarkTuple) { + outStream.write(1); + new DataOutputStream(outStream).writeLong(((WatermarkTuple)value).getTimestamp()); + } else { + outStream.write(0); + valueCoder.encode(value.getValue(), outStream, context); + } + } + + @Override + public ApexStreamTuple decode(InputStream inStream, Context context) + throws CoderException, IOException + { + int b = inStream.read(); + if (b == 1) { + return new WatermarkTuple(new DataInputStream(inStream).readLong()); + } else { + return new DataTuple(valueCoder.decode(inStream, context)); + } + } + + @Override + public List> getCoderArguments() + { + return Arrays.>asList(valueCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException + { + verifyDeterministic( + this.getClass().getSimpleName() + " requires a deterministic valueCoder", + valueCoder); + } + + /** + * Returns the value coder. + */ + public Coder getValueCoder() { + return valueCoder; + } + + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java new file mode 100644 index 000000000000..c18765bb0121 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.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.apex.translators.utils; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; + +import com.datatorrent.api.StreamCodec; +import com.datatorrent.netlet.util.Slice; +import com.google.common.base.Throwables; + +/** + * The Apex {@link StreamCodec} adapter for using Beam {@link Coder}. + */ +public class CoderAdapterStreamCodec implements StreamCodec, Serializable { + + private static final long serialVersionUID = 1L; + private final Coder coder; + + public CoderAdapterStreamCodec(Coder coder) { + this.coder = coder; + } + + @Override + public Object fromByteArray(Slice fragment) + { + ByteArrayInputStream bis = new ByteArrayInputStream(fragment.buffer, fragment.offset, fragment.length); + try { + return coder.decode(bis, Context.OUTER); + } catch (IOException e) { + throw Throwables.propagate(e); + } + } + + @Override + public Slice toByteArray(Object wv) + { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + try { + coder.encode(wv, bos, Context.OUTER); + } catch (IOException e) { + Throwables.propagate(e); + } + return new Slice(bos.toByteArray()); + } + + @Override + public int getPartition(Object o) + { + return o.hashCode(); + } + +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.java new file mode 100644 index 000000000000..ffe1a29d9ddf --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.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.apex.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/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java new file mode 100644 index 000000000000..43d92f61116d --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.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.apex.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 does nothing. + */ +public class NoOpStepContext implements ExecutionContext.StepContext, Serializable { + + private static final long serialVersionUID = 1L; + + @Override + public String getStepName() { + return null; + } + + @Override + public String getTransformName() { + return null; + } + + @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() { + return null; + } + + @Override + public TimerInternals timerInternals() { + return null; + } +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java new file mode 100644 index 000000000000..7f7b3ef21662 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.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.runners.apex.translators.utils; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptions; + +import com.fasterxml.jackson.databind.ObjectMapper; + +/** + * A wrapper to enable serialization of {@link PipelineOptions} + */ +public class SerializablePipelineOptions implements Externalizable { + + 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(new ObjectMapper().writeValueAsString(pipelineOptions)); + } + + @Override + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException + { + String s = in.readUTF(); + this.pipelineOptions = new ObjectMapper().readValue(s, PipelineOptions.class).as(ApexPipelineOptions.class); + } + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java new file mode 100644 index 000000000000..582d83986699 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex.examples; + +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.runners.apex.ApexRunner; +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.joda.time.Duration; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Windowed word count example on Apex runner. + */ +public class StreamingWordCountTest { + + static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @ProcessElement + public void processElement(ProcessContext c) { + if (c.element().trim().isEmpty()) { + emptyLines.addValue(1L); + } + + // Split the line into words. + String[] words = c.element().split("[^a-zA-Z']+"); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + c.output(word); + } + } + } + } + + static class FormatAsStringFn extends DoFn, String> { + private static final Logger LOG = LoggerFactory.getLogger(FormatAsStringFn.class); + static final ConcurrentHashMap RESULTS = new ConcurrentHashMap<>(); + + @ProcessElement + public void processElement(ProcessContext c) { + String row = c.element().getKey() + " - " + c.element().getValue() + + " @ " + c.timestamp().toString(); + LOG.debug("output {}", row); + c.output(row); + RESULTS.put(c.element().getKey(), c.element().getValue()); + } + } + + @Test + public void testWindowedWordCount() throws Exception { + String[] args = new String[] { + "--runner=" + ApexRunner.class.getName() + }; + ApexPipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() + .as(ApexPipelineOptions.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())); + + ApexRunnerResult result = (ApexRunnerResult)p.run(); + Assert.assertNotNull(result.getApexDAG().getOperatorMeta("Read(UnboundedTextSource)")); + long timeout = System.currentTimeMillis() + 30000; + while (System.currentTimeMillis() < timeout) { + if (FormatAsStringFn.RESULTS.containsKey("foo") && FormatAsStringFn.RESULTS.containsKey("bar")) { + break; + } + Thread.sleep(1000); + } + result.cancel(); + Assert.assertTrue(FormatAsStringFn.RESULTS.containsKey("foo") && FormatAsStringFn.RESULTS.containsKey("bar")); + FormatAsStringFn.RESULTS.clear(); + + } +} 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 new file mode 100644 index 000000000000..29351e97bc51 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex.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 com.google.common.base.Throwables; + +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 { + private static final long serialVersionUID = 1L; + + @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); + try { + Thread.sleep(index); // allow for downstream processing to complete + } catch (InterruptedException e) { + Throwables.propagate(e); + } + 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/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java new file mode 100644 index 000000000000..d3b56bcb19b7 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.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.apex.translators; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.runners.apex.ApexRunner; +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.transforms.Create; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.List; + +/** + * integration test for {@link FlattenPCollectionTranslator}. + */ +public class FlattenPCollectionTranslatorTest { + private static final Logger LOG = LoggerFactory.getLogger(FlattenPCollectionTranslatorTest.class); + + @Test + public void test() throws Exception { + ApexPipelineOptions options = + PipelineOptionsFactory.as(ApexPipelineOptions.class); + options.setApplicationName("FlattenPCollection"); + options.setRunner(ApexRunner.class); + Pipeline p = Pipeline.create(options); + + List collection1 = Lists.newArrayList("1", "2", "3"); + List collection2 = Lists.newArrayList("4", "5"); + List expected = Lists.newArrayList("1", "2", "3", "4", "5"); + PCollection pc1 = + p.apply(Create.of(collection1).withCoder(StringUtf8Coder.of())); + PCollection pc2 = + p.apply(Create.of(collection2).withCoder(StringUtf8Coder.of())); + PCollectionList pcs = PCollectionList.of(pc1).and(pc2); + PCollection actual = pcs.apply(Flatten.pCollections()); + actual.apply(ParDo.of(new EmbeddedCollector())); + + ApexRunnerResult result = (ApexRunnerResult)p.run(); + // TODO: verify translation + result.getApexDAG(); + long timeout = System.currentTimeMillis() + 30000; + while (System.currentTimeMillis() < timeout) { + if (EmbeddedCollector.results.containsAll(expected)) { + break; + } + LOG.info("Waiting for expected results."); + Thread.sleep(1000); + } + org.junit.Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + + } + + @SuppressWarnings("serial") + private static class EmbeddedCollector extends OldDoFn { + protected static final HashSet results = new HashSet<>(); + + public EmbeddedCollector() { + } + + @Override + public void processElement(ProcessContext c) throws Exception { + results.add(c.element()); + } + } + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java new file mode 100644 index 000000000000..e4d4606db450 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.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.runners.apex.translators; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; + +import com.datatorrent.api.DAG; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * integration test for {@link GroupByKeyTranslator}. + */ +public class GroupByKeyTranslatorTest { + + @SuppressWarnings({"unchecked"}) + @Test + public void test() throws Exception { + ApexPipelineOptions options = + PipelineOptionsFactory.as(ApexPipelineOptions.class); + options.setApplicationName("GroupByKey"); + options.setRunner(ApexRunner.class); + Pipeline p = Pipeline.create(options); + + List> data = + Lists.newArrayList( + KV.of("foo", new Instant(1000)), KV.of("foo", new Instant(1000)), + KV.of("foo", new Instant(2000)), + KV.of("bar", new Instant(1000)), KV.of("bar", new Instant(2000)), + KV.of("bar", new Instant(2000)) + ); + + // expected results assume outputAtLatestInputTimestamp + List>> expected = + Lists.newArrayList( + KV.of(new Instant(1000), KV.of("foo", 2L)), + KV.of(new Instant(1000), KV.of("bar", 1L)), + KV.of(new Instant(2000), KV.of("foo", 1L)), + KV.of(new Instant(2000), KV.of("bar", 2L)) + ); + + p.apply(Read.from(new TestSource(data, new Instant(5000)))) + .apply(Window.into(FixedWindows.of(Duration.standardSeconds(1))) + .withOutputTimeFn(OutputTimeFns.outputAtLatestInputTimestamp())) + .apply(Count.perElement()) + .apply(ParDo.of(new KeyedByTimestamp>())) + .apply(ParDo.of(new EmbeddedCollector())) + ; + + ApexRunnerResult result = (ApexRunnerResult)p.run(); + // TODO: verify translation + DAG dag = result.getApexDAG(); + + long timeout = System.currentTimeMillis() + 30000; + while (System.currentTimeMillis() < timeout) { + if (EmbeddedCollector.results.containsAll(expected)) { + break; + } + Thread.sleep(1000); + } + Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + + } + + @SuppressWarnings("serial") + private static class EmbeddedCollector extends OldDoFn { + protected static final HashSet results = new HashSet<>(); + + public EmbeddedCollector() { + } + + @Override + public void processElement(ProcessContext c) throws Exception { + results.add(c.element()); + } + } + + private static class KeyedByTimestamp extends OldDoFn> { + + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(KV.of(c.timestamp(), c.element())); + } + } + + private static class TestSource extends UnboundedSource { + + private final List> data; + private final Instant watermark; + + public TestSource(List> data, Instant watermark) { + this.data = data; + this.watermark = watermark; + } + + @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 TestReader(data, watermark, this); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() { + } + + @Override + public Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + + private static class TestReader extends UnboundedReader implements Serializable { + + private static final long serialVersionUID = 7526472295622776147L; + + private final List> data; + private final TestSource source; + + private Iterator> iterator; + private String currentRecord; + private Instant currentTimestamp; + private Instant watermark; + private boolean collected; + + public TestReader(List> data, Instant watermark, TestSource source) { + this.data = data; + this.source = source; + this.watermark = watermark; + } + + @Override + public boolean start() throws IOException { + iterator = data.iterator(); + return advance(); + } + + @Override + public boolean advance() throws IOException { + if (iterator.hasNext()) { + KV kv = iterator.next(); + collected = false; + currentRecord = kv.getKey(); + currentTimestamp = kv.getValue(); + return true; + } else { + return false; + } + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return new byte[0]; + } + + @Override + public String getCurrent() throws NoSuchElementException { + collected = true; + return this.currentRecord; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return currentTimestamp; + } + + @Override + public void close() throws IOException { + } + + @Override + public Instant getWatermark() { + if (!iterator.hasNext() && collected) { + return watermark; + } else { + return new Instant(0); + } + } + + @Override + public CheckpointMark getCheckpointMark() { + return null; + } + + @Override + public UnboundedSource getCurrentSource() { + return this.source; + } + } + } +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java new file mode 100644 index 000000000000..06aaf550a1b7 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.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.apex.translators; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; +import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.SerializableCoder; +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.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + +import com.datatorrent.api.DAG; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.HashSet; +import java.util.List; +import java.util.regex.Pattern; + +/** + * integration test for {@link ParDoBoundTranslator}. + */ +@RunWith(JUnit4.class) +public class ParDoBoundTranslatorTest { + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorTest.class); + + @Test + public void test() throws Exception { + ApexPipelineOptions options = PipelineOptionsFactory.create() + .as(ApexPipelineOptions.class); + options.setApplicationName("ParDoBound"); + options.setRunner(ApexRunner.class); + + Pipeline p = Pipeline.create(options); + + List collection = Lists.newArrayList(1, 2, 3, 4, 5); + List expected = Lists.newArrayList(6, 7, 8, 9, 10); + p.apply(Create.of(collection).withCoder(SerializableCoder.of(Integer.class))) + .apply(ParDo.of(new Add(5))) + .apply(ParDo.of(new EmbeddedCollector())); + + ApexRunnerResult result = (ApexRunnerResult)p.run(); + DAG dag = result.getApexDAG(); + + DAG.OperatorMeta om = dag.getOperatorMeta("Create.Values"); + Assert.assertNotNull(om); + Assert.assertEquals(om.getOperator().getClass(), ApexReadUnboundedInputOperator.class); + + om = dag.getOperatorMeta("ParDo(Add)"); + Assert.assertNotNull(om); + Assert.assertEquals(om.getOperator().getClass(), ApexParDoOperator.class); + + long timeout = System.currentTimeMillis() + 30000; + while (System.currentTimeMillis() < timeout) { + if (EmbeddedCollector.results.containsAll(expected)) { + break; + } + LOG.info("Waiting for expected results."); + Thread.sleep(1000); + } + Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + } + + @SuppressWarnings("serial") + private static class Add extends OldDoFn { + private final Integer number; + + public Add(Integer number) { + this.number = number; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element() + number); + } + } + + @SuppressWarnings("serial") + private static class EmbeddedCollector extends OldDoFn { + protected static final HashSet results = new HashSet<>(); + + public EmbeddedCollector() { + } + + @Override + public void processElement(ProcessContext c) throws Exception { + results.add(c.element()); + } + } + + + @Ignore + @Test + public void testAssertionFailure() throws Exception { + ApexPipelineOptions options = PipelineOptionsFactory.create() + .as(ApexPipelineOptions.class); + options.setRunner(ApexRunner.class); + Pipeline pipeline = Pipeline.create(options); + + PCollection pcollection = pipeline + .apply(Create.of(1, 2, 3, 4)); + PAssert.that(pcollection).containsInAnyOrder(2, 1, 4, 3, 7); + + Throwable exc = runExpectingAssertionFailure(pipeline); + Pattern expectedPattern = Pattern.compile( + "Expected: iterable over \\[((<4>|<7>|<3>|<2>|<1>)(, )?){5}\\] in any order"); + // A loose pattern, but should get the job done. + assertTrue( + "Expected error message from PAssert with substring matching " + + expectedPattern + + " but the message was \"" + + exc.getMessage() + + "\"", + expectedPattern.matcher(exc.getMessage()).find()); + } + + private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { + // We cannot use thrown.expect(AssertionError.class) because the AssertionError + // is first caught by JUnit and causes a test failure. + try { + pipeline.run(); + } catch (AssertionError exc) { + return exc; + } + fail("assertion should have failed"); + throw new RuntimeException("unreachable"); + } + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java new file mode 100644 index 000000000000..6260632544a1 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.apex.translators; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.runners.apex.translators.utils.CollectionSource; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.ParDo; + +import com.datatorrent.api.DAG; +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; + +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * integration test for {@link ReadUnboundedTranslator}. + */ +public class ReadUnboundTranslatorTest { + private static final Logger LOG = LoggerFactory.getLogger(ReadUnboundTranslatorTest.class); + + @Test + public void test() throws Exception { + ApexPipelineOptions options = PipelineOptionsFactory.create() + .as(ApexPipelineOptions.class); + EmbeddedCollector.results.clear(); + options.setApplicationName("ReadUnbound"); + options.setRunner(ApexRunner.class); + Pipeline p = Pipeline.create(options); + + List collection = Lists.newArrayList("1", "2", "3", "4", "5"); + CollectionSource source = new CollectionSource<>(collection, StringUtf8Coder.of()); + p.apply(Read.from(source)) + .apply(ParDo.of(new EmbeddedCollector())); + + ApexRunnerResult result = (ApexRunnerResult)p.run(); + DAG dag = result.getApexDAG(); + DAG.OperatorMeta om = dag.getOperatorMeta("Read(CollectionSource)"); + Assert.assertNotNull(om); + Assert.assertEquals(om.getOperator().getClass(), ApexReadUnboundedInputOperator.class); + + long timeout = System.currentTimeMillis() + 30000; + while (System.currentTimeMillis() < timeout) { + if (EmbeddedCollector.results.containsAll(collection)) { + break; + } + LOG.info("Waiting for expected results."); + Thread.sleep(1000); + } + Assert.assertEquals(Sets.newHashSet(collection), EmbeddedCollector.results); + } + + @Test + public void testReadBounded() throws Exception { + ApexPipelineOptions options = PipelineOptionsFactory.create() + .as(ApexPipelineOptions.class); + EmbeddedCollector.results.clear(); + options.setApplicationName("ReadBounded"); + options.setRunner(ApexRunner.class); + Pipeline p = Pipeline.create(options); + + Set expected = ContiguousSet.create(Range.closedOpen(0L, 10L), DiscreteDomain.longs()); + p.apply(Read.from(CountingSource.upTo(10))) + .apply(ParDo.of(new EmbeddedCollector())); + + ApexRunnerResult result = (ApexRunnerResult)p.run(); + DAG dag = result.getApexDAG(); + DAG.OperatorMeta om = dag.getOperatorMeta("Read(BoundedCountingSource)/Read(BoundedCountingSource)/Read(BoundedToUnboundedSourceAdapter)"); + Assert.assertNotNull(om); + Assert.assertEquals(om.getOperator().getClass(), ApexReadUnboundedInputOperator.class); + + long timeout = System.currentTimeMillis() + 30000; + while (System.currentTimeMillis() < timeout) { + if (EmbeddedCollector.results.containsAll(expected)) { + break; + } + LOG.info("Waiting for expected results."); + Thread.sleep(1000); + } + Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + } + + @SuppressWarnings("serial") + private static class EmbeddedCollector extends OldDoFn { + protected static final HashSet results = new HashSet<>(); + + public EmbeddedCollector() { + } + + @Override + public void processElement(ProcessContext c) throws Exception { + results.add(c.element()); + } + } + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java new file mode 100644 index 000000000000..a1e8b3e55425 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.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.apex.translators.utils; + +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.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * collection as {@link UnboundedSource}, used for tests. + */ +public class CollectionSource extends UnboundedSource { + + private final Collection collection; + private final Coder coder; + + public CollectionSource(Collection collection, Coder coder) { + this.collection = collection; + this.coder = coder; + } + + @Override + public List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + return Collections.singletonList(this); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, + @Nullable UnboundedSource.CheckpointMark checkpointMark) { + return new CollectionReader<>(collection, this); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() { + } + + @Override + public Coder getDefaultOutputCoder() { + return coder; + } + + private static class CollectionReader extends UnboundedSource.UnboundedReader + implements Serializable { + + private T current; + private final CollectionSource source; + private final Collection collection; + private Iterator iterator; + + public CollectionReader(Collection collection, CollectionSource source) { + this.collection = collection; + this.source = source; + } + + @Override + public boolean start() throws IOException { + if (null == iterator) { + iterator = collection.iterator(); + } + return advance(); + } + + @Override + public boolean advance() throws IOException { + if (iterator.hasNext()) { + current = iterator.next(); + return true; + } else { + return false; + } + } + + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + return null; + } + + @Override + public UnboundedSource getCurrentSource() { + return source; + } + + @Override + public T getCurrent() throws NoSuchElementException { + return current; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return Instant.now(); + } + + @Override + public void close() throws IOException { + } + } +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java new file mode 100644 index 000000000000..e2fa9d972937 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.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.apex.translators.utils; + +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.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datatorrent.common.util.FSStorageAgent; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; + +/** + * Tests the serialization of PipelineOptions. + */ +public class PipelineOptionsTest { + + public interface MyOptions extends ApexPipelineOptions { + @Description("Bla bla bla") + @Default.String("Hello") + String getTestOption(); + void setTestOption(String value); + } + + private static class MyOptionsWrapper { + private MyOptionsWrapper() { + this(null); // required for Kryo + } + private MyOptionsWrapper(ApexPipelineOptions options) { + this.options = new SerializablePipelineOptions(options); + } + @Bind(JavaSerializer.class) + private final SerializablePipelineOptions options; + } + + private static MyOptions options; + + private final static String[] args = new String[]{"--testOption=nothing"}; + + @BeforeClass + public static void beforeTest() { + options = PipelineOptionsFactory.fromArgs(args).as(MyOptions.class); + } + + @Test + public void testSerialization() { + MyOptionsWrapper wrapper = new MyOptionsWrapper(PipelineOptionsTest.options); + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + FSStorageAgent.store(bos, wrapper); + + ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); + MyOptionsWrapper wrapperCopy = (MyOptionsWrapper)FSStorageAgent.retrieve(bis); + assertNotNull(wrapperCopy.options); + assertEquals("nothing", wrapperCopy.options.get().as(MyOptions.class).getTestOption()); + } + +} diff --git a/runners/apex/src/test/resources/log4j.properties b/runners/apex/src/test/resources/log4j.properties new file mode 100644 index 000000000000..84a6f6897743 --- /dev/null +++ b/runners/apex/src/test/resources/log4j.properties @@ -0,0 +1,33 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +log4j.rootLogger=DEBUG, testlogger + +# A1 is set to be a ConsoleAppender. +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.logger.org=info +#log4j.logger.org.apache.commons.beanutils=warn +log4j.logger.com.datatorrent=info +log4j.logger.org.apache.apex=debug +log4j.logger.org.apache.beam.runners.apex=debug diff --git a/runners/pom.xml b/runners/pom.xml index 605c3b29fcae..ff800d183d3f 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -37,6 +37,7 @@ direct-java flink spark + apex From a7e430d6b24de53a116258af75c7eb15d6133b4d Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Wed, 31 Aug 2016 16:41:52 -0700 Subject: [PATCH 2/8] BEAM-261 Use Apex 3.5.0-SNAPSHOT to use loopback as connect address. --- runners/apex/pom.xml | 6 +++--- .../runners/apex/translators/CreateValuesTranslator.java | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index bb08b3ca0ae6..21e53a8ea94a 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -35,9 +35,9 @@ jar - 3.4.0 + 3.5.0-SNAPSHOT 3.4.0 - true + false -Xmx2048m @@ -206,7 +206,7 @@ - org.apache.apex:apex-api:jar:3.4.0 + org.apache.apex:apex-api:jar:3.5.0-SNAPSHOT org.apache.commons:commons-lang3::3.1 com.esotericsoftware.kryo:kryo::2.24.0 com.datatorrent:netlet::1.2.1 diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java index 387b19f5e39a..7a29057272a6 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java @@ -23,6 +23,7 @@ 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.values.PBegin; import com.google.common.base.Throwables; @@ -38,7 +39,7 @@ public class CreateValuesTranslator implements TransformTranslator transform, TranslationContext context) { try { UnboundedSource unboundedSource = new ValuesSource<>(transform.getElements(), - transform.getDefaultOutputCoder(context.getInput())); + transform.getDefaultOutputCoder((PBegin)context.getInput())); ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>(unboundedSource, context.getPipelineOptions()); context.addOperator(operator, operator.output); From 074b18f6ae0cfc1a3cc986f89ded6a45e0a3eb57 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Sun, 11 Sep 2016 20:34:08 -0700 Subject: [PATCH 3/8] BEAM-261 Read.Bounded and FlattenPCollection. --- runners/apex/pom.xml | 2 +- .../runners/apex/ApexPipelineTranslator.java | 16 ++ .../apache/beam/runners/apex/ApexRunner.java | 10 +- .../FlattenPCollectionTranslator.java | 53 ++++- .../apex/translators/TranslationContext.java | 24 +- .../functions/ApexGroupByKeyOperator.java | 6 +- .../functions/ApexParDoOperator.java | 6 +- .../beam/runners/apex/examples/IntTests.java | 207 ++++++++++++++++++ .../ReadUnboundTranslatorTest.java | 2 +- 9 files changed, 284 insertions(+), 42 deletions(-) create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 21e53a8ea94a..e9377b4d9aa6 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -28,7 +28,7 @@ ../pom.xml - beam-runners-apex_3.4.0 + beam-runners-apex Apache Beam :: Runners :: Apex diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java index 8ea7139808f9..b0391b426e97 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java @@ -25,6 +25,8 @@ import org.apache.beam.runners.apex.translators.ReadUnboundedTranslator; import org.apache.beam.runners.apex.translators.TransformTranslator; import org.apache.beam.runners.apex.translators.TranslationContext; +import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.runners.TransformTreeNode; @@ -64,6 +66,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { // 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()); @@ -130,5 +133,18 @@ TransformTranslator getTransformTranslator(Class transfo return transformTranslators.get(transformClass); } + private static class ReadBoundedTranslator implements TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(Read.Bounded transform, TranslationContext context) { + // TODO: adapter is visibleForTesting + BoundedToUnboundedSourceAdapter unboundedSource = new BoundedToUnboundedSourceAdapter<>(transform.getSource()); + ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>( + unboundedSource, context.getPipelineOptions()); + context.addOperator(operator, operator.output); + } + + } } 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 87c8f97a4417..5fa3f23e7f69 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 @@ -20,10 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import org.apache.beam.runners.apex.translators.TranslationContext; -import org.apache.beam.runners.core.UnboundedReadFromBoundedSource; -import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Create; @@ -33,9 +30,8 @@ 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.runners.core.AssignWindows; 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; import org.apache.beam.sdk.values.POutput; @@ -70,6 +66,8 @@ public static ApexRunner fromOptions(PipelineOptions options) { @Override public OutputT apply( PTransform transform, InputT input) { +//System.out.println("transform: " + transform); + if (Window.Bound.class.equals(transform.getClass())) { return (OutputT) ((PCollection) input).apply( new AssignWindowsAndSetStrategy((Window.Bound) transform)); @@ -79,8 +77,6 @@ public OutputT apply( input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); - } else if (Read.Bounded.class.equals(transform.getClass())) { - return (OutputT) ((PBegin) input).apply(new UnboundedReadFromBoundedSource<>(((Read.Bounded)transform).getSource())); } else { return super.apply(transform, input); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java index f22814936a94..e1538675af0f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java @@ -18,11 +18,15 @@ package org.apache.beam.runners.apex.translators; +import java.util.List; + +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import com.datatorrent.lib.stream.StreamMerger; +import com.google.common.collect.Lists; /** * Flatten.FlattenPCollectionList translation to Apex operator. @@ -34,19 +38,46 @@ public class FlattenPCollectionTranslator implements @Override public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { - StreamMerger operator = null; - PCollectionList collections = context.getInput(); - if (collections.size() > 2) { - throw new UnsupportedOperationException("Currently supports only 2 collections: " + transform); - } - for (PCollection collection : collections.getAll()) { - if (null == operator) { - operator = new StreamMerger(); - context.addStream(collection, operator.data1); + PCollection firstCollection = null; + PCollectionList input = context.getInput(); + List> collections = input.getAll(); + List> remainingCollections = Lists.newArrayList(); + while (!collections.isEmpty()) { + for (PCollection collection : collections) { + if (null == firstCollection) { + firstCollection = collection; + } else { + StreamMerger operator = new StreamMerger<>(); + context.addStream(firstCollection, operator.data1); + context.addStream(collection, operator.data2); + if (collections.size() > 2) { + PCollection resultCollection = intermediateCollection(collection, collection.getCoder()); + context.addOperator(operator, operator.out, resultCollection); + remainingCollections.add(resultCollection); + } else { + // final stream merge + context.addOperator(operator, operator.out); + } + firstCollection = null; + } + } + if (firstCollection != null) { + // push to next merge level + remainingCollections.add(firstCollection); + } + if (remainingCollections.size() > 1) { + collections = remainingCollections; + remainingCollections = Lists.newArrayList(); } else { - context.addStream(collection, operator.data2); + collections = Lists.newArrayList(); } } - context.addOperator(operator, operator.out); } + + public static 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/translators/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java index 92afd583be9e..ab7cd0a53123 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java @@ -82,30 +82,22 @@ public OutputT getOutput() { } public void addOperator(Operator operator, OutputPort port) { - // Apex DAG requires a unique operator name - // use the transform's name and make it unique - String name = getCurrentTransform().getFullName(); - for (int i=1; this.operators.containsKey(name); name = getCurrentTransform().getFullName() + i++); - this.operators.put(name, operator); - PCollection output = getOutput(); - this.streams.put(output, (Pair)new ImmutablePair<>(port, new ArrayList<>())); + addOperator(operator, port, this.>getOutput()); } /** - * Add operator that is internal to a transformation. - * @param output + * Add intermediate operator for the current transformation. * @param operator * @param port - * @param name + * @param output */ - public PInput addInternalOperator(Operator operator, OutputPort port, String name, Coder coder) { - checkArgument(this.operators.get(name) == null, "duplicate operator " + name); + public void addOperator(Operator operator, OutputPort port, PCollection output) { + // Apex DAG requires a unique operator name + // use the transform's name and make it unique + String name = getCurrentTransform().getFullName(); + for (int i=1; this.operators.containsKey(name); name = getCurrentTransform().getFullName() + i++); this.operators.put(name, operator); - PCollection input = getInput(); - PCollection output = PCollection.createPrimitiveOutputInternal(input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); - output.setCoder(coder); this.streams.put(output, (Pair)new ImmutablePair<>(port, new ArrayList<>())); - return output; } public void addStream(PInput input, InputPort inputPort) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java index 4608c92249a4..29e1b32243f5 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java @@ -31,6 +31,7 @@ import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; @@ -42,7 +43,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingInternals; @@ -56,14 +56,14 @@ import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Instant; +import com.datatorrent.api.Context.OperatorContext; import com.datatorrent.api.DefaultInputPort; import com.datatorrent.api.DefaultOutputPort; import com.datatorrent.api.Operator; import com.datatorrent.api.StreamCodec; -import com.datatorrent.api.Context.OperatorContext; import com.datatorrent.api.annotation.OutputPortFieldAnnotation; -import com.esotericsoftware.kryo.serializers.JavaSerializer; import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.esotericsoftware.kryo.serializers.JavaSerializer; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java index 8005832dc786..d358d14770bb 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -22,14 +22,14 @@ import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.NoOpStepContext; import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; +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.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; -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.ExecutionContext; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java new file mode 100644 index 000000000000..0ee3442652fd --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java @@ -0,0 +1,207 @@ + /* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.examples; + + + import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.is; + import static org.junit.Assert.assertThat; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.TestApexRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.CountingInput; +import org.apache.beam.sdk.io.CountingInput.UnboundedCountingInput; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.NeedsRunner; + import org.apache.beam.sdk.testing.PAssert; + import org.apache.beam.sdk.testing.RunnableOnService; + import org.apache.beam.sdk.testing.TestPipeline; + import org.apache.beam.sdk.transforms.Count; + 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.RemoveDuplicates; + import org.apache.beam.sdk.transforms.SerializableFunction; + import org.apache.beam.sdk.transforms.display.DisplayData; +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.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; + + /** + * Tests for {@link CountingInput}. + */ + @RunWith(JUnit4.class) + public class IntTests { + public static void addCountingAsserts(PCollection input, long numElements) { + // Count == numElements + PAssert.thatSingleton(input.apply("Count", Count.globally())) + .isEqualTo(numElements); + // Unique count == numElements + PAssert.thatSingleton( + input + .apply(RemoveDuplicates.create()) + .apply("UniqueCount", Count.globally())) + .isEqualTo(numElements); + // Min == 0 + PAssert.thatSingleton(input.apply("Min", Min.globally())).isEqualTo(0L); + // Max == numElements-1 + PAssert.thatSingleton(input.apply("Max", Max.globally())) + .isEqualTo(numElements - 1); + } + + @Test + @Category(RunnableOnService.class) + public void testBoundedInput() { + //Pipeline p = TestPipeline.create(); + ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); + options.setRunner(TestApexRunner.class); + Pipeline p = Pipeline.create(options); + + long numElements = 1000; + PCollection input = p.apply(CountingInput.upTo(numElements)); + + addCountingAsserts(input, numElements); + p.run(); + } + + @Test + public void testBoundedDisplayData() { + PTransform input = CountingInput.upTo(1234); + DisplayData displayData = DisplayData.from(input); + assertThat(displayData, hasDisplayItem("upTo", 1234)); + } + + @Test + @Category(RunnableOnService.class) + public void testUnboundedInput() { + //Pipeline p = TestPipeline.create(); + ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); + options.setRunner(TestApexRunner.class); + Pipeline p = Pipeline.create(options); + + + long numElements = 1000; + + PCollection input = p.apply(CountingInput.unbounded().withMaxNumRecords(numElements)); + +// input = input.apply(Window.into(FixedWindows.of(Duration.standardSeconds(10)))); + + addCountingAsserts(input, numElements); + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testUnboundedInputRate() { + Pipeline p = TestPipeline.create(); + long numElements = 5000; + + long elemsPerPeriod = 10L; + Duration periodLength = Duration.millis(8); + PCollection input = + p.apply( + CountingInput.unbounded() + .withRate(elemsPerPeriod, periodLength) + .withMaxNumRecords(numElements)); + + addCountingAsserts(input, numElements); + long expectedRuntimeMillis = (periodLength.getMillis() * numElements) / elemsPerPeriod; + Instant startTime = Instant.now(); + p.run(); + Instant endTime = Instant.now(); + assertThat(endTime.isAfter(startTime.plus(expectedRuntimeMillis)), is(true)); + } + + private static class ElementValueDiff extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + c.output(c.element() - c.timestamp().getMillis()); + } + } + + @Test + @Category(RunnableOnService.class) + public void testUnboundedInputTimestamps() { + Pipeline p = TestPipeline.create(); + long numElements = 1000; + + PCollection input = + p.apply( + CountingInput.unbounded() + .withTimestampFn(new ValueAsTimestampFn()) + .withMaxNumRecords(numElements)); + addCountingAsserts(input, numElements); + + PCollection diffs = + input + .apply("TimestampDiff", ParDo.of(new ElementValueDiff())) + .apply("RemoveDuplicateTimestamps", RemoveDuplicates.create()); + // This assert also confirms that diffs only has one unique value. + PAssert.thatSingleton(diffs).isEqualTo(0L); + + p.run(); + } + + @Test + public void testUnboundedDisplayData() { + Duration maxReadTime = Duration.standardHours(5); + SerializableFunction timestampFn = new SerializableFunction() { + @Override + public Instant apply(Long input) { + return Instant.now(); + } + }; + + PTransform input = CountingInput.unbounded() + .withMaxNumRecords(1234) + .withMaxReadTime(maxReadTime) + .withTimestampFn(timestampFn); + + DisplayData displayData = DisplayData.from(input); + + assertThat(displayData, hasDisplayItem("maxRecords", 1234)); + assertThat(displayData, hasDisplayItem("maxReadTime", maxReadTime)); + assertThat(displayData, hasDisplayItem("timestampFn", timestampFn.getClass())); + } + + /** + * A timestamp function that uses the given value as the timestamp. Because the input values will + * not wrap, this function is non-decreasing and meets the timestamp function criteria laid out + * in {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)}. + */ + private static class ValueAsTimestampFn implements SerializableFunction { + @Override + public Instant apply(Long input) { + return new Instant(input); + } + } + + + + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java index 6260632544a1..f954537db408 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java @@ -99,7 +99,7 @@ public void testReadBounded() throws Exception { ApexRunnerResult result = (ApexRunnerResult)p.run(); DAG dag = result.getApexDAG(); - DAG.OperatorMeta om = dag.getOperatorMeta("Read(BoundedCountingSource)/Read(BoundedCountingSource)/Read(BoundedToUnboundedSourceAdapter)"); + DAG.OperatorMeta om = dag.getOperatorMeta("Read(BoundedCountingSource)"); Assert.assertNotNull(om); Assert.assertEquals(om.getOperator().getClass(), ApexReadUnboundedInputOperator.class); From 09754942c66c9befffc8df9b3c8a75b819a672e6 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Sun, 25 Sep 2016 16:46:44 -0700 Subject: [PATCH 4/8] BEAM-261 PCollectionView and side inputs. --- .../runners/apex/ApexPipelineOptions.java | 6 + .../runners/apex/ApexPipelineTranslator.java | 19 +- .../apache/beam/runners/apex/ApexRunner.java | 397 +++++++++++++++++- .../FlattenPCollectionTranslator.java | 26 +- .../translators/ParDoBoundTranslator.java | 22 +- .../apex/translators/TranslationContext.java | 14 +- .../functions/ApexFlattenOperator.java | 113 +++++ .../functions/ApexGroupByKeyOperator.java | 78 +++- .../functions/ApexParDoOperator.java | 210 +++++++-- .../io/ApexReadUnboundedInputOperator.java | 31 +- .../translators/utils/ApexStreamTuple.java | 11 + .../utils/NoOpSideInputReader.java | 47 --- .../beam/runners/apex/examples/IntTest.java | 133 ++++++ .../beam/runners/apex/examples/IntTests.java | 207 --------- .../translators/ParDoBoundTranslatorTest.java | 37 +- .../apex/src/test/resources/log4j.properties | 4 +- 16 files changed, 1028 insertions(+), 327 deletions(-) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java delete mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java delete mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java index f70d24cc8cea..141a8c10a5a9 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineOptions.java @@ -50,6 +50,12 @@ public interface ApexPipelineOptions extends PipelineOptions, java.io.Serializab @Default.Boolean(true) boolean isEmbeddedExecutionDebugMode(); + @Description("output data received and emitted on ports (for debugging)") + void setTupleTracingEnabled(boolean enabled); + + @Default.Boolean(false) + boolean isTupleTracingEnabled(); + @Description("how long the client should wait for the pipeline to run") void setRunMillis(long runMillis); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java index b0391b426e97..ad8c28321581 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.apex; +import org.apache.beam.runners.apex.ApexRunner.CreateApexPCollectionView; import org.apache.beam.runners.apex.translators.CreateValuesTranslator; import org.apache.beam.runners.apex.translators.FlattenPCollectionTranslator; import org.apache.beam.runners.apex.translators.GroupByKeyTranslator; @@ -35,8 +36,8 @@ 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.PCollectionView; import org.apache.beam.sdk.values.PValue; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,6 +72,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { registerTransformTranslator(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); + registerTransformTranslator(CreateApexPCollectionView.class, new CreatePCollectionViewTranslator()); } public ApexPipelineTranslator(TranslationContext translationContext) { @@ -98,7 +100,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { PTransform transform = node.getTransform(); TransformTranslator translator = getTransformTranslator(transform.getClass()); if (null == translator) { - throw new IllegalStateException( + throw new UnsupportedOperationException( "no translator registered for " + transform); } translationContext.setCurrentTransform(node); @@ -147,4 +149,17 @@ public void translate(Read.Bounded transform, TranslationContext context) { } + private static class CreatePCollectionViewTranslator implements TransformTranslator> + { + private static final long serialVersionUID = 1L; + + @Override + public void translate(CreateApexPCollectionView transform, TranslationContext context) + { + PCollectionView view = transform.getView(); + context.addView(view); + LOG.debug("view {}", view.getName()); + } + } + } 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 5fa3f23e7f69..ae79a2049d11 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 @@ -19,20 +19,36 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + import org.apache.beam.runners.apex.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.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.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.OldDoFn.ProcessContext; 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.runners.core.AssignWindows; +import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.hadoop.conf.Configuration; @@ -55,6 +71,13 @@ public class ApexRunner extends PipelineRunner { private final ApexPipelineOptions options; + /** + * TODO: this isn't thread sa + * Holds any most resent assertion error that was raised while processing elements. + * Used in the unit test driver in embedded to propagate the exception. + */ + public static volatile AssertionError assertionError; + public ApexRunner(ApexPipelineOptions options) { this.options = options; } @@ -77,6 +100,32 @@ public OutputT apply( input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); +// TODO: replace this with a mapping + } else if (Combine.GloballyAsSingletonView.class.equals(transform.getClass())) { + PTransform customTransform = (PTransform)new StreamingCombineGloballyAsSingletonView(this, + (Combine.GloballyAsSingletonView)transform); + return Pipeline.applyTransform(input, customTransform); + } else if (View.AsSingleton.class.equals(transform.getClass())) { + // note this assumes presence of above Combine.GloballyAsSingletonView mapping + PTransform customTransform = (PTransform)new StreamingViewAsSingleton(this, + (View.AsSingleton)transform); + return Pipeline.applyTransform(input, customTransform); + } else if (View.AsIterable.class.equals(transform.getClass())) { + PTransform customTransform = (PTransform)new StreamingViewAsIterable(this, + (View.AsIterable)transform); + return Pipeline.applyTransform(input, customTransform); + } else if (View.AsList.class.equals(transform.getClass())) { + PTransform customTransform = (PTransform)new StreamingViewAsList(this, + (View.AsList)transform); + return Pipeline.applyTransform(input, customTransform); + } else if (View.AsMap.class.equals(transform.getClass())) { + PTransform customTransform = new StreamingViewAsMap(this, + (View.AsMap)transform); + return Pipeline.applyTransform(input, customTransform); + } else if (View.AsMultimap.class.equals(transform.getClass())) { + PTransform customTransform = new StreamingViewAsMultimap(this, + (View.AsMultimap)transform); + return Pipeline.applyTransform(input, customTransform); } else { return super.apply(transform, input); } @@ -109,10 +158,19 @@ public void populateDAG(DAG dag, Configuration conf) // turns off timeout checking for operator progress lc.setHeartbeatMonitoringEnabled(false); } + assertionError = null; + lc.runAsync(); if (options.getRunMillis() > 0) { - lc.run(options.getRunMillis()); - } else { - lc.runAsync(); + try { + long timeout = System.currentTimeMillis() + options.getRunMillis(); + while (System.currentTimeMillis() < timeout) { + if (assertionError != null) { + throw assertionError; + } + } + } finally { + lc.shutdown(); + } } return new ApexRunnerResult(lma.getDAG(), lc); } catch (Exception e) { @@ -158,10 +216,343 @@ public PCollection apply(PCollection input) { } private static class IdentityFn extends DoFn { + private static final long serialVersionUID = 1L; @ProcessElement public void processElement(ProcessContext c) { c.output(c.element()); } } +//////////////////////////////////////////// +// Adapted from FlinkRunner for View support + + /** + * Records that the {@link PTransform} requires a deterministic key coder. + */ + private void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { + throw new UnsupportedOperationException(); + } + + /** + * 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 CreateApexPCollectionView + extends PTransform>, PCollectionView> { + private PCollectionView view; + + private CreateApexPCollectionView(PCollectionView view) { + this.view = view; + } + + public static CreateApexPCollectionView of( + PCollectionView view) { + return new CreateApexPCollectionView<>(view); + } + + public PCollectionView getView() { + return view; + } + + @Override + public PCollectionView apply(PCollection> input) { + return view; + } + } + + private static class WrapAsList extends OldDoFn> { + @Override + public void processElement(ProcessContext c) { + c.output(Arrays.asList(c.element())); + } + } + + private static class StreamingCombineGloballyAsSingletonView + extends PTransform, PCollectionView> + { + Combine.GloballyAsSingletonView transform; + + /** + * Builds an instance of this class from the overridden transform. + */ + public StreamingCombineGloballyAsSingletonView(ApexRunner runner, + Combine.GloballyAsSingletonView transform) + { + this.transform = transform; + } + + @Override + public PCollectionView apply(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(CreateApexPCollectionView. of(view)); + } + + @Override + protected String getKindString() + { + return "StreamingCombineGloballyAsSingletonView"; + } + } + + private static class StreamingViewAsSingleton extends PTransform, PCollectionView> + { + private static final long serialVersionUID = 1L; + private View.AsSingleton transform; + + public StreamingViewAsSingleton(ApexRunner runner, View.AsSingleton transform) + { + this.transform = transform; + } + + @Override + public PCollectionView apply(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 StreamingViewAsMap + extends PTransform>, PCollectionView>> { + + private final ApexRunner runner; + + @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() + public StreamingViewAsMap(ApexRunner runner, View.AsMap transform) { + this.runner = runner; + } + + @Override + public PCollectionView> apply(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) { + runner.recordViewUsesNonDeterministicKeyCoder(this); + } + + return input + .apply(Combine.globally(new Concatenate>()).withoutDefaults()) + .apply(CreateApexPCollectionView., Map>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsMap"; + } + } + + /** + * Specialized expansion for {@link + * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the + * Flink runner in streaming mode. + */ + private static class StreamingViewAsMultimap + extends PTransform>, PCollectionView>>> { + + private final ApexRunner runner; + + /** + * Builds an instance of this class from the overridden transform. + */ + @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() + public StreamingViewAsMultimap(ApexRunner runner, View.AsMultimap transform) { + this.runner = runner; + } + + @Override + public PCollectionView>> apply(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) { + runner.recordViewUsesNonDeterministicKeyCoder(this); + } + + return input + .apply(Combine.globally(new Concatenate>()).withoutDefaults()) + .apply(CreateApexPCollectionView., Map>>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsMultimap"; + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the + * Flink runner in streaming mode. + */ + private static class StreamingViewAsList + extends PTransform, PCollectionView>> { + /** + * Builds an instance of this class from the overridden transform. + */ + @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() + public StreamingViewAsList(ApexRunner runner, View.AsList transform) {} + + @Override + public PCollectionView> apply(PCollection input) { + PCollectionView> view = + PCollectionViews.listView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateApexPCollectionView.>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsList"; + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the + * Flink runner in streaming mode. + */ + private static class StreamingViewAsIterable + extends PTransform, PCollectionView>> { + /** + * Builds an instance of this class from the overridden transform. + */ + @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() + public StreamingViewAsIterable(ApexRunner runner, View.AsIterable transform) { } + + @Override + public PCollectionView> apply(PCollection input) { + PCollectionView> view = + PCollectionViews.iterableView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateApexPCollectionView.>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsIterable"; + } + } + + /** + * 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/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java index e1538675af0f..712466a10c40 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java @@ -18,19 +18,23 @@ package org.apache.beam.runners.apex.translators; +import java.util.Collections; import java.util.List; +import org.apache.beam.runners.apex.translators.functions.ApexFlattenOperator; +import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.runners.apex.translators.io.ValuesSource; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import com.datatorrent.lib.stream.StreamMerger; import com.google.common.collect.Lists; /** - * Flatten.FlattenPCollectionList translation to Apex operator. - * TODO: support more than two streams + * {@link Flatten.FlattenPCollectionList} translation to Apex operator. */ public class FlattenPCollectionTranslator implements TransformTranslator> { @@ -38,16 +42,28 @@ public class FlattenPCollectionTranslator implements @Override public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { - PCollection firstCollection = null; PCollectionList input = context.getInput(); List> collections = input.getAll(); + + if (collections.isEmpty()) { + // create a dummy source that never emits anything + @SuppressWarnings("unchecked") + UnboundedSource unboundedSource = new ValuesSource<>(Collections.EMPTY_LIST, + (Coder) VoidCoder.of()); + ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>( + unboundedSource, context.getPipelineOptions()); + context.addOperator(operator, operator.output); + return; + } + List> remainingCollections = Lists.newArrayList(); + PCollection firstCollection = null; while (!collections.isEmpty()) { for (PCollection collection : collections) { if (null == firstCollection) { firstCollection = collection; } else { - StreamMerger operator = new StreamMerger<>(); + ApexFlattenOperator operator = new ApexFlattenOperator<>(); context.addStream(firstCollection, operator.data1); context.addStream(collection, operator.data2); if (collections.size() > 2) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java index a9582349df7a..632829ace7fd 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java @@ -18,11 +18,15 @@ package org.apache.beam.runners.apex.translators; +import java.util.List; + import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; -import org.apache.beam.runners.apex.translators.utils.NoOpSideInputReader; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +import com.datatorrent.api.Operator; /** * {@link ParDo.Bound} is translated to Apex operator that wraps the {@link DoFn} @@ -35,9 +39,23 @@ public class ParDoBoundTranslator implements public void translate(ParDo.Bound transform, TranslationContext context) { OldDoFn doFn = transform.getFn(); PCollection output = context.getOutput(); + List> sideInputs = transform.getSideInputs(); ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), - doFn, output.getWindowingStrategy(), new NoOpSideInputReader()); + doFn, output.getWindowingStrategy(), sideInputs); context.addOperator(operator, operator.output); context.addStream(context.getInput(), operator.input); + if (!sideInputs.isEmpty()) { + Operator.InputPort[] sideInputPorts = {operator.sideInput1}; + for (int i=0; i currentTransform; private final Map, List>>> streams = new HashMap<>(); private final Map operators = new HashMap<>(); + private final Map, PInput> viewInputs = new HashMap<>(); + + public void addView(PCollectionView view) { + this.viewInputs.put(view, this.getInput()); + } + + public InputT getViewInput(PCollectionView view) { + PInput input = this.viewInputs.get(view); + checkArgument(input != null, "unknown view " + view.getName()); + return (InputT)input; + } public TranslationContext(ApexPipelineOptions pipelineOptions) { this.pipelineOptions = pipelineOptions; @@ -102,7 +114,7 @@ public void addOperator(Operator operator, OutputPort port, PCollection output) public void addStream(PInput input, InputPort inputPort) { Pair, List>> stream = this.streams.get(input); - checkArgument(stream != null, "no upstream operator defined"); + checkArgument(stream != null, "no upstream operator defined for %s", input); stream.getRight().add(inputPort); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java new file mode 100644 index 000000000000..ce27abb5096c --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.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.runners.apex.translators.functions; + +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple.WatermarkTuple; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.util.WindowedValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.common.util.BaseOperator; + +/** + * Apex operator for Beam {@link Flatten.FlattenPCollectionList}. + */ +public class ApexFlattenOperator extends BaseOperator +{ + private static final Logger LOG = LoggerFactory.getLogger(ApexFlattenOperator.class); + private boolean traceTuples = true; + + private long inputWM1; + private long inputWM2; + private long outputWM; + + /** + * Data input port 1. + */ + public final transient DefaultInputPort>> data1 = new DefaultInputPort>>() + { + /** + * Emits to port "out" + */ + @Override + public void process(ApexStreamTuple> tuple) + { + if (tuple instanceof WatermarkTuple) { + WatermarkTuple wmTuple = (WatermarkTuple)tuple; + if (wmTuple.getTimestamp() > inputWM1) { + inputWM1 = wmTuple.getTimestamp(); + if (inputWM1 <= inputWM2) { + // move output watermark and emit it + outputWM = inputWM1; + if (traceTuples) { + LOG.debug("\nemitting watermark {}\n", outputWM); + } + out.emit(tuple); + } + } + return; + } + if (traceTuples) { + LOG.debug("\nemitting {}\n", tuple); + } + out.emit(tuple); + } + }; + + /** + * Data input port 2. + */ + public final transient DefaultInputPort>> data2 = new DefaultInputPort>>() + { + /** + * Emits to port "out" + */ + @Override + public void process(ApexStreamTuple> tuple) + { + if (tuple instanceof WatermarkTuple) { + WatermarkTuple wmTuple = (WatermarkTuple)tuple; + if (wmTuple.getTimestamp() > inputWM2) { + inputWM2 = wmTuple.getTimestamp(); + if (inputWM2 <= inputWM1) { + // move output watermark and emit it + outputWM = inputWM2; + if (traceTuples) { + LOG.debug("\nemitting watermark {}\n", outputWM); + } + out.emit(tuple); + } + } + return; + } + if (traceTuples) { + LOG.debug("\nemitting {}\n", tuple); + } + out.emit(tuple); + } + }; + + /** + * Output port. + */ + public final transient DefaultOutputPort>> out = new DefaultOutputPort>>(); +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java index 29e1b32243f5..5970f3627c65 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.io.Serializable; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -33,6 +34,7 @@ import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.core.SystemReduceFn; 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.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; @@ -41,6 +43,7 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.TimerInternals; @@ -55,6 +58,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.datatorrent.api.Context.OperatorContext; import com.datatorrent.api.DefaultInputPort; @@ -79,16 +84,22 @@ */ public class ApexGroupByKeyOperator implements Operator { + private static final Logger LOG = LoggerFactory.getLogger(ApexGroupByKeyOperator.class); + private boolean traceTuples = true; + @Bind(JavaSerializer.class) private WindowingStrategy windowingStrategy; @Bind(JavaSerializer.class) + private Coder keyCoder; + @Bind(JavaSerializer.class) private Coder valueCoder; @Bind(JavaSerializer.class) private final SerializablePipelineOptions serializedOptions; @Bind(JavaSerializer.class) - private Map> perKeyStateInternals = new HashMap<>(); - private Map> activeTimers = new HashMap<>(); +// TODO: InMemoryStateInternals not serializable +transient private Map> perKeyStateInternals = new HashMap<>(); + private Map> activeTimers = new HashMap<>(); private transient ProcessContext context; private transient OldDoFn, KV>> fn; @@ -100,14 +111,19 @@ public class ApexGroupByKeyOperator implements Operator @Override public void process(ApexStreamTuple>> t) { - //System.out.println("\n***RECEIVED: " +t); try { if (t instanceof ApexStreamTuple.WatermarkTuple) { ApexStreamTuple.WatermarkTuple mark = (ApexStreamTuple.WatermarkTuple)t; processWatermark(mark); + if (traceTuples) { + LOG.debug("\nemitting watermark {}\n", mark.getTimestamp()); + } output.emit(ApexStreamTuple.WatermarkTuple.>>>of(mark.getTimestamp())); return; } + if (traceTuples) { + LOG.debug("\ninput {}\n", t.getValue()); + } processElement(t.getValue()); } catch (Exception e) { Throwables.propagate(e); @@ -124,6 +140,7 @@ public ApexGroupByKeyOperator(ApexPipelineOptions pipelineOptions, PCollection)input.getWindowingStrategy(); + this.keyCoder = ((KvCoder)input.getCoder()).getKeyCoder(); this.valueCoder = ((KvCoder)input.getCoder()).getValueCoder(); } @@ -146,6 +163,7 @@ public void endWindow() @Override public void setup(OperatorContext context) { + this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(serializedOptions.get(), this); StateInternalsFactory stateInternalsFactory = new GroupByKeyStateInternalsFactory(); this.fn = GroupAlsoByWindowViaWindowSetDoFn.create(this.windowingStrategy, stateInternalsFactory, SystemReduceFn.buffering(this.valueCoder)); @@ -163,16 +181,16 @@ public void teardown() * We keep these timers in a Set, so that they are deduplicated, as the same * timer can be registered multiple times. */ - private Multimap getTimersReadyToProcess(long currentWatermark) { + private Multimap getTimersReadyToProcess(long currentWatermark) { // we keep the timers to return in a different list and launch them later // because we cannot prevent a trigger from registering another trigger, // which would lead to concurrent modification exception. - Multimap toFire = HashMultimap.create(); + Multimap toFire = HashMultimap.create(); - Iterator>> it = activeTimers.entrySet().iterator(); + Iterator>> it = activeTimers.entrySet().iterator(); while (it.hasNext()) { - Map.Entry> keyWithTimers = it.next(); + Map.Entry> keyWithTimers = it.next(); Iterator timerIt = keyWithTimers.getValue().iterator(); while (timerIt.hasNext()) { @@ -205,44 +223,64 @@ private void processElement(WindowedValue> windowedValue) throws Except fn.processElement(context); } - private StateInternals getStateInternalsForKey(K key) { - StateInternals stateInternals = perKeyStateInternals.get(key); + private StateInternals getStateInternalsForKey(K key) + { + final ByteBuffer keyBytes; + try { + keyBytes = ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, key)); + } catch (CoderException e) { + throw Throwables.propagate(e); + } + StateInternals stateInternals = perKeyStateInternals.get(keyBytes); if (stateInternals == null) { //Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); //OutputTimeFn outputTimeFn = this.windowingStrategy.getOutputTimeFn(); stateInternals = InMemoryStateInternals.forKey(key); - perKeyStateInternals.put(key, stateInternals); + perKeyStateInternals.put(keyBytes, stateInternals); } return stateInternals; } private void registerActiveTimer(K key, TimerInternals.TimerData timer) { - Set timersForKey = activeTimers.get(key); + final ByteBuffer keyBytes; + try { + keyBytes = ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, key)); + } catch (CoderException e) { + throw Throwables.propagate(e); + } + Set timersForKey = activeTimers.get(keyBytes); if (timersForKey == null) { timersForKey = new HashSet<>(); } timersForKey.add(timer); - activeTimers.put(key, timersForKey); + activeTimers.put(keyBytes, timersForKey); } private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { - Set timersForKey = activeTimers.get(key); + final ByteBuffer keyBytes; + try { + keyBytes = ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, key)); + } catch (CoderException e) { + throw Throwables.propagate(e); + } + Set timersForKey = activeTimers.get(keyBytes); if (timersForKey != null) { timersForKey.remove(timer); if (timersForKey.isEmpty()) { - activeTimers.remove(key); + activeTimers.remove(keyBytes); } else { - activeTimers.put(key, timersForKey); + activeTimers.put(keyBytes, timersForKey); } } } private void processWatermark(ApexStreamTuple.WatermarkTuple mark) throws Exception { this.inputWatermark = new Instant(mark.getTimestamp()); - Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); + Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); if (!timers.isEmpty()) { - for (K key : timers.keySet()) { - KeyedWorkItem kwi = KeyedWorkItems.timersWorkItem(key, timers.get(key)); + for (ByteBuffer keyBytes : timers.keySet()) { + K key = CoderUtils.decodeFromByteArray(keyCoder, keyBytes.array()); + KeyedWorkItem kwi = KeyedWorkItems.timersWorkItem(key, timers.get(keyBytes)); context.setElement(kwi, getStateInternalsForKey(kwi.key())); fn.processElement(context); } @@ -315,7 +353,9 @@ public StateInternals stateInternals() { @Override public void outputWindowedValue(KV> output, Instant timestamp, Collection windows, PaneInfo pane) { - System.out.println("\n***EMITTING: " + output + ", timestamp=" + timestamp); + if (traceTuples) { + LOG.debug("\nemitting {} timestamp {}\n", output, timestamp); + } ApexGroupByKeyOperator.this.output.emit(ApexStreamTuple.DataTuple.of(WindowedValue.of(output, timestamp, windows, pane))); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java index d358d14770bb..13a8fc9f50da 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -18,40 +18,58 @@ package org.apache.beam.runners.apex.translators.functions; +import java.util.ArrayList; +import java.util.List; + import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.NoOpStepContext; import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; 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.core.DoFnRunners.OutputManager; +import org.apache.beam.sdk.repackaged.com.google.common.base.Throwables; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.ExecutionContext; +import org.apache.beam.sdk.util.NullSideInputReader; import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.UserCodeException; 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.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.datatorrent.api.Context.OperatorContext; import com.datatorrent.api.DefaultInputPort; import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.annotation.InputPortFieldAnnotation; import com.datatorrent.api.annotation.OutputPortFieldAnnotation; import com.datatorrent.common.util.BaseOperator; import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.google.common.collect.Iterables; import com.esotericsoftware.kryo.serializers.JavaSerializer; /** * Apex operator for Beam {@link DoFn}. */ public class ApexParDoOperator extends BaseOperator implements OutputManager { + private static final Logger LOG = LoggerFactory.getLogger(ApexParDoOperator.class); + private boolean traceTuples = true; private transient final TupleTag mainTag = new TupleTag(); - private transient DoFnRunner doFnRunner; + private transient PushbackSideInputDoFnRunner pushbackDoFnRunner; @Bind(JavaSerializer.class) private final SerializablePipelineOptions pipelineOptions; @@ -60,17 +78,28 @@ public class ApexParDoOperator extends BaseOperator implements @Bind(JavaSerializer.class) private final WindowingStrategy windowingStrategy; @Bind(JavaSerializer.class) - private final SideInputReader sideInputReader; + List> sideInputs; +// TODO: not Kryo serializable, integrate codec +//@Bind(JavaSerializer.class) +private transient StateInternals sideInputStateInternals = InMemoryStateInternals.forKey(null); + private transient SideInputHandler sideInputHandler; + // TODO: not Kryo serializable, integrate codec + private List> pushedBack = new ArrayList<>(); + private LongMin pushedBackWatermark = new LongMin(); + private long currentInputWatermark = Long.MIN_VALUE; + private long currentOutputWatermark = currentInputWatermark; public ApexParDoOperator( ApexPipelineOptions pipelineOptions, OldDoFn doFn, WindowingStrategy windowingStrategy, - SideInputReader sideInputReader) { + List> sideInputs + ) + { this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions); this.doFn = doFn; this.windowingStrategy = windowingStrategy; - this.sideInputReader = sideInputReader; + this.sideInputs = sideInputs; } @SuppressWarnings("unused") // for Kryo @@ -78,17 +107,60 @@ private ApexParDoOperator() { this(null, null, null, null); } + public final transient DefaultInputPort>> input = new DefaultInputPort>>() { @Override public void process(ApexStreamTuple> t) { if (t instanceof ApexStreamTuple.WatermarkTuple) { - output.emit(t); + processWatermark((ApexStreamTuple.WatermarkTuple)t); } else { - System.out.println("\n" + Thread.currentThread().getName() + "\n" + t.getValue() + "\n"); - doFnRunner.processElement(t.getValue()); + if (traceTuples) { + LOG.debug("\ninput {}\n", t.getValue()); + } + Iterable> justPushedBack = processElementInReadyWindows(t.getValue()); + for (WindowedValue pushedBackValue : justPushedBack) { + pushedBackWatermark.add(pushedBackValue.getTimestamp().getMillis()); + pushedBack.add(pushedBackValue); + } + } + } + }; + + @InputPortFieldAnnotation(optional=true) + public final transient DefaultInputPort>>> sideInput1 = new DefaultInputPort>>>() + { + private final int sideInputIndex = 0; + + @Override + public void process(ApexStreamTuple>> t) + { + if (t instanceof ApexStreamTuple.WatermarkTuple) { + // ignore side input watermarks + return; + } + if (traceTuples) { + LOG.debug("\nsideInput {}\n", t.getValue()); } + PCollectionView sideInput = sideInputs.get(sideInputIndex); + sideInputHandler.addSideInputValue(sideInput, t.getValue()); + + List> newPushedBack = new ArrayList<>(); + for (WindowedValue elem : pushedBack) { + Iterable> justPushedBack = processElementInReadyWindows(elem); + Iterables.addAll(newPushedBack, justPushedBack); + } + + pushedBack.clear(); + pushedBackWatermark.clear(); + for (WindowedValue pushedBackValue : newPushedBack) { + pushedBackWatermark.add(pushedBackValue.getTimestamp().getMillis()); + pushedBack.add(pushedBackValue); + } + + // potentially emit watermark + processWatermark(ApexStreamTuple.WatermarkTuple.of(currentInputWatermark)); } }; @@ -99,27 +171,82 @@ public void process(ApexStreamTuple> t) public void output(TupleTag tag, WindowedValue tuple) { output.emit(ApexStreamTuple.DataTuple.of(tuple)); + if (traceTuples) { + LOG.debug("\nemitting {}\n", tuple); + } + } + + private Iterable> processElementInReadyWindows(WindowedValue elem) { + try { + return pushbackDoFnRunner.processElementInReadyWindows(elem); + } catch (UserCodeException ue) { + if (ue.getCause() instanceof AssertionError) { + ApexRunner.assertionError = (AssertionError)ue.getCause(); + } + throw ue; + } + } + + private void processWatermark(ApexStreamTuple.WatermarkTuple mark) + { + this.currentInputWatermark = mark.getTimestamp(); + + if (sideInputs.isEmpty()) { + if (traceTuples) { + LOG.debug("\nemitting watermark {}\n", mark); + } + output.emit(mark); + return; + } + + long potentialOutputWatermark = + Math.min(pushedBackWatermark.get(), currentInputWatermark); + if (potentialOutputWatermark > currentOutputWatermark) { + currentOutputWatermark = potentialOutputWatermark; + if (traceTuples) { + LOG.debug("\nemitting watermark {}\n", currentOutputWatermark); + } + output.emit(ApexStreamTuple.WatermarkTuple.of(currentOutputWatermark)); + } } @Override public void setup(OperatorContext context) { - this.doFnRunner = DoFnRunners.simpleRunner(pipelineOptions.get(), + this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(pipelineOptions.get(), this); + SideInputReader sideInputReader = NullSideInputReader.of(sideInputs); + if (!sideInputs.isEmpty()) { + sideInputHandler = new SideInputHandler(sideInputs, sideInputStateInternals); + sideInputReader = sideInputHandler; + } + + DoFnRunner doFnRunner = DoFnRunners.createDefault( + pipelineOptions.get(), doFn, sideInputReader, this, mainTag, - TupleTagList.empty().getAll(), + TupleTagList.empty().getAll() /*sideOutputTags*/, new NoOpStepContext(), new NoOpAggregatorFactory(), windowingStrategy ); + + pushbackDoFnRunner = + PushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler); + + try { + doFn.setup(); + } catch (Exception e) { + Throwables.propagate(e); + } + } @Override public void beginWindow(long windowId) { - doFnRunner.startBundle(); + pushbackDoFnRunner.startBundle(); /* Collection> aggregators = AggregatorRetriever.getAggregators(doFn); if (!aggregators.isEmpty()) { @@ -131,14 +258,14 @@ public void beginWindow(long windowId) @Override public void endWindow() { - doFnRunner.finishBundle(); + pushbackDoFnRunner.finishBundle(); } /** * TODO: Placeholder for aggregation, to be implemented for embedded and cluster mode. * It is called from {@link org.apache.beam.sdk.util.SimpleDoFnRunner}. */ - public class NoOpAggregatorFactory implements AggregatorFactory { + public static class NoOpAggregatorFactory implements AggregatorFactory { private NoOpAggregatorFactory() { } @@ -147,31 +274,52 @@ private NoOpAggregatorFactory() { public Aggregator createAggregatorForDoFn( Class fnClass, ExecutionContext.StepContext step, String name, CombineFn combine) { - return new Aggregator() { + return new NoOpAggregator(); + } - @Override - public void addValue(InputT value) - { - } + private static class NoOpAggregator implements Aggregator, java.io.Serializable + { + private static final long serialVersionUID = 1L; - @Override - public String getName() - { - // TODO Auto-generated method stub - return null; - } + @Override + public void addValue(InputT value) + { + } + + @Override + public String getName() + { + // TODO Auto-generated method stub + return null; + } + + @Override + public CombineFn getCombineFn() + { + // TODO Auto-generated method stub + return null; + } + + }; - @Override - public CombineFn getCombineFn() - { - // TODO Auto-generated method stub - return null; - } - }; - } } + private static class LongMin { + long state = Long.MAX_VALUE; + public void add(long l) { + state = Math.min(state, l); + } + + public long get() { + return state; + } + + public void clear() { + state = Long.MAX_VALUE; + } + + } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java index 39114feb849b..6ee82ea7554b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java @@ -28,8 +28,11 @@ import org.apache.beam.sdk.util.WindowedValue; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; import com.google.common.base.Throwables; import com.datatorrent.api.DefaultOutputPort; import com.datatorrent.api.InputOperator; @@ -40,10 +43,14 @@ import java.io.IOException; /** - * Apex input operator that wraps Beam UnboundedSource. + * Apex input operator that wraps Beam {@link UnboundedSource}. */ public class ApexReadUnboundedInputOperator implements InputOperator { + private static final Logger LOG = LoggerFactory.getLogger( + ApexReadUnboundedInputOperator.class); + private boolean traceTuples = false; + private long outputWatermark = 0; @Bind(JavaSerializer.class) private final SerializablePipelineOptions pipelineOptions; @@ -51,6 +58,7 @@ public class ApexReadUnboundedInputOperator source; private transient UnboundedSource.UnboundedReader reader; private transient boolean available = false; + @OutputPortFieldAnnotation(optional=true) public final transient DefaultOutputPort>> output = new DefaultOutputPort<>(); public ApexReadUnboundedInputOperator(UnboundedSource source, ApexPipelineOptions options) { @@ -66,12 +74,23 @@ private ApexReadUnboundedInputOperator() { @Override public void beginWindow(long windowId) { - Instant mark = reader.getWatermark(); - output.emit(ApexStreamTuple.WatermarkTuple.>of(mark.getMillis())); if (!available && source instanceof ValuesSource) { - // if it's a Create transformation and the input was consumed, + // if it's a Create and the input was consumed, emit final watermark + emitWatermarkIfNecessary(GlobalWindow.TIMESTAMP_MAX_VALUE.getMillis()); // terminate the stream (allows tests to finish faster) BaseOperator.shutdown(); + } else { + emitWatermarkIfNecessary(reader.getWatermark().getMillis()); + } + } + + private void emitWatermarkIfNecessary(long mark) { + if (mark > outputWatermark) { + outputWatermark = mark; + if (traceTuples) { + LOG.debug("\nemitting watermark {}\n", mark); + } + output.emit(ApexStreamTuple.WatermarkTuple.>of(mark)); } } @@ -83,6 +102,7 @@ public void endWindow() @Override public void setup(OperatorContext context) { + this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(pipelineOptions.get(), this); try { reader = source.createReader(this.pipelineOptions.get(), null); available = reader.start(); @@ -114,6 +134,9 @@ public void emitTuples() OutputT data = reader.getCurrent(); Instant timestamp = reader.getCurrentTimestamp(); available = reader.advance(); + if (traceTuples) { + LOG.debug("\nemitting {}\n", data); + } output.emit(DataTuple.of(WindowedValue.of( data, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java index efb69eef8ed3..06940aa28b94 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java @@ -27,10 +27,13 @@ import java.util.Arrays; import java.util.List; +import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StandardCoder; +import com.datatorrent.api.Operator; + public interface ApexStreamTuple { /** @@ -188,4 +191,12 @@ public Coder getValueCoder() { } + final class Logging + { + public static boolean isDebugEnabled(ApexPipelineOptions options, Operator operator) + { + return options.isTupleTracingEnabled(); + } + } + } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.java deleted file mode 100644 index ffe1a29d9ddf..000000000000 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpSideInputReader.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.apex.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/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java new file mode 100644 index 000000000000..3573d31dc741 --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.examples; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.TestApexRunner; +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.runners.dataflow.TestCountingSource; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * For debugging only. + */ +@Ignore +@RunWith(JUnit4.class) +public class IntTest implements java.io.Serializable +{ + + @Test + public void test() + { + ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); + options.setTupleTracingEnabled(true); + options.setRunner(TestApexRunner.class); + Pipeline p = Pipeline.create(options); +boolean timeBound = false; + + + TestCountingSource source = new TestCountingSource(Integer.MAX_VALUE).withoutSplitting(); +//List> values = Lists.newArrayList( +// KV.of(0, 99),KV.of(0, 99),KV.of(0, 98)); + +//UnboundedSource, ?> source = new ValuesSource<>(values, +// KvCoder.of(VarIntCoder.of(), VarIntCoder.of())); + + if (true) { + source = source.withDedup(); + } + + PCollection> output = + timeBound + ? p.apply(Read.from(source).withMaxReadTime(Duration.millis(200))) + : p.apply(Read.from(source).withMaxNumRecords(NUM_RECORDS)); + + List> expectedOutput = new ArrayList<>(); + for (int i = 0; i < NUM_RECORDS; i++) { + expectedOutput.add(KV.of(0, i)); + } + + // Because some of the NUM_RECORDS elements read are dupes, the final output + // will only have output from 0 to n where n < NUM_RECORDS. + PAssert.that(output).satisfies(new Checker(true, timeBound)); + + + p.run(); + return; + } + + private static final int NUM_RECORDS = 10; + private static class Checker implements SerializableFunction>, Void> + { + private final boolean dedup; + private final boolean timeBound; + + Checker(boolean dedup, boolean timeBound) + { + this.dedup = dedup; + this.timeBound = timeBound; + } + + @Override + public Void apply(Iterable> input) + { + List values = new ArrayList<>(); + for (KV kv : input) { + assertEquals(0, (int)kv.getKey()); + values.add(kv.getValue()); + } + if (timeBound) { + assertTrue(values.size() >= 1); + } else if (dedup) { + // Verify that at least some data came through. The chance of 90% of the input + // being duplicates is essentially zero. + assertTrue(values.size() > NUM_RECORDS / 10 && values.size() <= NUM_RECORDS); + } else { + assertEquals(NUM_RECORDS, values.size()); + } + Collections.sort(values); + for (int i = 0; i < values.size(); i++) { + assertEquals(i, (int)values.get(i)); + } + //if (finalizeTracker != null) { + // assertThat(finalizeTracker, containsInAnyOrder(values.size() - 1)); + //} + return null; + } + } + + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java deleted file mode 100644 index 0ee3442652fd..000000000000 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTests.java +++ /dev/null @@ -1,207 +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.examples; - - - import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.is; - import static org.junit.Assert.assertThat; - -import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.TestApexRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.CountingInput; -import org.apache.beam.sdk.io.CountingInput.UnboundedCountingInput; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.NeedsRunner; - import org.apache.beam.sdk.testing.PAssert; - import org.apache.beam.sdk.testing.RunnableOnService; - import org.apache.beam.sdk.testing.TestPipeline; - import org.apache.beam.sdk.transforms.Count; - 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.RemoveDuplicates; - import org.apache.beam.sdk.transforms.SerializableFunction; - import org.apache.beam.sdk.transforms.display.DisplayData; -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.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; - - /** - * Tests for {@link CountingInput}. - */ - @RunWith(JUnit4.class) - public class IntTests { - public static void addCountingAsserts(PCollection input, long numElements) { - // Count == numElements - PAssert.thatSingleton(input.apply("Count", Count.globally())) - .isEqualTo(numElements); - // Unique count == numElements - PAssert.thatSingleton( - input - .apply(RemoveDuplicates.create()) - .apply("UniqueCount", Count.globally())) - .isEqualTo(numElements); - // Min == 0 - PAssert.thatSingleton(input.apply("Min", Min.globally())).isEqualTo(0L); - // Max == numElements-1 - PAssert.thatSingleton(input.apply("Max", Max.globally())) - .isEqualTo(numElements - 1); - } - - @Test - @Category(RunnableOnService.class) - public void testBoundedInput() { - //Pipeline p = TestPipeline.create(); - ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); - options.setRunner(TestApexRunner.class); - Pipeline p = Pipeline.create(options); - - long numElements = 1000; - PCollection input = p.apply(CountingInput.upTo(numElements)); - - addCountingAsserts(input, numElements); - p.run(); - } - - @Test - public void testBoundedDisplayData() { - PTransform input = CountingInput.upTo(1234); - DisplayData displayData = DisplayData.from(input); - assertThat(displayData, hasDisplayItem("upTo", 1234)); - } - - @Test - @Category(RunnableOnService.class) - public void testUnboundedInput() { - //Pipeline p = TestPipeline.create(); - ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); - options.setRunner(TestApexRunner.class); - Pipeline p = Pipeline.create(options); - - - long numElements = 1000; - - PCollection input = p.apply(CountingInput.unbounded().withMaxNumRecords(numElements)); - -// input = input.apply(Window.into(FixedWindows.of(Duration.standardSeconds(10)))); - - addCountingAsserts(input, numElements); - p.run(); - } - - @Test - @Category(NeedsRunner.class) - public void testUnboundedInputRate() { - Pipeline p = TestPipeline.create(); - long numElements = 5000; - - long elemsPerPeriod = 10L; - Duration periodLength = Duration.millis(8); - PCollection input = - p.apply( - CountingInput.unbounded() - .withRate(elemsPerPeriod, periodLength) - .withMaxNumRecords(numElements)); - - addCountingAsserts(input, numElements); - long expectedRuntimeMillis = (periodLength.getMillis() * numElements) / elemsPerPeriod; - Instant startTime = Instant.now(); - p.run(); - Instant endTime = Instant.now(); - assertThat(endTime.isAfter(startTime.plus(expectedRuntimeMillis)), is(true)); - } - - private static class ElementValueDiff extends DoFn { - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - c.output(c.element() - c.timestamp().getMillis()); - } - } - - @Test - @Category(RunnableOnService.class) - public void testUnboundedInputTimestamps() { - Pipeline p = TestPipeline.create(); - long numElements = 1000; - - PCollection input = - p.apply( - CountingInput.unbounded() - .withTimestampFn(new ValueAsTimestampFn()) - .withMaxNumRecords(numElements)); - addCountingAsserts(input, numElements); - - PCollection diffs = - input - .apply("TimestampDiff", ParDo.of(new ElementValueDiff())) - .apply("RemoveDuplicateTimestamps", RemoveDuplicates.create()); - // This assert also confirms that diffs only has one unique value. - PAssert.thatSingleton(diffs).isEqualTo(0L); - - p.run(); - } - - @Test - public void testUnboundedDisplayData() { - Duration maxReadTime = Duration.standardHours(5); - SerializableFunction timestampFn = new SerializableFunction() { - @Override - public Instant apply(Long input) { - return Instant.now(); - } - }; - - PTransform input = CountingInput.unbounded() - .withMaxNumRecords(1234) - .withMaxReadTime(maxReadTime) - .withTimestampFn(timestampFn); - - DisplayData displayData = DisplayData.from(input); - - assertThat(displayData, hasDisplayItem("maxRecords", 1234)); - assertThat(displayData, hasDisplayItem("maxReadTime", maxReadTime)); - assertThat(displayData, hasDisplayItem("timestampFn", timestampFn.getClass())); - } - - /** - * A timestamp function that uses the given value as the timestamp. Because the input values will - * not wrap, this function is non-decreasing and meets the timestamp function criteria laid out - * in {@link UnboundedCountingInput#withTimestampFn(SerializableFunction)}. - */ - private static class ValueAsTimestampFn implements SerializableFunction { - @Override - public Instant apply(Long input) { - return new Instant(input); - } - } - - - - -} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java index 06aaf550a1b7..6239021b7ca5 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java @@ -20,20 +20,25 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.runners.apex.TestApexRunner; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; +import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.SerializableCoder; 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.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; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import com.datatorrent.api.DAG; +import com.datatorrent.lib.util.KryoCloneUtils; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -48,6 +53,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.regex.Pattern; @@ -123,13 +129,11 @@ public void processElement(ProcessContext c) throws Exception { } } - - @Ignore @Test public void testAssertionFailure() throws Exception { ApexPipelineOptions options = PipelineOptionsFactory.create() .as(ApexPipelineOptions.class); - options.setRunner(ApexRunner.class); + options.setRunner(TestApexRunner.class); Pipeline pipeline = Pipeline.create(options); PCollection pcollection = pipeline @@ -149,6 +153,16 @@ public void testAssertionFailure() throws Exception { expectedPattern.matcher(exc.getMessage()).find()); } + @Test + public void testContainsInAnyOrder() throws Exception { + ApexPipelineOptions options = PipelineOptionsFactory.create().as(ApexPipelineOptions.class); + options.setRunner(TestApexRunner.class); + Pipeline pipeline = Pipeline.create(options); + PCollection pcollection = pipeline.apply(Create.of(1, 2, 3, 4)); + PAssert.that(pcollection).containsInAnyOrder(2, 1, 4, 3); + pipeline.run(); + } + private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { // We cannot use thrown.expect(AssertionError.class) because the AssertionError // is first caught by JUnit and causes a test failure. @@ -161,4 +175,19 @@ private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { throw new RuntimeException("unreachable"); } + @Test + public void testSerialization() throws Exception { + ApexPipelineOptions options = PipelineOptionsFactory.create() + .as(ApexPipelineOptions.class); + ApexParDoOperator operator = new ApexParDoOperator<>(options, + new Add(0), WindowingStrategy.globalDefault(), Collections.> emptyList()); + operator.setup(null); + operator.beginWindow(0); + WindowedValue wv = WindowedValue.valueInGlobalWindow(0); + operator.input.process(ApexStreamTuple.DataTuple.of(wv)); + operator.input.process(ApexStreamTuple.WatermarkTuple.>of(0)); + operator.endWindow(); + Assert.assertNotNull("Serialization", KryoCloneUtils.cloneObject(operator)); + + } } diff --git a/runners/apex/src/test/resources/log4j.properties b/runners/apex/src/test/resources/log4j.properties index 84a6f6897743..c0efc5de4f39 100644 --- a/runners/apex/src/test/resources/log4j.properties +++ b/runners/apex/src/test/resources/log4j.properties @@ -26,8 +26,8 @@ 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.logger.org=info -#log4j.logger.org.apache.commons.beanutils=warn +log4j.logger.org=debug +log4j.logger.org.apache.commons.beanutils=warn log4j.logger.com.datatorrent=info log4j.logger.org.apache.apex=debug log4j.logger.org.apache.beam.runners.apex=debug From 047cff492f1f804785dee73b4768293d3569e8de Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Thu, 6 Oct 2016 22:36:01 -0700 Subject: [PATCH 5/8] BEAM-261 Add support for ParDo.BoundMulti --- runners/apex/pom.xml | 3 +- .../runners/apex/ApexPipelineTranslator.java | 2 + .../apache/beam/runners/apex/ApexRunner.java | 3 +- .../FlattenPCollectionTranslator.java | 1 + .../ParDoBoundMultiTranslator.java | 74 +++++++++++++++++++ .../translators/ParDoBoundTranslator.java | 5 +- .../apex/translators/TranslationContext.java | 17 +++++ .../functions/ApexFlattenOperator.java | 2 + .../functions/ApexParDoOperator.java | 68 +++++++++++++---- .../FlattenPCollectionTranslatorTest.java | 42 ++++++----- .../translators/ParDoBoundTranslatorTest.java | 29 ++++---- 11 files changed, 194 insertions(+), 52 deletions(-) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index e9377b4d9aa6..929feb4733c7 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -185,8 +185,7 @@ [ - "--runner=org.apache.beam.runners.apex.TestApexRunner", - "--streaming=true" + "--runner=org.apache.beam.runners.apex.TestApexRunner" ] diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java index ad8c28321581..40edfb16e475 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java @@ -22,6 +22,7 @@ import org.apache.beam.runners.apex.translators.CreateValuesTranslator; import org.apache.beam.runners.apex.translators.FlattenPCollectionTranslator; import org.apache.beam.runners.apex.translators.GroupByKeyTranslator; +import org.apache.beam.runners.apex.translators.ParDoBoundMultiTranslator; import org.apache.beam.runners.apex.translators.ParDoBoundTranslator; import org.apache.beam.runners.apex.translators.ReadUnboundedTranslator; import org.apache.beam.runners.apex.translators.TransformTranslator; @@ -66,6 +67,7 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { static { // register TransformTranslators registerTransformTranslator(ParDo.Bound.class, new ParDoBoundTranslator()); + registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator<>()); registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); 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 ae79a2049d11..e2ebc2961705 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 @@ -230,7 +230,7 @@ public void processElement(ProcessContext c) { * Records that the {@link PTransform} requires a deterministic key coder. */ private void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { - throw new UnsupportedOperationException(); + //throw new UnsupportedOperationException(); } /** @@ -369,7 +369,6 @@ private static class StreamingViewAsMap private final ApexRunner runner; - @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() public StreamingViewAsMap(ApexRunner runner, View.AsMap transform) { this.runner = runner; } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java index 712466a10c40..90ab81f436e9 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java @@ -80,6 +80,7 @@ public void translate(Flatten.FlattenPCollectionList transform, TranslationCo if (firstCollection != null) { // push to next merge level remainingCollections.add(firstCollection); + firstCollection = null; } if (remainingCollections.size() > 1) { collections = remainingCollections; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java new file mode 100644 index 000000000000..6488bf682f54 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.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.apex.translators; + +import java.util.List; +import java.util.Map; + +import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; + +import com.datatorrent.api.Operator; +import com.datatorrent.api.Operator.OutputPort; +import com.google.common.collect.Maps; + +/** + * {@link ParDo.BoundMulti} is translated to Apex operator that wraps the {@link DoFn} + */ +public class ParDoBoundMultiTranslator implements TransformTranslator> { + private static final long serialVersionUID = 1L; + + @Override + public void translate(ParDo.BoundMulti transform, TranslationContext context) { + OldDoFn doFn = transform.getFn(); + PCollectionTuple output = context.getOutput(); + List> sideInputs = transform.getSideInputs(); + ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), + doFn, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), + context.>getInput().getWindowingStrategy(), sideInputs); + + Map, PCollection> outputs = output.getAll(); + Map, OutputPort> ports = Maps.newHashMapWithExpectedSize(outputs.size()); + int i = 0; + for (Map.Entry, PCollection> outputEntry : outputs.entrySet()) { + ports.put(outputEntry.getValue(), operator.sideOutputPorts[i++]); + } + context.addOperator(operator, ports); + + context.addStream(context.getInput(), operator.input); + if (!sideInputs.isEmpty()) { + Operator.InputPort[] sideInputPorts = {operator.sideInput1}; + for (i=0; i transform, TranslationContext PCollection output = context.getOutput(); List> sideInputs = transform.getSideInputs(); ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), - doFn, output.getWindowingStrategy(), sideInputs); + doFn, new TupleTag(), TupleTagList.empty().getAll() /*sideOutputTags*/, + output.getWindowingStrategy(), sideInputs); context.addOperator(operator, operator.output); context.addStream(context.getInput(), operator.input); if (!sideInputs.isEmpty()) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java index 163cfd43ec8a..bd44a20a911d 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java @@ -97,6 +97,23 @@ public void addOperator(Operator operator, OutputPort port) { addOperator(operator, port, this.>getOutput()); } + /** + * Register operator and output ports for the given collections. + * @param operator + * @param ports + */ + public void addOperator(Operator operator, Map, OutputPort> ports) { + boolean first = true; + for (Map.Entry, OutputPort> portEntry : ports.entrySet()) { + if (first) { + addOperator(operator, portEntry.getValue(), portEntry.getKey()); + first = false; + } else { + this.streams.put(portEntry.getKey(), (Pair)new ImmutablePair<>(portEntry.getValue(), new ArrayList<>())); + } + } + } + /** * Add intermediate operator for the current transformation. * @param operator diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java index ce27abb5096c..467524411794 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java @@ -26,6 +26,7 @@ import com.datatorrent.api.DefaultInputPort; import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; import com.datatorrent.common.util.BaseOperator; /** @@ -109,5 +110,6 @@ public void process(ApexStreamTuple> tuple) /** * Output port. */ + @OutputPortFieldAnnotation(optional=true) public final transient DefaultOutputPort>> out = new DefaultOutputPort>>(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java index 13a8fc9f50da..995fee1e8234 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.runners.apex.translators.functions; import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; @@ -47,7 +47,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,6 +58,7 @@ import com.datatorrent.common.util.BaseOperator; import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import com.esotericsoftware.kryo.serializers.JavaSerializer; /** @@ -68,43 +68,58 @@ public class ApexParDoOperator extends BaseOperator implements private static final Logger LOG = LoggerFactory.getLogger(ApexParDoOperator.class); private boolean traceTuples = true; - private transient final TupleTag mainTag = new TupleTag(); - private transient PushbackSideInputDoFnRunner pushbackDoFnRunner; - @Bind(JavaSerializer.class) private final SerializablePipelineOptions pipelineOptions; @Bind(JavaSerializer.class) private final OldDoFn doFn; @Bind(JavaSerializer.class) + private final TupleTag mainOutputTag; + @Bind(JavaSerializer.class) + private final List> sideOutputTags; + @Bind(JavaSerializer.class) private final WindowingStrategy windowingStrategy; @Bind(JavaSerializer.class) - List> sideInputs; + private final List> sideInputs; + // TODO: not Kryo serializable, integrate codec //@Bind(JavaSerializer.class) private transient StateInternals sideInputStateInternals = InMemoryStateInternals.forKey(null); - private transient SideInputHandler sideInputHandler; // TODO: not Kryo serializable, integrate codec private List> pushedBack = new ArrayList<>(); private LongMin pushedBackWatermark = new LongMin(); private long currentInputWatermark = Long.MIN_VALUE; private long currentOutputWatermark = currentInputWatermark; + private transient PushbackSideInputDoFnRunner pushbackDoFnRunner; + private transient SideInputHandler sideInputHandler; + private transient Map, DefaultOutputPort>> sideOutputPortMapping = Maps.newHashMapWithExpectedSize(5); + public ApexParDoOperator( ApexPipelineOptions pipelineOptions, OldDoFn doFn, + TupleTag mainOutputTag, + List> sideOutputTags, WindowingStrategy windowingStrategy, List> sideInputs ) { this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions); this.doFn = doFn; + this.mainOutputTag = mainOutputTag; + this.sideOutputTags = sideOutputTags; this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; + + if (sideOutputTags != null && sideOutputTags.size() > sideOutputPorts.length) { + String msg = String.format("Too many side outputs (currently only supporting %s).", + sideOutputPorts.length); + throw new UnsupportedOperationException(msg); + } } @SuppressWarnings("unused") // for Kryo private ApexParDoOperator() { - this(null, null, null, null); + this(null, null, null, null, null, null); } @@ -167,10 +182,28 @@ public void process(ApexStreamTuple>> t) @OutputPortFieldAnnotation(optional=true) public final transient DefaultOutputPort> output = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort> sideOutput1 = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort> sideOutput2 = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort> sideOutput3 = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort> sideOutput4 = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional=true) + public final transient DefaultOutputPort> sideOutput5 = new DefaultOutputPort<>(); + + public final transient DefaultOutputPort[] sideOutputPorts = {sideOutput1, sideOutput2, sideOutput3, sideOutput4, sideOutput5}; + @Override public void output(TupleTag tag, WindowedValue tuple) { - output.emit(ApexStreamTuple.DataTuple.of(tuple)); + DefaultOutputPort> sideOutputPort = sideOutputPortMapping.get(tag); + if (sideOutputPort != null) { + sideOutputPort.emit(ApexStreamTuple.DataTuple.of(tuple)); + } else { + output.emit(ApexStreamTuple.DataTuple.of(tuple)); + } if (traceTuples) { LOG.debug("\nemitting {}\n", tuple); } @@ -178,7 +211,10 @@ public void output(TupleTag tag, WindowedValue tuple) private Iterable> processElementInReadyWindows(WindowedValue elem) { try { - return pushbackDoFnRunner.processElementInReadyWindows(elem); + pushbackDoFnRunner.startBundle(); + Iterable> pushedBack = pushbackDoFnRunner.processElementInReadyWindows(elem); + pushbackDoFnRunner.finishBundle(); + return pushedBack; } catch (UserCodeException ue) { if (ue.getCause() instanceof AssertionError) { ApexRunner.assertionError = (AssertionError)ue.getCause(); @@ -220,13 +256,19 @@ public void setup(OperatorContext context) sideInputReader = sideInputHandler; } + for (int i=0; i < sideOutputTags.size(); i++) { + @SuppressWarnings("unchecked") + DefaultOutputPort> port = (DefaultOutputPort>)sideOutputPorts[i]; + sideOutputPortMapping.put(sideOutputTags.get(i), port); + } + DoFnRunner doFnRunner = DoFnRunners.createDefault( pipelineOptions.get(), doFn, sideInputReader, this, - mainTag, - TupleTagList.empty().getAll() /*sideOutputTags*/, + mainOutputTag, + sideOutputTags, new NoOpStepContext(), new NoOpAggregatorFactory(), windowingStrategy @@ -246,7 +288,6 @@ public void setup(OperatorContext context) @Override public void beginWindow(long windowId) { - pushbackDoFnRunner.startBundle(); /* Collection> aggregators = AggregatorRetriever.getAggregators(doFn); if (!aggregators.isEmpty()) { @@ -258,7 +299,6 @@ public void beginWindow(long windowId) @Override public void endWindow() { - pushbackDoFnRunner.finishBundle(); } /** diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java index d3b56bcb19b7..6b181ba4deca 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java @@ -31,15 +31,17 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashSet; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.Set; /** * integration test for {@link FlattenPCollectionTranslator}. @@ -49,41 +51,41 @@ public class FlattenPCollectionTranslatorTest { @Test public void test() throws Exception { - ApexPipelineOptions options = - PipelineOptionsFactory.as(ApexPipelineOptions.class); + ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); options.setApplicationName("FlattenPCollection"); options.setRunner(ApexRunner.class); Pipeline p = Pipeline.create(options); - List collection1 = Lists.newArrayList("1", "2", "3"); - List collection2 = Lists.newArrayList("4", "5"); - List expected = Lists.newArrayList("1", "2", "3", "4", "5"); - PCollection pc1 = - p.apply(Create.of(collection1).withCoder(StringUtf8Coder.of())); - PCollection pc2 = - p.apply(Create.of(collection2).withCoder(StringUtf8Coder.of())); - PCollectionList pcs = PCollectionList.of(pc1).and(pc2); - PCollection actual = pcs.apply(Flatten.pCollections()); + String[][] collections = { + {"1"}, {"2"}, {"3"}, {"4"}, {"5"} + }; + + Set expected = Sets.newHashSet(); + List> pcList = new ArrayList>(); + for (String[] collection : collections) { + pcList.add(p.apply(Create.of(collection).withCoder(StringUtf8Coder.of()))); + expected.addAll(Arrays.asList(collection)); + } + + PCollection actual = PCollectionList.of(pcList).apply(Flatten.pCollections()); actual.apply(ParDo.of(new EmbeddedCollector())); ApexRunnerResult result = (ApexRunnerResult)p.run(); // TODO: verify translation result.getApexDAG(); long timeout = System.currentTimeMillis() + 30000; - while (System.currentTimeMillis() < timeout) { - if (EmbeddedCollector.results.containsAll(expected)) { - break; - } + while (System.currentTimeMillis() < timeout && EmbeddedCollector.results.size() < expected.size()) { LOG.info("Waiting for expected results."); - Thread.sleep(1000); + Thread.sleep(500); } - org.junit.Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + Assert.assertEquals("number results", expected.size(), EmbeddedCollector.results.size()); + Assert.assertEquals(expected, Sets.newHashSet(EmbeddedCollector.results)); } @SuppressWarnings("serial") private static class EmbeddedCollector extends OldDoFn { - protected static final HashSet results = new HashSet<>(); + protected static final ArrayList results = new ArrayList<>(); public EmbeddedCollector() { } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java index 6239021b7ca5..301f6f89937d 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java @@ -36,6 +36,8 @@ 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 com.datatorrent.api.DAG; import com.datatorrent.lib.util.KryoCloneUtils; @@ -129,6 +131,18 @@ public void processElement(ProcessContext c) throws Exception { } } + private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { + // We cannot use thrown.expect(AssertionError.class) because the AssertionError + // is first caught by JUnit and causes a test failure. + try { + pipeline.run(); + } catch (AssertionError exc) { + return exc; + } + fail("assertion should have failed"); + throw new RuntimeException("unreachable"); + } + @Test public void testAssertionFailure() throws Exception { ApexPipelineOptions options = PipelineOptionsFactory.create() @@ -163,24 +177,13 @@ public void testContainsInAnyOrder() throws Exception { pipeline.run(); } - private static Throwable runExpectingAssertionFailure(Pipeline pipeline) { - // We cannot use thrown.expect(AssertionError.class) because the AssertionError - // is first caught by JUnit and causes a test failure. - try { - pipeline.run(); - } catch (AssertionError exc) { - return exc; - } - fail("assertion should have failed"); - throw new RuntimeException("unreachable"); - } - @Test public void testSerialization() throws Exception { ApexPipelineOptions options = PipelineOptionsFactory.create() .as(ApexPipelineOptions.class); ApexParDoOperator operator = new ApexParDoOperator<>(options, - new Add(0), WindowingStrategy.globalDefault(), Collections.> emptyList()); + new Add(0), new TupleTag(), TupleTagList.empty().getAll(), + WindowingStrategy.globalDefault(), Collections.> emptyList()); operator.setup(null); operator.beginWindow(0); WindowedValue wv = WindowedValue.valueInGlobalWindow(0); From fd7f46c19b9c95a63b522793bb6fb8a849167cbc Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Thu, 13 Oct 2016 00:56:37 -0700 Subject: [PATCH 6/8] BEAM-261 Checkpointing for pushed back inputs. --- .../apache/beam/runners/apex/ApexRunner.java | 3 +- .../ParDoBoundMultiTranslator.java | 10 ++- .../translators/ParDoBoundTranslator.java | 11 ++- .../functions/ApexParDoOperator.java | 32 +++++--- .../utils/ValueAndCoderKryoSerializable.java | 81 +++++++++++++++++++ .../translators/ParDoBoundTranslatorTest.java | 42 ++++++---- 6 files changed, 149 insertions(+), 30 deletions(-) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.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 e2ebc2961705..ad49f08b8333 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 @@ -72,7 +72,7 @@ public class ApexRunner extends PipelineRunner { private final ApexPipelineOptions options; /** - * TODO: this isn't thread sa + * TODO: this isn't thread safe and may cause issues when tests run in parallel * Holds any most resent assertion error that was raised while processing elements. * Used in the unit test driver in embedded to propagate the exception. */ @@ -89,7 +89,6 @@ public static ApexRunner fromOptions(PipelineOptions options) { @Override public OutputT apply( PTransform transform, InputT input) { -//System.out.println("transform: " + transform); if (Window.Bound.class.equals(transform.getClass())) { return (OutputT) ((PCollection) input).apply( diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java index 6488bf682f54..9c5f2b566180 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java @@ -22,8 +22,11 @@ import java.util.Map; import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; @@ -43,10 +46,15 @@ public class ParDoBoundMultiTranslator implements TransformTran public void translate(ParDo.BoundMulti transform, TranslationContext context) { OldDoFn doFn = transform.getFn(); PCollectionTuple output = context.getOutput(); + PCollection input = context.getInput(); List> sideInputs = transform.getSideInputs(); + Coder inputCoder = input.getCoder(); + WindowedValueCoder wvInputCoder = FullWindowedValueCoder.of(inputCoder, + input.getWindowingStrategy().getWindowFn().windowCoder()); + ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), doFn, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), - context.>getInput().getWindowingStrategy(), sideInputs); + context.>getInput().getWindowingStrategy(), sideInputs, wvInputCoder); Map, PCollection> outputs = output.getAll(); Map, OutputPort> ports = Maps.newHashMapWithExpectedSize(outputs.size()); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java index fa3df7c7ddd6..8a7dd4b5c512 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java @@ -21,8 +21,12 @@ import java.util.List; import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; @@ -41,10 +45,15 @@ public class ParDoBoundTranslator implements public void translate(ParDo.Bound transform, TranslationContext context) { OldDoFn doFn = transform.getFn(); PCollection output = context.getOutput(); + PCollection input = context.getInput(); List> sideInputs = transform.getSideInputs(); + Coder inputCoder = input.getCoder(); + WindowedValueCoder wvInputCoder = FullWindowedValueCoder.of(inputCoder, + input.getWindowingStrategy().getWindowFn().windowCoder()); + ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), doFn, new TupleTag(), TupleTagList.empty().getAll() /*sideOutputTags*/, - output.getWindowingStrategy(), sideInputs); + output.getWindowingStrategy(), sideInputs, wvInputCoder); context.addOperator(operator, operator.output); context.addStream(context.getInput(), operator.input); if (!sideInputs.isEmpty()) { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java index 995fee1e8234..a951ca757f56 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -26,11 +26,14 @@ import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.NoOpStepContext; import org.apache.beam.runners.apex.translators.utils.SerializablePipelineOptions; +import org.apache.beam.runners.apex.translators.utils.ValueAndCoderKryoSerializable; 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.core.DoFnRunners.OutputManager; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.repackaged.com.google.common.base.Throwables; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; @@ -84,8 +87,7 @@ public class ApexParDoOperator extends BaseOperator implements // TODO: not Kryo serializable, integrate codec //@Bind(JavaSerializer.class) private transient StateInternals sideInputStateInternals = InMemoryStateInternals.forKey(null); - // TODO: not Kryo serializable, integrate codec - private List> pushedBack = new ArrayList<>(); + private final ValueAndCoderKryoSerializable>> pushedBack; private LongMin pushedBackWatermark = new LongMin(); private long currentInputWatermark = Long.MIN_VALUE; private long currentOutputWatermark = currentInputWatermark; @@ -100,7 +102,8 @@ public ApexParDoOperator( TupleTag mainOutputTag, List> sideOutputTags, WindowingStrategy windowingStrategy, - List> sideInputs + List> sideInputs, + Coder> inputCoder ) { this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions); @@ -110,19 +113,28 @@ public ApexParDoOperator( this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; - if (sideOutputTags != null && sideOutputTags.size() > sideOutputPorts.length) { + if (sideOutputTags.size() > sideOutputPorts.length) { String msg = String.format("Too many side outputs (currently only supporting %s).", sideOutputPorts.length); throw new UnsupportedOperationException(msg); } + + Coder>> coder = ListCoder.of(inputCoder); + this.pushedBack = new ValueAndCoderKryoSerializable<>(new ArrayList>(), coder); + } @SuppressWarnings("unused") // for Kryo private ApexParDoOperator() { - this(null, null, null, null, null, null); + this.pipelineOptions = null; + this.doFn = null; + this.mainOutputTag = null; + this.sideOutputTags = null; + this.windowingStrategy = null; + this.sideInputs = null; + this.pushedBack = null; } - public final transient DefaultInputPort>> input = new DefaultInputPort>>() { @Override @@ -137,7 +149,7 @@ public void process(ApexStreamTuple> t) Iterable> justPushedBack = processElementInReadyWindows(t.getValue()); for (WindowedValue pushedBackValue : justPushedBack) { pushedBackWatermark.add(pushedBackValue.getTimestamp().getMillis()); - pushedBack.add(pushedBackValue); + pushedBack.get().add(pushedBackValue); } } } @@ -162,16 +174,16 @@ public void process(ApexStreamTuple>> t) sideInputHandler.addSideInputValue(sideInput, t.getValue()); List> newPushedBack = new ArrayList<>(); - for (WindowedValue elem : pushedBack) { + for (WindowedValue elem : pushedBack.get()) { Iterable> justPushedBack = processElementInReadyWindows(elem); Iterables.addAll(newPushedBack, justPushedBack); } - pushedBack.clear(); + pushedBack.get().clear(); pushedBackWatermark.clear(); for (WindowedValue pushedBackValue : newPushedBack) { pushedBackWatermark.add(pushedBackValue.getTimestamp().getMillis()); - pushedBack.add(pushedBackValue); + pushedBack.get().add(pushedBackValue); } // potentially emit watermark diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.java new file mode 100644 index 000000000000..2de737d406e2 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.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.runners.apex.translators.utils; + +import java.io.IOException; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.base.Throwables; + + +/** + * A {@link KryoSerializable} holder that uses the specified {@link Coder}. + * @param + */ +public class ValueAndCoderKryoSerializable implements KryoSerializable +{ + private static JavaSerializer JAVA_SERIALIZER = new JavaSerializer(); + private T value; + private Coder coder; + + public ValueAndCoderKryoSerializable(T value, Coder coder) { + this.value = value; + this.coder = coder; + } + + @SuppressWarnings("unused") // for Kryo + private ValueAndCoderKryoSerializable() { + } + + public T get() { + return value; + } + + @Override + public void write(Kryo kryo, Output output) + { + try { + kryo.writeClass(output, coder.getClass()); + kryo.writeObject(output, coder, JAVA_SERIALIZER); + coder.encode(value, output, Context.OUTER); + } catch (IOException e) { + Throwables.propagate(e); + } + } + + @Override + public void read(Kryo kryo, Input input) + { + try { + @SuppressWarnings("unchecked") + Class> type = kryo.readClass(input).getType(); + coder = kryo.readObject(input, type, JAVA_SERIALIZER); + value = coder.decode(input, Context.OUTER); + } catch (IOException e) { + Throwables.propagate(e); + } + } + +} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java index 301f6f89937d..b9748eebeb08 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java @@ -18,47 +18,48 @@ package org.apache.beam.runners.apex.translators; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.regex.Pattern; + import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.runners.apex.TestApexRunner; -import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sum; 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 com.datatorrent.api.DAG; -import com.datatorrent.lib.util.KryoCloneUtils; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.regex.Pattern; +import com.datatorrent.api.DAG; +import com.datatorrent.lib.util.KryoCloneUtils; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; /** * integration test for {@link ParDoBoundTranslator}. @@ -181,9 +182,18 @@ public void testContainsInAnyOrder() throws Exception { public void testSerialization() throws Exception { ApexPipelineOptions options = PipelineOptionsFactory.create() .as(ApexPipelineOptions.class); + options.setRunner(TestApexRunner.class); + Pipeline pipeline = Pipeline.create(options); + Coder> coder = WindowedValue.getValueOnlyCoder(VarIntCoder.of()); + + PCollectionView singletonView = pipeline.apply(Create.of(1)) + .apply(Sum.integersGlobally().asSingletonView()); + ApexParDoOperator operator = new ApexParDoOperator<>(options, new Add(0), new TupleTag(), TupleTagList.empty().getAll(), - WindowingStrategy.globalDefault(), Collections.> emptyList()); + WindowingStrategy.globalDefault(), + Collections.>singletonList(singletonView), + coder); operator.setup(null); operator.beginWindow(0); WindowedValue wv = WindowedValue.valueInGlobalWindow(0); From 1ec7cd9129fc31ece7554e2ea18535ce15e46bcf Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Thu, 13 Oct 2016 14:38:06 -0700 Subject: [PATCH 7/8] BEAM-261 Support multiple side inputs. --- .../runners/apex/ApexPipelineTranslator.java | 19 ++++++- .../apache/beam/runners/apex/ApexRunner.java | 7 ++- .../beam/runners/apex/ApexRunnerResult.java | 7 +++ .../FlattenPCollectionTranslator.java | 38 +++++++++++-- .../ParDoBoundMultiTranslator.java | 55 ++++++++++++++++--- .../translators/ParDoBoundTranslator.java | 14 +---- .../functions/ApexFlattenOperator.java | 11 ++++ .../functions/ApexParDoOperator.java | 13 +++-- .../translators/utils/ApexStreamTuple.java | 22 ++++++-- 9 files changed, 148 insertions(+), 38 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java index 40edfb16e475..a16f55141255 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java @@ -37,6 +37,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.CreatePCollectionView; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.slf4j.Logger; @@ -74,7 +75,8 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { registerTransformTranslator(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); - registerTransformTranslator(CreateApexPCollectionView.class, new CreatePCollectionViewTranslator()); + registerTransformTranslator(CreateApexPCollectionView.class, new CreateApexPCollectionViewTranslator()); + registerTransformTranslator(CreatePCollectionView.class, new CreatePCollectionViewTranslator()); } public ApexPipelineTranslator(TranslationContext translationContext) { @@ -151,7 +153,7 @@ public void translate(Read.Bounded transform, TranslationContext context) { } - private static class CreatePCollectionViewTranslator implements TransformTranslator> + private static class CreateApexPCollectionViewTranslator implements TransformTranslator> { private static final long serialVersionUID = 1L; @@ -164,4 +166,17 @@ public void translate(CreateApexPCollectionView transform, Transla } } + private static class CreatePCollectionViewTranslator implements TransformTranslator> + { + private static final long serialVersionUID = 1L; + + @Override + public void translate(CreatePCollectionView transform, TranslationContext context) + { + PCollectionView view = transform.getView(); + context.addView(view); + LOG.debug("view {}", view.getName()); + } + } + } 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 ad49f08b8333..667f1c806377 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 @@ -74,7 +74,7 @@ public class ApexRunner extends PipelineRunner { /** * TODO: this isn't thread safe and may cause issues when tests run in parallel * Holds any most resent assertion error that was raised while processing elements. - * Used in the unit test driver in embedded to propagate the exception. + * Used in the unit test driver in embedded mode to propagate the exception. */ public static volatile AssertionError assertionError; @@ -100,6 +100,8 @@ public OutputT apply( WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); // TODO: replace this with a mapping +//// + } else if (Combine.GloballyAsSingletonView.class.equals(transform.getClass())) { PTransform customTransform = (PTransform)new StreamingCombineGloballyAsSingletonView(this, (Combine.GloballyAsSingletonView)transform); @@ -109,6 +111,7 @@ public OutputT apply( PTransform customTransform = (PTransform)new StreamingViewAsSingleton(this, (View.AsSingleton)transform); return Pipeline.applyTransform(input, customTransform); +/* } else if (View.AsIterable.class.equals(transform.getClass())) { PTransform customTransform = (PTransform)new StreamingViewAsIterable(this, (View.AsIterable)transform); @@ -125,6 +128,8 @@ public OutputT apply( PTransform customTransform = new StreamingViewAsMultimap(this, (View.AsMultimap)transform); return Pipeline.applyTransform(input, customTransform); +*/ +//// } else { return super.apply(transform, input); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java index f28c8dc5d42b..68176844290f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java @@ -19,6 +19,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.metrics.MetricResults; import java.io.IOException; @@ -74,6 +75,12 @@ public State waitUntilFinish() throws IOException, InterruptedException throw new UnsupportedOperationException(); } + @Override + public MetricResults metrics() + { + throw new UnsupportedOperationException(); + } + /** * Return the DAG executed by the pipeline. * @return diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java index 90ab81f436e9..673776707e77 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.List; +import java.util.Map; import org.apache.beam.runners.apex.translators.functions.ApexFlattenOperator; import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; @@ -53,9 +54,25 @@ public void translate(Flatten.FlattenPCollectionList transform, TranslationCo ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>( unboundedSource, context.getPipelineOptions()); context.addOperator(operator, operator.output); - return; + } else { + PCollection output = context.getOutput(); + Map, Integer> unionTags = Collections.emptyMap(); + flattenCollections(collections, unionTags, output, context); } + } + /** + * Flatten the given collections into the given result collection. Translates + * into a cascading merge with 2 input ports per operator. The optional union + * tags can be used to identify the source in the result stream, used to + * channel multiple side inputs to a single Apex operator port. + * + * @param collections + * @param unionTags + * @param finalCollection + * @param context + */ + static void flattenCollections(List> collections, Map, Integer> unionTags, PCollection finalCollection, TranslationContext context) { List> remainingCollections = Lists.newArrayList(); PCollection firstCollection = null; while (!collections.isEmpty()) { @@ -65,14 +82,23 @@ public void translate(Flatten.FlattenPCollectionList transform, TranslationCo } else { ApexFlattenOperator operator = new ApexFlattenOperator<>(); context.addStream(firstCollection, operator.data1); + Integer unionTag = unionTags.get(firstCollection); + operator.data1Tag = (unionTag != null) ? unionTag : 0; context.addStream(collection, operator.data2); + unionTag = unionTags.get(collection); + operator.data2Tag = (unionTag != null) ? unionTag : 0; + + if (!collection.getCoder().equals(firstCollection.getCoder())) { + throw new UnsupportedOperationException("coders don't match"); + } + if (collections.size() > 2) { - PCollection resultCollection = intermediateCollection(collection, collection.getCoder()); - context.addOperator(operator, operator.out, resultCollection); - remainingCollections.add(resultCollection); + PCollection intermediateCollection = intermediateCollection(collection, collection.getCoder()); + context.addOperator(operator, operator.out, intermediateCollection); + remainingCollections.add(intermediateCollection); } else { // final stream merge - context.addOperator(operator, operator.out); + context.addOperator(operator, operator.out, finalCollection); } firstCollection = null; } @@ -91,7 +117,7 @@ public void translate(Flatten.FlattenPCollectionList transform, TranslationCo } } - public static PCollection intermediateCollection(PCollection input, Coder outputCoder) { + static 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/translators/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java index 9c5f2b566180..a229a8191e3c 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java @@ -18,6 +18,10 @@ package org.apache.beam.runners.apex.translators; +import static com.google.common.base.Preconditions.checkArgument; + +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -63,20 +67,55 @@ public void translate(ParDo.BoundMulti transform, TranslationCo ports.put(outputEntry.getValue(), operator.sideOutputPorts[i++]); } context.addOperator(operator, ports); - context.addStream(context.getInput(), operator.input); if (!sideInputs.isEmpty()) { - Operator.InputPort[] sideInputPorts = {operator.sideInput1}; - for (i=0; i operator, List> sideInputs, TranslationContext context) { + Operator.InputPort[] sideInputPorts = {operator.sideInput1}; + if (sideInputs.size() > sideInputPorts.length) { + // String msg = String.format("Too many side inputs in %s (currently only supporting %s).", + // transform.toString(), sideInputPorts.length); + // throw new UnsupportedOperationException(msg); + PCollection unionCollection = unionSideInputs(sideInputs, context); + context.addStream(unionCollection, sideInputPorts[0]); + } else { + for (int i=0; i unionSideInputs(List> sideInputs, TranslationContext context) { + checkArgument(sideInputs.size() > 1, "requires multiple side inputs"); + // flatten and assign union tag + List> sourceCollections = new ArrayList<>(); + Map, Integer> unionTags = new HashMap<>(); + PCollection firstSideInput = context.getViewInput(sideInputs.get(0)); + for (int i=0; i < sideInputs.size(); i++) { + PCollectionView sideInput = sideInputs.get(i); + PCollection sideInputCollection = context.getViewInput(sideInput); + if (!sideInputCollection.getWindowingStrategy().equals(firstSideInput.getWindowingStrategy())) { + // TODO: check how to handle this in stream codec + //String msg = "Multiple side inputs with different window strategies."; + //throw new UnsupportedOperationException(msg); + } + if (!sideInputCollection.getCoder().equals(firstSideInput.getCoder())) { + String msg = "Multiple side inputs with different coders."; + throw new UnsupportedOperationException(msg); + } + sourceCollections.add(context.>getViewInput(sideInput)); + unionTags.put(sideInputCollection, i); + } + + PCollection resultCollection = FlattenPCollectionTranslator.intermediateCollection(firstSideInput, firstSideInput.getCoder()); + FlattenPCollectionTranslator.flattenCollections(sourceCollections, unionTags, resultCollection, context); + return resultCollection; + + } + } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java index 8a7dd4b5c512..7749a0635ac8 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java @@ -32,8 +32,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.datatorrent.api.Operator; - /** * {@link ParDo.Bound} is translated to Apex operator that wraps the {@link DoFn} */ @@ -57,17 +55,7 @@ public void translate(ParDo.Bound transform, TranslationContext context.addOperator(operator, operator.output); context.addStream(context.getInput(), operator.input); if (!sideInputs.isEmpty()) { - Operator.InputPort[] sideInputPorts = {operator.sideInput1}; - for (int i=0; i extends BaseOperator private long inputWM2; private long outputWM; + public int data1Tag; + public int data2Tag; + /** * Data input port 1. */ @@ -70,6 +73,10 @@ public void process(ApexStreamTuple> tuple) if (traceTuples) { LOG.debug("\nemitting {}\n", tuple); } + + if (data1Tag > 0 && tuple instanceof ApexStreamTuple.DataTuple) { + ((ApexStreamTuple.DataTuple)tuple).setUnionTag(data1Tag); + } out.emit(tuple); } }; @@ -103,6 +110,10 @@ public void process(ApexStreamTuple> tuple) if (traceTuples) { LOG.debug("\nemitting {}\n", tuple); } + + if (data2Tag > 0 && tuple instanceof ApexStreamTuple.DataTuple) { + ((ApexStreamTuple.DataTuple)tuple).setUnionTag(data2Tag); + } out.emit(tuple); } }; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java index a951ca757f56..96be11d101d3 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -34,7 +34,6 @@ import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.repackaged.com.google.common.base.Throwables; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -60,6 +59,7 @@ import com.datatorrent.api.annotation.OutputPortFieldAnnotation; import com.datatorrent.common.util.BaseOperator; import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.esotericsoftware.kryo.serializers.JavaSerializer; @@ -158,8 +158,6 @@ public void process(ApexStreamTuple> t) @InputPortFieldAnnotation(optional=true) public final transient DefaultInputPort>>> sideInput1 = new DefaultInputPort>>>() { - private final int sideInputIndex = 0; - @Override public void process(ApexStreamTuple>> t) { @@ -167,9 +165,16 @@ public void process(ApexStreamTuple>> t) // ignore side input watermarks return; } + + int sideInputIndex = 0; + if (t instanceof ApexStreamTuple.DataTuple) { + sideInputIndex = ((ApexStreamTuple.DataTuple)t).getUnionTag(); + } + if (traceTuples) { - LOG.debug("\nsideInput {}\n", t.getValue()); + LOG.debug("\nsideInput {} {}\n", sideInputIndex, t.getValue()); } + PCollectionView sideInput = sideInputs.get(sideInputIndex); sideInputHandler.addSideInputValue(sideInput, t.getValue()); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java index 06940aa28b94..c9bf6dc2f5cb 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java @@ -50,15 +50,17 @@ public interface ApexStreamTuple */ class DataTuple implements ApexStreamTuple { + private int unionTag; private T value; public static DataTuple of(T value) { - return new DataTuple<>(value); + return new DataTuple<>(value, 0); } - private DataTuple(T value) + private DataTuple(T value, int unionTag) { this.value = value; + this.unionTag = unionTag; } @Override @@ -72,6 +74,16 @@ public void setValue(T value) this.value = value; } + public int getUnionTag() + { + return unionTag; + } + + public void setUnionTag(int unionTag) + { + this.unionTag = unionTag; + } + @Override public String toString() { @@ -91,7 +103,7 @@ class TimestampedTuple extends DataTuple public TimestampedTuple(long timestamp, T value) { - super(value); + super(value, 0); this.timestamp = timestamp; } @@ -152,6 +164,7 @@ public void encode(ApexStreamTuple value, OutputStream outStream, Context con new DataOutputStream(outStream).writeLong(((WatermarkTuple)value).getTimestamp()); } else { outStream.write(0); + outStream.write(((DataTuple)value).unionTag); valueCoder.encode(value.getValue(), outStream, context); } } @@ -164,7 +177,8 @@ public ApexStreamTuple decode(InputStream inStream, Context context) if (b == 1) { return new WatermarkTuple(new DataInputStream(inStream).readLong()); } else { - return new DataTuple(valueCoder.decode(inStream, context)); + int unionTag = inStream.read(); + return new DataTuple(valueCoder.decode(inStream, context), unionTag); } } From 9454b3bdc6f6ff69363dcd339cfb069c2c2f8cc9 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Sun, 16 Oct 2016 17:36:01 -0700 Subject: [PATCH 8/8] BEAM-261 Enable checkstyle and cleanup. --- runners/apex/pom.xml | 2 - .../runners/apex/ApexPipelineTranslator.java | 39 ++- .../apache/beam/runners/apex/ApexRunner.java | 314 +++--------------- .../beam/runners/apex/ApexRunnerResult.java | 23 +- .../beam/runners/apex/TestApexRunner.java | 9 +- .../beam/runners/apex/package-info.java | 22 ++ .../translators/CreateValuesTranslator.java | 12 +- .../FlattenPCollectionTranslator.java | 13 +- .../translators/GroupByKeyTranslator.java | 4 +- .../ParDoBoundMultiTranslator.java | 47 +-- .../translators/ParDoBoundTranslator.java | 5 +- .../translators/ReadUnboundedTranslator.java | 4 +- .../apex/translators/TransformTranslator.java | 8 +- .../apex/translators/TranslationContext.java | 40 ++- .../functions/ApexFlattenOperator.java | 42 ++- .../functions/ApexGroupByKeyOperator.java | 155 +++++---- .../functions/ApexParDoOperator.java | 140 ++++---- .../translators/functions/package-info.java | 22 ++ .../io/ApexReadUnboundedInputOperator.java | 57 ++-- .../apex/translators/io/ValuesSource.java | 23 +- .../apex/translators/io/package-info.java | 22 ++ .../apex/translators/package-info.java | 22 ++ .../translators/utils/ApexStreamTuple.java | 85 +++-- .../utils/CoderAdapterStreamCodec.java | 24 +- .../translators/utils/NoOpStepContext.java | 7 +- .../utils/SerializablePipelineOptions.java | 21 +- .../utils/ValueAndCoderKryoSerializable.java | 26 +- .../apex/translators/utils/package-info.java | 22 ++ .../beam/runners/apex/examples/IntTest.java | 133 -------- .../apex/examples/StreamingWordCountTest.java | 15 +- .../apex/examples/UnboundedTextSource.java | 16 +- .../runners/apex/examples/package-info.java | 22 ++ .../FlattenPCollectionTranslatorTest.java | 32 +- .../translators/GroupByKeyTranslatorTest.java | 45 ++- .../translators/ParDoBoundTranslatorTest.java | 20 +- .../ReadUnboundTranslatorTest.java | 45 ++- .../translators/utils/CollectionSource.java | 13 +- .../utils/PipelineOptionsTest.java | 28 +- .../apex/src/test/resources/log4j.properties | 8 +- 39 files changed, 662 insertions(+), 925 deletions(-) create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/package-info.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/package-info.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/package-info.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/package-info.java create mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/package-info.java delete mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java create mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/examples/package-info.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index 929feb4733c7..8b624105fad4 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -148,12 +148,10 @@ - diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java index a16f55141255..a6857eea1b3f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexPipelineTranslator.java @@ -18,6 +18,11 @@ package org.apache.beam.runners.apex; +import com.datatorrent.api.DAG; + +import java.util.HashMap; +import java.util.Map; + import org.apache.beam.runners.apex.ApexRunner.CreateApexPCollectionView; import org.apache.beam.runners.apex.translators.CreateValuesTranslator; import org.apache.beam.runners.apex.translators.FlattenPCollectionTranslator; @@ -43,18 +48,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; - /** * {@link ApexPipelineTranslator} translates {@link Pipeline} objects * into Apex logical plan {@link DAG}. */ @SuppressWarnings({"rawtypes", "unchecked"}) public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { - - private static final Logger LOG = LoggerFactory.getLogger( - ApexPipelineTranslator.class); + private static final Logger LOG = LoggerFactory.getLogger(ApexPipelineTranslator.class); /** * A map from {@link PTransform} subclass to the corresponding @@ -75,8 +75,10 @@ public class ApexPipelineTranslator implements Pipeline.PipelineVisitor { registerTransformTranslator(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); - registerTransformTranslator(CreateApexPCollectionView.class, new CreateApexPCollectionViewTranslator()); - registerTransformTranslator(CreatePCollectionView.class, new CreatePCollectionViewTranslator()); + registerTransformTranslator(CreateApexPCollectionView.class, + new CreateApexPCollectionViewTranslator()); + registerTransformTranslator(CreatePCollectionView.class, + new CreatePCollectionViewTranslator()); } public ApexPipelineTranslator(TranslationContext translationContext) { @@ -134,7 +136,7 @@ private static void registerTransformTranslator( * Returns the {@link TransformTranslator} to use for instances of the * specified PTransform class, or null if none registered. */ - private > + private > TransformTranslator getTransformTranslator(Class transformClass) { return transformTranslators.get(transformClass); } @@ -145,7 +147,8 @@ private static class ReadBoundedTranslator implements TransformTranslator transform, TranslationContext context) { // TODO: adapter is visibleForTesting - BoundedToUnboundedSourceAdapter unboundedSource = new BoundedToUnboundedSourceAdapter<>(transform.getSource()); + BoundedToUnboundedSourceAdapter unboundedSource = new BoundedToUnboundedSourceAdapter<>( + transform.getSource()); ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>( unboundedSource, context.getPipelineOptions()); context.addOperator(operator, operator.output); @@ -153,26 +156,26 @@ public void translate(Read.Bounded transform, TranslationContext context) { } - private static class CreateApexPCollectionViewTranslator implements TransformTranslator> - { + private static class CreateApexPCollectionViewTranslator + implements TransformTranslator> { private static final long serialVersionUID = 1L; @Override - public void translate(CreateApexPCollectionView transform, TranslationContext context) - { + public void translate(CreateApexPCollectionView transform, + TranslationContext context) { PCollectionView view = transform.getView(); context.addView(view); LOG.debug("view {}", view.getName()); } } - private static class CreatePCollectionViewTranslator implements TransformTranslator> - { + private static class CreatePCollectionViewTranslator + implements TransformTranslator> { private static final long serialVersionUID = 1L; @Override - public void translate(CreatePCollectionView transform, TranslationContext context) - { + public void translate(CreatePCollectionView transform, + TranslationContext context) { PCollectionView view = transform.getView(); context.addView(view); LOG.debug("view {}", view.getName()); 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 667f1c806377..f3c44bb0dbde 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 @@ -19,17 +19,18 @@ import static com.google.common.base.Preconditions.checkArgument; -import java.util.ArrayList; +import com.datatorrent.api.Context.DAGContext; +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; +import com.datatorrent.api.StreamingApplication; +import com.google.common.base.Throwables; + import java.util.Arrays; import java.util.List; -import java.util.Map; import org.apache.beam.runners.apex.translators.TranslationContext; +import org.apache.beam.runners.core.AssignWindows; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; -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.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Combine; @@ -39,31 +40,22 @@ 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.OldDoFn.ProcessContext; 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.runners.core.AssignWindows; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.hadoop.conf.Configuration; -import com.datatorrent.api.Context.DAGContext; -import com.datatorrent.api.DAG; -import com.datatorrent.api.LocalMode; -import com.datatorrent.api.StreamingApplication; -import com.google.common.base.Throwables; - /** * A {@link PipelineRunner} that translates the * pipeline to an Apex DAG and executes it on an Apex cluster. - *

- * Currently execution is always in embedded mode, + * + *

Currently execution is always in embedded mode, * launch on Hadoop cluster will be added in subsequent iteration. */ @SuppressWarnings({"rawtypes", "unchecked"}) @@ -99,37 +91,16 @@ public OutputT apply( input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.BOUNDED); -// TODO: replace this with a mapping -//// - } else if (Combine.GloballyAsSingletonView.class.equals(transform.getClass())) { - PTransform customTransform = (PTransform)new StreamingCombineGloballyAsSingletonView(this, - (Combine.GloballyAsSingletonView)transform); + PTransform customTransform = (PTransform) + new StreamingCombineGloballyAsSingletonView( + this, (Combine.GloballyAsSingletonView) transform); return Pipeline.applyTransform(input, customTransform); } else if (View.AsSingleton.class.equals(transform.getClass())) { - // note this assumes presence of above Combine.GloballyAsSingletonView mapping - PTransform customTransform = (PTransform)new StreamingViewAsSingleton(this, - (View.AsSingleton)transform); + // assumes presence of above Combine.GloballyAsSingletonView mapping + PTransform customTransform = (PTransform) + new StreamingViewAsSingleton(this, (View.AsSingleton) transform); return Pipeline.applyTransform(input, customTransform); -/* - } else if (View.AsIterable.class.equals(transform.getClass())) { - PTransform customTransform = (PTransform)new StreamingViewAsIterable(this, - (View.AsIterable)transform); - return Pipeline.applyTransform(input, customTransform); - } else if (View.AsList.class.equals(transform.getClass())) { - PTransform customTransform = (PTransform)new StreamingViewAsList(this, - (View.AsList)transform); - return Pipeline.applyTransform(input, customTransform); - } else if (View.AsMap.class.equals(transform.getClass())) { - PTransform customTransform = new StreamingViewAsMap(this, - (View.AsMap)transform); - return Pipeline.applyTransform(input, customTransform); - } else if (View.AsMultimap.class.equals(transform.getClass())) { - PTransform customTransform = new StreamingViewAsMultimap(this, - (View.AsMultimap)transform); - return Pipeline.applyTransform(input, customTransform); -*/ -//// } else { return super.apply(transform, input); } @@ -142,17 +113,16 @@ public ApexRunnerResult run(Pipeline pipeline) { ApexPipelineTranslator translator = new ApexPipelineTranslator(translationContext); translator.translate(pipeline); - StreamingApplication apexApp = new StreamingApplication() - { + StreamingApplication apexApp = new StreamingApplication() { @Override - public void populateDAG(DAG dag, Configuration conf) - { + public void populateDAG(DAG dag, Configuration conf) { dag.setAttribute(DAGContext.APPLICATION_NAME, options.getApplicationName()); translationContext.populateDAG(dag); } }; - checkArgument(options.isEmbeddedExecution(), "only embedded execution is supported at this time"); + checkArgument(options.isEmbeddedExecution(), + "only embedded execution is supported at this time"); LocalMode lma = LocalMode.newInstance(); Configuration conf = new Configuration(false); try { @@ -178,7 +148,8 @@ public void populateDAG(DAG dag, Configuration conf) } return new ApexRunnerResult(lma.getDAG(), lc); } catch (Exception e) { - throw Throwables.propagate(e); + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); } } @@ -230,13 +201,6 @@ public void processElement(ProcessContext c) { //////////////////////////////////////////// // Adapted from FlinkRunner for View support - /** - * Records that the {@link PTransform} requires a deterministic key coder. - */ - private void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { - //throw new UnsupportedOperationException(); - } - /** * Creates a primitive {@link PCollectionView}. * @@ -247,6 +211,7 @@ private void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) */ public static class CreateApexPCollectionView extends PTransform>, PCollectionView> { + private static final long serialVersionUID = 1L; private PCollectionView view; private CreateApexPCollectionView(PCollectionView view) { @@ -276,52 +241,50 @@ public void processElement(ProcessContext c) { } private static class StreamingCombineGloballyAsSingletonView - extends PTransform, PCollectionView> - { + extends PTransform, PCollectionView> { + private static final long serialVersionUID = 1L; Combine.GloballyAsSingletonView transform; /** * Builds an instance of this class from the overridden transform. */ public StreamingCombineGloballyAsSingletonView(ApexRunner runner, - Combine.GloballyAsSingletonView transform) - { + Combine.GloballyAsSingletonView transform) { this.transform = transform; } @Override - public PCollectionView apply(PCollection input) - { + public PCollectionView apply(PCollection input) { PCollection combined = input - .apply(Combine.globally(transform.getCombineFn()).withoutDefaults().withFanout(transform.getFanout())); + .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()); + transform.getInsertDefault() ? transform.getCombineFn().defaultValue() : null, + combined.getCoder()); return combined.apply(ParDo.of(new WrapAsList())) .apply(CreateApexPCollectionView. of(view)); } @Override - protected String getKindString() - { + protected String getKindString() { return "StreamingCombineGloballyAsSingletonView"; } } - private static class StreamingViewAsSingleton extends PTransform, PCollectionView> - { + private static class StreamingViewAsSingleton + extends PTransform, PCollectionView> { private static final long serialVersionUID = 1L; + private View.AsSingleton transform; - public StreamingViewAsSingleton(ApexRunner runner, View.AsSingleton transform) - { + public StreamingViewAsSingleton(ApexRunner runner, View.AsSingleton transform) { this.transform = transform; } @Override - public PCollectionView apply(PCollection input) - { + public PCollectionView apply(PCollection input) { Combine.Globally combine = Combine .globally(new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); if (!transform.hasDefaultValue()) { @@ -331,33 +294,28 @@ public PCollectionView apply(PCollection input) } @Override - protected String getKindString() - { + protected String getKindString() { return "StreamingViewAsSingleton"; } - private static class SingletonCombine extends Combine.BinaryCombineFn - { + private static class SingletonCombine extends Combine.BinaryCombineFn { private boolean hasDefaultValue; private T defaultValue; - SingletonCombine(boolean hasDefaultValue, T defaultValue) - { + SingletonCombine(boolean hasDefaultValue, T defaultValue) { this.hasDefaultValue = hasDefaultValue; this.defaultValue = defaultValue; } @Override - public T apply(T left, T right) - { + 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() - { + public T identity() { if (hasDefaultValue) { return defaultValue; } else { @@ -368,194 +326,4 @@ public T identity() } } - private static class StreamingViewAsMap - extends PTransform>, PCollectionView>> { - - private final ApexRunner runner; - - public StreamingViewAsMap(ApexRunner runner, View.AsMap transform) { - this.runner = runner; - } - - @Override - public PCollectionView> apply(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) { - runner.recordViewUsesNonDeterministicKeyCoder(this); - } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(CreateApexPCollectionView., Map>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMap"; - } - } - - /** - * Specialized expansion for {@link - * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the - * Flink runner in streaming mode. - */ - private static class StreamingViewAsMultimap - extends PTransform>, PCollectionView>>> { - - private final ApexRunner runner; - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsMultimap(ApexRunner runner, View.AsMultimap transform) { - this.runner = runner; - } - - @Override - public PCollectionView>> apply(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) { - runner.recordViewUsesNonDeterministicKeyCoder(this); - } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(CreateApexPCollectionView., Map>>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMultimap"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the - * Flink runner in streaming mode. - */ - private static class StreamingViewAsList - extends PTransform, PCollectionView>> { - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsList(ApexRunner runner, View.AsList transform) {} - - @Override - public PCollectionView> apply(PCollection input) { - PCollectionView> view = - PCollectionViews.listView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(CreateApexPCollectionView.>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsList"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the - * Flink runner in streaming mode. - */ - private static class StreamingViewAsIterable - extends PTransform, PCollectionView>> { - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsIterable(ApexRunner runner, View.AsIterable transform) { } - - @Override - public PCollectionView> apply(PCollection input) { - PCollectionView> view = - PCollectionViews.iterableView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(CreateApexPCollectionView.>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsIterable"; - } - } - - /** - * 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/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java index 68176844290f..d5613fe88303 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/ApexRunnerResult.java @@ -17,20 +17,19 @@ */ package org.apache.beam.runners.apex; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.metrics.MetricResults; +import com.datatorrent.api.DAG; +import com.datatorrent.api.LocalMode; 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.metrics.MetricResults; import org.apache.beam.sdk.transforms.Aggregator; import org.joda.time.Duration; -import com.datatorrent.api.DAG; -import com.datatorrent.api.LocalMode; - /** * Result of executing a {@link Pipeline} with Apex in embedded mode. */ @@ -56,28 +55,24 @@ public AggregatorValues getAggregatorValues(Aggregator aggregator) } @Override - public State cancel() throws IOException - { + public State cancel() throws IOException { ctrl.shutdown(); state = State.CANCELLED; return state; } @Override - public State waitUntilFinish(Duration duration) throws IOException, InterruptedException - { + public State waitUntilFinish(Duration duration) throws IOException, InterruptedException { throw new UnsupportedOperationException(); } @Override - public State waitUntilFinish() throws IOException, InterruptedException - { + public State waitUntilFinish() throws IOException, InterruptedException { throw new UnsupportedOperationException(); } @Override - public MetricResults metrics() - { + public MetricResults metrics() { throw new UnsupportedOperationException(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java index 45c143e32074..2e048f0cc961 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/TestApexRunner.java @@ -25,7 +25,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - +/** + * Apex {@link PipelineRunner} for testing. + */ public class TestApexRunner extends PipelineRunner { private ApexRunner delegate; @@ -38,13 +40,14 @@ private TestApexRunner(ApexPipelineOptions options) { } public static TestApexRunner fromOptions(PipelineOptions options) { - ApexPipelineOptions apexOptions = PipelineOptionsValidator.validate(ApexPipelineOptions.class, options); + ApexPipelineOptions apexOptions = PipelineOptionsValidator + .validate(ApexPipelineOptions.class, options); return new TestApexRunner(apexOptions); } @Override public - OutputT apply(PTransform transform, InputT input) { + OutputT apply(PTransform transform, InputT input) { return delegate.apply(transform, input); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/package-info.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/package-info.java new file mode 100644 index 000000000000..4d2f417161db --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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. + */ + +/** + * Implementation of the Beam runner for Apache Apex. + */ +package org.apache.beam.runners.apex; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java index 7a29057272a6..539f31162f5f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/CreateValuesTranslator.java @@ -25,12 +25,10 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PBegin; -import com.google.common.base.Throwables; - /** * Wraps elements from Create.Values into an {@link UnboundedSource}. - * mainly used for test + * mainly used for testing */ public class CreateValuesTranslator implements TransformTranslator> { private static final long serialVersionUID = 1451000241832745629L; @@ -39,12 +37,12 @@ public class CreateValuesTranslator implements TransformTranslator transform, TranslationContext context) { try { UnboundedSource unboundedSource = new ValuesSource<>(transform.getElements(), - transform.getDefaultOutputCoder((PBegin)context.getInput())); - ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>(unboundedSource, - context.getPipelineOptions()); + transform.getDefaultOutputCoder((PBegin) context.getInput())); + ApexReadUnboundedInputOperator operator = new ApexReadUnboundedInputOperator<>( + unboundedSource, context.getPipelineOptions()); context.addOperator(operator, operator.output); } catch (CannotProvideCoderException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java index 673776707e77..a39aacb130c9 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslator.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.apex.translators; +import com.google.common.collect.Lists; + import java.util.Collections; import java.util.List; import java.util.Map; @@ -32,8 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import com.google.common.collect.Lists; - /** * {@link Flatten.FlattenPCollectionList} translation to Apex operator. */ @@ -72,7 +72,8 @@ public void translate(Flatten.FlattenPCollectionList transform, TranslationCo * @param finalCollection * @param context */ - static void flattenCollections(List> collections, Map, Integer> unionTags, PCollection finalCollection, TranslationContext context) { + static void flattenCollections(List> collections, Map, + Integer> unionTags, PCollection finalCollection, TranslationContext context) { List> remainingCollections = Lists.newArrayList(); PCollection firstCollection = null; while (!collections.isEmpty()) { @@ -93,7 +94,8 @@ static void flattenCollections(List> collections, Map 2) { - PCollection intermediateCollection = intermediateCollection(collection, collection.getCoder()); + PCollection intermediateCollection = intermediateCollection(collection, + collection.getCoder()); context.addOperator(operator, operator.out, intermediateCollection); remainingCollections.add(intermediateCollection); } else { @@ -118,7 +120,8 @@ static void flattenCollections(List> collections, Map PCollection intermediateCollection(PCollection input, Coder outputCoder) { - PCollection output = PCollection.createPrimitiveOutputInternal(input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + 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/translators/GroupByKeyTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.java index 43c82a90745b..d3e7d2de6211 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslator.java @@ -31,9 +31,9 @@ public class GroupByKeyTranslator implements TransformTranslator transform, TranslationContext context) { - PCollection> input = context.getInput(); - ApexGroupByKeyOperator group = new ApexGroupByKeyOperator<>(context.getPipelineOptions(), input); + ApexGroupByKeyOperator group = new ApexGroupByKeyOperator<>(context.getPipelineOptions(), + input); context.addOperator(group, group.output); context.addStream(input, group.input); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java index a229a8191e3c..13f07c1b1140 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundMultiTranslator.java @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.datatorrent.api.Operator; +import com.datatorrent.api.Operator.OutputPort; +import com.google.common.collect.Maps; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -27,6 +31,7 @@ import org.apache.beam.runners.apex.translators.functions.ApexParDoOperator; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; @@ -35,16 +40,16 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.datatorrent.api.Operator; -import com.datatorrent.api.Operator.OutputPort; -import com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * {@link ParDo.BoundMulti} is translated to Apex operator that wraps the {@link DoFn} + * {@link ParDo.BoundMulti} is translated to {@link ApexParDoOperator} that wraps the {@link DoFn}. */ -public class ParDoBoundMultiTranslator implements TransformTranslator> { +public class ParDoBoundMultiTranslator + implements TransformTranslator> { private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslator.class); @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { @@ -56,7 +61,8 @@ public void translate(ParDo.BoundMulti transform, TranslationCo WindowedValueCoder wvInputCoder = FullWindowedValueCoder.of(inputCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); - ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), + ApexParDoOperator operator = new ApexParDoOperator<>( + context.getPipelineOptions(), doFn, transform.getMainOutputTag(), transform.getSideOutputTags().getAll(), context.>getInput().getWindowingStrategy(), sideInputs, wvInputCoder); @@ -73,36 +79,37 @@ public void translate(ParDo.BoundMulti transform, TranslationCo } } - static void addSideInputs(ApexParDoOperator operator, List> sideInputs, TranslationContext context) { + static void addSideInputs(ApexParDoOperator operator, List> sideInputs, + TranslationContext context) { Operator.InputPort[] sideInputPorts = {operator.sideInput1}; if (sideInputs.size() > sideInputPorts.length) { - // String msg = String.format("Too many side inputs in %s (currently only supporting %s).", - // transform.toString(), sideInputPorts.length); - // throw new UnsupportedOperationException(msg); PCollection unionCollection = unionSideInputs(sideInputs, context); context.addStream(unionCollection, sideInputPorts[0]); } else { - for (int i=0; i unionSideInputs(List> sideInputs, TranslationContext context) { + private static PCollection unionSideInputs(List> sideInputs, + TranslationContext context) { checkArgument(sideInputs.size() > 1, "requires multiple side inputs"); // flatten and assign union tag List> sourceCollections = new ArrayList<>(); Map, Integer> unionTags = new HashMap<>(); PCollection firstSideInput = context.getViewInput(sideInputs.get(0)); - for (int i=0; i < sideInputs.size(); i++) { + for (int i = 0; i < sideInputs.size(); i++) { PCollectionView sideInput = sideInputs.get(i); PCollection sideInputCollection = context.getViewInput(sideInput); - if (!sideInputCollection.getWindowingStrategy().equals(firstSideInput.getWindowingStrategy())) { + if (!sideInputCollection.getWindowingStrategy().equals( + firstSideInput.getWindowingStrategy())) { // TODO: check how to handle this in stream codec //String msg = "Multiple side inputs with different window strategies."; //throw new UnsupportedOperationException(msg); + LOG.warn("Side inputs union with different windowing strategies {} {}", + firstSideInput.getWindowingStrategy(), sideInputCollection.getWindowingStrategy()); } if (!sideInputCollection.getCoder().equals(firstSideInput.getCoder())) { String msg = "Multiple side inputs with different coders."; @@ -112,8 +119,10 @@ private static PCollection unionSideInputs(List> sideInput unionTags.put(sideInputCollection, i); } - PCollection resultCollection = FlattenPCollectionTranslator.intermediateCollection(firstSideInput, firstSideInput.getCoder()); - FlattenPCollectionTranslator.flattenCollections(sourceCollections, unionTags, resultCollection, context); + PCollection resultCollection = FlattenPCollectionTranslator.intermediateCollection( + firstSideInput, firstSideInput.getCoder()); + FlattenPCollectionTranslator.flattenCollections(sourceCollections, unionTags, resultCollection, + context); return resultCollection; } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java index 7749a0635ac8..bd7115e431cb 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslator.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.values.TupleTagList; /** - * {@link ParDo.Bound} is translated to Apex operator that wraps the {@link DoFn} + * {@link ParDo.Bound} is translated to {link ApexParDoOperator} that wraps the {@link DoFn}. */ public class ParDoBoundTranslator implements TransformTranslator> { @@ -49,7 +49,8 @@ public void translate(ParDo.Bound transform, TranslationContext WindowedValueCoder wvInputCoder = FullWindowedValueCoder.of(inputCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); - ApexParDoOperator operator = new ApexParDoOperator<>(context.getPipelineOptions(), + ApexParDoOperator operator = new ApexParDoOperator<>( + context.getPipelineOptions(), doFn, new TupleTag(), TupleTagList.empty().getAll() /*sideOutputTags*/, output.getWindowingStrategy(), sideInputs, wvInputCoder); context.addOperator(operator, operator.output); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java index b53e4ddee721..30972767f772 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/ReadUnboundedTranslator.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.apex.translators; +import com.datatorrent.api.InputOperator; + import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.UnboundedSource; -import com.datatorrent.api.InputOperator; - /** * {@link Read.Unbounded} is translated to Apex {@link InputOperator} * that wraps {@link UnboundedSource}. diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java index 1a99885d1f9f..dfd20450b9ae 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TransformTranslator.java @@ -19,13 +19,13 @@ package org.apache.beam.runners.apex.translators; -import org.apache.beam.sdk.transforms.PTransform; - import java.io.Serializable; +import org.apache.beam.sdk.transforms.PTransform; + /** - * translates {@link PTransform} to Apex functions. + * Translates {@link PTransform} to Apex functions. */ -public interface TransformTranslator> extends Serializable { +public interface TransformTranslator> extends Serializable { void translate(T transform, TranslationContext context); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java index bd44a20a911d..ddacc29d659b 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/TranslationContext.java @@ -19,6 +19,17 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.datatorrent.api.Context.PortContext; +import com.datatorrent.api.DAG; +import com.datatorrent.api.Operator; +import com.datatorrent.api.Operator.InputPort; +import com.datatorrent.api.Operator.OutputPort; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.CoderAdapterStreamCodec; @@ -34,17 +45,6 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import com.datatorrent.api.DAG; -import com.datatorrent.api.Operator; -import com.datatorrent.api.Context.PortContext; -import com.datatorrent.api.Operator.InputPort; -import com.datatorrent.api.Operator.OutputPort; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Maintains context data for {@link TransformTranslator}s. */ @@ -64,7 +64,7 @@ public void addView(PCollectionView view) { public InputT getViewInput(PCollectionView view) { PInput input = this.viewInputs.get(view); checkArgument(input != null, "unknown view " + view.getName()); - return (InputT)input; + return (InputT) input; } public TranslationContext(ApexPipelineOptions pipelineOptions) { @@ -109,13 +109,14 @@ public void addOperator(Operator operator, Map, OutputPort> po addOperator(operator, portEntry.getValue(), portEntry.getKey()); first = false; } else { - this.streams.put(portEntry.getKey(), (Pair)new ImmutablePair<>(portEntry.getValue(), new ArrayList<>())); + this.streams.put(portEntry.getKey(), (Pair) new ImmutablePair<>(portEntry.getValue(), + new ArrayList<>())); } } } /** - * Add intermediate operator for the current transformation. + * Add the operator with its output port for the given result {link PCollection}. * @param operator * @param port * @param output @@ -124,9 +125,11 @@ public void addOperator(Operator operator, OutputPort port, PCollection output) // Apex DAG requires a unique operator name // use the transform's name and make it unique String name = getCurrentTransform().getFullName(); - for (int i=1; this.operators.containsKey(name); name = getCurrentTransform().getFullName() + i++); + for (int i = 1; this.operators.containsKey(name); i++) { + name = getCurrentTransform().getFullName() + i; + } this.operators.put(name, operator); - this.streams.put(output, (Pair)new ImmutablePair<>(port, new ArrayList<>())); + this.streams.put(output, (Pair) new ImmutablePair<>(port, new ArrayList<>())); } public void addStream(PInput input, InputPort inputPort) { @@ -140,11 +143,12 @@ public void populateDAG(DAG dag) { dag.addOperator(nameAndOperator.getKey(), nameAndOperator.getValue()); } int streamIndex = 0; - for (Map.Entry, List>>> streamEntry : this.streams.entrySet()) { + for (Map.Entry, List>>> streamEntry : this. + streams.entrySet()) { List> sinksList = streamEntry.getValue().getRight(); InputPort[] sinks = sinksList.toArray(new InputPort[sinksList.size()]); if (sinks.length > 0) { - dag.addStream("stream"+streamIndex++, streamEntry.getValue().getLeft(), sinks); + dag.addStream("stream" + streamIndex++, streamEntry.getValue().getLeft(), sinks); for (InputPort port : sinks) { PCollection pc = streamEntry.getKey(); Coder coder = pc.getCoder(); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java index 202f2d397cea..dd8fcd168724 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexFlattenOperator.java @@ -17,23 +17,22 @@ */ package org.apache.beam.runners.apex.translators.functions; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; +import com.datatorrent.common.util.BaseOperator; + import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple.WatermarkTuple; -import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.util.WindowedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.datatorrent.api.DefaultInputPort; -import com.datatorrent.api.DefaultOutputPort; -import com.datatorrent.api.annotation.OutputPortFieldAnnotation; -import com.datatorrent.common.util.BaseOperator; - /** * Apex operator for Beam {@link Flatten.FlattenPCollectionList}. */ -public class ApexFlattenOperator extends BaseOperator -{ +public class ApexFlattenOperator extends BaseOperator { + private static final Logger LOG = LoggerFactory.getLogger(ApexFlattenOperator.class); private boolean traceTuples = true; @@ -47,16 +46,15 @@ public class ApexFlattenOperator extends BaseOperator /** * Data input port 1. */ - public final transient DefaultInputPort>> data1 = new DefaultInputPort>>() - { + public final transient DefaultInputPort>> data1 = + new DefaultInputPort>>() { /** * Emits to port "out" */ @Override - public void process(ApexStreamTuple> tuple) - { + public void process(ApexStreamTuple> tuple) { if (tuple instanceof WatermarkTuple) { - WatermarkTuple wmTuple = (WatermarkTuple)tuple; + WatermarkTuple wmTuple = (WatermarkTuple) tuple; if (wmTuple.getTimestamp() > inputWM1) { inputWM1 = wmTuple.getTimestamp(); if (inputWM1 <= inputWM2) { @@ -75,7 +73,7 @@ public void process(ApexStreamTuple> tuple) } if (data1Tag > 0 && tuple instanceof ApexStreamTuple.DataTuple) { - ((ApexStreamTuple.DataTuple)tuple).setUnionTag(data1Tag); + ((ApexStreamTuple.DataTuple) tuple).setUnionTag(data1Tag); } out.emit(tuple); } @@ -84,16 +82,15 @@ public void process(ApexStreamTuple> tuple) /** * Data input port 2. */ - public final transient DefaultInputPort>> data2 = new DefaultInputPort>>() - { + public final transient DefaultInputPort>> data2 = + new DefaultInputPort>>() { /** * Emits to port "out" */ @Override - public void process(ApexStreamTuple> tuple) - { + public void process(ApexStreamTuple> tuple) { if (tuple instanceof WatermarkTuple) { - WatermarkTuple wmTuple = (WatermarkTuple)tuple; + WatermarkTuple wmTuple = (WatermarkTuple) tuple; if (wmTuple.getTimestamp() > inputWM2) { inputWM2 = wmTuple.getTimestamp(); if (inputWM2 <= inputWM1) { @@ -112,7 +109,7 @@ public void process(ApexStreamTuple> tuple) } if (data2Tag > 0 && tuple instanceof ApexStreamTuple.DataTuple) { - ((ApexStreamTuple.DataTuple)tuple).setUnionTag(data2Tag); + ((ApexStreamTuple.DataTuple) tuple).setUnionTag(data2Tag); } out.emit(tuple); } @@ -121,6 +118,7 @@ public void process(ApexStreamTuple> tuple) /** * Output port. */ - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort>> out = new DefaultOutputPort>>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort>> out = + new DefaultOutputPort>>(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java index 5970f3627c65..845618dc2e16 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexGroupByKeyOperator.java @@ -17,6 +17,20 @@ */ package org.apache.beam.runners.apex.translators.functions; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.Operator; +import com.datatorrent.api.StreamCodec; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.base.Throwables; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + import java.io.IOException; import java.io.Serializable; import java.nio.ByteBuffer; @@ -61,19 +75,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.datatorrent.api.Context.OperatorContext; -import com.datatorrent.api.DefaultInputPort; -import com.datatorrent.api.DefaultOutputPort; -import com.datatorrent.api.Operator; -import com.datatorrent.api.StreamCodec; -import com.datatorrent.api.annotation.OutputPortFieldAnnotation; -import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; -import com.esotericsoftware.kryo.serializers.JavaSerializer; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; - /** * Apex operator for Beam {@link GroupByKey}. * This operator expects the input stream already partitioned by K, @@ -82,8 +83,7 @@ * @param * @param */ -public class ApexGroupByKeyOperator implements Operator -{ +public class ApexGroupByKeyOperator implements Operator { private static final Logger LOG = LoggerFactory.getLogger(ApexGroupByKeyOperator.class); private boolean traceTuples = true; @@ -98,7 +98,7 @@ public class ApexGroupByKeyOperator implements Operator private final SerializablePipelineOptions serializedOptions; @Bind(JavaSerializer.class) // TODO: InMemoryStateInternals not serializable -transient private Map> perKeyStateInternals = new HashMap<>(); + private transient Map> perKeyStateInternals = new HashMap<>(); private Map> activeTimers = new HashMap<>(); private transient ProcessContext context; @@ -106,19 +106,19 @@ public class ApexGroupByKeyOperator implements Operator private transient ApexTimerInternals timerInternals = new ApexTimerInternals(); private Instant inputWatermark = new Instant(0); - public final transient DefaultInputPort>>> input = new DefaultInputPort>>>() - { + public final transient DefaultInputPort>>> input = + new DefaultInputPort>>>() { @Override - public void process(ApexStreamTuple>> t) - { + public void process(ApexStreamTuple>> t) { try { if (t instanceof ApexStreamTuple.WatermarkTuple) { - ApexStreamTuple.WatermarkTuple mark = (ApexStreamTuple.WatermarkTuple)t; + ApexStreamTuple.WatermarkTuple mark = (ApexStreamTuple.WatermarkTuple) t; processWatermark(mark); if (traceTuples) { LOG.debug("\nemitting watermark {}\n", mark.getTimestamp()); } - output.emit(ApexStreamTuple.WatermarkTuple.>>>of(mark.getTimestamp())); + output.emit(ApexStreamTuple.WatermarkTuple.>>>of( + mark.getTimestamp())); return; } if (traceTuples) { @@ -126,53 +126,49 @@ public void process(ApexStreamTuple>> t) } processElement(t.getValue()); } catch (Exception e) { - Throwables.propagate(e); + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); } } }; - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort>>>> output = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort>>>> + output = new DefaultOutputPort<>(); @SuppressWarnings("unchecked") - public ApexGroupByKeyOperator(ApexPipelineOptions pipelineOptions, PCollection> input) - { - Preconditions.checkNotNull(pipelineOptions); + public ApexGroupByKeyOperator(ApexPipelineOptions pipelineOptions, PCollection> input) { + checkNotNull(pipelineOptions); this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); - this.windowingStrategy = (WindowingStrategy)input.getWindowingStrategy(); - this.keyCoder = ((KvCoder)input.getCoder()).getKeyCoder(); - this.valueCoder = ((KvCoder)input.getCoder()).getValueCoder(); + this.windowingStrategy = (WindowingStrategy) input.getWindowingStrategy(); + this.keyCoder = ((KvCoder) input.getCoder()).getKeyCoder(); + this.valueCoder = ((KvCoder) input.getCoder()).getValueCoder(); } @SuppressWarnings("unused") // for Kryo - private ApexGroupByKeyOperator() - { + private ApexGroupByKeyOperator() { this.serializedOptions = null; } @Override - public void beginWindow(long l) - { + public void beginWindow(long l) { } @Override - public void endWindow() - { + public void endWindow() { } @Override - public void setup(OperatorContext context) - { + public void setup(OperatorContext context) { this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(serializedOptions.get(), this); StateInternalsFactory stateInternalsFactory = new GroupByKeyStateInternalsFactory(); - this.fn = GroupAlsoByWindowViaWindowSetDoFn.create(this.windowingStrategy, stateInternalsFactory, - SystemReduceFn.buffering(this.valueCoder)); + this.fn = GroupAlsoByWindowViaWindowSetDoFn.create(this.windowingStrategy, + stateInternalsFactory, SystemReduceFn.buffering(this.valueCoder)); this.context = new ProcessContext(fn, this.timerInternals); } @Override - public void teardown() - { + public void teardown() { } /** @@ -181,14 +177,16 @@ public void teardown() * We keep these timers in a Set, so that they are deduplicated, as the same * timer can be registered multiple times. */ - private Multimap getTimersReadyToProcess(long currentWatermark) { + private Multimap getTimersReadyToProcess( + long currentWatermark) { // we keep the timers to return in a different list and launch them later // because we cannot prevent a trigger from registering another trigger, // which would lead to concurrent modification exception. Multimap toFire = HashMultimap.create(); - Iterator>> it = activeTimers.entrySet().iterator(); + Iterator>> it = + activeTimers.entrySet().iterator(); while (it.hasNext()) { Map.Entry> keyWithTimers = it.next(); @@ -223,18 +221,15 @@ private void processElement(WindowedValue> windowedValue) throws Except fn.processElement(context); } - private StateInternals getStateInternalsForKey(K key) - { + private StateInternals getStateInternalsForKey(K key) { final ByteBuffer keyBytes; try { keyBytes = ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, key)); } catch (CoderException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } StateInternals stateInternals = perKeyStateInternals.get(keyBytes); if (stateInternals == null) { - //Coder windowCoder = this.windowingStrategy.getWindowFn().windowCoder(); - //OutputTimeFn outputTimeFn = this.windowingStrategy.getOutputTimeFn(); stateInternals = InMemoryStateInternals.forKey(key); perKeyStateInternals.put(keyBytes, stateInternals); } @@ -246,7 +241,7 @@ private void registerActiveTimer(K key, TimerInternals.TimerData timer) { try { keyBytes = ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, key)); } catch (CoderException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } Set timersForKey = activeTimers.get(keyBytes); if (timersForKey == null) { @@ -261,7 +256,7 @@ private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { try { keyBytes = ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, key)); } catch (CoderException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } Set timersForKey = activeTimers.get(keyBytes); if (timersForKey != null) { @@ -276,7 +271,8 @@ private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) { private void processWatermark(ApexStreamTuple.WatermarkTuple mark) throws Exception { this.inputWatermark = new Instant(mark.getTimestamp()); - Multimap timers = getTimersReadyToProcess(mark.getTimestamp()); + Multimap timers = getTimersReadyToProcess( + mark.getTimestamp()); if (!timers.isEmpty()) { for (ByteBuffer keyBytes : timers.keySet()) { K key = CoderUtils.decodeFromByteArray(keyCoder, keyBytes.array()); @@ -287,7 +283,8 @@ private void processWatermark(ApexStreamTuple.WatermarkTuple mark) throws Exc } } - private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn, ?, KeyedWorkItem>.ProcessContext { + private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn, ?, + KeyedWorkItem>.ProcessContext { private final ApexTimerInternals timerInternals; private StateInternals stateInternals; @@ -296,7 +293,7 @@ private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn, KV>> function, ApexTimerInternals timerInternals) { function.super(); - this.timerInternals = Preconditions.checkNotNull(timerInternals); + this.timerInternals = checkNotNull(timerInternals); } public void setElement(KeyedWorkItem element, StateInternals stateForKey) { @@ -311,7 +308,8 @@ public KeyedWorkItem element() { @Override public Instant timestamp() { - throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems."); + throw new UnsupportedOperationException( + "timestamp() is not available when processing KeyedWorkItems."); } @Override @@ -333,7 +331,8 @@ public void outputWithTimestamp(KV> output, Instant timestamp) { @Override public PaneInfo pane() { - throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems."); + throw new UnsupportedOperationException( + "pane() is not available when processing KeyedWorkItems."); } @Override @@ -352,11 +351,13 @@ public StateInternals stateInternals() { } @Override - public void outputWindowedValue(KV> output, Instant timestamp, Collection windows, PaneInfo pane) { + public void outputWindowedValue(KV> output, Instant timestamp, + Collection windows, PaneInfo pane) { if (traceTuples) { LOG.debug("\nemitting {} timestamp {}\n", output, timestamp); } - ApexGroupByKeyOperator.this.output.emit(ApexStreamTuple.DataTuple.of(WindowedValue.of(output, timestamp, windows, pane))); + ApexGroupByKeyOperator.this.output.emit(ApexStreamTuple.DataTuple.of( + WindowedValue.of(output, timestamp, windows, pane))); } @Override @@ -375,7 +376,8 @@ public PaneInfo pane() { } @Override - public void writePCollectionViewData(TupleTag tag, Iterable> data, Coder elemCoder) throws IOException { + public void writePCollectionViewData(TupleTag tag, Iterable> data, + Coder elemCoder) throws IOException { throw new RuntimeException("writePCollectionViewData() not available in Streaming mode."); } @@ -404,7 +406,8 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times } @Override - protected Aggregator createAggregatorInternal(String name, Combine.CombineFn combiner) { + protected Aggregator createAggregatorInternal( + String name, Combine.CombineFn combiner) { throw new UnsupportedOperationException(); } } @@ -416,52 +419,44 @@ protected Aggregator createAggreg public class ApexTimerInternals implements TimerInternals { @Override - public void setTimer(TimerData timerKey) - { + public void setTimer(TimerData timerKey) { registerActiveTimer(context.element().key(), timerKey); } @Override - public void deleteTimer(TimerData timerKey) - { + public void deleteTimer(TimerData timerKey) { unregisterActiveTimer(context.element().key(), timerKey); } @Override - public Instant currentProcessingTime() - { + public Instant currentProcessingTime() { return Instant.now(); } @Override - public Instant currentSynchronizedProcessingTime() - { + public Instant currentSynchronizedProcessingTime() { // TODO Auto-generated method stub return null; } @Override - public Instant currentInputWatermarkTime() - { + public Instant currentInputWatermarkTime() { return inputWatermark; } @Override - public Instant currentOutputWatermarkTime() - { + public Instant currentOutputWatermarkTime() { // TODO Auto-generated method stub return null; } - } - private class GroupByKeyStateInternalsFactory implements StateInternalsFactory, Serializable - { + private class GroupByKeyStateInternalsFactory implements StateInternalsFactory, Serializable { + private static final long serialVersionUID = 1L; + @Override - public StateInternals stateInternalsForKey(K key) - { + public StateInternals stateInternalsForKey(K key) { return getStateInternalsForKey(key); } } - -} \ No newline at end of file +} diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java index 96be11d101d3..9e8f3dc3d478 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/ApexParDoOperator.java @@ -17,6 +17,18 @@ */ package org.apache.beam.runners.apex.translators.functions; +import com.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.DefaultInputPort; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.annotation.InputPortFieldAnnotation; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; +import com.datatorrent.common.util.BaseOperator; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; + import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -29,9 +41,9 @@ import org.apache.beam.runners.apex.translators.utils.ValueAndCoderKryoSerializable; 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.core.SideInputHandler; -import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.transforms.Aggregator; @@ -52,18 +64,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.datatorrent.api.Context.OperatorContext; -import com.datatorrent.api.DefaultInputPort; -import com.datatorrent.api.DefaultOutputPort; -import com.datatorrent.api.annotation.InputPortFieldAnnotation; -import com.datatorrent.api.annotation.OutputPortFieldAnnotation; -import com.datatorrent.common.util.BaseOperator; -import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; -import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.esotericsoftware.kryo.serializers.JavaSerializer; - /** * Apex operator for Beam {@link DoFn}. */ @@ -85,8 +85,8 @@ public class ApexParDoOperator extends BaseOperator implements private final List> sideInputs; // TODO: not Kryo serializable, integrate codec -//@Bind(JavaSerializer.class) -private transient StateInternals sideInputStateInternals = InMemoryStateInternals.forKey(null); + private transient StateInternals sideInputStateInternals = InMemoryStateInternals + .forKey(null); private final ValueAndCoderKryoSerializable>> pushedBack; private LongMin pushedBackWatermark = new LongMin(); private long currentInputWatermark = Long.MIN_VALUE; @@ -94,7 +94,8 @@ public class ApexParDoOperator extends BaseOperator implements private transient PushbackSideInputDoFnRunner pushbackDoFnRunner; private transient SideInputHandler sideInputHandler; - private transient Map, DefaultOutputPort>> sideOutputPortMapping = Maps.newHashMapWithExpectedSize(5); + private transient Map, DefaultOutputPort>> sideOutputPortMapping = + Maps.newHashMapWithExpectedSize(5); public ApexParDoOperator( ApexPipelineOptions pipelineOptions, @@ -104,8 +105,7 @@ public ApexParDoOperator( WindowingStrategy windowingStrategy, List> sideInputs, Coder> inputCoder - ) - { + ) { this.pipelineOptions = new SerializablePipelineOptions(pipelineOptions); this.doFn = doFn; this.mainOutputTag = mainOutputTag; @@ -120,7 +120,8 @@ public ApexParDoOperator( } Coder>> coder = ListCoder.of(inputCoder); - this.pushedBack = new ValueAndCoderKryoSerializable<>(new ArrayList>(), coder); + this.pushedBack = new ValueAndCoderKryoSerializable<>(new ArrayList>(), + coder); } @@ -135,13 +136,12 @@ private ApexParDoOperator() { this.pushedBack = null; } - public final transient DefaultInputPort>> input = new DefaultInputPort>>() - { + public final transient DefaultInputPort>> input = + new DefaultInputPort>>() { @Override - public void process(ApexStreamTuple> t) - { + public void process(ApexStreamTuple> t) { if (t instanceof ApexStreamTuple.WatermarkTuple) { - processWatermark((ApexStreamTuple.WatermarkTuple)t); + processWatermark((ApexStreamTuple.WatermarkTuple) t); } else { if (traceTuples) { LOG.debug("\ninput {}\n", t.getValue()); @@ -155,12 +155,11 @@ public void process(ApexStreamTuple> t) } }; - @InputPortFieldAnnotation(optional=true) - public final transient DefaultInputPort>>> sideInput1 = new DefaultInputPort>>>() - { + @InputPortFieldAnnotation(optional = true) + public final transient DefaultInputPort>>> sideInput1 = + new DefaultInputPort>>>() { @Override - public void process(ApexStreamTuple>> t) - { + public void process(ApexStreamTuple>> t) { if (t instanceof ApexStreamTuple.WatermarkTuple) { // ignore side input watermarks return; @@ -168,7 +167,7 @@ public void process(ApexStreamTuple>> t) int sideInputIndex = 0; if (t instanceof ApexStreamTuple.DataTuple) { - sideInputIndex = ((ApexStreamTuple.DataTuple)t).getUnionTag(); + sideInputIndex = ((ApexStreamTuple.DataTuple) t).getUnionTag(); } if (traceTuples) { @@ -196,25 +195,30 @@ public void process(ApexStreamTuple>> t) } }; - @OutputPortFieldAnnotation(optional=true) + @OutputPortFieldAnnotation(optional = true) public final transient DefaultOutputPort> output = new DefaultOutputPort<>(); - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort> sideOutput1 = new DefaultOutputPort<>(); - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort> sideOutput2 = new DefaultOutputPort<>(); - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort> sideOutput3 = new DefaultOutputPort<>(); - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort> sideOutput4 = new DefaultOutputPort<>(); - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort> sideOutput5 = new DefaultOutputPort<>(); - - public final transient DefaultOutputPort[] sideOutputPorts = {sideOutput1, sideOutput2, sideOutput3, sideOutput4, sideOutput5}; + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort> sideOutput1 = + new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort> sideOutput2 = + new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort> sideOutput3 = + new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort> sideOutput4 = + new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort> sideOutput5 = + new DefaultOutputPort<>(); + + public final transient DefaultOutputPort[] sideOutputPorts = {sideOutput1, sideOutput2, + sideOutput3, sideOutput4, sideOutput5}; @Override - public void output(TupleTag tag, WindowedValue tuple) - { + public void output(TupleTag tag, WindowedValue tuple) { DefaultOutputPort> sideOutputPort = sideOutputPortMapping.get(tag); if (sideOutputPort != null) { sideOutputPort.emit(ApexStreamTuple.DataTuple.of(tuple)); @@ -229,19 +233,19 @@ public void output(TupleTag tag, WindowedValue tuple) private Iterable> processElementInReadyWindows(WindowedValue elem) { try { pushbackDoFnRunner.startBundle(); - Iterable> pushedBack = pushbackDoFnRunner.processElementInReadyWindows(elem); + Iterable> pushedBack = pushbackDoFnRunner + .processElementInReadyWindows(elem); pushbackDoFnRunner.finishBundle(); return pushedBack; } catch (UserCodeException ue) { if (ue.getCause() instanceof AssertionError) { - ApexRunner.assertionError = (AssertionError)ue.getCause(); + ApexRunner.assertionError = (AssertionError) ue.getCause(); } throw ue; } } - private void processWatermark(ApexStreamTuple.WatermarkTuple mark) - { + private void processWatermark(ApexStreamTuple.WatermarkTuple mark) { this.currentInputWatermark = mark.getTimestamp(); if (sideInputs.isEmpty()) { @@ -264,8 +268,7 @@ private void processWatermark(ApexStreamTuple.WatermarkTuple mark) } @Override - public void setup(OperatorContext context) - { + public void setup(OperatorContext context) { this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(pipelineOptions.get(), this); SideInputReader sideInputReader = NullSideInputReader.of(sideInputs); if (!sideInputs.isEmpty()) { @@ -273,9 +276,10 @@ public void setup(OperatorContext context) sideInputReader = sideInputHandler; } - for (int i=0; i < sideOutputTags.size(); i++) { + for (int i = 0; i < sideOutputTags.size(); i++) { @SuppressWarnings("unchecked") - DefaultOutputPort> port = (DefaultOutputPort>)sideOutputPorts[i]; + DefaultOutputPort> port = (DefaultOutputPort>) + sideOutputPorts[i]; sideOutputPortMapping.put(sideOutputTags.get(i), port); } @@ -297,25 +301,18 @@ public void setup(OperatorContext context) try { doFn.setup(); } catch (Exception e) { - Throwables.propagate(e); + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); } } @Override - public void beginWindow(long windowId) - { - /* - Collection> aggregators = AggregatorRetriever.getAggregators(doFn); - if (!aggregators.isEmpty()) { - System.out.println("\n" + Thread.currentThread().getName() + "\n" +AggregatorRetriever.getAggregators(doFn) + "\n"); - } - */ + public void beginWindow(long windowId) { } @Override - public void endWindow() - { + public void endWindow() { } /** @@ -334,32 +331,27 @@ public Aggregator createAggregatorFor return new NoOpAggregator(); } - private static class NoOpAggregator implements Aggregator, java.io.Serializable - { + private static class NoOpAggregator implements Aggregator, + java.io.Serializable { private static final long serialVersionUID = 1L; @Override - public void addValue(InputT value) - { + public void addValue(InputT value) { } @Override - public String getName() - { + public String getName() { // TODO Auto-generated method stub return null; } @Override - public CombineFn getCombineFn() - { + public CombineFn getCombineFn() { // TODO Auto-generated method stub return null; } }; - - } private static class LongMin { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/package-info.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/functions/package-info.java new file mode 100644 index 000000000000..ecb0adb2d184 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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. + */ + +/** + * Implementation of the Beam runner for Apache Apex. + */ +package org.apache.beam.runners.apex.translators.functions; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java index 6ee82ea7554b..3188dfa73ba8 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ApexReadUnboundedInputOperator.java @@ -18,6 +18,17 @@ package org.apache.beam.runners.apex.translators.io; +import com.datatorrent.api.Context.OperatorContext; +import com.datatorrent.api.DefaultOutputPort; +import com.datatorrent.api.InputOperator; +import com.datatorrent.api.annotation.OutputPortFieldAnnotation; +import com.datatorrent.common.util.BaseOperator; +import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import com.google.common.base.Throwables; + +import java.io.IOException; + import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translators.utils.ApexStreamTuple.DataTuple; @@ -26,27 +37,15 @@ 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.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.datatorrent.api.Context.OperatorContext; -import com.datatorrent.api.annotation.OutputPortFieldAnnotation; -import com.google.common.base.Throwables; -import com.datatorrent.api.DefaultOutputPort; -import com.datatorrent.api.InputOperator; -import com.datatorrent.common.util.BaseOperator; -import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; -import com.esotericsoftware.kryo.serializers.JavaSerializer; - -import java.io.IOException; - /** * Apex input operator that wraps Beam {@link UnboundedSource}. */ -public class ApexReadUnboundedInputOperator - implements InputOperator { +public class ApexReadUnboundedInputOperator implements InputOperator { private static final Logger LOG = LoggerFactory.getLogger( ApexReadUnboundedInputOperator.class); private boolean traceTuples = false; @@ -58,10 +57,12 @@ public class ApexReadUnboundedInputOperator source; private transient UnboundedSource.UnboundedReader reader; private transient boolean available = false; - @OutputPortFieldAnnotation(optional=true) - public final transient DefaultOutputPort>> output = new DefaultOutputPort<>(); + @OutputPortFieldAnnotation(optional = true) + public final transient DefaultOutputPort>> output = + new DefaultOutputPort<>(); - public ApexReadUnboundedInputOperator(UnboundedSource source, ApexPipelineOptions options) { + public ApexReadUnboundedInputOperator(UnboundedSource source, + ApexPipelineOptions options) { this.pipelineOptions = new SerializablePipelineOptions(options); this.source = source; } @@ -72,8 +73,7 @@ private ApexReadUnboundedInputOperator() { } @Override - public void beginWindow(long windowId) - { + public void beginWindow(long windowId) { if (!available && source instanceof ValuesSource) { // if it's a Create and the input was consumed, emit final watermark emitWatermarkIfNecessary(GlobalWindow.TIMESTAMP_MAX_VALUE.getMillis()); @@ -95,37 +95,33 @@ private void emitWatermarkIfNecessary(long mark) { } @Override - public void endWindow() - { + public void endWindow() { } @Override - public void setup(OperatorContext context) - { + public void setup(OperatorContext context) { this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(pipelineOptions.get(), this); try { reader = source.createReader(this.pipelineOptions.get(), null); available = reader.start(); } catch (IOException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } @Override - public void teardown() - { + public void teardown() { try { if (reader != null) { reader.close(); } } catch (IOException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } @Override - public void emitTuples() - { + public void emitTuples() { try { if (!available) { available = reader.advance(); @@ -141,7 +137,8 @@ public void emitTuples() data, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); } } catch (Exception e) { - Throwables.propagate(e); + Throwables.propagateIfPossible(e); + throw new RuntimeException(e); } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java index 2c4b298c56ee..fadf8ec4e99f 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/ValuesSource.java @@ -18,16 +18,6 @@ package org.apache.beam.runners.apex.translators.io; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - -import com.google.common.base.Throwables; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -37,8 +27,15 @@ 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.IterableCoder; +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}. + * Unbounded source that reads from a Java {@link Iterable}. */ public class ValuesSource extends UnboundedSource { private static final long serialVersionUID = 1L; @@ -52,7 +49,7 @@ public ValuesSource(Iterable values, Coder coder) { try { iterableCoder.encode(values, bos, Context.OUTER); } catch (IOException ex) { - Throwables.propagate(ex); + throw new RuntimeException(ex); } this.codedValues = bos.toByteArray(); } @@ -71,7 +68,7 @@ public UnboundedReader createReader(PipelineOptions options, Iterable values = this.iterableCoder.decode(bis, Context.OUTER); return new ValuesReader<>(values, this); } catch (IOException ex) { - throw Throwables.propagate(ex); + throw new RuntimeException(ex); } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/package-info.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/io/package-info.java new file mode 100644 index 000000000000..0d17f1943efc --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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. + */ + +/** + * Implementation of the Beam runner for Apache Apex. + */ +package org.apache.beam.runners.apex.translators.io; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/package-info.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/package-info.java new file mode 100644 index 000000000000..7d7c6cc97f49 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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. + */ + +/** + * Implementation of the Beam runner for Apache Apex. + */ +package org.apache.beam.runners.apex.translators; diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java index c9bf6dc2f5cb..a260a661a0c7 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ApexStreamTuple.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.datatorrent.api.Operator; + import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -32,24 +34,25 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StandardCoder; -import com.datatorrent.api.Operator; - -public interface ApexStreamTuple -{ +/** + * The common interface for all objects transmitted through streams. + * + * @param The actual payload type. + */ +public interface ApexStreamTuple { /** - * Gets the value of the tuple + * Gets the value of the tuple. * * @return */ T getValue(); /** - * Plain tuple class + * Data tuple class. * * @param */ - class DataTuple implements ApexStreamTuple - { + class DataTuple implements ApexStreamTuple { private int unionTag; private T value; @@ -57,86 +60,73 @@ public static DataTuple of(T value) { return new DataTuple<>(value, 0); } - private DataTuple(T value, int unionTag) - { + private DataTuple(T value, int unionTag) { this.value = value; this.unionTag = unionTag; } @Override - public T getValue() - { + public T getValue() { return value; } - public void setValue(T value) - { + public void setValue(T value) { this.value = value; } - public int getUnionTag() - { + public int getUnionTag() { return unionTag; } - public void setUnionTag(int unionTag) - { + public void setUnionTag(int unionTag) { this.unionTag = unionTag; } @Override - public String toString() - { + public String toString() { return value.toString(); } } /** - * Tuple that includes a timestamp + * Tuple that includes a timestamp. * * @param */ - class TimestampedTuple extends DataTuple - { + class TimestampedTuple extends DataTuple { private long timestamp; - public TimestampedTuple(long timestamp, T value) - { + public TimestampedTuple(long timestamp, T value) { super(value, 0); this.timestamp = timestamp; } - public long getTimestamp() - { + public long getTimestamp() { return timestamp; } - public void setTimestamp(long timestamp) - { + public void setTimestamp(long timestamp) { this.timestamp = timestamp; } } /** - * Tuple that represents a watermark + * Tuple that represents a watermark. * * @param */ - class WatermarkTuple extends TimestampedTuple - { + class WatermarkTuple extends TimestampedTuple { public static WatermarkTuple of(long timestamp) { return new WatermarkTuple<>(timestamp); } - protected WatermarkTuple(long timestamp) - { + protected WatermarkTuple(long timestamp) { super(timestamp, null); } @Override - public String toString() - { + public String toString() { return "[Watermark " + getTimestamp() + "]"; } } @@ -161,18 +151,17 @@ public void encode(ApexStreamTuple value, OutputStream outStream, Context con throws CoderException, IOException { if (value instanceof WatermarkTuple) { outStream.write(1); - new DataOutputStream(outStream).writeLong(((WatermarkTuple)value).getTimestamp()); + new DataOutputStream(outStream).writeLong(((WatermarkTuple) value).getTimestamp()); } else { outStream.write(0); - outStream.write(((DataTuple)value).unionTag); + outStream.write(((DataTuple) value).unionTag); valueCoder.encode(value.getValue(), outStream, context); } } @Override public ApexStreamTuple decode(InputStream inStream, Context context) - throws CoderException, IOException - { + throws CoderException, IOException { int b = inStream.read(); if (b == 1) { return new WatermarkTuple(new DataInputStream(inStream).readLong()); @@ -183,14 +172,12 @@ public ApexStreamTuple decode(InputStream inStream, Context context) } @Override - public List> getCoderArguments() - { + public List> getCoderArguments() { return Arrays.>asList(valueCoder); } @Override - public void verifyDeterministic() throws NonDeterministicException - { + public void verifyDeterministic() throws NonDeterministicException { verifyDeterministic( this.getClass().getSimpleName() + " requires a deterministic valueCoder", valueCoder); @@ -205,10 +192,12 @@ public Coder getValueCoder() { } - final class Logging - { - public static boolean isDebugEnabled(ApexPipelineOptions options, Operator operator) - { + /** + * Central if data tuples received on and emitted from ports should be logged. + * Should be called in setup and value cached in operator. + */ + final class Logging { + public static boolean isDebugEnabled(ApexPipelineOptions options, Operator operator) { return options.isTupleTracingEnabled(); } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java index c18765bb0121..61e3b83091b6 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/CoderAdapterStreamCodec.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.apex.translators.utils; +import com.datatorrent.api.StreamCodec; +import com.datatorrent.netlet.util.Slice; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -25,15 +28,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; -import com.datatorrent.api.StreamCodec; -import com.datatorrent.netlet.util.Slice; -import com.google.common.base.Throwables; - /** * The Apex {@link StreamCodec} adapter for using Beam {@link Coder}. */ public class CoderAdapterStreamCodec implements StreamCodec, Serializable { - private static final long serialVersionUID = 1L; private final Coder coder; @@ -42,31 +40,29 @@ public CoderAdapterStreamCodec(Coder coder) { } @Override - public Object fromByteArray(Slice fragment) - { - ByteArrayInputStream bis = new ByteArrayInputStream(fragment.buffer, fragment.offset, fragment.length); + public Object fromByteArray(Slice fragment) { + ByteArrayInputStream bis = new ByteArrayInputStream(fragment.buffer, fragment.offset, + fragment.length); try { return coder.decode(bis, Context.OUTER); } catch (IOException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } @Override - public Slice toByteArray(Object wv) - { + public Slice toByteArray(Object wv) { ByteArrayOutputStream bos = new ByteArrayOutputStream(); try { coder.encode(wv, bos, Context.OUTER); } catch (IOException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } return new Slice(bos.toByteArray()); } @Override - public int getPartition(Object o) - { + public int getPartition(Object o) { return o.hashCode(); } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java index 43d92f61116d..3b19c37333ab 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/NoOpStepContext.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.apex.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; @@ -25,14 +28,10 @@ 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 does nothing. */ public class NoOpStepContext implements ExecutionContext.StepContext, Serializable { - private static final long serialVersionUID = 1L; @Override diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java index 7f7b3ef21662..d32b869fec08 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/SerializablePipelineOptions.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.apex.translators.utils; +import com.fasterxml.jackson.databind.ObjectMapper; + import java.io.Externalizable; import java.io.IOException; import java.io.ObjectInput; @@ -25,37 +27,34 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; -import com.fasterxml.jackson.databind.ObjectMapper; - /** - * A wrapper to enable serialization of {@link PipelineOptions} + * A wrapper to enable serialization of {@link PipelineOptions}. */ public class SerializablePipelineOptions implements Externalizable { 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 - { + public void writeExternal(ObjectOutput out) throws IOException { out.writeUTF(new ObjectMapper().writeValueAsString(pipelineOptions)); } @Override - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException - { + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { String s = in.readUTF(); - this.pipelineOptions = new ObjectMapper().readValue(s, PipelineOptions.class).as(ApexPipelineOptions.class); + this.pipelineOptions = new ObjectMapper().readValue(s, PipelineOptions.class) + .as(ApexPipelineOptions.class); } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.java index 2de737d406e2..c06c500ef528 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/ValueAndCoderKryoSerializable.java @@ -17,26 +17,24 @@ */ package org.apache.beam.runners.apex.translators.utils; -import java.io.IOException; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; - import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.KryoSerializable; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import com.esotericsoftware.kryo.serializers.JavaSerializer; -import com.google.common.base.Throwables; + +import java.io.IOException; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; /** * A {@link KryoSerializable} holder that uses the specified {@link Coder}. * @param */ -public class ValueAndCoderKryoSerializable implements KryoSerializable -{ - private static JavaSerializer JAVA_SERIALIZER = new JavaSerializer(); +public class ValueAndCoderKryoSerializable implements KryoSerializable { + private static final JavaSerializer JAVA_SERIALIZER = new JavaSerializer(); private T value; private Coder coder; @@ -54,27 +52,25 @@ public T get() { } @Override - public void write(Kryo kryo, Output output) - { + public void write(Kryo kryo, Output output) { try { kryo.writeClass(output, coder.getClass()); kryo.writeObject(output, coder, JAVA_SERIALIZER); coder.encode(value, output, Context.OUTER); } catch (IOException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } @Override - public void read(Kryo kryo, Input input) - { + public void read(Kryo kryo, Input input) { try { @SuppressWarnings("unchecked") Class> type = kryo.readClass(input).getType(); coder = kryo.readObject(input, type, JAVA_SERIALIZER); value = coder.decode(input, Context.OUTER); } catch (IOException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/package-info.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translators/utils/package-info.java new file mode 100644 index 000000000000..4aeba3583d31 --- /dev/null +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/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. + */ + +/** + * Implementation of the Beam runner for Apache Apex. + */ +package org.apache.beam.runners.apex.translators.utils; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java deleted file mode 100644 index 3573d31dc741..000000000000 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/IntTest.java +++ /dev/null @@ -1,133 +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.examples; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.TestApexRunner; -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.runners.dataflow.TestCountingSource; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.joda.time.Duration; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * For debugging only. - */ -@Ignore -@RunWith(JUnit4.class) -public class IntTest implements java.io.Serializable -{ - - @Test - public void test() - { - ApexPipelineOptions options = PipelineOptionsFactory.as(ApexPipelineOptions.class); - options.setTupleTracingEnabled(true); - options.setRunner(TestApexRunner.class); - Pipeline p = Pipeline.create(options); -boolean timeBound = false; - - - TestCountingSource source = new TestCountingSource(Integer.MAX_VALUE).withoutSplitting(); -//List> values = Lists.newArrayList( -// KV.of(0, 99),KV.of(0, 99),KV.of(0, 98)); - -//UnboundedSource, ?> source = new ValuesSource<>(values, -// KvCoder.of(VarIntCoder.of(), VarIntCoder.of())); - - if (true) { - source = source.withDedup(); - } - - PCollection> output = - timeBound - ? p.apply(Read.from(source).withMaxReadTime(Duration.millis(200))) - : p.apply(Read.from(source).withMaxNumRecords(NUM_RECORDS)); - - List> expectedOutput = new ArrayList<>(); - for (int i = 0; i < NUM_RECORDS; i++) { - expectedOutput.add(KV.of(0, i)); - } - - // Because some of the NUM_RECORDS elements read are dupes, the final output - // will only have output from 0 to n where n < NUM_RECORDS. - PAssert.that(output).satisfies(new Checker(true, timeBound)); - - - p.run(); - return; - } - - private static final int NUM_RECORDS = 10; - private static class Checker implements SerializableFunction>, Void> - { - private final boolean dedup; - private final boolean timeBound; - - Checker(boolean dedup, boolean timeBound) - { - this.dedup = dedup; - this.timeBound = timeBound; - } - - @Override - public Void apply(Iterable> input) - { - List values = new ArrayList<>(); - for (KV kv : input) { - assertEquals(0, (int)kv.getKey()); - values.add(kv.getValue()); - } - if (timeBound) { - assertTrue(values.size() >= 1); - } else if (dedup) { - // Verify that at least some data came through. The chance of 90% of the input - // being duplicates is essentially zero. - assertTrue(values.size() > NUM_RECORDS / 10 && values.size() <= NUM_RECORDS); - } else { - assertEquals(NUM_RECORDS, values.size()); - } - Collections.sort(values); - for (int i = 0; i < values.size(); i++) { - assertEquals(i, (int)values.get(i)); - } - //if (finalizeTracker != null) { - // assertThat(finalizeTracker, containsInAnyOrder(values.size() - 1)); - //} - return null; - } - } - - -} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java index 582d83986699..6ab2e8e459c8 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/StreamingWordCountTest.java @@ -21,8 +21,8 @@ import java.util.concurrent.ConcurrentHashMap; import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -37,7 +37,6 @@ import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,8 +75,8 @@ 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(); LOG.debug("output {}", row); c.output(row); RESULTS.put(c.element().getKey(), c.element().getValue()); @@ -103,17 +102,19 @@ public void testWindowedWordCount() throws Exception { wordCounts.apply(ParDo.of(new FormatAsStringFn())); - ApexRunnerResult result = (ApexRunnerResult)p.run(); + ApexRunnerResult result = (ApexRunnerResult) p.run(); Assert.assertNotNull(result.getApexDAG().getOperatorMeta("Read(UnboundedTextSource)")); long timeout = System.currentTimeMillis() + 30000; while (System.currentTimeMillis() < timeout) { - if (FormatAsStringFn.RESULTS.containsKey("foo") && FormatAsStringFn.RESULTS.containsKey("bar")) { + if (FormatAsStringFn.RESULTS.containsKey("foo") + && FormatAsStringFn.RESULTS.containsKey("bar")) { break; } Thread.sleep(1000); } result.cancel(); - Assert.assertTrue(FormatAsStringFn.RESULTS.containsKey("foo") && FormatAsStringFn.RESULTS.containsKey("bar")); + Assert.assertTrue( + FormatAsStringFn.RESULTS.containsKey("foo") && FormatAsStringFn.RESULTS.containsKey("bar")); FormatAsStringFn.RESULTS.clear(); } 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 29351e97bc51..8132ee526e11 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 @@ -18,14 +18,6 @@ package org.apache.beam.runners.apex.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 com.google.common.base.Throwables; - import java.io.IOException; import java.io.Serializable; import java.util.Collections; @@ -34,6 +26,12 @@ 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. */ @@ -102,7 +100,7 @@ public boolean advance() throws IOException { try { Thread.sleep(index); // allow for downstream processing to complete } catch (InterruptedException e) { - Throwables.propagate(e); + throw new RuntimeException(e); } return true; } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/package-info.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/package-info.java new file mode 100644 index 000000000000..4308c80113ad --- /dev/null +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/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. + */ + +/** + * Implementation of the Beam runner for Apache Apex. + */ +package org.apache.beam.runners.apex.examples; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java index 6b181ba4deca..7defc77517be 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/FlattenPCollectionTranslatorTest.java @@ -18,9 +18,16 @@ package org.apache.beam.runners.apex.translators; +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -30,21 +37,13 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -import com.google.common.collect.Sets; - import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** - * integration test for {@link FlattenPCollectionTranslator}. + * Integration test for {@link FlattenPCollectionTranslator}. */ public class FlattenPCollectionTranslatorTest { private static final Logger LOG = LoggerFactory.getLogger(FlattenPCollectionTranslatorTest.class); @@ -70,29 +69,30 @@ public void test() throws Exception { PCollection actual = PCollectionList.of(pcList).apply(Flatten.pCollections()); actual.apply(ParDo.of(new EmbeddedCollector())); - ApexRunnerResult result = (ApexRunnerResult)p.run(); + ApexRunnerResult result = (ApexRunnerResult) p.run(); // TODO: verify translation result.getApexDAG(); long timeout = System.currentTimeMillis() + 30000; - while (System.currentTimeMillis() < timeout && EmbeddedCollector.results.size() < expected.size()) { + while (System.currentTimeMillis() < timeout + && EmbeddedCollector.RESULTS.size() < expected.size()) { LOG.info("Waiting for expected results."); Thread.sleep(500); } - Assert.assertEquals("number results", expected.size(), EmbeddedCollector.results.size()); - Assert.assertEquals(expected, Sets.newHashSet(EmbeddedCollector.results)); + Assert.assertEquals("number results", expected.size(), EmbeddedCollector.RESULTS.size()); + Assert.assertEquals(expected, Sets.newHashSet(EmbeddedCollector.RESULTS)); } @SuppressWarnings("serial") private static class EmbeddedCollector extends OldDoFn { - protected static final ArrayList results = new ArrayList<>(); + protected static final ArrayList RESULTS = new ArrayList<>(); public EmbeddedCollector() { } @Override public void processElement(ProcessContext c) throws Exception { - results.add(c.element()); + RESULTS.add(c.element()); } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java index e4d4606db450..cb764d6f44ae 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/GroupByKeyTranslatorTest.java @@ -18,9 +18,22 @@ package org.apache.beam.runners.apex.translators; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -35,28 +48,13 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; - -import com.datatorrent.api.DAG; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** - * integration test for {@link GroupByKeyTranslator}. + * Integration test for {@link GroupByKeyTranslator}. */ public class GroupByKeyTranslatorTest { @@ -94,31 +92,30 @@ public void test() throws Exception { .apply(ParDo.of(new EmbeddedCollector())) ; - ApexRunnerResult result = (ApexRunnerResult)p.run(); - // TODO: verify translation - DAG dag = result.getApexDAG(); + ApexRunnerResult result = (ApexRunnerResult) p.run(); + result.getApexDAG(); long timeout = System.currentTimeMillis() + 30000; while (System.currentTimeMillis() < timeout) { - if (EmbeddedCollector.results.containsAll(expected)) { + if (EmbeddedCollector.RESULTS.containsAll(expected)) { break; } Thread.sleep(1000); } - Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.RESULTS); } @SuppressWarnings("serial") private static class EmbeddedCollector extends OldDoFn { - protected static final HashSet results = new HashSet<>(); + protected static final HashSet RESULTS = new HashSet<>(); public EmbeddedCollector() { } @Override public void processElement(ProcessContext c) throws Exception { - results.add(c.element()); + RESULTS.add(c.element()); } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java index b9748eebeb08..ad22acde5cf6 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ParDoBoundTranslatorTest.java @@ -21,6 +21,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.datatorrent.api.DAG; +import com.datatorrent.lib.util.KryoCloneUtils; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -56,11 +61,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.datatorrent.api.DAG; -import com.datatorrent.lib.util.KryoCloneUtils; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - /** * integration test for {@link ParDoBoundTranslator}. */ @@ -83,7 +83,7 @@ public void test() throws Exception { .apply(ParDo.of(new Add(5))) .apply(ParDo.of(new EmbeddedCollector())); - ApexRunnerResult result = (ApexRunnerResult)p.run(); + ApexRunnerResult result = (ApexRunnerResult) p.run(); DAG dag = result.getApexDAG(); DAG.OperatorMeta om = dag.getOperatorMeta("Create.Values"); @@ -96,13 +96,13 @@ public void test() throws Exception { long timeout = System.currentTimeMillis() + 30000; while (System.currentTimeMillis() < timeout) { - if (EmbeddedCollector.results.containsAll(expected)) { + if (EmbeddedCollector.RESULTS.containsAll(expected)) { break; } LOG.info("Waiting for expected results."); Thread.sleep(1000); } - Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.RESULTS); } @SuppressWarnings("serial") @@ -121,14 +121,14 @@ public void processElement(ProcessContext c) throws Exception { @SuppressWarnings("serial") private static class EmbeddedCollector extends OldDoFn { - protected static final HashSet results = new HashSet<>(); + protected static final HashSet RESULTS = new HashSet<>(); public EmbeddedCollector() { } @Override public void processElement(ProcessContext c) throws Exception { - results.add(c.element()); + RESULTS.add(c.element()); } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java index f954537db408..71c5354b4368 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/ReadUnboundTranslatorTest.java @@ -18,9 +18,20 @@ package org.apache.beam.runners.apex.translators; +import com.datatorrent.api.DAG; +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.runners.apex.ApexRunnerResult; import org.apache.beam.runners.apex.translators.io.ApexReadUnboundedInputOperator; import org.apache.beam.runners.apex.translators.utils.CollectionSource; import org.apache.beam.sdk.Pipeline; @@ -30,23 +41,11 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - -import com.datatorrent.api.DAG; -import com.google.common.collect.ContiguousSet; -import com.google.common.collect.DiscreteDomain; -import com.google.common.collect.Lists; -import com.google.common.collect.Range; -import com.google.common.collect.Sets; - import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - /** * integration test for {@link ReadUnboundedTranslator}. */ @@ -57,7 +56,7 @@ public class ReadUnboundTranslatorTest { public void test() throws Exception { ApexPipelineOptions options = PipelineOptionsFactory.create() .as(ApexPipelineOptions.class); - EmbeddedCollector.results.clear(); + EmbeddedCollector.RESULTS.clear(); options.setApplicationName("ReadUnbound"); options.setRunner(ApexRunner.class); Pipeline p = Pipeline.create(options); @@ -67,7 +66,7 @@ public void test() throws Exception { p.apply(Read.from(source)) .apply(ParDo.of(new EmbeddedCollector())); - ApexRunnerResult result = (ApexRunnerResult)p.run(); + ApexRunnerResult result = (ApexRunnerResult) p.run(); DAG dag = result.getApexDAG(); DAG.OperatorMeta om = dag.getOperatorMeta("Read(CollectionSource)"); Assert.assertNotNull(om); @@ -75,20 +74,20 @@ public void test() throws Exception { long timeout = System.currentTimeMillis() + 30000; while (System.currentTimeMillis() < timeout) { - if (EmbeddedCollector.results.containsAll(collection)) { + if (EmbeddedCollector.RESULTS.containsAll(collection)) { break; } LOG.info("Waiting for expected results."); Thread.sleep(1000); } - Assert.assertEquals(Sets.newHashSet(collection), EmbeddedCollector.results); + Assert.assertEquals(Sets.newHashSet(collection), EmbeddedCollector.RESULTS); } @Test public void testReadBounded() throws Exception { ApexPipelineOptions options = PipelineOptionsFactory.create() .as(ApexPipelineOptions.class); - EmbeddedCollector.results.clear(); + EmbeddedCollector.RESULTS.clear(); options.setApplicationName("ReadBounded"); options.setRunner(ApexRunner.class); Pipeline p = Pipeline.create(options); @@ -97,7 +96,7 @@ public void testReadBounded() throws Exception { p.apply(Read.from(CountingSource.upTo(10))) .apply(ParDo.of(new EmbeddedCollector())); - ApexRunnerResult result = (ApexRunnerResult)p.run(); + ApexRunnerResult result = (ApexRunnerResult) p.run(); DAG dag = result.getApexDAG(); DAG.OperatorMeta om = dag.getOperatorMeta("Read(BoundedCountingSource)"); Assert.assertNotNull(om); @@ -105,25 +104,25 @@ public void testReadBounded() throws Exception { long timeout = System.currentTimeMillis() + 30000; while (System.currentTimeMillis() < timeout) { - if (EmbeddedCollector.results.containsAll(expected)) { + if (EmbeddedCollector.RESULTS.containsAll(expected)) { break; } LOG.info("Waiting for expected results."); Thread.sleep(1000); } - Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.results); + Assert.assertEquals(Sets.newHashSet(expected), EmbeddedCollector.RESULTS); } @SuppressWarnings("serial") private static class EmbeddedCollector extends OldDoFn { - protected static final HashSet results = new HashSet<>(); + protected static final HashSet RESULTS = new HashSet<>(); public EmbeddedCollector() { } @Override public void processElement(ProcessContext c) throws Exception { - results.add(c.element()); + RESULTS.add(c.element()); } } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java index a1e8b3e55425..c368bb22bd1b 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/CollectionSource.java @@ -18,12 +18,6 @@ package org.apache.beam.runners.apex.translators.utils; -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.IOException; import java.io.Serializable; import java.util.Collection; @@ -34,11 +28,16 @@ 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; + /** * collection as {@link UnboundedSource}, used for tests. */ public class CollectionSource extends UnboundedSource { - + private static final long serialVersionUID = 1L; private final Collection collection; private final Coder coder; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java index e2fa9d972937..e67efa99beff 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translators/utils/PipelineOptionsTest.java @@ -17,29 +17,31 @@ */ package org.apache.beam.runners.apex.translators.utils; -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.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.junit.BeforeClass; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import com.datatorrent.common.util.FSStorageAgent; -import com.esotericsoftware.kryo.serializers.JavaSerializer; import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import com.esotericsoftware.kryo.serializers.JavaSerializer; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +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.BeforeClass; +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") @@ -60,7 +62,7 @@ private MyOptionsWrapper(ApexPipelineOptions options) { private static MyOptions options; - private final static String[] args = new String[]{"--testOption=nothing"}; + private static final String[] args = new String[]{"--testOption=nothing"}; @BeforeClass public static void beforeTest() { @@ -74,7 +76,7 @@ public void testSerialization() { FSStorageAgent.store(bos, wrapper); ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); - MyOptionsWrapper wrapperCopy = (MyOptionsWrapper)FSStorageAgent.retrieve(bis); + MyOptionsWrapper wrapperCopy = (MyOptionsWrapper) FSStorageAgent.retrieve(bis); assertNotNull(wrapperCopy.options); assertEquals("nothing", wrapperCopy.options.get().as(MyOptions.class).getTestOption()); } diff --git a/runners/apex/src/test/resources/log4j.properties b/runners/apex/src/test/resources/log4j.properties index c0efc5de4f39..d1e6b44b03d9 100644 --- a/runners/apex/src/test/resources/log4j.properties +++ b/runners/apex/src/test/resources/log4j.properties @@ -18,16 +18,18 @@ # Set root logger level to OFF to not flood build logs # set manually to INFO for debugging purposes -log4j.rootLogger=DEBUG, testlogger +log4j.rootLogger=OFF, testlogger # A1 is set to be a ConsoleAppender. 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.threshold=${test.log.threshold} +test.log.threshold=DEBUG -log4j.logger.org=debug +log4j.logger.org=info log4j.logger.org.apache.commons.beanutils=warn log4j.logger.com.datatorrent=info -log4j.logger.org.apache.apex=debug +log4j.logger.org.apache.apex=info log4j.logger.org.apache.beam.runners.apex=debug