You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by pa...@apache.org on 2019/05/20 22:14:48 UTC
[beam] branch master updated: [BEAM-6880] Remove deprecated
Reference Runner code. (#8380)
This is an automated email from the ASF dual-hosted git repository.
pabloem pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new eddc83a [BEAM-6880] Remove deprecated Reference Runner code. (#8380)
eddc83a is described below
commit eddc83a33e74a606b0584eda75e4c2257e666032
Author: Daniel Oliveira <da...@gmail.com>
AuthorDate: Mon May 20 15:14:35 2019 -0700
[BEAM-6880] Remove deprecated Reference Runner code. (#8380)
* [BEAM-6880] Remove Java Reference Runner test code.
* [BEAM-6880] Remove additional mentions of removed code.
---
runners/direct-java/build.gradle | 43 --
.../runners/direct/portable/BundleFactory.java | 47 --
.../BundleFactoryOutputReceiverFactory.java | 68 ---
.../runners/direct/portable/BundleProcessor.java | 33 --
.../runners/direct/portable/CommittedBundle.java | 86 ---
.../runners/direct/portable/CommittedResult.java | 63 ---
.../direct/portable/CompletionCallback.java | 42 --
.../CopyOnAccessInMemoryStateInternals.java | 463 ----------------
.../runners/direct/portable/DirectGroupByKey.java | 133 -----
.../runners/direct/portable/DirectMetrics.java | 307 -----------
.../direct/portable/DirectStateAndTimers.java | 70 ---
.../direct/portable/DirectTimerInternals.java | 105 ----
.../direct/portable/DirectTransformExecutor.java | 151 ------
.../direct/portable/DisplayDataValidator.java | 67 ---
.../direct/portable/EmptyInputProvider.java | 38 --
.../runners/direct/portable/EvaluationContext.java | 333 ------------
...valuationContextStepStateAndTimersProvider.java | 39 --
.../direct/portable/ExecutorServiceFactory.java | 30 --
.../portable/ExecutorServiceParallelExecutor.java | 392 --------------
.../direct/portable/FlattenEvaluatorFactory.java | 79 ---
.../GroupAlsoByWindowEvaluatorFactory.java | 282 ----------
.../portable/GroupByKeyOnlyEvaluatorFactory.java | 157 ------
.../portable/ImmutableListBundleFactory.java | 171 ------
.../direct/portable/ImpulseEvaluatorFactory.java | 110 ----
.../direct/portable/PCollectionViewWindow.java | 66 ---
.../direct/portable/PCollectionViewWriter.java | 34 --
.../portable/PassthroughTransformEvaluator.java | 47 --
.../runners/direct/portable/PipelineExecutor.java | 61 ---
.../runners/direct/portable/PortableGraph.java | 73 ---
.../runners/direct/portable/QuiescenceDriver.java | 328 ------------
.../runners/direct/portable/ReferenceRunner.java | 559 -------------------
.../portable/RemoteStageEvaluatorFactory.java | 95 ----
.../runners/direct/portable/RootInputProvider.java | 44 --
.../direct/portable/RootProviderRegistry.java | 68 ---
.../beam/runners/direct/portable/SourceShard.java | 33 --
.../SplittableRemoteStageEvaluatorFactory.java | 174 ------
.../beam/runners/direct/portable/StepAndKey.java | 67 ---
.../direct/portable/StepStateAndTimers.java | 34 --
.../direct/portable/StepTransformResult.java | 135 -----
.../direct/portable/TransformEvaluator.java | 45 --
.../direct/portable/TransformEvaluatorFactory.java | 58 --
.../portable/TransformEvaluatorRegistry.java | 126 -----
.../runners/direct/portable/TransformExecutor.java | 21 -
.../direct/portable/TransformExecutorFactory.java | 29 -
.../direct/portable/TransformExecutorService.java | 38 --
.../direct/portable/TransformExecutorServices.java | 180 -------
.../runners/direct/portable/TransformResult.java | 101 ----
.../runners/direct/portable/UncommittedBundle.java | 56 --
.../direct/portable/WatermarkCallbackExecutor.java | 171 ------
.../direct/portable/WindowEvaluatorFactory.java | 123 -----
.../artifact/LocalArtifactStagingLocation.java | 116 ----
.../LocalFileSystemArtifactRetrievalService.java | 119 -----
.../LocalFileSystemArtifactStagerService.java | 282 ----------
.../UnsupportedArtifactRetrievalService.java | 42 --
.../direct/portable/artifact/package-info.java | 20 -
.../runners/direct/portable/job/PreparingJob.java | 59 --
.../portable/job/ReferenceRunnerJobServer.java | 164 ------
.../portable/job/ReferenceRunnerJobService.java | 308 -----------
.../runners/direct/portable/job/package-info.java | 23 -
.../beam/runners/direct/portable/package-info.java | 25 -
.../runners/direct/DirectRunnerApiSurfaceTest.java | 2 -
.../BundleFactoryOutputReceiverFactoryTest.java | 262 ---------
.../direct/portable/CommittedResultTest.java | 129 -----
.../CopyOnAccessInMemoryStateInternalsTest.java | 591 ---------------------
.../runners/direct/portable/DirectMetricsTest.java | 232 --------
.../direct/portable/DirectTimerInternalsTest.java | 132 -----
.../portable/DirectTransformExecutorTest.java | 330 ------------
.../direct/portable/EvaluationContextTest.java | 315 -----------
.../direct/portable/ExecutableGraphBuilder.java | 81 ---
.../portable/FlattenEvaluatorFactoryTest.java | 122 -----
.../GroupByKeyOnlyEvaluatorFactoryTest.java | 247 ---------
.../portable/ImmutableListBundleFactoryTest.java | 234 --------
.../portable/ImpulseEvaluatorFactoryTest.java | 130 -----
.../beam/runners/direct/portable/MockClock.java | 65 ---
.../runners/direct/portable/PortableGraphTest.java | 176 ------
.../direct/portable/ReferenceRunnerTest.java | 213 --------
.../portable/RemoteStageEvaluatorFactoryTest.java | 238 ---------
.../direct/portable/StepTransformResultTest.java | 95 ----
.../portable/TransformExecutorServicesTest.java | 143 -----
.../portable/WatermarkCallbackExecutorTest.java | 136 -----
.../portable/WindowEvaluatorFactoryTest.java | 314 -----------
.../artifact/LocalArtifactStagingLocationTest.java | 122 -----
...ocalFileSystemArtifactRetrievalServiceTest.java | 240 ---------
.../LocalFileSystemArtifactStagerServiceTest.java | 313 -----------
.../UnsupportedArtifactRetrievalServiceTest.java | 150 ------
.../job/ReferenceRunnerJobServiceTest.java | 150 ------
runners/reference/job-server/build.gradle | 54 --
.../src/main/resources/beam/spotbugs-filter.xml | 19 -
settings.gradle | 1 -
89 files changed, 12469 deletions(-)
diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle
index 328de71..e06194c 100644
--- a/runners/direct-java/build.gradle
+++ b/runners/direct-java/build.gradle
@@ -54,7 +54,6 @@ evaluationDependsOn(":sdks:java:core")
configurations {
needsRunner
validatesRunner
- validatesPortableRunner
}
dependencies {
@@ -70,7 +69,6 @@ dependencies {
shadow library.java.args4j
provided library.java.hamcrest_core
provided library.java.junit
- testRuntime project(path: ":sdks:java:harness")
shadowTest project(path: ":sdks:java:core", configuration: "shadowTest")
shadowTest project(path: ":runners:core-java", configuration: "shadowTest")
shadowTest library.java.guava_testlib
@@ -90,13 +88,6 @@ dependencies {
validatesRunner project(path: ":sdks:java:core", configuration: "shadowTest")
validatesRunner project(path: project.path, configuration: "shadow")
validatesRunner project(path: project.path, configuration: "shadowTest")
- validatesPortableRunner project(path: ":runners:core-construction-java", configuration: "shadowTest")
- validatesPortableRunner project(path: ":runners:core-java", configuration: "shadowTest")
- validatesPortableRunner project(path: ":runners:java-fn-execution", configuration: "shadowTest")
- validatesPortableRunner project(path: ":runners:reference:java", configuration: "shadowTest")
- validatesPortableRunner project(path: ":sdks:java:core", configuration: "shadowTest")
- validatesPortableRunner project(path: project.path, configuration: "shadow")
- validatesPortableRunner project(path: project.path, configuration: "shadowTest")
}
task needsRunnerTests(type: Test) {
@@ -160,37 +151,3 @@ createJavaExamplesArchetypeValidationTask(type: 'MobileGaming',
gcsBucket: gcsBucket,
bqDataset: bqDataset,
pubsubTopic: pubsubTopic)
-
-createPortableValidatesRunnerTask(
- jobServerDriver: "org.apache.beam.runners.direct.portable.job.ReferenceRunnerJobServer",
- testClasspathConfiguration: configurations.validatesPortableRunner,
- jobServerConfig: "--port=0",
- testCategories: {
- includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
- excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
- excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCrossLanguageTransforms'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
- excludeCategories 'org.apache.beam.sdk.testing.UsesDistributionMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
- excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
- excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
- // TODO(BEAM-5452): Support metrics.
- excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesAttemptedMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
- excludeCategories 'org.apache.beam.sdk.testing.UsesCounterMetrics'
- // TODO(BEAM-3743): Support SplittableDoFn
- excludeCategories 'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
- excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs'
- excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
- // TODO(BEAM-2928): Support sideinput.
- excludeCategories 'org.apache.beam.sdk.testing.UsesSideInputs'
- // TODO(BEAM-2917): Support user state.
- excludeCategories 'org.apache.beam.sdk.testing.UsesStatefulParDo'
- excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
- excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
- // TODO(BEAM-4680): Support user timers.
- excludeCategories 'org.apache.beam.sdk.testing.UsesTimersInParDo'
- },
-)
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactory.java
deleted file mode 100644
index 118892c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-
-/** A factory that creates {@link UncommittedBundle UncommittedBundles}. */
-interface BundleFactory {
- /**
- * Create an {@link UncommittedBundle} from an empty input. Elements added to the bundle do not
- * belong to a {@link PCollection}.
- *
- * <p>For use in creating inputs to root transforms.
- */
- <T> UncommittedBundle<T> createRootBundle();
-
- /**
- * Create an {@link UncommittedBundle} from the specified input. Elements added to the bundle
- * belong to the {@code output} {@link PCollection}.
- */
- <T> UncommittedBundle<T> createBundle(PCollectionNode output);
-
- /**
- * Create an {@link UncommittedBundle} with the specified keys at the specified step. For use by
- * {@code DirectGroupByKeyOnly} {@link PTransform PTransforms}. Elements added to the bundle
- * belong to the {@code output} {@link PCollection}.
- */
- <K, T> UncommittedBundle<T> createKeyedBundle(StructuralKey<K> key, PCollectionNode output);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactory.java
deleted file mode 100644
index 53c2cb9..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactory.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.function.Consumer;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An {@link OutputReceiverFactory} which adds received elements to {@link UncommittedBundle}
- * instances. The produced {@link UncommittedBundle bundles} are added to a provided {@link
- * StepTransformResult.Builder StepTransformResult Builder}.
- */
-class BundleFactoryOutputReceiverFactory implements OutputReceiverFactory {
- private final BundleFactory bundleFactory;
- private final RunnerApi.Components components;
-
- private final Consumer<UncommittedBundle<?>> bundleConsumer;
-
- private BundleFactoryOutputReceiverFactory(
- BundleFactory bundleFactory,
- Components components,
- Consumer<UncommittedBundle<?>> bundleConsumer) {
- this.bundleFactory = bundleFactory;
- this.components = components;
- this.bundleConsumer = bundleConsumer;
- }
-
- public static OutputReceiverFactory create(
- BundleFactory bundleFactory,
- Components components,
- Consumer<UncommittedBundle<?>> resultBuilder) {
- return new BundleFactoryOutputReceiverFactory(bundleFactory, components, resultBuilder);
- }
-
- @Override
- public <OutputT> FnDataReceiver<OutputT> create(String pCollectionId) {
- PCollectionNode pcollection =
- PipelineNode.pCollection(pCollectionId, components.getPcollectionsOrThrow(pCollectionId));
- return create(pcollection);
- }
-
- private <ElemT, OutputT> FnDataReceiver<OutputT> create(PCollectionNode pcollection) {
- UncommittedBundle<ElemT> bundle = bundleFactory.createBundle(pcollection);
- bundleConsumer.accept(bundle);
- return input -> bundle.add((WindowedValue<ElemT>) input);
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleProcessor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleProcessor.java
deleted file mode 100644
index a08e1b8..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/BundleProcessor.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.local.Bundle;
-
-/**
- * An executor that is capable of processing some bundle of input over some executable stage or
- * step.
- */
-interface BundleProcessor<
- CollectionT, BundleT extends Bundle<?, ? extends CollectionT>, ExecutableT> {
- /**
- * Execute the provided bundle using the provided Executable, calling back to the {@link
- * CompletionCallback} when execution completes.
- */
- void process(BundleT bundle, ExecutableT consumer, CompletionCallback onComplete);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedBundle.java
deleted file mode 100644
index 93926d6..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedBundle.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.local.Bundle;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-
-/**
- * Part of a {@link PCollection}. Elements are output to an {@link UncommittedBundle}, which will
- * eventually committed. Committed elements are executed by the {@link PTransform PTransforms} that
- * consume the {@link PCollection} this bundle is a part of at a later point.
- *
- * @param <T> the type of elements contained within this bundle
- */
-interface CommittedBundle<T> extends Bundle<T, PCollectionNode> {
- /** Returns the PCollection that the elements of this bundle belong to. */
- @Nullable
- @Override
- PCollectionNode getPCollection();
-
- /**
- * Returns the key that was output in the most recent {@code GroupByKey} in the execution of this
- * bundle.
- */
- @Override
- StructuralKey<?> getKey();
-
- /**
- * Returns an {@link Iterable} containing all of the elements that have been added to this {@link
- * CommittedBundle}.
- */
- Iterable<WindowedValue<T>> getElements();
-
- /**
- * Return the minimum timestamp among elements in this bundle.
- *
- * <p>This should be equivalent to iterating over all of the elements within a bundle and
- * selecting the minimum timestamp from among them.
- */
- @Override
- Instant getMinimumTimestamp();
-
- /**
- * Returns the processing time output watermark at the time the producing {@code Executable}
- * committed this bundle. Downstream synchronized processing time watermarks cannot progress past
- * this point before consuming this bundle.
- *
- * <p>This value is no greater than the earliest incomplete processing time or synchronized
- * processing time {@link TimerData timer} at the time this bundle was committed, including any
- * timers that fired to produce this bundle.
- */
- @Override
- Instant getSynchronizedProcessingOutputWatermark();
- /**
- * Return a new {@link CommittedBundle} that is like this one, except calls to {@link
- * #getElements()} will return the provided elements. This bundle is unchanged.
- *
- * <p>The value of the {@link #getSynchronizedProcessingOutputWatermark() synchronized processing
- * output watermark} of the returned {@link CommittedBundle} is equal to the value returned from
- * the current bundle. This is used to ensure a {@link PTransform} that could not complete
- * processing on input elements properly holds the synchronized processing time to the appropriate
- * value.
- */
- CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedResult.java
deleted file mode 100644
index 5487118..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CommittedResult.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import com.google.auto.value.AutoValue;
-import java.util.Set;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-
-/** A {@link TransformResult} that has been committed. */
-@AutoValue
-abstract class CommittedResult<ExecutableT> {
- /** Returns the {@link PTransformNode} that produced this result. */
- public abstract ExecutableT getExecutable();
-
- /**
- * Returns the {@link CommittedBundle} that contains the input elements that could not be
- * processed by the evaluation. The returned optional is present if there were any unprocessed
- * input elements, and absent otherwise.
- */
- public abstract Optional<? extends CommittedBundle<?>> getUnprocessedInputs();
- /** Returns the outputs produced by the transform. */
- public abstract Iterable<? extends CommittedBundle<?>> getOutputs();
-
- /**
- * Returns if the transform that produced this result produced outputs.
- *
- * <p>Transforms that produce output via modifying the state of the runner (e.g. {@link
- * CreatePCollectionView}) should explicitly set this to true. If {@link #getOutputs()} returns a
- * nonempty iterable, this will also return true.
- */
- public abstract Set<OutputType> getProducedOutputTypes();
-
- public static CommittedResult<PTransformNode> create(
- TransformResult<?> original,
- Optional<? extends CommittedBundle<?>> unprocessedElements,
- Iterable<? extends CommittedBundle<?>> outputs,
- Set<OutputType> producedOutputs) {
- return new AutoValue_CommittedResult<>(
- original.getTransform(), unprocessedElements, outputs, producedOutputs);
- }
-
- enum OutputType {
- PCOLLECTION_VIEW,
- BUNDLE
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CompletionCallback.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CompletionCallback.java
deleted file mode 100644
index 4d69044..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CompletionCallback.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-
-/** A callback for completing a bundle of input. */
-interface CompletionCallback {
- /** Handle a successful result, returning the committed outputs of the result. */
- CommittedResult handleResult(CommittedBundle<?> inputBundle, TransformResult<?> result);
-
- /**
- * Handle an input bundle that did not require processing.
- *
- * <p>This occurs when a Source has no splits that can currently produce outputs.
- */
- void handleEmpty(PTransformNode transform);
-
- /** Handle a result that terminated abnormally due to the provided {@link Exception}. */
- void handleException(CommittedBundle<?> inputBundle, Exception t);
-
- /**
- * Handle a result that terminated abnormally due to the provided {@link Error}. The pipeline
- * should be shut down, and the Error propagated.
- */
- void handleError(Error err);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternals.java
deleted file mode 100644
index d217121..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternals.java
+++ /dev/null
@@ -1,463 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryBag;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryCombiningState;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryMap;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemorySet;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryState;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryStateBinder;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryValue;
-import org.apache.beam.runners.core.InMemoryStateInternals.InMemoryWatermarkHold;
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateTable;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTag.StateBinder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.CombiningState;
-import org.apache.beam.sdk.state.MapState;
-import org.apache.beam.sdk.state.SetState;
-import org.apache.beam.sdk.state.State;
-import org.apache.beam.sdk.state.StateContext;
-import org.apache.beam.sdk.state.StateContexts;
-import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.state.WatermarkHoldState;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
-import org.apache.beam.sdk.util.CombineFnUtil;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.joda.time.Instant;
-
-/**
- * {@link StateInternals} built on top of an underlying {@link StateTable} that contains instances
- * of {@link InMemoryState}. Whenever state that exists in the underlying {@link StateTable} is
- * accessed, an independent copy will be created within this table.
- */
-class CopyOnAccessInMemoryStateInternals<K> implements StateInternals {
- private final CopyOnAccessInMemoryStateTable table;
-
- private K key;
-
- /**
- * Creates a new {@link CopyOnAccessInMemoryStateInternals} with the underlying (possibly null)
- * StateInternals.
- */
- public static <K> CopyOnAccessInMemoryStateInternals withUnderlying(
- K key, @Nullable CopyOnAccessInMemoryStateInternals underlying) {
- return new CopyOnAccessInMemoryStateInternals<>(key, underlying);
- }
-
- private CopyOnAccessInMemoryStateInternals(K key, CopyOnAccessInMemoryStateInternals underlying) {
- this.key = key;
- table = new CopyOnAccessInMemoryStateTable(underlying == null ? null : underlying.table);
- }
-
- /**
- * Ensures this {@link CopyOnAccessInMemoryStateInternals} is complete. Other copies of state for
- * the same Step and Key may be discarded after invoking this method.
- *
- * <p>For each {@link StateNamespace}, for each {@link StateTag address} in that namespace that
- * has not been bound in this {@link CopyOnAccessInMemoryStateInternals}, put a reference to that
- * state within this {@link StateInternals}.
- *
- * <p>Additionally, stores the {@link WatermarkHoldState} with the earliest time bound in the
- * state table after the commit is completed, enabling calls to {@link
- * #getEarliestWatermarkHold()}.
- *
- * @return this table
- */
- public CopyOnAccessInMemoryStateInternals<K> commit() {
- table.commit();
- return this;
- }
-
- /**
- * Gets the earliest Watermark Hold present in this table.
- *
- * <p>Must be called after this state has been committed. Will throw an {@link
- * IllegalStateException} if the state has not been committed.
- */
- public Instant getEarliestWatermarkHold() {
- // After commit, the watermark hold is always present, but may be
- // BoundedWindow#TIMESTAMP_MAX_VALUE if there is no hold set.
- checkState(
- table.earliestWatermarkHold.isPresent(),
- "Can't get the earliest watermark hold in a %s before it is committed",
- getClass().getSimpleName());
- return table.earliestWatermarkHold.get();
- }
-
- @Override
- public <T extends State> T state(
- StateNamespace namespace, StateTag<T> address, StateContext<?> c) {
- return table.get(namespace, address, c);
- }
-
- @Override
- public Object getKey() {
- return key;
- }
-
- public boolean isEmpty() {
- return Iterables.isEmpty(table.values());
- }
-
- /**
- * A {@link StateTable} that, when a value is retrieved with {@link StateTable#get(StateNamespace,
- * StateTag, StateContext)}, first attempts to obtain a copy of existing {@link State} from an
- * underlying {@link StateTable}.
- */
- private static class CopyOnAccessInMemoryStateTable extends StateTable {
- private Optional<StateTable> underlying;
-
- /**
- * The StateBinderFactory currently in use by this {@link CopyOnAccessInMemoryStateTable}.
- *
- * <p>There are three {@link StateBinderFactory} implementations used by the {@link
- * CopyOnAccessInMemoryStateTable}.
- *
- * <ul>
- * <li>The default {@link StateBinderFactory} is a {@link CopyOnBindBinderFactory}, allowing
- * the table to copy any existing {@link State} values to this {@link StateTable} from the
- * underlying table when accessed, at which point mutations will not be visible to the
- * underlying table - effectively a "Copy by Value" binder.
- * <li>During the execution of the {@link #commit()} method, this is a {@link
- * ReadThroughBinderFactory}, which copies the references to the existing {@link State}
- * objects to this {@link StateTable}.
- * <li>After the execution of the {@link #commit()} method, this is an instance of {@link
- * InMemoryStateBinderFactory}, which constructs new instances of state when a {@link
- * StateTag} is bound.
- * </ul>
- */
- private StateBinderFactory binderFactory;
-
- /** The earliest watermark hold in this table. */
- private Optional<Instant> earliestWatermarkHold;
-
- public CopyOnAccessInMemoryStateTable(StateTable underlying) {
- this.underlying = Optional.fromNullable(underlying);
- binderFactory = new CopyOnBindBinderFactory(this.underlying);
- earliestWatermarkHold = Optional.absent();
- }
-
- /**
- * Copies all values in the underlying table to this table, then discards the underlying table.
- *
- * <p>If there is an underlying table, this replaces the existing {@link
- * CopyOnBindBinderFactory} with a {@link ReadThroughBinderFactory}, then reads all of the
- * values in the existing table, binding the state values to this table. The old StateTable
- * should be discarded after the call to {@link #commit()}.
- *
- * <p>After copying all of the existing values, replace the binder factory with an instance of
- * {@link InMemoryStateBinderFactory} to construct new values, since all existing values are
- * bound in this {@link StateTable table} and this table represents the canonical state.
- */
- private void commit() {
- Instant earliestHold = getEarliestWatermarkHold();
- if (underlying.isPresent()) {
- ReadThroughBinderFactory readThroughBinder =
- new ReadThroughBinderFactory<>(underlying.get());
- binderFactory = readThroughBinder;
- Instant earliestUnderlyingHold = readThroughBinder.readThroughAndGetEarliestHold(this);
- if (earliestUnderlyingHold.isBefore(earliestHold)) {
- earliestHold = earliestUnderlyingHold;
- }
- }
- earliestWatermarkHold = Optional.of(earliestHold);
- clearEmpty();
- binderFactory = new InMemoryStateBinderFactory();
- underlying = Optional.absent();
- }
-
- /**
- * Get the earliest watermark hold in this table. Ignores the contents of any underlying table.
- */
- private Instant getEarliestWatermarkHold() {
- Instant earliest = BoundedWindow.TIMESTAMP_MAX_VALUE;
- for (State existingState : this.values()) {
- if (existingState instanceof WatermarkHoldState) {
- Instant hold = ((WatermarkHoldState) existingState).read();
- if (hold != null && hold.isBefore(earliest)) {
- earliest = hold;
- }
- }
- }
- return earliest;
- }
-
- /**
- * Clear all empty {@link StateNamespace StateNamespaces} from this table. If all states are
- * empty, clear the entire table.
- *
- * <p>Because {@link InMemoryState} is not removed from the {@link StateTable} after it is
- * cleared, in case contents are modified after being cleared, the table must be explicitly
- * checked to ensure that it contains state and removed if not (otherwise we may never use the
- * table again).
- */
- private void clearEmpty() {
- Collection<StateNamespace> emptyNamespaces = new HashSet<>(this.getNamespacesInUse());
- for (StateNamespace namespace : this.getNamespacesInUse()) {
- for (State existingState : this.getTagsInUse(namespace).values()) {
- if (!((InMemoryState<?>) existingState).isCleared()) {
- emptyNamespaces.remove(namespace);
- break;
- }
- }
- }
- for (StateNamespace empty : emptyNamespaces) {
- this.clearNamespace(empty);
- }
- }
-
- @Override
- protected StateBinder binderForNamespace(final StateNamespace namespace, StateContext<?> c) {
- return binderFactory.forNamespace(namespace, c);
- }
-
- private interface StateBinderFactory {
- StateBinder forNamespace(StateNamespace namespace, StateContext<?> c);
- }
-
- /**
- * {@link StateBinderFactory} that creates a copy of any existing state when the state is bound.
- */
- private static class CopyOnBindBinderFactory implements StateBinderFactory {
- private final Optional<StateTable> underlying;
-
- public CopyOnBindBinderFactory(Optional<StateTable> underlying) {
- this.underlying = underlying;
- }
-
- private boolean containedInUnderlying(StateNamespace namespace, StateTag<?> tag) {
- return underlying.isPresent()
- && underlying.get().isNamespaceInUse(namespace)
- && underlying.get().getTagsInUse(namespace).containsKey(tag);
- }
-
- @Override
- public StateBinder forNamespace(final StateNamespace namespace, final StateContext<?> c) {
- return new StateBinder() {
- @Override
- public WatermarkHoldState bindWatermark(
- StateTag<WatermarkHoldState> address, TimestampCombiner timestampCombiner) {
- if (containedInUnderlying(namespace, address)) {
- @SuppressWarnings("unchecked")
- InMemoryState<? extends WatermarkHoldState> existingState =
- (InMemoryState<? extends WatermarkHoldState>)
- underlying.get().get(namespace, address, c);
- return existingState.copy();
- } else {
- return new InMemoryWatermarkHold<>(timestampCombiner);
- }
- }
-
- @Override
- public <T> ValueState<T> bindValue(StateTag<ValueState<T>> address, Coder<T> coder) {
- if (containedInUnderlying(namespace, address)) {
- @SuppressWarnings("unchecked")
- InMemoryState<? extends ValueState<T>> existingState =
- (InMemoryState<? extends ValueState<T>>)
- underlying.get().get(namespace, address, c);
- return existingState.copy();
- } else {
- return new InMemoryValue<>(coder);
- }
- }
-
- @Override
- public <InputT, AccumT, OutputT>
- CombiningState<InputT, AccumT, OutputT> bindCombiningValue(
- StateTag<CombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- CombineFn<InputT, AccumT, OutputT> combineFn) {
- if (containedInUnderlying(namespace, address)) {
- @SuppressWarnings("unchecked")
- InMemoryState<? extends CombiningState<InputT, AccumT, OutputT>> existingState =
- (InMemoryState<? extends CombiningState<InputT, AccumT, OutputT>>)
- underlying.get().get(namespace, address, c);
- return existingState.copy();
- } else {
- return new InMemoryCombiningState<>(combineFn, accumCoder);
- }
- }
-
- @Override
- public <T> BagState<T> bindBag(StateTag<BagState<T>> address, Coder<T> elemCoder) {
- if (containedInUnderlying(namespace, address)) {
- @SuppressWarnings("unchecked")
- InMemoryState<? extends BagState<T>> existingState =
- (InMemoryState<? extends BagState<T>>)
- underlying.get().get(namespace, address, c);
- return existingState.copy();
- } else {
- return new InMemoryBag<>(elemCoder);
- }
- }
-
- @Override
- public <T> SetState<T> bindSet(StateTag<SetState<T>> address, Coder<T> elemCoder) {
- if (containedInUnderlying(namespace, address)) {
- @SuppressWarnings("unchecked")
- InMemoryState<? extends SetState<T>> existingState =
- (InMemoryState<? extends SetState<T>>)
- underlying.get().get(namespace, address, c);
- return existingState.copy();
- } else {
- return new InMemorySet<>(elemCoder);
- }
- }
-
- @Override
- public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
- StateTag<MapState<KeyT, ValueT>> address,
- Coder<KeyT> mapKeyCoder,
- Coder<ValueT> mapValueCoder) {
- if (containedInUnderlying(namespace, address)) {
- @SuppressWarnings("unchecked")
- InMemoryState<? extends MapState<KeyT, ValueT>> existingState =
- (InMemoryState<? extends MapState<KeyT, ValueT>>)
- underlying.get().get(namespace, address, c);
- return existingState.copy();
- } else {
- return new InMemoryMap<>(mapKeyCoder, mapValueCoder);
- }
- }
-
- @Override
- public <InputT, AccumT, OutputT>
- CombiningState<InputT, AccumT, OutputT> bindCombiningValueWithContext(
- StateTag<CombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- CombineFnWithContext<InputT, AccumT, OutputT> combineFn) {
- return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c));
- }
- };
- }
- }
-
- /**
- * {@link StateBinderFactory} that reads directly from the underlying table. Used during calls
- * to {@link CopyOnAccessInMemoryStateTable#commit()} to read all values from the underlying
- * table.
- */
- private static class ReadThroughBinderFactory<K> implements StateBinderFactory {
- private final StateTable underlying;
-
- public ReadThroughBinderFactory(StateTable underlying) {
- this.underlying = underlying;
- }
-
- public Instant readThroughAndGetEarliestHold(StateTable readTo) {
- Instant earliestHold = BoundedWindow.TIMESTAMP_MAX_VALUE;
- for (StateNamespace namespace : underlying.getNamespacesInUse()) {
- for (Map.Entry<StateTag, State> existingState :
- underlying.getTagsInUse(namespace).entrySet()) {
- if (!((InMemoryState<?>) existingState.getValue()).isCleared()) {
- // Only read through non-cleared values to ensure that completed windows are
- // eventually discarded, and remember the earliest watermark hold from among those
- // values.
- State state =
- readTo.get(namespace, existingState.getKey(), StateContexts.nullContext());
- if (state instanceof WatermarkHoldState) {
- Instant hold = ((WatermarkHoldState) state).read();
- if (hold != null && hold.isBefore(earliestHold)) {
- earliestHold = hold;
- }
- }
- }
- }
- }
- return earliestHold;
- }
-
- @Override
- public StateBinder forNamespace(final StateNamespace namespace, final StateContext<?> c) {
- return new StateBinder() {
- @Override
- public WatermarkHoldState bindWatermark(
- StateTag<WatermarkHoldState> address, TimestampCombiner timestampCombiner) {
- return underlying.get(namespace, address, c);
- }
-
- @Override
- public <T> ValueState<T> bindValue(StateTag<ValueState<T>> address, Coder<T> coder) {
- return underlying.get(namespace, address, c);
- }
-
- @Override
- public <InputT, AccumT, OutputT>
- CombiningState<InputT, AccumT, OutputT> bindCombiningValue(
- StateTag<CombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- CombineFn<InputT, AccumT, OutputT> combineFn) {
- return underlying.get(namespace, address, c);
- }
-
- @Override
- public <T> BagState<T> bindBag(StateTag<BagState<T>> address, Coder<T> elemCoder) {
- return underlying.get(namespace, address, c);
- }
-
- @Override
- public <T> SetState<T> bindSet(StateTag<SetState<T>> address, Coder<T> elemCoder) {
- return underlying.get(namespace, address, c);
- }
-
- @Override
- public <KeyT, ValueT> MapState<KeyT, ValueT> bindMap(
- StateTag<MapState<KeyT, ValueT>> address,
- Coder<KeyT> mapKeyCoder,
- Coder<ValueT> mapValueCoder) {
- return underlying.get(namespace, address, c);
- }
-
- @Override
- public <InputT, AccumT, OutputT>
- CombiningState<InputT, AccumT, OutputT> bindCombiningValueWithContext(
- StateTag<CombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- CombineFnWithContext<InputT, AccumT, OutputT> combineFn) {
- return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c));
- }
- };
- }
- }
-
- private static class InMemoryStateBinderFactory implements StateBinderFactory {
-
- public InMemoryStateBinderFactory() {}
-
- @Override
- public StateBinder forNamespace(StateNamespace namespace, StateContext<?> c) {
- return new InMemoryStateBinder(c);
- }
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectGroupByKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectGroupByKey.java
deleted file mode 100644
index 37b5430..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectGroupByKey.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.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItemCoder;
-import org.apache.beam.runners.core.construction.ForwardingPTransform;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.GroupByKey;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.WindowingStrategy;
-
-class DirectGroupByKey<K, V>
- extends ForwardingPTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> {
- private final PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> original;
-
- static final String DIRECT_GBKO_URN = "urn:beam:directrunner:transforms:gbko:v1";
- static final String DIRECT_GABW_URN = "urn:beam:directrunner:transforms:gabw:v1";
- private final WindowingStrategy<?, ?> outputWindowingStrategy;
-
- DirectGroupByKey(
- PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> original,
- WindowingStrategy<?, ?> outputWindowingStrategy) {
- this.original = original;
- this.outputWindowingStrategy = outputWindowingStrategy;
- }
-
- @Override
- public PTransform<PCollection<KV<K, V>>, PCollection<KV<K, Iterable<V>>>> delegate() {
- return original;
- }
-
- @Override
- public PCollection<KV<K, Iterable<V>>> expand(PCollection<KV<K, V>> input) {
- // This operation groups by the combination of key and window,
- // merging windows as needed, using the windows assigned to the
- // key/value input elements and the window merge operation of the
- // window function associated with the input PCollection.
- WindowingStrategy<?, ?> inputWindowingStrategy = input.getWindowingStrategy();
-
- // By default, implement GroupByKey via a series of lower-level operations.
- return input
- .apply(new DirectGroupByKeyOnly<>())
-
- // Group each key's values by window, merging windows as needed.
- .apply(
- "GroupAlsoByWindow",
- new DirectGroupAlsoByWindow<>(inputWindowingStrategy, outputWindowingStrategy));
- }
-
- static final class DirectGroupByKeyOnly<K, V>
- extends PTransform<PCollection<KV<K, V>>, PCollection<KeyedWorkItem<K, V>>> {
- @Override
- public PCollection<KeyedWorkItem<K, V>> expand(PCollection<KV<K, V>> input) {
- return PCollection.createPrimitiveOutputInternal(
- input.getPipeline(),
- WindowingStrategy.globalDefault(),
- input.isBounded(),
- KeyedWorkItemCoder.of(
- GroupByKey.getKeyCoder(input.getCoder()),
- GroupByKey.getInputValueCoder(input.getCoder()),
- input.getWindowingStrategy().getWindowFn().windowCoder()));
- }
-
- DirectGroupByKeyOnly() {}
- }
-
- static final class DirectGroupAlsoByWindow<K, V>
- extends PTransform<PCollection<KeyedWorkItem<K, V>>, PCollection<KV<K, Iterable<V>>>> {
-
- private final WindowingStrategy<?, ?> inputWindowingStrategy;
- private final WindowingStrategy<?, ?> outputWindowingStrategy;
-
- public DirectGroupAlsoByWindow(
- WindowingStrategy<?, ?> inputWindowingStrategy,
- WindowingStrategy<?, ?> outputWindowingStrategy) {
- this.inputWindowingStrategy = inputWindowingStrategy;
- this.outputWindowingStrategy = outputWindowingStrategy;
- }
-
- public WindowingStrategy<?, ?> getInputWindowingStrategy() {
- return inputWindowingStrategy;
- }
-
- private KeyedWorkItemCoder<K, V> getKeyedWorkItemCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
- // Coder<KV<...>> --> KvCoder<...>
- checkArgument(
- inputCoder instanceof KeyedWorkItemCoder,
- "%s requires a %s<...> but got %s",
- getClass().getSimpleName(),
- KvCoder.class.getSimpleName(),
- inputCoder);
- @SuppressWarnings("unchecked")
- KeyedWorkItemCoder<K, V> kvCoder = (KeyedWorkItemCoder<K, V>) inputCoder;
- return kvCoder;
- }
-
- public Coder<V> getValueCoder(Coder<KeyedWorkItem<K, V>> inputCoder) {
- return getKeyedWorkItemCoder(inputCoder).getElementCoder();
- }
-
- @Override
- public PCollection<KV<K, Iterable<V>>> expand(PCollection<KeyedWorkItem<K, V>> input) {
- KeyedWorkItemCoder<K, V> inputCoder = getKeyedWorkItemCoder(input.getCoder());
- return PCollection.createPrimitiveOutputInternal(
- input.getPipeline(),
- outputWindowingStrategy,
- input.isBounded(),
- KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getElementCoder())));
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectMetrics.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectMetrics.java
deleted file mode 100644
index a057260..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectMetrics.java
+++ /dev/null
@@ -1,307 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static java.util.Arrays.asList;
-
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicReference;
-import javax.annotation.Nullable;
-import javax.annotation.concurrent.GuardedBy;
-import org.apache.beam.runners.core.metrics.DistributionData;
-import org.apache.beam.runners.core.metrics.GaugeData;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
-import org.apache.beam.runners.core.metrics.MetricsMap;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.GaugeResult;
-import org.apache.beam.sdk.metrics.MetricFiltering;
-import org.apache.beam.sdk.metrics.MetricKey;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.MetricResult;
-import org.apache.beam.sdk.metrics.MetricResults;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/** Implementation of {@link MetricResults} for the Direct Runner. */
-class DirectMetrics extends MetricResults {
-
- // TODO: (BEAM-723) Create a shared ExecutorService for maintenance tasks in the DirectRunner.
- private static final ExecutorService COUNTER_COMMITTER =
- Executors.newCachedThreadPool(
- new ThreadFactoryBuilder()
- .setThreadFactory(MoreExecutors.platformThreadFactory())
- .setDaemon(true)
- .setNameFormat("direct-metrics-counter-committer")
- .build());
-
- private interface MetricAggregation<UpdateT, ResultT> {
- UpdateT zero();
-
- UpdateT combine(Iterable<UpdateT> updates);
-
- ResultT extract(UpdateT data);
- }
-
- /**
- * Implementation of a metric in the direct runner.
- *
- * @param <UpdateT> The type of raw data received and aggregated across updates.
- * @param <ResultT> The type of result extracted from the data.
- */
- private static class DirectMetric<UpdateT, ResultT> {
- private final MetricAggregation<UpdateT, ResultT> aggregation;
-
- private final AtomicReference<UpdateT> finishedCommitted;
-
- private final Object attemptedLock = new Object();
-
- @GuardedBy("attemptedLock")
- private volatile UpdateT finishedAttempted;
-
- private final ConcurrentMap<CommittedBundle<?>, UpdateT> inflightAttempted =
- new ConcurrentHashMap<>();
-
- public DirectMetric(MetricAggregation<UpdateT, ResultT> aggregation) {
- this.aggregation = aggregation;
- finishedCommitted = new AtomicReference<>(aggregation.zero());
- finishedAttempted = aggregation.zero();
- }
-
- /**
- * Add the given {@code tentativeCumulative} update to the physical aggregate.
- *
- * @param bundle The bundle receiving an update.
- * @param tentativeCumulative The new cumulative value for the given bundle.
- */
- public void updatePhysical(CommittedBundle<?> bundle, UpdateT tentativeCumulative) {
- // Add (or update) the cumulatiev value for the given bundle.
- inflightAttempted.put(bundle, tentativeCumulative);
- }
-
- /**
- * Commit a physical value for the given {@code bundle}.
- *
- * @param bundle The bundle being committed.
- * @param finalCumulative The final cumulative value for the given bundle.
- */
- @SuppressWarnings("FutureReturnValueIgnored") // direct runner metrics are best-effort;
- // we choose not to block on async commit
- public void commitPhysical(final CommittedBundle<?> bundle, final UpdateT finalCumulative) {
- // To prevent a query from blocking the commit, we perform the commit in two steps.
- // 1. We perform a non-blocking write to the uncommitted table to make the new value
- // available immediately.
- // 2. We submit a runnable that will commit the update and remove the tentative value in
- // a synchronized block.
- inflightAttempted.put(bundle, finalCumulative);
- COUNTER_COMMITTER.submit(
- () -> {
- synchronized (attemptedLock) {
- finishedAttempted = aggregation.combine(asList(finishedAttempted, finalCumulative));
- inflightAttempted.remove(bundle);
- }
- });
- }
-
- /** Extract the latest values from all attempted and in-progress bundles. */
- public ResultT extractLatestAttempted() {
- ArrayList<UpdateT> updates = new ArrayList<>(inflightAttempted.size() + 1);
- // Within this block we know that will be consistent. Specifically, the only change that can
- // happen concurrently is the addition of new (larger) values to inflightAttempted.
- synchronized (attemptedLock) {
- updates.add(finishedAttempted);
- updates.addAll(inflightAttempted.values());
- }
- return aggregation.extract(aggregation.combine(updates));
- }
-
- /**
- * Commit a logical value for the given {@code bundle}.
- *
- * @param bundle The bundle being committed.
- * @param finalCumulative The final cumulative value for the given bundle.
- */
- public void commitLogical(final CommittedBundle<?> bundle, final UpdateT finalCumulative) {
- UpdateT current;
- do {
- current = finishedCommitted.get();
- } while (!finishedCommitted.compareAndSet(
- current, aggregation.combine(asList(current, finalCumulative))));
- }
-
- /** Extract the value from all successfully committed bundles. */
- public ResultT extractCommitted() {
- return aggregation.extract(finishedCommitted.get());
- }
- }
-
- private static final MetricAggregation<Long, Long> COUNTER =
- new MetricAggregation<Long, Long>() {
- @Override
- public Long zero() {
- return 0L;
- }
-
- @Override
- public Long combine(Iterable<Long> updates) {
- long value = 0;
- for (long update : updates) {
- value += update;
- }
- return value;
- }
-
- @Override
- public Long extract(Long data) {
- return data;
- }
- };
-
- private static final MetricAggregation<DistributionData, DistributionResult> DISTRIBUTION =
- new MetricAggregation<DistributionData, DistributionResult>() {
- @Override
- public DistributionData zero() {
- return DistributionData.EMPTY;
- }
-
- @Override
- public DistributionData combine(Iterable<DistributionData> updates) {
- DistributionData result = DistributionData.EMPTY;
- for (DistributionData update : updates) {
- result = result.combine(update);
- }
- return result;
- }
-
- @Override
- public DistributionResult extract(DistributionData data) {
- return data.extractResult();
- }
- };
-
- private static final MetricAggregation<GaugeData, GaugeResult> GAUGE =
- new MetricAggregation<GaugeData, GaugeResult>() {
- @Override
- public GaugeData zero() {
- return GaugeData.empty();
- }
-
- @Override
- public GaugeData combine(Iterable<GaugeData> updates) {
- GaugeData result = GaugeData.empty();
- for (GaugeData update : updates) {
- result = result.combine(update);
- }
- return result;
- }
-
- @Override
- public GaugeResult extract(GaugeData data) {
- return data.extractResult();
- }
- };
-
- /** The current values of counters in memory. */
- private MetricsMap<MetricKey, DirectMetric<Long, Long>> counters =
- new MetricsMap<>(unusedKey -> new DirectMetric<>(COUNTER));
-
- private MetricsMap<MetricKey, DirectMetric<DistributionData, DistributionResult>> distributions =
- new MetricsMap<>(unusedKey -> new DirectMetric<>(DISTRIBUTION));
- private MetricsMap<MetricKey, DirectMetric<GaugeData, GaugeResult>> gauges =
- new MetricsMap<>(unusedKey -> new DirectMetric<>(GAUGE));
-
- @Override
- public MetricQueryResults queryMetrics(@Nullable MetricsFilter filter) {
- ImmutableList.Builder<MetricResult<Long>> counterResults = ImmutableList.builder();
- for (Entry<MetricKey, DirectMetric<Long, Long>> counter : counters.entries()) {
- maybeExtractResult(filter, counterResults, counter);
- }
- ImmutableList.Builder<MetricResult<DistributionResult>> distributionResults =
- ImmutableList.builder();
- for (Entry<MetricKey, DirectMetric<DistributionData, DistributionResult>> distribution :
- distributions.entries()) {
- maybeExtractResult(filter, distributionResults, distribution);
- }
- ImmutableList.Builder<MetricResult<GaugeResult>> gaugeResults = ImmutableList.builder();
- for (Entry<MetricKey, DirectMetric<GaugeData, GaugeResult>> gauge : gauges.entries()) {
- maybeExtractResult(filter, gaugeResults, gauge);
- }
-
- return MetricQueryResults.create(
- counterResults.build(), distributionResults.build(), gaugeResults.build());
- }
-
- private <ResultT> void maybeExtractResult(
- MetricsFilter filter,
- ImmutableList.Builder<MetricResult<ResultT>> resultsBuilder,
- Map.Entry<MetricKey, ? extends DirectMetric<?, ResultT>> entry) {
- if (MetricFiltering.matches(filter, entry.getKey())) {
- resultsBuilder.add(
- MetricResult.create(
- entry.getKey(),
- entry.getValue().extractCommitted(),
- entry.getValue().extractLatestAttempted()));
- }
- }
-
- /** Apply metric updates that represent physical counter deltas to the current metric values. */
- public void updatePhysical(CommittedBundle<?> bundle, MetricUpdates updates) {
- for (MetricUpdate<Long> counter : updates.counterUpdates()) {
- counters.get(counter.getKey()).updatePhysical(bundle, counter.getUpdate());
- }
- for (MetricUpdate<DistributionData> distribution : updates.distributionUpdates()) {
- distributions.get(distribution.getKey()).updatePhysical(bundle, distribution.getUpdate());
- }
- for (MetricUpdate<GaugeData> gauge : updates.gaugeUpdates()) {
- gauges.get(gauge.getKey()).updatePhysical(bundle, gauge.getUpdate());
- }
- }
-
- public void commitPhysical(CommittedBundle<?> bundle, MetricUpdates updates) {
- for (MetricUpdate<Long> counter : updates.counterUpdates()) {
- counters.get(counter.getKey()).commitPhysical(bundle, counter.getUpdate());
- }
- for (MetricUpdate<DistributionData> distribution : updates.distributionUpdates()) {
- distributions.get(distribution.getKey()).commitPhysical(bundle, distribution.getUpdate());
- }
- for (MetricUpdate<GaugeData> gauge : updates.gaugeUpdates()) {
- gauges.get(gauge.getKey()).commitPhysical(bundle, gauge.getUpdate());
- }
- }
-
- /** Apply metric updates that represent new logical values from a bundle being committed. */
- public void commitLogical(CommittedBundle<?> bundle, MetricUpdates updates) {
- for (MetricUpdate<Long> counter : updates.counterUpdates()) {
- counters.get(counter.getKey()).commitLogical(bundle, counter.getUpdate());
- }
- for (MetricUpdate<DistributionData> distribution : updates.distributionUpdates()) {
- distributions.get(distribution.getKey()).commitLogical(bundle, distribution.getUpdate());
- }
- for (MetricUpdate<GaugeData> gauge : updates.gaugeUpdates()) {
- gauges.get(gauge.getKey()).commitLogical(bundle, gauge.getUpdate());
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectStateAndTimers.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectStateAndTimers.java
deleted file mode 100644
index ca36781..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectStateAndTimers.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.direct.Clock;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.local.StructuralKey;
-
-/**
- * State and Timer access for the {@link ReferenceRunner}.
- *
- * <p>This provides per-key, per-stage access to {@link CopyOnAccessInMemoryStateInternals} and
- * {@link DirectTimerInternals} for transforms that require access to state or timers.
- *
- * <p>This implementation is not thread safe. A new {@link DirectStateAndTimers} must be created for
- * each thread that requires it.
- */
-class DirectStateAndTimers<K> implements StepStateAndTimers<K> {
- private final StructuralKey<K> key;
- private final CopyOnAccessInMemoryStateInternals existingState;
-
- private final Clock clock;
- private final TransformWatermarks watermarks;
-
- private CopyOnAccessInMemoryStateInternals<K> stateInternals;
- private DirectTimerInternals timerInternals;
-
- DirectStateAndTimers(
- StructuralKey<K> key,
- CopyOnAccessInMemoryStateInternals existingState,
- Clock clock,
- TransformWatermarks watermarks) {
- this.key = key;
- this.existingState = existingState;
- this.clock = clock;
- this.watermarks = watermarks;
- }
-
- @Override
- public CopyOnAccessInMemoryStateInternals<K> stateInternals() {
- if (stateInternals == null) {
- stateInternals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, existingState);
- }
- return stateInternals;
- }
-
- @Override
- public DirectTimerInternals timerInternals() {
- if (timerInternals == null) {
- timerInternals = DirectTimerInternals.create(clock, watermarks, TimerUpdate.builder(key));
- }
- return timerInternals;
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTimerInternals.java
deleted file mode 100644
index 15d7b6c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTimerInternals.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.direct.Clock;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.joda.time.Instant;
-
-/** An implementation of {@link TimerInternals} where all relevant data exists in memory. */
-class DirectTimerInternals implements TimerInternals {
- private final Clock processingTimeClock;
- private final TransformWatermarks watermarks;
- private final TimerUpdateBuilder timerUpdateBuilder;
-
- public static DirectTimerInternals create(
- Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
- return new DirectTimerInternals(clock, watermarks, timerUpdateBuilder);
- }
-
- private DirectTimerInternals(
- Clock clock, TransformWatermarks watermarks, TimerUpdateBuilder timerUpdateBuilder) {
- this.processingTimeClock = clock;
- this.watermarks = watermarks;
- this.timerUpdateBuilder = timerUpdateBuilder;
- }
-
- @Override
- public void setTimer(
- StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
- timerUpdateBuilder.setTimer(TimerData.of(timerId, namespace, target, timeDomain));
- }
-
- /** @deprecated use {@link #setTimer(StateNamespace, String, Instant, TimeDomain)}. */
- @Deprecated
- @Override
- public void setTimer(TimerData timerData) {
- timerUpdateBuilder.setTimer(timerData);
- }
-
- @Override
- public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
- throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
- }
-
- /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
- @Deprecated
- @Override
- public void deleteTimer(StateNamespace namespace, String timerId) {
- throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
- }
-
- /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
- @Deprecated
- @Override
- public void deleteTimer(TimerData timerKey) {
- timerUpdateBuilder.deletedTimer(timerKey);
- }
-
- public TimerUpdate getTimerUpdate() {
- return timerUpdateBuilder.build();
- }
-
- @Override
- public Instant currentProcessingTime() {
- return processingTimeClock.now();
- }
-
- @Override
- @Nullable
- public Instant currentSynchronizedProcessingTime() {
- return watermarks.getSynchronizedProcessingInputTime();
- }
-
- @Override
- public Instant currentInputWatermarkTime() {
- return watermarks.getInputWatermark();
- }
-
- @Override
- @Nullable
- public Instant currentOutputWatermarkTime() {
- return watermarks.getOutputWatermark();
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTransformExecutor.java
deleted file mode 100644
index b4e742a..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DirectTransformExecutor.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.io.Closeable;
-import java.util.concurrent.Callable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.core.metrics.MetricsContainerImpl;
-import org.apache.beam.sdk.metrics.MetricsEnvironment;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a {@link
- * TransformEvaluatorFactory} and evaluating it on some bundle of input, and registering the result
- * using a registered {@link CompletionCallback}.
- */
-class DirectTransformExecutor<T> implements TransformExecutor {
- private static final Logger LOG = LoggerFactory.getLogger(DirectTransformExecutor.class);
-
- static class Factory implements TransformExecutorFactory {
- private final EvaluationContext context;
- private final TransformEvaluatorRegistry registry;
-
- Factory(EvaluationContext context, TransformEvaluatorRegistry registry) {
- this.context = context;
- this.registry = registry;
- }
-
- @Override
- public TransformExecutor create(
- CommittedBundle<?> bundle,
- PTransformNode transform,
- CompletionCallback onComplete,
- TransformExecutorService executorService) {
- return new DirectTransformExecutor<>(
- context, registry, bundle, transform, onComplete, executorService);
- }
- }
-
- private final TransformEvaluatorRegistry evaluatorRegistry;
-
- /** The transform that will be evaluated. */
- private final PTransformNode transform;
- /** The inputs this {@link DirectTransformExecutor} will deliver to the transform. */
- private final CommittedBundle<T> inputBundle;
-
- private final CompletionCallback onComplete;
- private final TransformExecutorService transformEvaluationState;
- private final EvaluationContext context;
-
- @VisibleForTesting
- DirectTransformExecutor(
- EvaluationContext context,
- TransformEvaluatorRegistry factory,
- CommittedBundle<T> inputBundle,
- PTransformNode transform,
- CompletionCallback completionCallback,
- TransformExecutorService transformEvaluationState) {
- this.evaluatorRegistry = factory;
-
- this.inputBundle = inputBundle;
- this.transform = transform;
-
- this.onComplete = completionCallback;
-
- this.transformEvaluationState = transformEvaluationState;
- this.context = context;
- }
-
- @Override
- public void run() {
- MetricsContainerImpl metricsContainer = new MetricsContainerImpl(transform.getId());
- try (Closeable metricsScope = MetricsEnvironment.scopedMetricsContainer(metricsContainer)) {
- TransformEvaluator<T> evaluator = evaluatorRegistry.forApplication(transform, inputBundle);
- if (evaluator == null) {
- onComplete.handleEmpty(transform);
- // Nothing to do
- return;
- }
-
- processElements(evaluator, metricsContainer);
-
- finishBundle(evaluator, metricsContainer);
- } catch (Exception e) {
- onComplete.handleException(inputBundle, e);
- if (e instanceof RuntimeException) {
- throw (RuntimeException) e;
- }
- throw new RuntimeException(e);
- } catch (Error err) {
- LOG.error("Error occurred within {}", this, err);
- onComplete.handleError(err);
- throw err;
- } finally {
- // Report the physical metrics from the end of this step.
- context.getMetrics().commitPhysical(inputBundle, metricsContainer.getCumulative());
-
- transformEvaluationState.complete(this);
- }
- }
-
- /** Processes all the elements in the input bundle using the transform evaluator. */
- private void processElements(
- TransformEvaluator<T> evaluator, MetricsContainerImpl metricsContainer) throws Exception {
- if (inputBundle != null) {
- for (WindowedValue<T> value : inputBundle.getElements()) {
- evaluator.processElement(value);
-
- // Report the physical metrics after each element
- MetricUpdates deltas = metricsContainer.getUpdates();
- if (deltas != null) {
- context.getMetrics().updatePhysical(inputBundle, deltas);
- metricsContainer.commitUpdates();
- }
- }
- }
- }
-
- /**
- * Finishes processing the input bundle and commit the result using the {@link
- * CompletionCallback}.
- *
- * @return the {@link TransformResult} produced by {@link TransformEvaluator#finishBundle()}
- */
- private TransformResult<T> finishBundle(
- TransformEvaluator<T> evaluator, MetricsContainerImpl metricsContainer) throws Exception {
- TransformResult<T> result =
- evaluator.finishBundle().withLogicalMetricUpdates(metricsContainer.getCumulative());
- onComplete.handleResult(inputBundle, result);
- return result;
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DisplayDataValidator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DisplayDataValidator.java
deleted file mode 100644
index c01dd62..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/DisplayDataValidator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.runners.TransformHierarchy;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.transforms.display.HasDisplayData;
-
-/**
- * Validate correct implementation of {@link DisplayData} by evaluating {@link
- * HasDisplayData#populateDisplayData(DisplayData.Builder)} during pipeline construction.
- */
-class DisplayDataValidator {
- // Do not instantiate
- private DisplayDataValidator() {}
-
- static void validatePipeline(Pipeline pipeline) {
- validateTransforms(pipeline);
- }
-
- static void validateOptions(PipelineOptions options) {
- evaluateDisplayData(options);
- }
-
- private static void validateTransforms(Pipeline pipeline) {
- pipeline.traverseTopologically(Visitor.INSTANCE);
- }
-
- private static void evaluateDisplayData(HasDisplayData component) {
- DisplayData.from(component);
- }
-
- private static class Visitor extends Pipeline.PipelineVisitor.Defaults {
- private static final Visitor INSTANCE = new Visitor();
-
- @Override
- public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) {
- if (!node.isRootNode()) {
- evaluateDisplayData(node.getTransform());
- }
-
- return CompositeBehavior.ENTER_TRANSFORM;
- }
-
- @Override
- public void visitPrimitiveTransform(TransformHierarchy.Node node) {
- evaluateDisplayData(node.getTransform());
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EmptyInputProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EmptyInputProvider.java
deleted file mode 100644
index 82edcef..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EmptyInputProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-
-/** A {@link RootInputProvider} that provides no input bundles. */
-class EmptyInputProvider implements RootInputProvider<Void> {
- EmptyInputProvider() {}
-
- /**
- * {@inheritDoc}.
- *
- * <p>Returns an empty collection.
- */
- @Override
- public Collection<CommittedBundle<Void>> getInitialInputs(
- PTransformNode transform, int targetParallelism) {
- return Collections.emptyList();
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContext.java
deleted file mode 100644
index a1b57a9..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContext.java
+++ /dev/null
@@ -1,333 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.Clock;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.WatermarkManager;
-import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.joda.time.Instant;
-
-/**
- * The evaluation context for a specific pipeline being executed by the {@code DirectRunner}.
- * Contains state shared within the execution across all transforms.
- *
- * <p>{@link EvaluationContext} contains shared state for an execution of the {@code DirectRunner}
- * that can be used while evaluating a {@link PTransform}. This consists of views into underlying
- * state and watermark implementations, access to read and write {@link PCollectionView
- * PCollectionViews}, and managing the {@link DirectStateAndTimers ExecutionContexts}. This includes
- * executing callbacks asynchronously when state changes to the appropriate point (e.g. when a
- * {@link PCollectionView} is requested and known to be empty).
- *
- * <p>{@link EvaluationContext} also handles results by committing finalizing bundles based on the
- * current global state and updating the global state appropriately. This includes updating the
- * per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that can be
- * executed.
- */
-class EvaluationContext {
- /** The graph representing this {@link Pipeline}. */
- private final ExecutableGraph<PTransformNode, ? super PCollectionNode> graph;
-
- private final Clock clock;
-
- private final BundleFactory bundleFactory;
-
- /** The current processing time and event time watermarks and timers. */
- private final WatermarkManager<PTransformNode, ? super PCollectionNode> watermarkManager;
-
- /** Executes callbacks based on the progression of the watermark. */
- private final WatermarkCallbackExecutor callbackExecutor;
-
- /** The stateInternals of the world, by applied PTransform and key. */
- private final ConcurrentMap<StepAndKey, CopyOnAccessInMemoryStateInternals>
- applicationStateInternals;
-
- private final DirectMetrics metrics;
-
- private final Set<PCollectionNode> keyedPValues;
-
- public static EvaluationContext create(
- Clock clock,
- BundleFactory bundleFactory,
- ExecutableGraph<PTransformNode, ? super PCollectionNode> graph,
- Set<PCollectionNode> keyedPValues) {
- return new EvaluationContext(clock, bundleFactory, graph, keyedPValues);
- }
-
- private EvaluationContext(
- Clock clock,
- BundleFactory bundleFactory,
- ExecutableGraph<PTransformNode, ? super PCollectionNode> graph,
- Set<PCollectionNode> keyedPValues) {
- this.clock = clock;
- this.bundleFactory = checkNotNull(bundleFactory);
- this.graph = checkNotNull(graph);
- this.keyedPValues = keyedPValues;
-
- this.watermarkManager = WatermarkManager.create(clock, graph, PTransformNode::getId);
-
- this.applicationStateInternals = new ConcurrentHashMap<>();
- this.metrics = new DirectMetrics();
-
- this.callbackExecutor = WatermarkCallbackExecutor.create(MoreExecutors.directExecutor());
- }
-
- public void initialize(
- Map<PTransformNode, ? extends Iterable<CommittedBundle<?>>> initialInputs) {
- watermarkManager.initialize((Map) initialInputs);
- }
-
- /**
- * Handle the provided {@link TransformResult}, produced after evaluating the provided {@link
- * CommittedBundle} (potentially null, if the result of a root {@link PTransform}).
- *
- * <p>The result is the output of running the transform contained in the {@link TransformResult}
- * on the contents of the provided bundle.
- *
- * @param completedBundle the bundle that was processed to produce the result. Potentially {@code
- * null} if the transform that produced the result is a root transform
- * @param completedTimers the timers that were delivered to produce the {@code completedBundle},
- * or an empty iterable if no timers were delivered
- * @param result the result of evaluating the input bundle
- * @return the committed bundles contained within the handled {@code result}
- */
- public CommittedResult<PTransformNode> handleResult(
- CommittedBundle<?> completedBundle,
- Iterable<TimerData> completedTimers,
- TransformResult<?> result) {
- Iterable<? extends CommittedBundle<?>> committedBundles =
- commitBundles(result.getOutputBundles());
- metrics.commitLogical(completedBundle, result.getLogicalMetricUpdates());
-
- // Update watermarks and timers
- EnumSet<OutputType> outputTypes = EnumSet.copyOf(result.getOutputTypes());
- if (Iterables.isEmpty(committedBundles)) {
- outputTypes.remove(OutputType.BUNDLE);
- } else {
- outputTypes.add(OutputType.BUNDLE);
- }
- CommittedResult<PTransformNode> committedResult =
- CommittedResult.create(
- result, getUnprocessedInput(completedBundle, result), committedBundles, outputTypes);
- // Update state internals
- CopyOnAccessInMemoryStateInternals theirState = result.getState();
- if (theirState != null) {
- CopyOnAccessInMemoryStateInternals committedState = theirState.commit();
- StepAndKey stepAndKey = StepAndKey.of(result.getTransform(), completedBundle.getKey());
- if (!committedState.isEmpty()) {
- applicationStateInternals.put(stepAndKey, committedState);
- } else {
- applicationStateInternals.remove(stepAndKey);
- }
- }
- // Watermarks are updated last to ensure visibility of any global state before progress is
- // permitted
- watermarkManager.updateWatermarks(
- completedBundle,
- result.getTimerUpdate().withCompletedTimers(completedTimers),
- committedResult.getExecutable(),
- committedResult.getUnprocessedInputs().orNull(),
- committedResult.getOutputs(),
- result.getWatermarkHold());
- return committedResult;
- }
-
- /**
- * Returns an {@link Optional} containing a bundle which contains all of the unprocessed elements
- * that were not processed from the {@code completedBundle}. If all of the elements of the {@code
- * completedBundle} were processed, or if {@code completedBundle} is null, returns an absent
- * {@link Optional}.
- */
- private Optional<? extends CommittedBundle<?>> getUnprocessedInput(
- CommittedBundle<?> completedBundle, TransformResult<?> result) {
- if (completedBundle == null || Iterables.isEmpty(result.getUnprocessedElements())) {
- return Optional.absent();
- }
- CommittedBundle<?> residual =
- completedBundle.withElements((Iterable) result.getUnprocessedElements());
- return Optional.of(residual);
- }
-
- private Iterable<? extends CommittedBundle<?>> commitBundles(
- Iterable<? extends UncommittedBundle<?>> bundles) {
- ImmutableList.Builder<CommittedBundle<?>> completed = ImmutableList.builder();
- for (UncommittedBundle<?> inProgress : bundles) {
- PTransformNode producing = graph.getProducer(inProgress.getPCollection());
- TransformWatermarks watermarks = watermarkManager.getWatermarks(producing);
- CommittedBundle<?> committed =
- inProgress.commit(watermarks.getSynchronizedProcessingOutputTime());
- // Empty bundles don't impact watermarks and shouldn't trigger downstream execution, so
- // filter them out
- if (!Iterables.isEmpty(committed.getElements())) {
- completed.add(committed);
- }
- }
- return completed.build();
- }
-
- private void fireAllAvailableCallbacks() {
- for (PTransformNode transform : graph.getExecutables()) {
- fireAvailableCallbacks(transform);
- }
- }
-
- private void fireAvailableCallbacks(PTransformNode producingTransform) {
- TransformWatermarks watermarks = watermarkManager.getWatermarks(producingTransform);
- Instant outputWatermark = watermarks.getOutputWatermark();
- callbackExecutor.fireForWatermark(producingTransform, outputWatermark);
- }
-
- /** Create a {@link UncommittedBundle} for use by a source. */
- public <T> UncommittedBundle<T> createRootBundle() {
- return bundleFactory.createRootBundle();
- }
-
- /**
- * Create a {@link UncommittedBundle} whose elements belong to the specified {@link PCollection}.
- */
- public <T> UncommittedBundle<T> createBundle(PCollectionNode output) {
- return bundleFactory.createBundle(output);
- }
-
- /**
- * Create a {@link UncommittedBundle} with the specified keys at the specified step. For use by
- * {@code DirectGroupByKeyOnly} {@link PTransform PTransforms}.
- */
- public <K, T> UncommittedBundle<T> createKeyedBundle(
- StructuralKey<K> key, PCollectionNode output) {
- return bundleFactory.createKeyedBundle(key, output);
- }
-
- /** Indicate whether or not this {@link PCollection} has been determined to be keyed. */
- public <T> boolean isKeyed(PCollectionNode pValue) {
- return keyedPValues.contains(pValue);
- }
-
- /**
- * Schedule a callback to be executed after output would be produced for the given window if there
- * had been input.
- *
- * <p>Output would be produced when the watermark for a {@link PValue} passes the point at which
- * the trigger for the specified window (with the specified windowing strategy) must have fired
- * from the perspective of that {@link PValue}, as specified by the value of {@link
- * Trigger#getWatermarkThatGuaranteesFiring(BoundedWindow)} for the trigger of the {@link
- * WindowingStrategy}. When the callback has fired, either values will have been produced for a
- * key in that window, the window is empty, or all elements in the window are late. The callback
- * will be executed regardless of whether values have been produced.
- */
- public void scheduleAfterOutputWouldBeProduced(
- PCollectionNode value,
- BoundedWindow window,
- WindowingStrategy<?, ?> windowingStrategy,
- Runnable runnable) {
- PTransformNode producing = graph.getProducer(value);
- callbackExecutor.callOnWindowExpiration(producing, window, windowingStrategy, runnable);
-
- fireAvailableCallbacks(producing);
- }
-
- /** Get a {@link DirectStateAndTimers} for the provided {@link PTransformNode} and key. */
- public <K> StepStateAndTimers<K> getStateAndTimers(
- PTransformNode application, StructuralKey<K> key) {
- StepAndKey stepAndKey = StepAndKey.of(application, key);
- return new DirectStateAndTimers<>(
- key,
- applicationStateInternals.get(stepAndKey),
- clock,
- watermarkManager.getWatermarks(application));
- }
-
- /** Returns all of the steps in this {@link Pipeline}. */
- Collection<PTransformNode> getSteps() {
- return graph.getExecutables();
- }
-
- /** Returns the metrics container for this pipeline. */
- public DirectMetrics getMetrics() {
- return metrics;
- }
-
- @VisibleForTesting
- void forceRefresh() {
- watermarkManager.refreshAll();
- fireAllAvailableCallbacks();
- }
-
- /**
- * Extracts all timers that have been fired and have not already been extracted.
- *
- * <p>This is a destructive operation. Timers will only appear in the result of this method once
- * for each time they are set.
- */
- public Collection<FiredTimers<PTransformNode>> extractFiredTimers() {
- forceRefresh();
- return watermarkManager.extractFiredTimers();
- }
-
- /** Returns true if the step will not produce additional output. */
- public boolean isDone(PTransformNode transform) {
- // the PTransform is done only if watermark is at the max value
- Instant stepWatermark = watermarkManager.getWatermarks(transform).getOutputWatermark();
- return !stepWatermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-
- /** Returns true if all steps are done. */
- public boolean isDone() {
- for (PTransformNode transform : graph.getExecutables()) {
- if (!isDone(transform)) {
- return false;
- }
- }
- return true;
- }
-
- public Instant now() {
- return clock.now();
- }
-
- Clock getClock() {
- return clock;
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContextStepStateAndTimersProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContextStepStateAndTimersProvider.java
deleted file mode 100644
index f6ef49c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/EvaluationContextStepStateAndTimersProvider.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.local.StructuralKey;
-
-/** A {@link StepStateAndTimers.Provider} that uses an {@link EvaluationContext}. */
-class EvaluationContextStepStateAndTimersProvider implements StepStateAndTimers.Provider {
- public static StepStateAndTimers.Provider forContext(EvaluationContext context) {
- return new EvaluationContextStepStateAndTimersProvider(context);
- }
-
- private final EvaluationContext context;
-
- private EvaluationContextStepStateAndTimersProvider(EvaluationContext context) {
- this.context = context;
- }
-
- @Override
- public <K> StepStateAndTimers<K> forStepAndKey(PTransformNode transform, StructuralKey<K> key) {
- return context.getStateAndTimers(transform, key);
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceFactory.java
deleted file mode 100644
index 8b097ad..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.concurrent.ExecutorService;
-
-/**
- * A factory that creates {@link ExecutorService ExecutorServices}. {@link ExecutorService
- * ExecutorServices} created by this factory should be independent of one another (e.g., if any
- * executor is shut down the remaining executors should continue to process work).
- */
-interface ExecutorServiceFactory {
- /** Create a new {@link ExecutorService}. */
- ExecutorService create();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceParallelExecutor.java
deleted file mode 100644
index 1a7a99a..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ExecutorServiceParallelExecutor.java
+++ /dev/null
@@ -1,392 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.local.ExecutionDriver;
-import org.apache.beam.runners.local.ExecutionDriver.DriverState;
-import org.apache.beam.runners.local.PipelineMessageReceiver;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.util.UserCodeException;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.CacheBuilder;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.CacheLoader;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.LoadingCache;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.cache.RemovalListener;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An {@link PipelineExecutor} that uses an underlying {@link ExecutorService} and {@link
- * EvaluationContext} to execute a {@link Pipeline}.
- */
-final class ExecutorServiceParallelExecutor
- implements PipelineExecutor,
- BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode> {
- private static final Logger LOG = LoggerFactory.getLogger(ExecutorServiceParallelExecutor.class);
-
- private final int targetParallelism;
- private final ExecutorService executorService;
-
- private final RootProviderRegistry rootRegistry;
- private final TransformEvaluatorRegistry transformRegistry;
-
- private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
- private final EvaluationContext evaluationContext;
-
- private final TransformExecutorFactory executorFactory;
- private final TransformExecutorService parallelExecutorService;
- private final LoadingCache<StepAndKey, TransformExecutorService> serialExecutorServices;
-
- private final QueueMessageReceiver visibleUpdates;
-
- private AtomicReference<State> pipelineState = new AtomicReference<>(State.RUNNING);
-
- public static ExecutorServiceParallelExecutor create(
- int targetParallelism,
- RootProviderRegistry rootRegistry,
- TransformEvaluatorRegistry transformRegistry,
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- EvaluationContext context) {
- return new ExecutorServiceParallelExecutor(
- targetParallelism, rootRegistry, transformRegistry, graph, context);
- }
-
- private ExecutorServiceParallelExecutor(
- int targetParallelism,
- RootProviderRegistry rootRegistry,
- TransformEvaluatorRegistry transformRegistry,
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- EvaluationContext context) {
- this.targetParallelism = targetParallelism;
- // Don't use Daemon threads for workers. The Pipeline should continue to execute even if there
- // are no other active threads (for example, because waitUntilFinish was not called)
- this.executorService =
- Executors.newFixedThreadPool(
- targetParallelism,
- new ThreadFactoryBuilder()
- .setThreadFactory(MoreExecutors.platformThreadFactory())
- .setNameFormat("direct-runner-worker")
- .build());
- this.rootRegistry = rootRegistry;
- this.transformRegistry = transformRegistry;
- this.graph = graph;
- this.evaluationContext = context;
-
- // Weak Values allows TransformExecutorServices that are no longer in use to be reclaimed.
- // Executing TransformExecutorServices have a strong reference to their TransformExecutorService
- // which stops the TransformExecutorServices from being prematurely garbage collected
- serialExecutorServices =
- CacheBuilder.newBuilder()
- .weakValues()
- .removalListener(shutdownExecutorServiceListener())
- .build(serialTransformExecutorServiceCacheLoader());
-
- this.visibleUpdates = new QueueMessageReceiver();
-
- parallelExecutorService = TransformExecutorServices.parallel(executorService);
- executorFactory = new DirectTransformExecutor.Factory(context, transformRegistry);
- }
-
- private CacheLoader<StepAndKey, TransformExecutorService>
- serialTransformExecutorServiceCacheLoader() {
- return new CacheLoader<StepAndKey, TransformExecutorService>() {
- @Override
- public TransformExecutorService load(StepAndKey stepAndKey) throws Exception {
- return TransformExecutorServices.serial(executorService);
- }
- };
- }
-
- private RemovalListener<StepAndKey, TransformExecutorService> shutdownExecutorServiceListener() {
- return notification -> {
- TransformExecutorService service = notification.getValue();
- if (service != null) {
- service.shutdown();
- }
- };
- }
-
- @Override
- // TODO: [BEAM-4563] Pass Future back to consumer to check for async errors
- @SuppressWarnings("FutureReturnValueIgnored")
- public void start() {
- int numTargetSplits = Math.max(3, targetParallelism);
- ImmutableMap.Builder<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>>
- pendingRootBundles = ImmutableMap.builder();
- for (PTransformNode root : graph.getRootTransforms()) {
- ConcurrentLinkedQueue<CommittedBundle<?>> pending = new ConcurrentLinkedQueue<>();
- try {
- Collection<CommittedBundle<?>> initialInputs =
- rootRegistry.getInitialInputs(root, numTargetSplits);
- pending.addAll(initialInputs);
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- pendingRootBundles.put(root, pending);
- }
- evaluationContext.initialize(pendingRootBundles.build());
- final ExecutionDriver executionDriver =
- QuiescenceDriver.create(
- evaluationContext, graph, this, visibleUpdates, pendingRootBundles.build());
- executorService.submit(
- new Runnable() {
- @Override
- public void run() {
- DriverState drive = executionDriver.drive();
- if (drive.isTermainal()) {
- State newPipelineState = State.UNKNOWN;
- switch (drive) {
- case FAILED:
- newPipelineState = State.FAILED;
- break;
- case SHUTDOWN:
- newPipelineState = State.DONE;
- break;
- case CONTINUE:
- throw new IllegalStateException(
- String.format("%s should not be a terminal state", DriverState.CONTINUE));
- default:
- throw new IllegalArgumentException(
- String.format("Unknown %s %s", DriverState.class.getSimpleName(), drive));
- }
- shutdownIfNecessary(newPipelineState);
- } else {
- executorService.submit(this);
- }
- }
- });
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void process(
- CommittedBundle<?> bundle, PTransformNode consumer, CompletionCallback onComplete) {
- evaluateBundle(consumer, bundle, onComplete);
- }
-
- private <T> void evaluateBundle(
- final PTransformNode transform,
- final CommittedBundle<T> bundle,
- final CompletionCallback onComplete) {
- TransformExecutorService transformExecutor;
-
- if (isKeyed(bundle.getPCollection())) {
- final StepAndKey stepAndKey = StepAndKey.of(transform, bundle.getKey());
- // This executor will remain reachable until it has executed all scheduled transforms.
- // The TransformExecutors keep a strong reference to the Executor, the ExecutorService keeps
- // a reference to the scheduled DirectTransformExecutor callable. Follow-up TransformExecutors
- // (scheduled due to the completion of another DirectTransformExecutor) are provided to the
- // ExecutorService before the Earlier DirectTransformExecutor callable completes.
- transformExecutor = serialExecutorServices.getUnchecked(stepAndKey);
- } else {
- transformExecutor = parallelExecutorService;
- }
-
- TransformExecutor callable =
- executorFactory.create(bundle, transform, onComplete, transformExecutor);
- if (!pipelineState.get().isTerminal()) {
- transformExecutor.schedule(callable);
- }
- }
-
- private boolean isKeyed(PCollectionNode pvalue) {
- return evaluationContext.isKeyed(pvalue);
- }
-
- @Override
- public State waitUntilFinish(Duration duration) throws Exception {
- Instant completionTime;
- if (duration.equals(Duration.ZERO)) {
- completionTime = new Instant(Long.MAX_VALUE);
- } else {
- completionTime = Instant.now().plus(duration);
- }
-
- VisibleExecutorUpdate update = null;
- while (Instant.now().isBefore(completionTime)
- && (update == null || isTerminalStateUpdate(update))) {
- // Get an update; don't block forever if another thread has handled it. The call to poll will
- // wait the entire timeout; this call primarily exists to relinquish any core.
- update = visibleUpdates.tryNext(Duration.millis(25L));
- if (update == null && pipelineState.get().isTerminal()) {
- // there are no updates to process and no updates will ever be published because the
- // executor is shutdown
- return pipelineState.get();
- } else if (update != null && update.thrown.isPresent()) {
- Throwable thrown = update.thrown.get();
- if (thrown instanceof Exception) {
- throw (Exception) thrown;
- } else if (thrown instanceof Error) {
- throw (Error) thrown;
- } else {
- throw new Exception("Unknown Type of Throwable", thrown);
- }
- }
- }
- return pipelineState.get();
- }
-
- @Override
- public State getPipelineState() {
- return pipelineState.get();
- }
-
- private boolean isTerminalStateUpdate(VisibleExecutorUpdate update) {
- return !(update.getNewState() == null && update.getNewState().isTerminal());
- }
-
- @Override
- public void stop() {
- shutdownIfNecessary(State.CANCELLED);
- visibleUpdates.cancelled();
- }
-
- private void shutdownIfNecessary(State newState) {
- if (!newState.isTerminal()) {
- return;
- }
- LOG.debug("Pipeline has terminated. Shutting down.");
-
- final Collection<Exception> errors = new ArrayList<>();
- // Stop accepting new work before shutting down the executor. This ensures that thread don't try
- // to add work to the shutdown executor.
- try {
- serialExecutorServices.invalidateAll();
- } catch (final RuntimeException re) {
- errors.add(re);
- }
- try {
- serialExecutorServices.cleanUp();
- } catch (final RuntimeException re) {
- errors.add(re);
- }
- try {
- parallelExecutorService.shutdown();
- } catch (final RuntimeException re) {
- errors.add(re);
- }
- try {
- executorService.shutdown();
- } catch (final RuntimeException re) {
- errors.add(re);
- }
- try {
- transformRegistry.cleanup();
- } catch (final Exception e) {
- errors.add(e);
- }
- pipelineState.compareAndSet(State.RUNNING, newState); // ensure we hit a terminal node
- if (!errors.isEmpty()) {
- final IllegalStateException exception =
- new IllegalStateException(
- "Error"
- + (errors.size() == 1 ? "" : "s")
- + " during executor shutdown:\n"
- + errors.stream()
- .map(Exception::getMessage)
- .collect(Collectors.joining("\n- ", "- ", "")));
- visibleUpdates.failed(exception);
- throw exception;
- }
- }
-
- /**
- * An update of interest to the user. Used in {@link #waitUntilFinish} to decide whether to return
- * normally or throw an exception.
- */
- private static class VisibleExecutorUpdate {
- private final Optional<? extends Throwable> thrown;
- @Nullable private final State newState;
-
- public static VisibleExecutorUpdate fromException(Exception e) {
- return new VisibleExecutorUpdate(null, e);
- }
-
- public static VisibleExecutorUpdate fromError(Error err) {
- return new VisibleExecutorUpdate(State.FAILED, err);
- }
-
- public static VisibleExecutorUpdate finished() {
- return new VisibleExecutorUpdate(State.DONE, null);
- }
-
- public static VisibleExecutorUpdate cancelled() {
- return new VisibleExecutorUpdate(State.CANCELLED, null);
- }
-
- private VisibleExecutorUpdate(State newState, @Nullable Throwable exception) {
- this.thrown = Optional.fromNullable(exception);
- this.newState = newState;
- }
-
- State getNewState() {
- return newState;
- }
- }
-
- private static class QueueMessageReceiver implements PipelineMessageReceiver {
- // If the type of BlockingQueue changes, ensure the findbugs filter is updated appropriately
- private final BlockingQueue<VisibleExecutorUpdate> updates = new LinkedBlockingQueue<>();
-
- @Override
- public void failed(Exception e) {
- updates.offer(VisibleExecutorUpdate.fromException(e));
- }
-
- @Override
- public void failed(Error e) {
- updates.offer(VisibleExecutorUpdate.fromError(e));
- }
-
- @Override
- public void cancelled() {
- updates.offer(VisibleExecutorUpdate.cancelled());
- }
-
- @Override
- public void completed() {
- updates.offer(VisibleExecutorUpdate.finished());
- }
-
- /** Try to get the next unconsumed message in this {@link QueueMessageReceiver}. */
- @Nullable
- private VisibleExecutorUpdate tryNext(Duration timeout) throws InterruptedException {
- return updates.poll(timeout.getMillis(), TimeUnit.MILLISECONDS);
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactory.java
deleted file mode 100644
index 31654e0..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactory.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link Flatten} {@link
- * PTransform}.
- */
-class FlattenEvaluatorFactory implements TransformEvaluatorFactory {
- private final BundleFactory bundleFactory;
- private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
- FlattenEvaluatorFactory(
- ExecutableGraph<PTransformNode, PCollectionNode> graph, BundleFactory bundleFactory) {
- this.bundleFactory = bundleFactory;
- this.graph = graph;
- }
-
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) {
- @SuppressWarnings({"cast", "unchecked", "rawtypes"})
- TransformEvaluator<InputT> evaluator = createInMemoryEvaluator(application);
- return evaluator;
- }
-
- @Override
- public void cleanup() {}
-
- private <InputT> TransformEvaluator<InputT> createInMemoryEvaluator(
- final PTransformNode transform) {
- return new FlattenEvaluator<>(transform);
- }
-
- private class FlattenEvaluator<InputT> implements TransformEvaluator<InputT> {
- private final PTransformNode transform;
- private final UncommittedBundle<InputT> bundle;
-
- FlattenEvaluator(PTransformNode transform) {
- this.transform = transform;
- PCollectionNode output = getOnlyElement(graph.getProduced(transform));
- bundle = bundleFactory.createBundle(output);
- }
-
- @Override
- public void processElement(WindowedValue<InputT> element) {
- bundle.add(element);
- }
-
- @Override
- public TransformResult<InputT> finishBundle() {
- return StepTransformResult.<InputT>withoutHold(transform).addOutput(bundle).build();
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupAlsoByWindowEvaluatorFactory.java
deleted file mode 100644
index 1c73908..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupAlsoByWindowEvaluatorFactory.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.runners.core.GroupAlsoByWindowsAggregators;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.OutputWindowedValue;
-import org.apache.beam.runners.core.ReduceFnRunner;
-import org.apache.beam.runners.core.SystemReduceFn;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.construction.RehydratedComponents;
-import org.apache.beam.runners.core.construction.TriggerTranslation;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
-import org.apache.beam.runners.core.triggers.TriggerStateMachines;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.portable.DirectGroupByKey.DirectGroupAlsoByWindow;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.IterableLikeCoder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.metrics.Counter;
-import org.apache.beam.sdk.metrics.Metrics;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowTracing;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.joda.time.Instant;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@code
- * DirectGroupAlsoByWindow} {@link PTransform}.
- */
-class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
- private final BundleFactory bundleFactory;
- private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
- private final Components components;
- private final StepStateAndTimers.Provider stp;
-
- GroupAlsoByWindowEvaluatorFactory(
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- Components components,
- BundleFactory bundleFactory,
- StepStateAndTimers.Provider stp) {
- this.bundleFactory = bundleFactory;
- this.graph = graph;
- this.components = components;
- this.stp = stp;
- }
-
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) {
- @SuppressWarnings({"unchecked", "rawtypes"})
- TransformEvaluator<InputT> evaluator =
- createEvaluator(application, (CommittedBundle) inputBundle);
- return evaluator;
- }
-
- @Override
- public void cleanup() {}
-
- private <K, V> TransformEvaluator<KeyedWorkItem<K, V>> createEvaluator(
- PTransformNode application, CommittedBundle<KeyedWorkItem<K, V>> inputBundle) {
- @SuppressWarnings("unchecked")
- StructuralKey<K> key = (StructuralKey<K>) inputBundle.getKey();
- return new GroupAlsoByWindowEvaluator<>(
- bundleFactory, key, application, graph, components, stp.forStepAndKey(application, key));
- }
-
- /**
- * A transform evaluator for the pseudo-primitive {@code DirectGroupAlsoByWindow}. The window of
- * the input {@link KeyedWorkItem} is ignored; it should be in the global window, as element
- * windows are reified in the {@link KeyedWorkItem#elementsIterable()}.
- *
- * @see GroupByKeyViaGroupByKeyOnly
- */
- private static class GroupAlsoByWindowEvaluator<K, V>
- implements TransformEvaluator<KeyedWorkItem<K, V>> {
- private final BundleFactory bundleFactory;
-
- private final PTransformNode application;
- private final PCollectionNode outputCollection;
-
- private final StructuralKey<?> key;
-
- private final CopyOnAccessInMemoryStateInternals<K> stateInternals;
- private final DirectTimerInternals timerInternals;
- private final WindowingStrategy<?, BoundedWindow> windowingStrategy;
-
- private final Collection<UncommittedBundle<?>> outputBundles;
-
- private final SystemReduceFn<K, V, Iterable<V>, Iterable<V>, BoundedWindow> reduceFn;
- private final Counter droppedDueToLateness;
-
- private GroupAlsoByWindowEvaluator(
- BundleFactory bundleFactory,
- StructuralKey<K> key,
- PTransformNode application,
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- Components components,
- StepStateAndTimers<K> stp) {
- this.bundleFactory = bundleFactory;
- this.application = application;
- this.outputCollection = getOnlyElement(graph.getProduced(application));
- this.key = key;
-
- this.stateInternals = stp.stateInternals();
- this.timerInternals = stp.timerInternals();
-
- PCollectionNode inputCollection = getOnlyElement(graph.getPerElementInputs(application));
- try {
- windowingStrategy =
- (WindowingStrategy<?, BoundedWindow>)
- RehydratedComponents.forComponents(components)
- .getWindowingStrategy(
- inputCollection.getPCollection().getWindowingStrategyId());
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- outputBundles = new ArrayList<>();
-
- Coder<V> valueCoder;
- try {
- Coder<WindowedValue<KV<K, Iterable<V>>>> windowedValueCoder =
- WireCoders.instantiateRunnerWireCoder(outputCollection, components);
- checkArgument(windowedValueCoder instanceof WindowedValue.WindowedValueCoder);
- Coder<KV<K, Iterable<V>>> outputKvCoder =
- ((WindowedValueCoder<KV<K, Iterable<V>>>) windowedValueCoder).getValueCoder();
- checkArgument(outputKvCoder instanceof KvCoder);
- Coder<Iterable<V>> iterVCoder = ((KvCoder<K, Iterable<V>>) outputKvCoder).getValueCoder();
- checkArgument(iterVCoder instanceof IterableLikeCoder);
- valueCoder = ((IterableLikeCoder<V, ?>) iterVCoder).getElemCoder();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- reduceFn = SystemReduceFn.buffering(valueCoder);
- droppedDueToLateness =
- Metrics.counter(
- GroupAlsoByWindowEvaluator.class,
- GroupAlsoByWindowsAggregators.DROPPED_DUE_TO_LATENESS_COUNTER);
- }
-
- @Override
- public void processElement(WindowedValue<KeyedWorkItem<K, V>> element) throws Exception {
- KeyedWorkItem<K, V> workItem = element.getValue();
-
- UncommittedBundle<KV<K, Iterable<V>>> bundle =
- bundleFactory.createKeyedBundle(this.key, outputCollection);
- outputBundles.add(bundle);
- RunnerApi.Trigger runnerApiTrigger =
- TriggerTranslation.toProto(windowingStrategy.getTrigger());
- ReduceFnRunner<K, V, Iterable<V>, BoundedWindow> reduceFnRunner =
- new ReduceFnRunner<>(
- workItem.key(),
- windowingStrategy,
- ExecutableTriggerStateMachine.create(
- TriggerStateMachines.stateMachineForTrigger(runnerApiTrigger)),
- stateInternals,
- timerInternals,
- new OutputWindowedValueToBundle<>(bundle),
- null,
- reduceFn,
- null);
-
- // Drop any elements within expired windows
- reduceFnRunner.processElements(
- dropExpiredWindows(workItem.key(), workItem.elementsIterable(), timerInternals));
- reduceFnRunner.onTimers(workItem.timersIterable());
- reduceFnRunner.persist();
- }
-
- @Override
- public TransformResult<KeyedWorkItem<K, V>> finishBundle() throws Exception {
- // State is initialized within the constructor. It can never be null.
- CopyOnAccessInMemoryStateInternals<?> state = stateInternals.commit();
- return StepTransformResult.<KeyedWorkItem<K, V>>withHold(
- application, state.getEarliestWatermarkHold())
- .withState(state)
- .addOutput(outputBundles)
- .withTimerUpdate(timerInternals.getTimerUpdate())
- .build();
- }
-
- /**
- * Returns an {@code Iterable<WindowedValue<InputT>>} that only contains non-late input
- * elements.
- */
- Iterable<WindowedValue<V>> dropExpiredWindows(
- final K key, Iterable<WindowedValue<V>> elements, final TimerInternals timerInternals) {
- return StreamSupport.stream(elements.spliterator(), false)
- .flatMap(wv -> StreamSupport.stream(wv.explodeWindows().spliterator(), false))
- .filter(
- input -> {
- BoundedWindow window = getOnlyElement(input.getWindows());
- boolean expired =
- window
- .maxTimestamp()
- .plus(windowingStrategy.getAllowedLateness())
- .isBefore(timerInternals.currentInputWatermarkTime());
- if (expired) {
- // The element is too late for this window.
- droppedDueToLateness.inc();
- WindowTracing.debug(
- "{}: Dropping element at {} for key: {}; "
- + "window: {} since it is too far behind inputWatermark: {}",
- DirectGroupAlsoByWindow.class.getSimpleName(),
- input.getTimestamp(),
- key,
- window,
- timerInternals.currentInputWatermarkTime());
- }
- // Keep the element if the window is not expired.
- return !expired;
- })
- .collect(Collectors.toList());
- }
- }
-
- private static class OutputWindowedValueToBundle<K, V>
- implements OutputWindowedValue<KV<K, Iterable<V>>> {
- private final UncommittedBundle<KV<K, Iterable<V>>> bundle;
-
- private OutputWindowedValueToBundle(UncommittedBundle<KV<K, Iterable<V>>> bundle) {
- this.bundle = bundle;
- }
-
- @Override
- public void outputWindowedValue(
- KV<K, Iterable<V>> output,
- Instant timestamp,
- Collection<? extends BoundedWindow> windows,
- PaneInfo pane) {
- bundle.add(WindowedValue.of(output, timestamp, windows, pane));
- }
-
- @Override
- public <AdditionalOutputT> void outputWindowedValue(
- TupleTag<AdditionalOutputT> tag,
- AdditionalOutputT output,
- Instant timestamp,
- Collection<? extends BoundedWindow> windows,
- PaneInfo pane) {
- throw new UnsupportedOperationException(
- String.format("%s should not use tagged outputs", "DirectGroupAlsoByWindow"));
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactory.java
deleted file mode 100644
index 46fee32..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactory.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components.Builder;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly;
-import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.construction.RehydratedComponents;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-
-/**
- * The {@code DirectRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKeyOnly} {@link
- * PTransform}.
- */
-class GroupByKeyOnlyEvaluatorFactory implements TransformEvaluatorFactory {
- private final Components components;
-
- private final BundleFactory bundleFactory;
- private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
- GroupByKeyOnlyEvaluatorFactory(
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- Components components,
- BundleFactory bundleFactory) {
- this.components = components;
- this.bundleFactory = bundleFactory;
- this.graph = graph;
- }
-
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) {
- @SuppressWarnings({"cast", "unchecked", "rawtypes"})
- TransformEvaluator<InputT> evaluator = (TransformEvaluator) createEvaluator(application);
- return evaluator;
- }
-
- @Override
- public void cleanup() {}
-
- private <K, V> TransformEvaluator<KV<K, V>> createEvaluator(final PTransformNode application) {
- return new GroupByKeyOnlyEvaluator<>(application);
- }
-
- /**
- * A transform evaluator for the pseudo-primitive {@link GroupByKeyOnly}. Windowing is ignored;
- * all input should be in the global window since all output will be as well.
- *
- * @see GroupByKeyViaGroupByKeyOnly
- */
- private class GroupByKeyOnlyEvaluator<K, V> implements TransformEvaluator<KV<K, V>> {
- private final Coder<K> keyCoder;
- private final Map<StructuralKey<K>, List<WindowedValue<V>>> groupingMap;
-
- private final PCollectionNode outputPCollection;
- private final StepTransformResult.Builder<KV<K, V>> resultBuilder;
-
- private GroupByKeyOnlyEvaluator(PTransformNode application) {
- keyCoder = getKeyCoder(application);
- groupingMap = new HashMap<>();
- outputPCollection = getOnlyElement(graph.getProduced(application));
- resultBuilder = StepTransformResult.withoutHold(application);
- }
-
- private Coder<K> getKeyCoder(PTransformNode application) {
- PCollectionNode inputPCollection = getOnlyElement(graph.getPerElementInputs(application));
- try {
- // We know the type restrictions on the input PCollection, and the restrictions on the
- // Wire coder
- Builder builder = GroupByKeyOnlyEvaluatorFactory.this.components.toBuilder();
- String wireCoderId = WireCoders.addRunnerWireCoder(inputPCollection, builder);
- Coder<WindowedValue<KV<K, V>>> wireCoder =
- (Coder<WindowedValue<KV<K, V>>>)
- RehydratedComponents.forComponents(builder.build()).getCoder(wireCoderId);
-
- checkArgument(
- wireCoder instanceof WindowedValue.WindowedValueCoder,
- "Wire %s must be a %s",
- Coder.class.getSimpleName(),
- WindowedValueCoder.class.getSimpleName());
- WindowedValueCoder<KV<K, V>> windowedValueCoder = (WindowedValueCoder<KV<K, V>>) wireCoder;
-
- checkArgument(
- windowedValueCoder.getValueCoder() instanceof KvCoder,
- "Input elements to %s must be encoded with a %s",
- DirectGroupByKey.DirectGroupByKeyOnly.class.getSimpleName(),
- KvCoder.class.getSimpleName());
- KvCoder<K, V> kvCoder = (KvCoder<K, V>) windowedValueCoder.getValueCoder();
-
- return kvCoder.getKeyCoder();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public void processElement(WindowedValue<KV<K, V>> element) {
- KV<K, V> kv = element.getValue();
- K key = kv.getKey();
- StructuralKey<K> groupingKey = StructuralKey.of(key, keyCoder);
- List<WindowedValue<V>> values =
- groupingMap.computeIfAbsent(groupingKey, k -> new ArrayList<>());
- values.add(element.withValue(kv.getValue()));
- }
-
- @Override
- public TransformResult<KV<K, V>> finishBundle() {
- for (Map.Entry<StructuralKey<K>, List<WindowedValue<V>>> groupedEntry :
- groupingMap.entrySet()) {
- K key = groupedEntry.getKey().getKey();
- KeyedWorkItem<K, V> groupedKv =
- KeyedWorkItems.elementsWorkItem(key, groupedEntry.getValue());
- UncommittedBundle<KeyedWorkItem<K, V>> bundle =
- bundleFactory.createKeyedBundle(StructuralKey.of(key, keyCoder), outputPCollection);
- bundle.add(WindowedValue.valueInGlobalWindow(groupedKv));
- resultBuilder.addOutput(bundle);
- }
- return resultBuilder.build();
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImmutableListBundleFactory.java
deleted file mode 100644
index 1531e25..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImmutableListBundleFactory.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import com.google.auto.value.AutoValue;
-import java.util.Iterator;
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.MoreObjects;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.joda.time.Instant;
-
-/** A factory that produces bundles that perform no additional validation. */
-class ImmutableListBundleFactory implements BundleFactory {
- private static final ImmutableListBundleFactory FACTORY = new ImmutableListBundleFactory();
-
- public static ImmutableListBundleFactory create() {
- return FACTORY;
- }
-
- private ImmutableListBundleFactory() {}
-
- @Override
- public <T> UncommittedBundle<T> createRootBundle() {
- return UncommittedImmutableListBundle.create(null, StructuralKey.empty());
- }
-
- @Override
- public <T> UncommittedBundle<T> createBundle(PCollectionNode output) {
- return UncommittedImmutableListBundle.create(output, StructuralKey.empty());
- }
-
- @Override
- public <K, T> UncommittedBundle<T> createKeyedBundle(
- StructuralKey<K> key, PCollectionNode output) {
- return UncommittedImmutableListBundle.create(output, key);
- }
-
- /** A {@link UncommittedBundle} that buffers elements in memory. */
- private static final class UncommittedImmutableListBundle<T> implements UncommittedBundle<T> {
- private final PCollectionNode pcollection;
- private final StructuralKey<?> key;
- private boolean committed = false;
- private ImmutableList.Builder<WindowedValue<T>> elements;
- private Instant minSoFar = BoundedWindow.TIMESTAMP_MAX_VALUE;
-
- /**
- * Create a new {@link UncommittedImmutableListBundle} for the specified {@link PCollection}.
- */
- public static <T> UncommittedImmutableListBundle<T> create(
- PCollectionNode pcollection, StructuralKey<?> key) {
- return new UncommittedImmutableListBundle<>(pcollection, key);
- }
-
- private UncommittedImmutableListBundle(PCollectionNode pcollection, StructuralKey<?> key) {
- this.pcollection = pcollection;
- this.key = key;
- this.elements = ImmutableList.builder();
- }
-
- @Override
- public PCollectionNode getPCollection() {
- return pcollection;
- }
-
- @Override
- public UncommittedImmutableListBundle<T> add(WindowedValue<T> element) {
- checkState(
- !committed,
- "Can't add element %s to committed bundle in PCollection %s",
- element,
- pcollection);
- checkArgument(
- element.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE),
- "Can't add an element past the end of time (%s), got timestamp %s",
- BoundedWindow.TIMESTAMP_MAX_VALUE,
- element.getTimestamp());
- elements.add(element);
- if (element.getTimestamp().isBefore(minSoFar)) {
- minSoFar = element.getTimestamp();
- }
- return this;
- }
-
- @Override
- public CommittedBundle<T> commit(final Instant synchronizedCompletionTime) {
- checkState(!committed, "Can't commit already committed bundle %s", this);
- committed = true;
- final Iterable<WindowedValue<T>> committedElements = elements.build();
- return CommittedImmutableListBundle.create(
- pcollection, key, committedElements, minSoFar, synchronizedCompletionTime);
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(this).add("elements", elements.build()).toString();
- }
- }
-
- @AutoValue
- abstract static class CommittedImmutableListBundle<T> implements CommittedBundle<T> {
- public static <T> CommittedImmutableListBundle<T> create(
- @Nullable PCollectionNode pcollection,
- StructuralKey<?> key,
- Iterable<WindowedValue<T>> committedElements,
- Instant minElementTimestamp,
- Instant synchronizedCompletionTime) {
- return new AutoValue_ImmutableListBundleFactory_CommittedImmutableListBundle<>(
- pcollection, key, committedElements, minElementTimestamp, synchronizedCompletionTime);
- }
-
- @Override
- @Nonnull
- public Iterator<WindowedValue<T>> iterator() {
- return getElements().iterator();
- }
-
- @Override
- public CommittedBundle<T> withElements(Iterable<WindowedValue<T>> elements) {
- return create(
- getPCollection(),
- getKey(),
- ImmutableList.copyOf(elements),
- minTimestamp(elements),
- getSynchronizedProcessingOutputWatermark());
- }
-
- @Override
- public int hashCode() {
- return System.identityHashCode(this);
- }
-
- @Override
- public boolean equals(Object obj) {
- return this == obj;
- }
- }
-
- private static Instant minTimestamp(Iterable<? extends WindowedValue<?>> elements) {
- Instant minTs = BoundedWindow.TIMESTAMP_MAX_VALUE;
- for (WindowedValue<?> element : elements) {
- if (element.getTimestamp().isBefore(minTs)) {
- minTs = element.getTimestamp();
- }
- }
- return minTs;
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImpulseEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImpulseEvaluatorFactory.java
deleted file mode 100644
index c2997ef..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ImpulseEvaluatorFactory.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.util.Collection;
-import java.util.Collections;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.sdk.transforms.Impulse;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-
-/** The evaluator for the {@link Impulse} transform. Produces only empty byte arrays. */
-class ImpulseEvaluatorFactory implements TransformEvaluatorFactory {
- private final BundleFactory bundleFactory;
- private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
- ImpulseEvaluatorFactory(
- ExecutableGraph<PTransformNode, PCollectionNode> graph, BundleFactory bundleFactory) {
- this.bundleFactory = bundleFactory;
- this.graph = graph;
- }
-
- @Nullable
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) {
- return (TransformEvaluator<InputT>)
- new ImpulseEvaluator(
- bundleFactory, application, getOnlyElement(graph.getProduced(application)));
- }
-
- @Override
- public void cleanup() {
- // Impulse has no state, so do nothing.
- }
-
- private static class ImpulseEvaluator implements TransformEvaluator<ImpulseShard> {
- private final StepTransformResult.Builder<ImpulseShard> result;
-
- private final BundleFactory factory;
- private final PCollectionNode outputPCollection;
-
- private ImpulseEvaluator(
- BundleFactory factory, PTransformNode application, PCollectionNode outputPCollection) {
- this.factory = factory;
- result = StepTransformResult.withoutHold(application);
- this.outputPCollection = outputPCollection;
- }
-
- @Override
- public void processElement(WindowedValue<ImpulseShard> element) throws Exception {
- result.addOutput(
- factory
- .createBundle(outputPCollection)
- .add(WindowedValue.valueInGlobalWindow(new byte[0])));
- }
-
- @Override
- public TransformResult<ImpulseShard> finishBundle() throws Exception {
- return result.build();
- }
- }
-
- /**
- * The {@link RootInputProvider} for the {@link Impulse} {@link PTransform}. Produces a single
- * {@link ImpulseShard}.
- */
- static class ImpulseRootProvider implements RootInputProvider<ImpulseShard> {
- private final BundleFactory bundleFactory;
-
- ImpulseRootProvider(BundleFactory bundleFactory) {
- this.bundleFactory = bundleFactory;
- }
-
- @Override
- public Collection<CommittedBundle<ImpulseShard>> getInitialInputs(
- PTransformNode transform, int targetParallelism) {
- return Collections.singleton(
- bundleFactory
- .<ImpulseShard>createRootBundle()
- .add(WindowedValue.valueInGlobalWindow(new ImpulseShard()))
- .commit(BoundedWindow.TIMESTAMP_MIN_VALUE));
- }
- }
-
- @VisibleForTesting
- static class ImpulseShard {}
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWindow.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWindow.java
deleted file mode 100644
index a632daf..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWindow.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Objects;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.PCollectionView;
-
-/**
- * A pair of a {@link PCollectionView} and a {@link BoundedWindow}, which can be thought of as
- * window "of" the view. This is a value class for use e.g. as a compound cache key.
- *
- * @param <T> the type of the underlying PCollectionView
- */
-final class PCollectionViewWindow<T> {
-
- private final PCollectionView<T> view;
- private final BoundedWindow window;
-
- private PCollectionViewWindow(PCollectionView<T> view, BoundedWindow window) {
- this.view = view;
- this.window = window;
- }
-
- public static <T> PCollectionViewWindow<T> of(PCollectionView<T> view, BoundedWindow window) {
- return new PCollectionViewWindow<>(view, window);
- }
-
- public PCollectionView<T> getView() {
- return view;
- }
-
- public BoundedWindow getWindow() {
- return window;
- }
-
- @Override
- public boolean equals(Object otherObject) {
- if (!(otherObject instanceof PCollectionViewWindow)) {
- return false;
- }
- @SuppressWarnings("unchecked")
- PCollectionViewWindow<T> other = (PCollectionViewWindow<T>) otherObject;
- return getView().equals(other.getView()) && getWindow().equals(other.getWindow());
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(getView(), getWindow());
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWriter.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWriter.java
deleted file mode 100644
index 944caca..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PCollectionViewWriter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionView;
-
-/**
- * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to a
- * storage mechanism that can be read from while constructing a {@link PCollectionView}.
- *
- * @param <ElemT> the type of elements the input {@link PCollection} contains.
- * @param <ViewT> the type of the PCollectionView this writer writes to.
- */
-@FunctionalInterface
-interface PCollectionViewWriter<ElemT, ViewT> {
- void add(Iterable<WindowedValue<ElemT>> values);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PassthroughTransformEvaluator.java
deleted file mode 100644
index 831bb93..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PassthroughTransformEvaluator.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.util.WindowedValue;
-
-class PassthroughTransformEvaluator<InputT> implements TransformEvaluator<InputT> {
- public static <InputT> PassthroughTransformEvaluator<InputT> create(
- PTransformNode transform, UncommittedBundle<InputT> output) {
- return new PassthroughTransformEvaluator<>(transform, output);
- }
-
- private final PTransformNode transform;
- private final UncommittedBundle<InputT> output;
-
- private PassthroughTransformEvaluator(
- PTransformNode transform, UncommittedBundle<InputT> output) {
- this.transform = transform;
- this.output = output;
- }
-
- @Override
- public void processElement(WindowedValue<InputT> element) throws Exception {
- output.add(element);
- }
-
- @Override
- public TransformResult<InputT> finishBundle() throws Exception {
- return StepTransformResult.<InputT>withoutHold(transform).addOutput(output).build();
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PipelineExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PipelineExecutor.java
deleted file mode 100644
index f169f3e..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PipelineExecutor.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult.State;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.joda.time.Duration;
-
-/**
- * An executor that schedules and executes {@link PTransformNode PTransformNodes} for both source
- * and intermediate {@link PTransform PTransforms}.
- */
-interface PipelineExecutor {
- /**
- * Starts this executor on the provided graph. The {@link RootProviderRegistry} will be used to
- * create initial inputs for the provide {@link ExecutableGraph graph}.
- */
- void start();
-
- /**
- * Blocks until the job being executed enters a terminal state. A job is completed after all root
- * {@link PTransformNode PTransformNodes} have completed, and all {@link CommittedBundle Bundles}
- * have been consumed. Jobs may also terminate abnormally.
- *
- * <p>Waits for up to the provided duration, or forever if the provided duration is less than or
- * equal to zero.
- *
- * @return The terminal state of the Pipeline.
- * @throws Exception whenever an executor thread throws anything, transfers to the waiting thread
- * and rethrows it
- */
- State waitUntilFinish(Duration duration) throws Exception;
-
- /** Gets the current state of the {@link Pipeline}. */
- State getPipelineState();
-
- /**
- * Shuts down the executor.
- *
- * <p>The executor may continue to run for a short time after this method returns.
- */
- void stop();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PortableGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PortableGraph.java
deleted file mode 100644
index 88b6e8c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/PortableGraph.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
-import org.apache.beam.runners.direct.ExecutableGraph;
-
-/** A {@link ExecutableGraph} for a Portable {@link RunnerApi.Pipeline}. */
-class PortableGraph implements ExecutableGraph<PTransformNode, PCollectionNode> {
- private final QueryablePipeline queryablePipeline;
-
- public static PortableGraph forPipeline(RunnerApi.Pipeline p) {
- return new PortableGraph(p);
- }
-
- private PortableGraph(RunnerApi.Pipeline p) {
- this.queryablePipeline =
- QueryablePipeline.forTransforms(p.getRootTransformIdsList(), p.getComponents());
- }
-
- @Override
- public Collection<PTransformNode> getRootTransforms() {
- return queryablePipeline.getRootTransforms();
- }
-
- @Override
- public Collection<PTransformNode> getExecutables() {
- return queryablePipeline.getTransforms();
- }
-
- @Override
- public PTransformNode getProducer(PCollectionNode collection) {
- return queryablePipeline.getProducer(collection);
- }
-
- @Override
- public Collection<PCollectionNode> getProduced(PTransformNode producer) {
- return queryablePipeline.getOutputPCollections(producer);
- }
-
- @Override
- public Collection<PCollectionNode> getPerElementInputs(PTransformNode transform) {
- return queryablePipeline.getPerElementInputPCollections(transform);
- }
-
- @Override
- public Collection<PTransformNode> getPerElementConsumers(PCollectionNode pCollection) {
- return queryablePipeline.getPerElementConsumers(pCollection);
- }
-
- public QueryablePipeline getQueryablePipeline() {
- return this.queryablePipeline;
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/QuiescenceDriver.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/QuiescenceDriver.java
deleted file mode 100644
index b11bc91..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/QuiescenceDriver.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import com.google.auto.value.AutoValue;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
-import org.apache.beam.runners.local.ExecutionDriver;
-import org.apache.beam.runners.local.PipelineMessageReceiver;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Pushes additional work onto a {@link BundleProcessor} based on the fact that a pipeline has
- * quiesced.
- */
-class QuiescenceDriver implements ExecutionDriver {
- private static final Logger LOG = LoggerFactory.getLogger(QuiescenceDriver.class);
-
- public static ExecutionDriver create(
- EvaluationContext context,
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode> bundleProcessor,
- PipelineMessageReceiver messageReceiver,
- Map<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> initialBundles) {
- return new QuiescenceDriver(context, graph, bundleProcessor, messageReceiver, initialBundles);
- }
-
- private final EvaluationContext evaluationContext;
- private final ExecutableGraph<PTransformNode, PCollectionNode> graph;
- private final BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode>
- bundleProcessor;
- private final PipelineMessageReceiver pipelineMessageReceiver;
-
- private final CompletionCallback defaultCompletionCallback =
- new TimerIterableCompletionCallback(Collections.emptyList());
-
- private final Map<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> pendingRootBundles;
- private final Queue<WorkUpdate> pendingWork = new ConcurrentLinkedQueue<>();
-
- private final AtomicReference<ExecutorState> state =
- new AtomicReference<>(ExecutorState.QUIESCENT);
- private final AtomicLong outstandingWork = new AtomicLong(0L);
- private boolean exceptionThrown = false;
-
- private QuiescenceDriver(
- EvaluationContext evaluationContext,
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- BundleProcessor<PCollectionNode, CommittedBundle<?>, PTransformNode> bundleProcessor,
- PipelineMessageReceiver pipelineMessageReceiver,
- Map<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> pendingRootBundles) {
- this.evaluationContext = evaluationContext;
- this.graph = graph;
- this.bundleProcessor = bundleProcessor;
- this.pipelineMessageReceiver = pipelineMessageReceiver;
- this.pendingRootBundles = pendingRootBundles;
- }
-
- @Override
- public DriverState drive() {
- boolean noWorkOutstanding = outstandingWork.get() == 0L;
- ExecutorState startingState = state.get();
- if (startingState == ExecutorState.ACTIVE) {
- // The remainder of this call will add all available work to the Executor, and there will
- // be no new work available
- state.compareAndSet(ExecutorState.ACTIVE, ExecutorState.PROCESSING);
- } else if (startingState == ExecutorState.PROCESSING && noWorkOutstanding) {
- // The executor has consumed all new work and no new work was added
- state.compareAndSet(ExecutorState.PROCESSING, ExecutorState.QUIESCING);
- } else if (startingState == ExecutorState.QUIESCING && noWorkOutstanding) {
- // The executor re-ran all blocked work and nothing could make progress.
- state.compareAndSet(ExecutorState.QUIESCING, ExecutorState.QUIESCENT);
- }
- fireTimers();
- Collection<WorkUpdate> updates = new ArrayList<>();
- // Pull all available updates off of the queue before adding additional work. This ensures
- // both loops terminate.
- WorkUpdate pendingUpdate = pendingWork.poll();
- while (pendingUpdate != null) {
- updates.add(pendingUpdate);
- pendingUpdate = pendingWork.poll();
- }
- for (WorkUpdate update : updates) {
- applyUpdate(noWorkOutstanding, startingState, update);
- }
- addWorkIfNecessary();
-
- if (exceptionThrown) {
- return DriverState.FAILED;
- } else if (evaluationContext.isDone()) {
- return DriverState.SHUTDOWN;
- } else {
- return DriverState.CONTINUE;
- }
- }
-
- private void applyUpdate(
- boolean noWorkOutstanding, ExecutorState startingState, WorkUpdate update) {
- LOG.debug("Executor Update: {}", update);
- if (update.getBundle().isPresent()) {
- if (ExecutorState.ACTIVE == startingState
- || (ExecutorState.PROCESSING == startingState && noWorkOutstanding)) {
- CommittedBundle<?> bundle = update.getBundle().get();
- for (PTransformNode consumer : update.getConsumers()) {
- outstandingWork.incrementAndGet();
- bundleProcessor.process(bundle, consumer, defaultCompletionCallback);
- }
- } else {
- pendingWork.offer(update);
- }
- } else if (update.getException().isPresent()) {
- pipelineMessageReceiver.failed(update.getException().get());
- exceptionThrown = true;
- }
- }
-
- /** Fires any available timers. */
- private void fireTimers() {
- try {
- for (FiredTimers<PTransformNode> transformTimers : evaluationContext.extractFiredTimers()) {
- Collection<TimerData> delivery = transformTimers.getTimers();
- KeyedWorkItem<?, Object> work =
- KeyedWorkItems.timersWorkItem(transformTimers.getKey().getKey(), delivery);
- PCollectionNode inputPCollection =
- Iterables.getOnlyElement(graph.getPerElementInputs(transformTimers.getExecutable()));
- @SuppressWarnings({"unchecked", "rawtypes"})
- CommittedBundle<?> bundle =
- evaluationContext
- .createKeyedBundle(transformTimers.getKey(), inputPCollection)
- .add(WindowedValue.valueInGlobalWindow(work))
- .commit(evaluationContext.now());
- outstandingWork.incrementAndGet();
- bundleProcessor.process(
- bundle, transformTimers.getExecutable(), new TimerIterableCompletionCallback(delivery));
- state.set(ExecutorState.ACTIVE);
- }
- } catch (Exception e) {
- LOG.error("Internal Error while delivering timers", e);
- pipelineMessageReceiver.failed(e);
- exceptionThrown = true;
- }
- }
-
- /**
- * If all active {@link DirectTransformExecutor TransformExecutors} are in a blocked state, add
- * more work from root nodes that may have additional work. This ensures that if a pipeline has
- * elements available from the root nodes it will add those elements when necessary.
- */
- private void addWorkIfNecessary() {
- // If any timers have fired, they will add more work; We don't need to add more
- if (state.get() == ExecutorState.QUIESCENT) {
- // All current TransformExecutors are blocked; add more work from the roots.
- for (Map.Entry<PTransformNode, ConcurrentLinkedQueue<CommittedBundle<?>>> pendingRootEntry :
- pendingRootBundles.entrySet()) {
- Collection<CommittedBundle<?>> bundles = new ArrayList<>();
- // Pull all available work off of the queue, then schedule it all, so this loop
- // terminates
- while (!pendingRootEntry.getValue().isEmpty()) {
- CommittedBundle<?> bundle = pendingRootEntry.getValue().poll();
- bundles.add(bundle);
- }
- for (CommittedBundle<?> bundle : bundles) {
- outstandingWork.incrementAndGet();
- bundleProcessor.process(bundle, pendingRootEntry.getKey(), defaultCompletionCallback);
- state.set(ExecutorState.ACTIVE);
- }
- }
- }
- }
-
- /**
- * The state of the executor. The state of the executor determines the behavior of the {@link
- * QuiescenceDriver} when it runs.
- */
- private enum ExecutorState {
- /**
- * Output has been produced since the last time the monitor ran. Work exists that has not yet
- * been evaluated, and all pending, including potentially blocked work, should be evaluated.
- *
- * <p>The executor becomes active whenever a timer fires, a {@link PCollectionView} is updated,
- * or output is produced by the evaluation of a {@link DirectTransformExecutor}.
- */
- ACTIVE,
- /**
- * The Executor does not have any unevaluated work available to it, but work is in progress.
- * Work should not be added until the Executor becomes active or no work is outstanding.
- *
- * <p>If all outstanding work completes without the executor becoming {@code ACTIVE}, the
- * Executor enters state {@code QUIESCING}. Previously evaluated work must be reevaluated, in
- * case a side input has made progress.
- */
- PROCESSING,
- /**
- * All outstanding work is work that may be blocked on a side input. When there is no
- * outstanding work, the executor becomes {@code QUIESCENT}.
- */
- QUIESCING,
- /**
- * All elements are either buffered in state or are blocked on a side input. There are no timers
- * that are permitted to fire but have not. There is no outstanding work.
- *
- * <p>The pipeline will not make progress without the progression of watermarks, the progression
- * of processing time, or the addition of elements.
- */
- QUIESCENT
- }
-
- /**
- * The base implementation of {@link CompletionCallback} that provides implementations for {@link
- * #handleResult(CommittedBundle, TransformResult)} and {@link #handleException(CommittedBundle,
- * Exception)}.
- */
- private class TimerIterableCompletionCallback implements CompletionCallback {
- private final Iterable<TimerData> timers;
-
- TimerIterableCompletionCallback(Iterable<TimerData> timers) {
- this.timers = timers;
- }
-
- @Override
- public final CommittedResult handleResult(
- CommittedBundle<?> inputBundle, TransformResult<?> result) {
- CommittedResult<PTransformNode> committedResult =
- evaluationContext.handleResult(inputBundle, timers, result);
- for (CommittedBundle<?> outputBundle : committedResult.getOutputs()) {
- pendingWork.offer(
- WorkUpdate.fromBundle(
- outputBundle, graph.getPerElementConsumers(outputBundle.getPCollection())));
- }
- Optional<? extends CommittedBundle<?>> unprocessedInputs =
- committedResult.getUnprocessedInputs();
- if (unprocessedInputs.isPresent()) {
- if (inputBundle.getPCollection() == null) {
- // TODO: Split this logic out of an if statement
- pendingRootBundles.get(result.getTransform()).offer(unprocessedInputs.get());
- } else {
- pendingWork.offer(
- WorkUpdate.fromBundle(
- unprocessedInputs.get(), Collections.singleton(committedResult.getExecutable())));
- }
- }
- if (!committedResult.getProducedOutputTypes().isEmpty()) {
- state.set(ExecutorState.ACTIVE);
- }
- outstandingWork.decrementAndGet();
- return committedResult;
- }
-
- @Override
- public void handleEmpty(PTransformNode transform) {
- outstandingWork.decrementAndGet();
- }
-
- @Override
- public final void handleException(CommittedBundle<?> inputBundle, Exception e) {
- pendingWork.offer(WorkUpdate.fromException(e));
- outstandingWork.decrementAndGet();
- }
-
- @Override
- public void handleError(Error err) {
- outstandingWork.decrementAndGet();
- pipelineMessageReceiver.failed(err);
- }
- }
-
- /**
- * An internal status update on the state of the executor.
- *
- * <p>Used to signal when the executor should be shut down (due to an exception).
- */
- @AutoValue
- abstract static class WorkUpdate {
- private static WorkUpdate fromBundle(
- CommittedBundle<?> bundle, Collection<PTransformNode> consumers) {
- return new AutoValue_QuiescenceDriver_WorkUpdate(
- Optional.of(bundle), consumers, Optional.absent());
- }
-
- private static WorkUpdate fromException(Exception e) {
- return new AutoValue_QuiescenceDriver_WorkUpdate(
- Optional.absent(), Collections.emptyList(), Optional.of(e));
- }
-
- /** Returns the bundle that produced this update. */
- public abstract Optional<? extends CommittedBundle<?>> getBundle();
-
- /**
- * Returns the transforms to process the bundle. If nonempty, {@link #getBundle()} will return a
- * present {@link Optional}.
- */
- public abstract Collection<PTransformNode> getConsumers();
-
- public abstract Optional<? extends Exception> getException();
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ReferenceRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ReferenceRunner.java
deleted file mode 100644
index eee804c..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/ReferenceRunner.java
+++ /dev/null
@@ -1,559 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.runners.core.construction.SyntheticComponents.uniqueId;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.stream.Collectors;
-import org.apache.beam.model.fnexecution.v1.ProvisionApi.ProvisionInfo;
-import org.apache.beam.model.fnexecution.v1.ProvisionApi.Resources;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Coder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ComponentsOrBuilder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.MessageWithComponents;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform.Builder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
-import org.apache.beam.runners.core.construction.ModelCoders;
-import org.apache.beam.runners.core.construction.ModelCoders.KvCoderComponents;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.construction.graph.PipelineValidator;
-import org.apache.beam.runners.core.construction.graph.ProtoOverrides;
-import org.apache.beam.runners.core.construction.graph.ProtoOverrides.TransformReplacement;
-import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.fnexecution.GrpcContextHeaderAccessorProvider;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.InProcessServerFactory;
-import org.apache.beam.runners.fnexecution.ServerFactory;
-import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
-import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactRetrievalService;
-import org.apache.beam.runners.fnexecution.control.ControlClientPool;
-import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.runners.fnexecution.control.MapControlClientPool;
-import org.apache.beam.runners.fnexecution.control.SingleEnvironmentInstanceJobBundleFactory;
-import org.apache.beam.runners.fnexecution.data.GrpcDataService;
-import org.apache.beam.runners.fnexecution.environment.DockerEnvironmentFactory;
-import org.apache.beam.runners.fnexecution.environment.EmbeddedEnvironmentFactory;
-import org.apache.beam.runners.fnexecution.environment.EnvironmentFactory;
-import org.apache.beam.runners.fnexecution.logging.GrpcLoggingService;
-import org.apache.beam.runners.fnexecution.logging.Slf4jLogWriter;
-import org.apache.beam.runners.fnexecution.provisioning.StaticGrpcProvisionService;
-import org.apache.beam.runners.fnexecution.state.GrpcStateService;
-import org.apache.beam.runners.fnexecution.wire.LengthPrefixUnknownCoders;
-import org.apache.beam.sdk.fn.IdGenerator;
-import org.apache.beam.sdk.fn.IdGenerators;
-import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.Struct;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Maps;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Sets;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-/**
- * The "ReferenceRunner" engine implementation. The ReferenceRunner uses the portability framework
- * to execute a Pipeline on a single machine.
- */
-public class ReferenceRunner {
- private final RunnerApi.Pipeline pipeline;
- private final Struct options;
- private final String artifactRetrievalToken;
-
- private final EnvironmentType environmentType;
-
- private final IdGenerator idGenerator = IdGenerators.incrementingLongs();
-
- /** @param environmentType The environment to use for the SDK Harness. */
- private ReferenceRunner(
- Pipeline p, Struct options, String artifactRetrievalToken, EnvironmentType environmentType) {
- this.pipeline = executable(p);
- this.options = options;
- this.environmentType = environmentType;
- this.artifactRetrievalToken = artifactRetrievalToken;
- }
-
- /**
- * Creates a "ReferenceRunner" engine for a single pipeline with a Dockerized SDK harness.
- *
- * @param p Pipeline being executed for this job.
- * @param options PipelineOptions for this job.
- * @param artifactRetrievalToken Token to retrieve artifacts that have been staged.
- */
- public static ReferenceRunner forPipeline(
- RunnerApi.Pipeline p, Struct options, String artifactRetrievalToken) {
- return new ReferenceRunner(p, options, artifactRetrievalToken, EnvironmentType.DOCKER);
- }
-
- static ReferenceRunner forInProcessPipeline(RunnerApi.Pipeline p, Struct options) {
- return new ReferenceRunner(p, options, "", EnvironmentType.IN_PROCESS);
- }
-
- private RunnerApi.Pipeline executable(RunnerApi.Pipeline original) {
- RunnerApi.Pipeline p = original;
- PipelineValidator.validate(p);
- p =
- ProtoOverrides.updateTransform(
- PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN,
- p,
- new SplittableProcessKeyedReplacer());
- p =
- ProtoOverrides.updateTransform(
- PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, p, new PortableGroupByKeyReplacer());
- p = GreedyPipelineFuser.fuse(p).toPipeline();
-
- p = foldFeedSDFIntoExecutableStage(p);
- PipelineValidator.validate(p);
-
- return p;
- }
-
- private static Set<PCollectionNode> getKeyedPCollections(
- ExecutableGraph<PTransformNode, PCollectionNode> graph) {
- // This mimics KeyedPValueTrackingVisitor behavior in regular direct runner,
- // but without propagating keyed-ness through key-preserving DoFn's.
- // That is not yet necessary, but will be necessary once we implement state and timers.
- // See https://issues.apache.org/jira/browse/BEAM-4557.
- Set<PCollectionNode> res = Sets.newHashSet();
- Set<String> keyedProducers =
- Sets.newHashSet(DirectGroupByKey.DIRECT_GBKO_URN, DirectGroupByKey.DIRECT_GABW_URN);
- for (PTransformNode transform : graph.getExecutables()) {
- if (keyedProducers.contains(transform.getTransform().getSpec().getUrn())) {
- res.addAll(graph.getProduced(transform));
- }
- }
- return res;
- }
-
- /**
- * First starts all the services needed, then configures and starts the {@link
- * ExecutorServiceParallelExecutor}.
- */
- public void execute() throws Exception {
- ExecutableGraph<PTransformNode, PCollectionNode> graph = PortableGraph.forPipeline(pipeline);
- BundleFactory bundleFactory = ImmutableListBundleFactory.create();
- EvaluationContext ctxt =
- EvaluationContext.create(Instant::new, bundleFactory, graph, getKeyedPCollections(graph));
- RootProviderRegistry rootRegistry = RootProviderRegistry.javaPortableRegistry(bundleFactory);
- int targetParallelism = Math.max(Runtime.getRuntime().availableProcessors(), 3);
- ServerFactory serverFactory = createServerFactory();
- ControlClientPool controlClientPool = MapControlClientPool.create();
- ExecutorService dataExecutor = Executors.newCachedThreadPool();
- ProvisionInfo provisionInfo =
- ProvisionInfo.newBuilder()
- .setJobId("id")
- .setJobName("reference")
- .setPipelineOptions(options)
- .setWorkerId("foo")
- .setResourceLimits(Resources.getDefaultInstance())
- .setRetrievalToken(artifactRetrievalToken)
- .build();
- try (GrpcFnServer<GrpcLoggingService> logging =
- GrpcFnServer.allocatePortAndCreateFor(
- GrpcLoggingService.forWriter(Slf4jLogWriter.getDefault()), serverFactory);
- GrpcFnServer<ArtifactRetrievalService> artifact =
- GrpcFnServer.allocatePortAndCreateFor(
- BeamFileSystemArtifactRetrievalService.create(), serverFactory);
- GrpcFnServer<StaticGrpcProvisionService> provisioning =
- GrpcFnServer.allocatePortAndCreateFor(
- StaticGrpcProvisionService.create(provisionInfo), serverFactory);
- GrpcFnServer<FnApiControlClientPoolService> control =
- GrpcFnServer.allocatePortAndCreateFor(
- FnApiControlClientPoolService.offeringClientsToPool(
- controlClientPool.getSink(),
- GrpcContextHeaderAccessorProvider.getHeaderAccessor()),
- serverFactory);
- GrpcFnServer<GrpcDataService> data =
- GrpcFnServer.allocatePortAndCreateFor(
- GrpcDataService.create(dataExecutor, OutboundObserverFactory.serverDirect()),
- serverFactory);
- GrpcFnServer<GrpcStateService> state =
- GrpcFnServer.allocatePortAndCreateFor(GrpcStateService.create(), serverFactory)) {
-
- EnvironmentFactory environmentFactory =
- createEnvironmentFactory(control, logging, artifact, provisioning, controlClientPool);
- JobBundleFactory jobBundleFactory =
- SingleEnvironmentInstanceJobBundleFactory.create(
- environmentFactory, data, state, idGenerator);
-
- TransformEvaluatorRegistry transformRegistry =
- TransformEvaluatorRegistry.portableRegistry(
- graph,
- pipeline.getComponents(),
- bundleFactory,
- jobBundleFactory,
- EvaluationContextStepStateAndTimersProvider.forContext(ctxt));
- ExecutorServiceParallelExecutor executor =
- ExecutorServiceParallelExecutor.create(
- targetParallelism, rootRegistry, transformRegistry, graph, ctxt);
- executor.start();
- executor.waitUntilFinish(Duration.ZERO);
- } finally {
- dataExecutor.shutdown();
- }
- }
-
- private ServerFactory createServerFactory() {
- switch (environmentType) {
- case DOCKER:
- return ServerFactory.createDefault();
- case IN_PROCESS:
- return InProcessServerFactory.create();
- default:
- throw new IllegalArgumentException(
- String.format("Unknown %s %s", EnvironmentType.class.getSimpleName(), environmentType));
- }
- }
-
- private EnvironmentFactory createEnvironmentFactory(
- GrpcFnServer<FnApiControlClientPoolService> control,
- GrpcFnServer<GrpcLoggingService> logging,
- GrpcFnServer<ArtifactRetrievalService> artifact,
- GrpcFnServer<StaticGrpcProvisionService> provisioning,
- ControlClientPool controlClient) {
- switch (environmentType) {
- case DOCKER:
- return new DockerEnvironmentFactory.Provider(PipelineOptionsTranslation.fromProto(options))
- .createEnvironmentFactory(
- control, logging, artifact, provisioning, controlClient, idGenerator);
- case IN_PROCESS:
- return EmbeddedEnvironmentFactory.create(
- PipelineOptionsFactory.create(), logging, control, controlClient.getSource());
- default:
- throw new IllegalArgumentException(
- String.format("Unknown %s %s", EnvironmentType.class.getSimpleName(), environmentType));
- }
- }
-
- @VisibleForTesting
- static class PortableGroupByKeyReplacer implements TransformReplacement {
- @Override
- public MessageWithComponents getReplacement(String gbkId, ComponentsOrBuilder components) {
- PTransform gbk = components.getTransformsOrThrow(gbkId);
- checkArgument(
- PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN.equals(gbk.getSpec().getUrn()),
- "URN must be %s, got %s",
- PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN,
- gbk.getSpec().getUrn());
-
- PTransform.Builder newTransform = gbk.toBuilder();
- Components.Builder newComponents = Components.newBuilder();
- String inputId = getOnlyElement(gbk.getInputsMap().values());
-
- // Add the GBKO transform
- String kwiCollectionId =
- uniqueId(String.format("%s.%s", inputId, "kwi"), components::containsPcollections);
- {
- PCollection input = components.getPcollectionsOrThrow(inputId);
- Coder inputCoder = components.getCodersOrThrow(input.getCoderId());
- KvCoderComponents kvComponents = ModelCoders.getKvCoderComponents(inputCoder);
- String windowCoderId =
- components
- .getWindowingStrategiesOrThrow(input.getWindowingStrategyId())
- .getWindowCoderId();
- // This coder isn't actually required for the pipeline to function properly - the KWIs can
- // be passed around as pure java objects with no coding of the values, but it approximates
- // a full pipeline.
- Coder kwiCoder =
- Coder.newBuilder()
- .setSpec(
- SdkFunctionSpec.newBuilder()
- .setSpec(FunctionSpec.newBuilder().setUrn("beam:direct:keyedworkitem:v1")))
- .addAllComponentCoderIds(
- ImmutableList.of(
- kvComponents.keyCoderId(), kvComponents.valueCoderId(), windowCoderId))
- .build();
- String kwiCoderId =
- uniqueId(
- String.format("kwi(%s:%s)", kvComponents.keyCoderId(), kvComponents.valueCoderId()),
- components::containsCoders);
- // The kwi PCollection has the same WindowingStrategy as the input, as no merging will
- // have been performed, so elements remain in their original windows
- PCollection kwi =
- input.toBuilder().setUniqueName(kwiCollectionId).setCoderId(kwiCoderId).build();
- String gbkoId = uniqueId(String.format("%s/GBKO", gbkId), components::containsTransforms);
- PTransform gbko =
- PTransform.newBuilder()
- .setUniqueName(String.format("%s/GBKO", gbk.getUniqueName()))
- .putAllInputs(gbk.getInputsMap())
- .setSpec(FunctionSpec.newBuilder().setUrn(DirectGroupByKey.DIRECT_GBKO_URN))
- .putOutputs("output", kwiCollectionId)
- .build();
-
- newTransform.addSubtransforms(gbkoId);
- newComponents
- .putCoders(kwiCoderId, kwiCoder)
- .putPcollections(kwiCollectionId, kwi)
- .putTransforms(gbkoId, gbko);
- }
-
- // Add the GABW transform
- {
- String gabwId = uniqueId(String.format("%s/GABW", gbkId), components::containsTransforms);
- PTransform gabw =
- PTransform.newBuilder()
- .setUniqueName(String.format("%s/GABW", gbk.getUniqueName()))
- .putInputs("input", kwiCollectionId)
- .setSpec(FunctionSpec.newBuilder().setUrn(DirectGroupByKey.DIRECT_GABW_URN))
- .putAllOutputs(gbk.getOutputsMap())
- .build();
- newTransform.addSubtransforms(gabwId);
- newComponents.putTransforms(gabwId, gabw);
- }
-
- return MessageWithComponents.newBuilder()
- .setPtransform(newTransform)
- .setComponents(newComponents)
- .build();
- }
- }
-
- /**
- * Replaces the {@link PTransformTranslation#SPLITTABLE_PROCESS_KEYED_URN} with a {@link
- * DirectGroupByKey#DIRECT_GBKO_URN} (construct keyed work items) followed by a {@link
- * SplittableRemoteStageEvaluatorFactory#FEED_SDF_URN} (convert the keyed work items to
- * element/restriction pairs that later go into {@link
- * PTransformTranslation#SPLITTABLE_PROCESS_ELEMENTS_URN}).
- */
- @VisibleForTesting
- static class SplittableProcessKeyedReplacer implements TransformReplacement {
- @Override
- public MessageWithComponents getReplacement(String spkId, ComponentsOrBuilder components) {
- PTransform spk = components.getTransformsOrThrow(spkId);
- checkArgument(
- PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN.equals(spk.getSpec().getUrn()),
- "URN must be %s, got %s",
- PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN,
- spk.getSpec().getUrn());
-
- Components.Builder newComponents = Components.newBuilder();
- newComponents.putAllCoders(components.getCodersMap());
-
- Builder newPTransform = spk.toBuilder();
-
- String inputId = getOnlyElement(spk.getInputsMap().values());
- PCollection input = components.getPcollectionsOrThrow(inputId);
-
- // This is a Coder<KV<String, KV<ElementT, RestrictionT>>>
- Coder inputCoder = components.getCodersOrThrow(input.getCoderId());
- KvCoderComponents kvComponents = ModelCoders.getKvCoderComponents(inputCoder);
- String windowCoderId =
- components
- .getWindowingStrategiesOrThrow(input.getWindowingStrategyId())
- .getWindowCoderId();
-
- // === Construct a raw GBK returning KeyedWorkItem's ===
- String kwiCollectionId =
- uniqueId(String.format("%s.kwi", spkId), components::containsPcollections);
- {
- // This coder isn't actually required for the pipeline to function properly - the KWIs can
- // be passed around as pure java objects with no coding of the values, but it approximates a
- // full pipeline.
- Coder kwiCoder =
- Coder.newBuilder()
- .setSpec(
- SdkFunctionSpec.newBuilder()
- .setSpec(FunctionSpec.newBuilder().setUrn("beam:direct:keyedworkitem:v1")))
- .addAllComponentCoderIds(
- ImmutableList.of(
- kvComponents.keyCoderId(), kvComponents.valueCoderId(), windowCoderId))
- .build();
- String kwiCoderId =
- uniqueId(
- String.format(
- "keyed_work_item(%s:%s)",
- kvComponents.keyCoderId(), kvComponents.valueCoderId()),
- components::containsCoders);
-
- PCollection kwiCollection =
- input.toBuilder().setUniqueName(kwiCollectionId).setCoderId(kwiCoderId).build();
- String rawGbkId =
- uniqueId(String.format("%s/RawGBK", spkId), components::containsTransforms);
- PTransform rawGbk =
- PTransform.newBuilder()
- .setUniqueName(String.format("%s/RawGBK", spk.getUniqueName()))
- .putAllInputs(spk.getInputsMap())
- .setSpec(FunctionSpec.newBuilder().setUrn(DirectGroupByKey.DIRECT_GBKO_URN))
- .putOutputs("output", kwiCollectionId)
- .build();
-
- newComponents
- .putCoders(kwiCoderId, kwiCoder)
- .putPcollections(kwiCollectionId, kwiCollection)
- .putTransforms(rawGbkId, rawGbk);
- newPTransform.addSubtransforms(rawGbkId);
- }
-
- // === Construct a "Feed SDF" operation that converts KWI to KV<ElementT, RestrictionT> ===
- String feedSDFCollectionId =
- uniqueId(String.format("%s.feed", spkId), components::containsPcollections);
- {
- String elementRestrictionCoderId = kvComponents.valueCoderId();
- String feedSDFCoderId =
- LengthPrefixUnknownCoders.addLengthPrefixedCoder(
- elementRestrictionCoderId, newComponents, false);
-
- PCollection feedSDFCollection =
- input.toBuilder().setUniqueName(feedSDFCollectionId).setCoderId(feedSDFCoderId).build();
- String feedSDFId =
- uniqueId(String.format("%s/FeedSDF", spkId), components::containsTransforms);
- PTransform feedSDF =
- PTransform.newBuilder()
- .setUniqueName(String.format("%s/FeedSDF", spk.getUniqueName()))
- .putInputs("input", kwiCollectionId)
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(SplittableRemoteStageEvaluatorFactory.FEED_SDF_URN))
- .putOutputs("output", feedSDFCollectionId)
- .build();
-
- newComponents
- .putPcollections(feedSDFCollectionId, feedSDFCollection)
- .putTransforms(feedSDFId, feedSDF);
- newPTransform.addSubtransforms(feedSDFId);
- }
-
- // === Construct the SPLITTABLE_PROCESS_ELEMENTS operation
- {
- String runSDFId =
- uniqueId(String.format("%s/RunSDF", spkId), components::containsTransforms);
- PTransform runSDF =
- PTransform.newBuilder()
- .setUniqueName(String.format("%s/RunSDF", spk.getUniqueName()))
- .putInputs("input", feedSDFCollectionId)
- .setSpec(
- FunctionSpec.newBuilder()
- .setUrn(PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN)
- .setPayload(spk.getSpec().getPayload()))
- .putAllOutputs(spk.getOutputsMap())
- .build();
- newComponents.putTransforms(runSDFId, runSDF);
- newPTransform.addSubtransforms(runSDFId);
- }
-
- return MessageWithComponents.newBuilder()
- .setPtransform(newPTransform.build())
- .setComponents(newComponents)
- .build();
- }
- }
-
- /**
- * Finds FEED_SDF nodes followed by an ExecutableStage and replaces them by a single {@link
- * SplittableRemoteStageEvaluatorFactory#URN} stage that feeds the ExecutableStage knowing that
- * the first instruction in the stage is an SDF.
- */
- private static Pipeline foldFeedSDFIntoExecutableStage(Pipeline p) {
- Pipeline.Builder newPipeline = p.toBuilder();
- Components.Builder newPipelineComponents = newPipeline.getComponentsBuilder();
-
- QueryablePipeline q = QueryablePipeline.forPipeline(p);
- String feedSdfUrn = SplittableRemoteStageEvaluatorFactory.FEED_SDF_URN;
- List<PTransformNode> feedSDFNodes =
- q.getTransforms().stream()
- .filter(node -> node.getTransform().getSpec().getUrn().equals(feedSdfUrn))
- .collect(Collectors.toList());
- Map<String, PTransformNode> stageToFeeder = Maps.newHashMap();
- for (PTransformNode node : feedSDFNodes) {
- PCollectionNode output = Iterables.getOnlyElement(q.getOutputPCollections(node));
- PTransformNode consumer = Iterables.getOnlyElement(q.getPerElementConsumers(output));
- String consumerUrn = consumer.getTransform().getSpec().getUrn();
- checkState(
- consumerUrn.equals(ExecutableStage.URN),
- "Expected all FeedSDF nodes to be consumed by an ExecutableStage, "
- + "but %s is consumed by %s which is %s",
- node.getId(),
- consumer.getId(),
- consumerUrn);
- stageToFeeder.put(consumer.getId(), node);
- }
-
- // Copy over root transforms except for the excluded FEED_SDF transforms.
- Set<String> feedSDFIds =
- feedSDFNodes.stream().map(PTransformNode::getId).collect(Collectors.toSet());
- newPipeline.clearRootTransformIds();
- for (String rootId : p.getRootTransformIdsList()) {
- if (!feedSDFIds.contains(rootId)) {
- newPipeline.addRootTransformIds(rootId);
- }
- }
- // Copy over all transforms, except FEED_SDF transforms are skipped, and ExecutableStage's
- // feeding from them are replaced.
- for (PTransformNode node : q.getTransforms()) {
- if (feedSDFNodes.contains(node)) {
- // These transforms are skipped and handled separately.
- continue;
- }
- if (!stageToFeeder.containsKey(node.getId())) {
- // This transform is unchanged
- newPipelineComponents.putTransforms(node.getId(), node.getTransform());
- continue;
- }
- // "node" is an ExecutableStage transform feeding from an SDF.
- PTransformNode feedSDFNode = stageToFeeder.get(node.getId());
- PCollectionNode rawGBKOutput =
- Iterables.getOnlyElement(q.getPerElementInputPCollections(feedSDFNode));
-
- // Replace the ExecutableStage transform.
- newPipelineComponents.putTransforms(
- node.getId(),
- node.getTransform()
- .toBuilder()
- .mergeSpec(
- // Change URN from ExecutableStage.URN to URN of the ULR's splittable executable
- // stage evaluator.
- FunctionSpec.newBuilder()
- .setUrn(SplittableRemoteStageEvaluatorFactory.URN)
- .build())
- .putInputs(
- // The splittable executable stage now reads from the raw GBK, instead of
- // from the now non-existent FEED_SDF.
- Iterables.getOnlyElement(node.getTransform().getInputsMap().keySet()),
- rawGBKOutput.getId())
- .build());
- }
-
- return newPipeline.build();
- }
-
- private enum EnvironmentType {
- DOCKER,
- IN_PROCESS
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RemoteStageEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RemoteStageEvaluatorFactory.java
deleted file mode 100644
index 335c1e7..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RemoteStageEvaluatorFactory.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.fnexecution.control.BundleProgressHandler;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.runners.fnexecution.control.RemoteBundle;
-import org.apache.beam.runners.fnexecution.control.StageBundleFactory;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-
-/**
- * The {@link TransformEvaluatorFactory} which produces {@link TransformEvaluator evaluators} for
- * stages which execute on an SDK harness via the Fn Execution APIs.
- */
-class RemoteStageEvaluatorFactory implements TransformEvaluatorFactory {
- private final BundleFactory bundleFactory;
-
- private final JobBundleFactory jobFactory;
-
- RemoteStageEvaluatorFactory(BundleFactory bundleFactory, JobBundleFactory jobFactory) {
- this.bundleFactory = bundleFactory;
- this.jobFactory = jobFactory;
- }
-
- @Nullable
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) throws Exception {
- return new RemoteStageEvaluator<>(application);
- }
-
- @Override
- public void cleanup() throws Exception {
- jobFactory.close();
- }
-
- private class RemoteStageEvaluator<T> implements TransformEvaluator<T> {
- private final PTransformNode transform;
- private final RemoteBundle bundle;
- private final FnDataReceiver<WindowedValue<?>> mainInput;
- private final Collection<UncommittedBundle<?>> outputs;
-
- private RemoteStageEvaluator(PTransformNode transform) throws Exception {
- this.transform = transform;
- ExecutableStage stage =
- ExecutableStage.fromPayload(
- ExecutableStagePayload.parseFrom(transform.getTransform().getSpec().getPayload()));
- this.outputs = new ArrayList<>();
- StageBundleFactory stageFactory = jobFactory.forStage(stage);
- this.bundle =
- stageFactory.getBundle(
- BundleFactoryOutputReceiverFactory.create(
- bundleFactory, stage.getComponents(), outputs::add),
- StateRequestHandler.unsupported(),
- BundleProgressHandler.ignored());
- // TODO(BEAM-4680): Add support for timers as inputs to the ULR
- this.mainInput = Iterables.getOnlyElement(bundle.getInputReceivers().values());
- }
-
- @Override
- public void processElement(WindowedValue<T> element) throws Exception {
- mainInput.accept(element);
- }
-
- @Override
- public TransformResult<T> finishBundle() throws Exception {
- bundle.close();
- return StepTransformResult.<T>withoutHold(transform).addOutput(outputs).build();
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootInputProvider.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootInputProvider.java
deleted file mode 100644
index 9f904c7..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootInputProvider.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-
-/**
- * Provides {@link CommittedBundle bundles} that will be provided to the {@link PTransform
- * PTransforms} that are at the root of a {@link Pipeline}.
- */
-interface RootInputProvider<ShardT> {
- /**
- * Get the initial inputs for the {@link PTransformNode}. The {@link PTransformNode} will be
- * provided with these {@link CommittedBundle bundles} as input when the {@link Pipeline} runs.
- *
- * <p>For source transforms, these should be sufficient that, when provided to the evaluators
- * produced by {@link TransformEvaluatorFactory#forApplication(PTransformNode, CommittedBundle)},
- * all of the elements contained in the source are eventually produced.
- *
- * @param transform the {@link PTransformNode} to get initial inputs for.
- * @param targetParallelism the target amount of parallelism to obtain from the source. Must be
- * greater than or equal to 1.
- */
- Collection<CommittedBundle<ShardT>> getInitialInputs(
- PTransformNode transform, int targetParallelism) throws Exception;
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootProviderRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootProviderRegistry.java
deleted file mode 100644
index 054a04d..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/RootProviderRegistry.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN;
-import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.Map;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
-
-/**
- * A {@link RootInputProvider} that delegates to primitive {@link RootInputProvider} implementations
- * based on the type of {@link PTransform} of the application.
- */
-class RootProviderRegistry {
-
- /**
- * Returns a {@link RootProviderRegistry} that supports the {@link Impulse} and {@link Flatten}
- * primitives.
- */
- static RootProviderRegistry javaPortableRegistry(BundleFactory bundleFactory) {
- return new RootProviderRegistry(
- ImmutableMap.<String, RootInputProvider<?>>builder()
- .put(
- IMPULSE_TRANSFORM_URN,
- new ImpulseEvaluatorFactory.ImpulseRootProvider(bundleFactory))
- .put(FLATTEN_TRANSFORM_URN, new EmptyInputProvider())
- .build());
- }
-
- private final Map<String, RootInputProvider<?>> providers;
-
- private RootProviderRegistry(Map<String, RootInputProvider<?>> providers) {
- this.providers = providers;
- }
-
- public Collection<CommittedBundle<?>> getInitialInputs(
- PTransformNode transform, int targetParallelism) throws Exception {
- String transformUrn = PTransformTranslation.urnForTransformOrNull(transform.getTransform());
- RootInputProvider provider =
- checkNotNull(
- providers.get(transformUrn),
- "Tried to get a %s for a transform \"%s\", but there is no such provider",
- RootInputProvider.class.getSimpleName(),
- transformUrn);
- return provider.getInitialInputs(transform, targetParallelism);
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SourceShard.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SourceShard.java
deleted file mode 100644
index cf978b8..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SourceShard.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.io.BoundedSource;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.Source;
-import org.apache.beam.sdk.io.UnboundedSource;
-
-/**
- * A shard for a source in the {@link Read} transform.
- *
- * <p>Since {@link UnboundedSource} and {@link BoundedSource} have radically different needs, this
- * is a mostly-empty interface.
- */
-interface SourceShard<T> {
- Source<T> getSource();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SplittableRemoteStageEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SplittableRemoteStageEvaluatorFactory.java
deleted file mode 100644
index 28f8789..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/SplittableRemoteStageEvaluatorFactory.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleProgressResponse;
-import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
-import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
-import org.apache.beam.runners.core.KeyedWorkItem;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.fnexecution.control.BundleProgressHandler;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.runners.fnexecution.control.RemoteBundle;
-import org.apache.beam.runners.fnexecution.splittabledofn.SDFFeederViaStateAndTimers;
-import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.KvCoder;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-
-/**
- * The {@link TransformEvaluatorFactory} for {@link #URN}, which reads from a {@link
- * DirectGroupByKey#DIRECT_GBKO_URN} and feeds the data, using state and timers, to a {@link
- * ExecutableStage} whose first instruction is an SDF.
- */
-class SplittableRemoteStageEvaluatorFactory implements TransformEvaluatorFactory {
- public static final String URN = "beam:directrunner:transforms:splittable_remote_stage:v1";
-
- // A fictional transform that transforms from KWI<unique key, KV<element, restriction>>
- // to simply KV<element, restriction> taken by the SDF inside the ExecutableStage.
- public static final String FEED_SDF_URN = "beam:directrunner:transforms:feed_sdf:v1";
-
- private final BundleFactory bundleFactory;
- private final JobBundleFactory jobBundleFactory;
- private final StepStateAndTimers.Provider stp;
-
- SplittableRemoteStageEvaluatorFactory(
- BundleFactory bundleFactory,
- JobBundleFactory jobBundleFactory,
- StepStateAndTimers.Provider stepStateAndTimers) {
- this.bundleFactory = bundleFactory;
- this.jobBundleFactory = jobBundleFactory;
- this.stp = stepStateAndTimers;
- }
-
- @Nullable
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) throws Exception {
- return new SplittableRemoteStageEvaluator(
- bundleFactory,
- jobBundleFactory,
- stp.forStepAndKey(application, inputBundle.getKey()),
- application);
- }
-
- @Override
- public void cleanup() throws Exception {
- jobBundleFactory.close();
- }
-
- private static class SplittableRemoteStageEvaluator<InputT, RestrictionT>
- implements TransformEvaluator<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> {
- private final PTransformNode transform;
- private final ExecutableStage stage;
-
- private final CopyOnAccessInMemoryStateInternals<byte[]> stateInternals;
- private final DirectTimerInternals timerInternals;
- private final RemoteBundle bundle;
- private final FnDataReceiver<WindowedValue<?>> mainInput;
- private final Collection<UncommittedBundle<?>> outputs;
-
- private final SDFFeederViaStateAndTimers<InputT, RestrictionT> feeder;
-
- private SplittableRemoteStageEvaluator(
- BundleFactory bundleFactory,
- JobBundleFactory jobBundleFactory,
- StepStateAndTimers<byte[]> stp,
- PTransformNode transform)
- throws Exception {
- this.stateInternals = stp.stateInternals();
- this.timerInternals = stp.timerInternals();
- this.transform = transform;
- this.stage =
- ExecutableStage.fromPayload(
- ExecutableStagePayload.parseFrom(transform.getTransform().getSpec().getPayload()));
- this.outputs = new ArrayList<>();
-
- FullWindowedValueCoder<KV<InputT, RestrictionT>> windowedValueCoder =
- (FullWindowedValueCoder<KV<InputT, RestrictionT>>)
- WireCoders.<KV<InputT, RestrictionT>>instantiateRunnerWireCoder(
- stage.getInputPCollection(), stage.getComponents());
- KvCoder<InputT, RestrictionT> kvCoder =
- (KvCoder<InputT, RestrictionT>) windowedValueCoder.getValueCoder();
- this.feeder =
- new SDFFeederViaStateAndTimers<>(
- stateInternals,
- timerInternals,
- kvCoder.getKeyCoder(),
- kvCoder.getValueCoder(),
- (Coder<BoundedWindow>) windowedValueCoder.getWindowCoder());
-
- this.bundle =
- jobBundleFactory
- .forStage(stage)
- .getBundle(
- BundleFactoryOutputReceiverFactory.create(
- bundleFactory, stage.getComponents(), outputs::add),
- StateRequestHandler.unsupported(),
- // TODO: Wire in splitting via a split listener
- new BundleProgressHandler() {
- @Override
- public void onProgress(ProcessBundleProgressResponse progress) {}
-
- @Override
- public void onCompleted(ProcessBundleResponse response) {}
- });
- this.mainInput = Iterables.getOnlyElement(bundle.getInputReceivers().values());
- }
-
- @Override
- public void processElement(
- WindowedValue<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> windowedWorkItem)
- throws Exception {
- KeyedWorkItem<byte[], KV<InputT, RestrictionT>> kwi = windowedWorkItem.getValue();
- WindowedValue<KV<InputT, RestrictionT>> elementRestriction =
- Iterables.getOnlyElement(kwi.elementsIterable(), null);
- if (elementRestriction != null) {
- feeder.seed(elementRestriction);
- } else {
- elementRestriction = feeder.resume(Iterables.getOnlyElement(kwi.timersIterable()));
- }
- mainInput.accept(elementRestriction);
- }
-
- @Override
- public TransformResult<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> finishBundle()
- throws Exception {
- bundle.close();
- feeder.commit();
- CopyOnAccessInMemoryStateInternals<byte[]> state = stateInternals.commit();
- StepTransformResult.Builder<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>> result =
- StepTransformResult.withHold(transform, state.getEarliestWatermarkHold());
- return result
- .addOutput(outputs)
- .withState(state)
- .withTimerUpdate(timerInternals.getTimerUpdate())
- .build();
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepAndKey.java
deleted file mode 100644
index 4c4a48b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepAndKey.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Objects;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.MoreObjects;
-
-/**
- * A (Step, Key) pair. This is useful as a map key or cache key for things that are available
- * per-step in a keyed manner (e.g. State).
- */
-final class StepAndKey {
- private final PTransformNode step;
- private final StructuralKey<?> key;
-
- /** Create a new {@link StepAndKey} with the provided step and key. */
- public static StepAndKey of(PTransformNode step, StructuralKey<?> key) {
- return new StepAndKey(step, key);
- }
-
- private StepAndKey(PTransformNode step, StructuralKey<?> key) {
- this.step = step;
- this.key = key;
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(StepAndKey.class)
- .add("step", step.getId())
- .add("key", key.getKey())
- .toString();
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(step, key);
- }
-
- @Override
- public boolean equals(Object other) {
- if (other == this) {
- return true;
- } else if (!(other instanceof StepAndKey)) {
- return false;
- } else {
- StepAndKey that = (StepAndKey) other;
- return Objects.equals(this.step, that.step) && Objects.equals(this.key, that.key);
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepStateAndTimers.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepStateAndTimers.java
deleted file mode 100644
index e5cfe55..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepStateAndTimers.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.StateInternals;
-import org.apache.beam.runners.core.TimerInternals;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.local.StructuralKey;
-
-/** A provider of {@link StateInternals} and {@link TimerInternals}. */
-interface StepStateAndTimers<K> {
- interface Provider {
- <K> StepStateAndTimers<K> forStepAndKey(PTransformNode transform, StructuralKey<K> key);
- }
-
- CopyOnAccessInMemoryStateInternals<K> stateInternals();
-
- DirectTimerInternals timerInternals();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepTransformResult.java
deleted file mode 100644
index b694d3b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/StepTransformResult.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import com.google.auto.value.AutoValue;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Set;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.joda.time.Instant;
-
-/** An immutable {@link TransformResult}. */
-@AutoValue
-abstract class StepTransformResult<InputT> implements TransformResult<InputT> {
-
- public static <InputT> Builder<InputT> withHold(PTransformNode transform, Instant watermarkHold) {
- return new Builder(transform, watermarkHold);
- }
-
- public static <InputT> Builder<InputT> withoutHold(PTransformNode transform) {
- return new Builder(transform, BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-
- @Override
- public TransformResult<InputT> withLogicalMetricUpdates(MetricUpdates metricUpdates) {
- return new AutoValue_StepTransformResult(
- getTransform(),
- getOutputBundles(),
- getUnprocessedElements(),
- metricUpdates,
- getWatermarkHold(),
- getState(),
- getTimerUpdate(),
- getOutputTypes());
- }
-
- /** A builder for creating instances of {@link StepTransformResult}. */
- public static class Builder<InputT> {
- private final PTransformNode transform;
- private final ImmutableList.Builder<UncommittedBundle<?>> bundlesBuilder;
- private final ImmutableList.Builder<WindowedValue<InputT>> unprocessedElementsBuilder;
- private MetricUpdates metricUpdates;
- private CopyOnAccessInMemoryStateInternals state;
- private TimerUpdate timerUpdate;
- private final Set<OutputType> producedOutputs;
- private final Instant watermarkHold;
-
- private Builder(PTransformNode transform, Instant watermarkHold) {
- this.transform = transform;
- this.watermarkHold = watermarkHold;
- this.bundlesBuilder = ImmutableList.builder();
- this.producedOutputs = EnumSet.noneOf(OutputType.class);
- this.unprocessedElementsBuilder = ImmutableList.builder();
- this.timerUpdate = TimerUpdate.builder(null).build();
- this.metricUpdates = MetricUpdates.EMPTY;
- }
-
- public StepTransformResult<InputT> build() {
- return new AutoValue_StepTransformResult<>(
- transform,
- bundlesBuilder.build(),
- unprocessedElementsBuilder.build(),
- metricUpdates,
- watermarkHold,
- state,
- timerUpdate,
- producedOutputs);
- }
-
- public Builder<InputT> withMetricUpdates(MetricUpdates metricUpdates) {
- this.metricUpdates = metricUpdates;
- return this;
- }
-
- public Builder<InputT> withState(CopyOnAccessInMemoryStateInternals state) {
- this.state = state;
- return this;
- }
-
- public Builder<InputT> withTimerUpdate(TimerUpdate timerUpdate) {
- this.timerUpdate = timerUpdate;
- return this;
- }
-
- public Builder<InputT> addUnprocessedElements(WindowedValue<InputT>... unprocessed) {
- unprocessedElementsBuilder.addAll(Arrays.asList(unprocessed));
- return this;
- }
-
- public Builder<InputT> addUnprocessedElements(
- Iterable<? extends WindowedValue<InputT>> unprocessed) {
- unprocessedElementsBuilder.addAll(unprocessed);
- return this;
- }
-
- public Builder<InputT> addOutput(
- UncommittedBundle<?> outputBundle, UncommittedBundle<?>... outputBundles) {
- bundlesBuilder.add(outputBundle);
- bundlesBuilder.add(outputBundles);
- return this;
- }
-
- public Builder<InputT> addOutput(Collection<UncommittedBundle<?>> outputBundles) {
- bundlesBuilder.addAll(outputBundles);
- return this;
- }
-
- public Builder<InputT> withAdditionalOutput(OutputType producedAdditionalOutput) {
- producedOutputs.add(producedAdditionalOutput);
- return this;
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluator.java
deleted file mode 100644
index 2579bcc..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluator.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.sdk.util.WindowedValue;
-
-/**
- * An evaluator of a specific application of a transform. Will be used for at least one {@link
- * CommittedBundle}.
- *
- * @param <InputT> the type of elements that will be passed to {@link #processElement}
- */
-interface TransformEvaluator<InputT> {
- /**
- * Process an element in the input {@link CommittedBundle}.
- *
- * @param element the element to process
- */
- void processElement(WindowedValue<InputT> element) throws Exception;
-
- /**
- * Finish processing the bundle of this {@link TransformEvaluator}.
- *
- * <p>After {@link #finishBundle()} is called, the {@link TransformEvaluator} will not be reused,
- * and no more elements will be processed.
- *
- * @return an {@link TransformResult} containing the results of this bundle evaluation.
- */
- TransformResult<InputT> finishBundle() throws Exception;
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorFactory.java
deleted file mode 100644
index 9892c36..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorFactory.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.transforms.PTransform;
-
-/**
- * A factory for creating instances of {@link TransformEvaluator} for the application of a {@link
- * PTransform}.
- *
- * <p>{@link TransformEvaluatorFactory TransformEvaluatorFactories} will be reused within a single
- * execution of a {@link Pipeline} but will not be reused across executions.
- */
-interface TransformEvaluatorFactory {
- /**
- * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}.
- *
- * <p>Any work that must be done before input elements are processed (such as calling {@code
- * DoFn.StartBundle}) must be done before the {@link TransformEvaluator} is made available to the
- * caller.
- *
- * <p>May return null if the application cannot produce an evaluator (for example, it is a {@link
- * Read} {@link PTransform} where all evaluators are in-use).
- *
- * @return An evaluator capable of processing the transform on the bundle, or null if no evaluator
- * can be constructed.
- * @throws Exception whenever constructing the underlying evaluator throws an exception
- */
- @Nullable
- <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) throws Exception;
-
- /**
- * Cleans up any state maintained by this {@link TransformEvaluatorFactory}. Called after a {@link
- * Pipeline} is shut down. No more calls to {@link #forApplication(PTransformNode,
- * CommittedBundle)} will be made after a call to {@link #cleanup()}.
- */
- void cleanup() throws Exception;
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorRegistry.java
deleted file mode 100644
index 6587882..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformEvaluatorRegistry.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.graph.ExecutableStage;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
- * implementations based on the type of {@link PTransform} of the application.
- */
-class TransformEvaluatorRegistry {
- private static final Logger LOG = LoggerFactory.getLogger(TransformEvaluatorRegistry.class);
-
- static TransformEvaluatorRegistry portableRegistry(
- ExecutableGraph<PTransformNode, PCollectionNode> graph,
- Components components,
- BundleFactory bundleFactory,
- JobBundleFactory jobBundleFactory,
- StepStateAndTimers.Provider stepStateAndTimers) {
- return new TransformEvaluatorRegistry(
- ImmutableMap.<String, TransformEvaluatorFactory>builder()
- .put(
- PTransformTranslation.IMPULSE_TRANSFORM_URN,
- new ImpulseEvaluatorFactory(graph, bundleFactory))
- .put(
- PTransformTranslation.FLATTEN_TRANSFORM_URN,
- new FlattenEvaluatorFactory(graph, bundleFactory))
- .put(
- DirectGroupByKey.DIRECT_GBKO_URN,
- new GroupByKeyOnlyEvaluatorFactory(graph, components, bundleFactory))
- .put(
- DirectGroupByKey.DIRECT_GABW_URN,
- new GroupAlsoByWindowEvaluatorFactory(
- graph, components, bundleFactory, stepStateAndTimers))
- .put(
- ExecutableStage.URN,
- new RemoteStageEvaluatorFactory(bundleFactory, jobBundleFactory))
- .put(
- SplittableRemoteStageEvaluatorFactory.URN,
- new SplittableRemoteStageEvaluatorFactory(
- bundleFactory, jobBundleFactory, stepStateAndTimers))
- .build());
- }
-
- // the TransformEvaluatorFactories can construct instances of all generic types of transform,
- // so all instances of a primitive can be handled with the same evaluator factory.
- private final Map<String, TransformEvaluatorFactory> factories;
-
- private final AtomicBoolean finished = new AtomicBoolean(false);
-
- private TransformEvaluatorRegistry(
- @SuppressWarnings("rawtypes") Map<String, TransformEvaluatorFactory> factories) {
- this.factories = factories;
- }
-
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, CommittedBundle<?> inputBundle) throws Exception {
- checkState(
- !finished.get(), "Tried to get an evaluator for a finished TransformEvaluatorRegistry");
-
- String urn = PTransformTranslation.urnForTransformOrNull(application.getTransform());
-
- TransformEvaluatorFactory factory =
- checkNotNull(factories.get(urn), "No evaluator for PTransform \"%s\"", urn);
- return factory.forApplication(application, inputBundle);
- }
-
- public void cleanup() throws Exception {
- Collection<Exception> thrownInCleanup = new ArrayList<>();
- for (TransformEvaluatorFactory factory : factories.values()) {
- try {
- factory.cleanup();
- } catch (Exception e) {
- if (e instanceof InterruptedException) {
- Thread.currentThread().interrupt();
- }
- thrownInCleanup.add(e);
- }
- }
- finished.set(true);
- if (!thrownInCleanup.isEmpty()) {
- LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup);
- Exception toThrow = null;
- for (Exception e : thrownInCleanup) {
- if (toThrow == null) {
- toThrow = e;
- } else {
- toThrow.addSuppressed(e);
- }
- }
- throw toThrow;
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutor.java
deleted file mode 100644
index f4f34cc..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutor.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-/** A {@link Runnable} that will execute a {@code PTransform} on some bundle of input. */
-public interface TransformExecutor extends Runnable {}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorFactory.java
deleted file mode 100644
index fe29aee..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorFactory.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-
-/** A Factory for creating {@link TransformExecutor Transform Executors} on an input. */
-interface TransformExecutorFactory {
- TransformExecutor create(
- CommittedBundle<?> bundle,
- PTransformNode transform,
- CompletionCallback onComplete,
- TransformExecutorService executorService);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorService.java
deleted file mode 100644
index f8dcf8d..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorService.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-/**
- * Schedules and completes {@link TransformExecutor TransformExecutors}, controlling concurrency as
- * appropriate for the {@link StepAndKey} the executor exists for.
- */
-interface TransformExecutorService {
- /** Schedule the provided work to be eventually executed. */
- void schedule(TransformExecutor work);
-
- /**
- * Finish executing the provided work. This may cause additional {@link DirectTransformExecutor
- * TransformExecutors} to be evaluated.
- */
- void complete(TransformExecutor completed);
-
- /**
- * Cancel any outstanding work, if possible. Any future calls to schedule should ignore any work.
- */
- void shutdown();
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorServices.java
deleted file mode 100644
index 24d7694..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformExecutorServices.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.MoreObjects;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Static factory methods for constructing instances of {@link TransformExecutorService}. */
-final class TransformExecutorServices {
- private TransformExecutorServices() {
- // Do not instantiate
- }
-
- /**
- * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
- * parallel.
- */
- public static TransformExecutorService parallel(ExecutorService executor) {
- return new ParallelTransformExecutor(executor);
- }
-
- /**
- * Returns an EvaluationState that evaluates {@link TransformExecutor TransformExecutors} in
- * serial.
- */
- public static TransformExecutorService serial(ExecutorService executor) {
- return new SerialTransformExecutor(executor);
- }
-
- /**
- * A {@link TransformExecutorService} with unlimited parallelism. Any {@link TransformExecutor}
- * scheduled will be immediately submitted to the {@link ExecutorService}.
- *
- * <p>A principal use of this is for the evaluation of an unkeyed Step. Unkeyed computations are
- * processed in parallel.
- */
- private static class ParallelTransformExecutor implements TransformExecutorService {
- private static final Logger LOG = LoggerFactory.getLogger(ParallelTransformExecutor.class);
-
- private final ExecutorService executor;
- private final AtomicBoolean active = new AtomicBoolean(true);
-
- private ParallelTransformExecutor(ExecutorService executor) {
- this.executor = executor;
- }
-
- @Override
- // TODO: [BEAM-4563] Pass Future back to consumer to check for async errors
- @SuppressWarnings("FutureReturnValueIgnored")
- public void schedule(TransformExecutor work) {
- if (active.get()) {
- try {
- executor.submit(work);
- } catch (RejectedExecutionException rejected) {
- boolean stillActive = active.get();
- if (stillActive) {
- throw new IllegalStateException(
- String.format(
- "Execution of Work %s was rejected, but the %s is still active",
- work, ParallelTransformExecutor.class.getSimpleName()));
- } else {
- LOG.debug(
- "Rejected execution of Work {} on executor {}. "
- + "Suppressed exception because evaluator is not active",
- work,
- this);
- }
- }
- }
- }
-
- @Override
- public void complete(TransformExecutor completed) {}
-
- @Override
- public void shutdown() {
- active.set(false);
- }
- }
-
- /**
- * A {@link TransformExecutorService} with a single work queue. Any {@link TransformExecutor}
- * scheduled will be placed on the work queue. Only one item of work will be submitted to the
- * {@link ExecutorService} at any time.
- *
- * <p>A principal use of this is for the serial evaluation of a (Step, Key) pair. Keyed
- * computations are processed serially per step.
- */
- private static class SerialTransformExecutor implements TransformExecutorService {
- private final ExecutorService executor;
-
- private AtomicReference<TransformExecutor> currentlyEvaluating;
- private final Queue<TransformExecutor> workQueue;
- private boolean active = true;
-
- private SerialTransformExecutor(ExecutorService executor) {
- this.executor = executor;
- this.currentlyEvaluating = new AtomicReference<>();
- this.workQueue = new ConcurrentLinkedQueue<>();
- }
-
- /**
- * Schedules the work, adding it to the work queue if there is a bundle currently being
- * evaluated and scheduling it immediately otherwise.
- */
- @Override
- public void schedule(TransformExecutor work) {
- workQueue.offer(work);
- updateCurrentlyEvaluating();
- }
-
- @Override
- public void complete(TransformExecutor completed) {
- if (!currentlyEvaluating.compareAndSet(completed, null)) {
- throw new IllegalStateException(
- "Finished work "
- + completed
- + " but could not complete due to unexpected currently executing "
- + currentlyEvaluating.get());
- }
- updateCurrentlyEvaluating();
- }
-
- @Override
- public void shutdown() {
- synchronized (this) {
- active = false;
- }
- workQueue.clear();
- }
-
- // TODO: [BEAM-4563] Pass Future back to consumer to check for async errors
- @SuppressWarnings("FutureReturnValueIgnored")
- private void updateCurrentlyEvaluating() {
- if (currentlyEvaluating.get() == null) {
- // Only synchronize if we need to update what's currently evaluating
- synchronized (this) {
- TransformExecutor newWork = workQueue.poll();
- if (active && newWork != null) {
- if (currentlyEvaluating.compareAndSet(null, newWork)) {
- executor.submit(newWork);
- } else {
- workQueue.offer(newWork);
- }
- }
- }
- }
- }
-
- @Override
- public String toString() {
- return MoreObjects.toStringHelper(SerialTransformExecutor.class)
- .add("currentlyEvaluating", currentlyEvaluating)
- .add("workQueue", workQueue)
- .toString();
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformResult.java
deleted file mode 100644
index 3f055e4..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/TransformResult.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Set;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.ParDo;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.joda.time.Instant;
-
-/**
- * The result of evaluating an {@link PTransformNode} with a {@link TransformEvaluator}.
- *
- * <p>Every transform evaluator has a defined input type, but {@link ParDo} has multiple outputs so
- * there is not necesssarily a defined output type.
- */
-interface TransformResult<InputT> {
- /**
- * Returns the {@link PTransformNode} that produced this result.
- *
- * <p>This is treated as an opaque identifier so evaluators can delegate to other evaluators that
- * may not have compatible types.
- */
- PTransformNode getTransform();
-
- /**
- * Returns the {@link UncommittedBundle (uncommitted) Bundles} output by this transform. These
- * will be committed by the evaluation context as part of completing this result.
- *
- * <p>Note that the bundles need not have a uniform type, for example in the case of multi-output
- * {@link ParDo}.
- */
- Iterable<? extends UncommittedBundle<?>> getOutputBundles();
-
- /**
- * Returns elements that were provided to the {@link TransformEvaluator} as input but were not
- * processed.
- */
- Iterable<? extends WindowedValue<InputT>> getUnprocessedElements();
-
- /** Returns the logical metric updates. */
- MetricUpdates getLogicalMetricUpdates();
-
- /**
- * Returns the Watermark Hold for the transform at the time this result was produced.
- *
- * <p>If the transform does not set any watermark hold, returns {@link
- * BoundedWindow#TIMESTAMP_MAX_VALUE}.
- */
- Instant getWatermarkHold();
-
- /**
- * Returns the State used by the transform.
- *
- * <p>If this evaluation did not access state, this may return null.
- */
- @Nullable
- CopyOnAccessInMemoryStateInternals getState();
-
- /**
- * Returns a TimerUpdateBuilder that was produced as a result of this evaluation. If the
- * evaluation was triggered due to the delivery of one or more timers, those timers must be added
- * to the builder before it is complete.
- *
- * <p>If this evaluation did not add or remove any timers, returns an empty TimerUpdate.
- */
- TimerUpdate getTimerUpdate();
-
- /**
- * Returns the types of output produced by this {@link PTransform}. This may not include {@link
- * OutputType#BUNDLE}, as empty bundles may be dropped when the transform is committed.
- */
- Set<OutputType> getOutputTypes();
-
- /**
- * Returns a new TransformResult based on this one but overwriting any existing logical metric
- * updates with {@code metricUpdates}.
- */
- TransformResult<InputT> withLogicalMetricUpdates(MetricUpdates metricUpdates);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/UncommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/UncommittedBundle.java
deleted file mode 100644
index a55d415..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/UncommittedBundle.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.joda.time.Instant;
-
-/**
- * Part of a {@link PCollection}. Elements are output to a bundle, which will cause them to be
- * executed by {@link PTransform PTransforms} that consume the {@link PCollection} this bundle is a
- * part of at a later point. This is an uncommitted bundle and can have elements added to it.
- *
- * @param <T> the type of elements that can be added to this bundle
- */
-interface UncommittedBundle<T> {
- /** Returns the PCollection that the elements of this {@link UncommittedBundle} belong to. */
- @Nullable
- PCollectionNode getPCollection();
-
- /**
- * Outputs an element to this bundle.
- *
- * @param element the element to add to this bundle
- * @return this bundle
- */
- UncommittedBundle<T> add(WindowedValue<T> element);
-
- /**
- * Commits this {@link UncommittedBundle}, returning an immutable {@link CommittedBundle}
- * containing all of the elements that were added to it. The {@link #add(WindowedValue)} method
- * will throw an {@link IllegalStateException} if called after a call to commit.
- *
- * @param synchronizedProcessingTime the synchronized processing time at which this bundle was
- * committed
- */
- CommittedBundle<T> commit(Instant synchronizedProcessingTime);
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WatermarkCallbackExecutor.java
deleted file mode 100644
index b7c8a83..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WatermarkCallbackExecutor.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.io.Serializable;
-import java.util.PriorityQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executor;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ComparisonChain;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Ordering;
-import org.joda.time.Instant;
-
-/**
- * Executes callbacks that occur based on the progression of the watermark per-step.
- *
- * <p>Callbacks are registered by calls to {@link #callOnGuaranteedFiring(PTransformNode,
- * BoundedWindow, WindowingStrategy, Runnable)}, and are executed after a call to {@link
- * #fireForWatermark(PTransformNode, Instant)} with the same {@link PTransformNode} and a watermark
- * sufficient to ensure that the trigger for the windowing strategy would have been produced.
- *
- * <p>NOTE: {@link WatermarkCallbackExecutor} does not track the latest observed watermark for any
- * {@link PTransformNode} - any call to {@link #callOnGuaranteedFiring(PTransformNode,
- * BoundedWindow, WindowingStrategy, Runnable)} that could have potentially already fired should be
- * followed by a call to {@link #fireForWatermark(PTransformNode, Instant)} for the same transform
- * with the current value of the watermark.
- */
-class WatermarkCallbackExecutor {
- /** Create a new {@link WatermarkCallbackExecutor}. */
- public static WatermarkCallbackExecutor create(Executor executor) {
- return new WatermarkCallbackExecutor(executor);
- }
-
- private final ConcurrentMap<PTransformNode, PriorityQueue<WatermarkCallback>> callbacks;
- private final Executor executor;
-
- private WatermarkCallbackExecutor(Executor executor) {
- this.callbacks = new ConcurrentHashMap<>();
- this.executor = executor;
- }
-
- /**
- * Execute the provided {@link Runnable} after the next call to {@link
- * #fireForWatermark(PTransformNode, Instant)} where the window is guaranteed to have produced
- * output.
- */
- public void callOnGuaranteedFiring(
- PTransformNode step,
- BoundedWindow window,
- WindowingStrategy<?, ?> windowingStrategy,
- Runnable runnable) {
- WatermarkCallback callback =
- WatermarkCallback.onGuaranteedFiring(window, windowingStrategy, runnable);
-
- PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
- if (callbackQueue == null) {
- callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
- if (callbacks.putIfAbsent(step, callbackQueue) != null) {
- callbackQueue = callbacks.get(step);
- }
- }
-
- synchronized (callbackQueue) {
- callbackQueue.offer(callback);
- }
- }
-
- /**
- * Execute the provided {@link Runnable} after the next call to {@link
- * #fireForWatermark(PTransformNode, Instant)} where the window is guaranteed to be expired.
- */
- public void callOnWindowExpiration(
- PTransformNode step,
- BoundedWindow window,
- WindowingStrategy<?, ?> windowingStrategy,
- Runnable runnable) {
- WatermarkCallback callback =
- WatermarkCallback.afterWindowExpiration(window, windowingStrategy, runnable);
-
- PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
- if (callbackQueue == null) {
- callbackQueue = new PriorityQueue<>(11, new CallbackOrdering());
- if (callbacks.putIfAbsent(step, callbackQueue) != null) {
- callbackQueue = callbacks.get(step);
- }
- }
-
- synchronized (callbackQueue) {
- callbackQueue.offer(callback);
- }
- }
-
- /**
- * Schedule all pending callbacks that must have produced output by the time of the provided
- * watermark.
- */
- public void fireForWatermark(PTransformNode step, Instant watermark) {
- PriorityQueue<WatermarkCallback> callbackQueue = callbacks.get(step);
- if (callbackQueue == null) {
- return;
- }
- synchronized (callbackQueue) {
- while (!callbackQueue.isEmpty() && callbackQueue.peek().shouldFire(watermark)) {
- executor.execute(callbackQueue.poll().getCallback());
- }
- }
- }
-
- private static class WatermarkCallback {
- public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
- BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
- @SuppressWarnings("unchecked")
- Instant firingAfter = strategy.getTrigger().getWatermarkThatGuaranteesFiring((W) window);
- return new WatermarkCallback(firingAfter, callback);
- }
-
- public static <W extends BoundedWindow> WatermarkCallback afterWindowExpiration(
- BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
- // Fire one milli past the end of the window. This ensures that all window expiration
- // timers are delivered first
- Instant firingAfter = window.maxTimestamp().plus(strategy.getAllowedLateness()).plus(1L);
- return new WatermarkCallback(firingAfter, callback);
- }
-
- private final Instant fireAfter;
- private final Runnable callback;
-
- private WatermarkCallback(Instant fireAfter, Runnable callback) {
- this.fireAfter = fireAfter;
- this.callback = callback;
- }
-
- public boolean shouldFire(Instant currentWatermark) {
- return currentWatermark.isAfter(fireAfter)
- || currentWatermark.equals(BoundedWindow.TIMESTAMP_MAX_VALUE);
- }
-
- public Runnable getCallback() {
- return callback;
- }
- }
-
- private static class CallbackOrdering extends Ordering<WatermarkCallback>
- implements Serializable {
- @Override
- public int compare(WatermarkCallback left, WatermarkCallback right) {
- return ComparisonChain.start()
- .compare(left.fireAfter, right.fireAfter)
- .compare(left.callback, right.callback, Ordering.arbitrary())
- .result();
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WindowEvaluatorFactory.java
deleted file mode 100644
index 83d745d..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/WindowEvaluatorFactory.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.joda.time.Instant;
-
-/**
- * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link Window.Assign}
- * primitive {@link PTransform}.
- */
-class WindowEvaluatorFactory implements TransformEvaluatorFactory {
- private final EvaluationContext evaluationContext;
-
- WindowEvaluatorFactory(EvaluationContext evaluationContext) {
- this.evaluationContext = evaluationContext;
- }
-
- @Override
- public <InputT> TransformEvaluator<InputT> forApplication(
- PTransformNode application, @Nullable CommittedBundle<?> inputBundle) {
- return createTransformEvaluator(application);
- }
-
- private <InputT> TransformEvaluator<InputT> createTransformEvaluator(PTransformNode transform) {
- WindowFn<? super InputT, ?> fn = null;
-
- PCollectionNode outputPCollection = null;
- evaluationContext.createBundle(outputPCollection);
- throw new UnsupportedOperationException("Not yet migrated");
- }
-
- @Override
- public void cleanup() {}
-
- private static class WindowIntoEvaluator<InputT> implements TransformEvaluator<InputT> {
- private final PTransformNode transform;
- private final WindowFn<InputT, ?> windowFn;
- private final UncommittedBundle<InputT> outputBundle;
-
- @SuppressWarnings("unchecked")
- public WindowIntoEvaluator(
- PTransformNode transform,
- WindowFn<? super InputT, ?> windowFn,
- UncommittedBundle<InputT> outputBundle) {
- this.outputBundle = outputBundle;
- this.transform = transform;
- // Safe contravariant cast
- this.windowFn = (WindowFn<InputT, ?>) windowFn;
- }
-
- @Override
- public void processElement(WindowedValue<InputT> compressedElement) throws Exception {
- for (WindowedValue<InputT> element : compressedElement.explodeWindows()) {
- Collection<? extends BoundedWindow> windows = assignWindows(windowFn, element);
- outputBundle.add(
- WindowedValue.of(
- element.getValue(), element.getTimestamp(), windows, element.getPane()));
- }
- }
-
- private <W extends BoundedWindow> Collection<? extends BoundedWindow> assignWindows(
- WindowFn<InputT, W> windowFn, WindowedValue<InputT> element) throws Exception {
- WindowFn<InputT, W>.AssignContext assignContext =
- new DirectAssignContext<>(windowFn, element);
- return windowFn.assignWindows(assignContext);
- }
-
- @Override
- public TransformResult<InputT> finishBundle() throws Exception {
- return StepTransformResult.<InputT>withoutHold(transform).addOutput(outputBundle).build();
- }
- }
-
- private static class DirectAssignContext<InputT, W extends BoundedWindow>
- extends WindowFn<InputT, W>.AssignContext {
- private final WindowedValue<InputT> value;
-
- public DirectAssignContext(WindowFn<InputT, W> fn, WindowedValue<InputT> value) {
- fn.super();
- this.value = value;
- }
-
- @Override
- public InputT element() {
- return value.getValue();
- }
-
- @Override
- public Instant timestamp() {
- return value.getTimestamp();
- }
-
- @Override
- public BoundedWindow window() {
- return Iterables.getOnlyElement(value.getWindows());
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalArtifactStagingLocation.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalArtifactStagingLocation.java
deleted file mode 100644
index eb9b33b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalArtifactStagingLocation.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.artifact;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.io.File;
-import java.io.IOException;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
-
-/**
- * A location where the results of an {@link LocalFileSystemArtifactStagerService} are stored and
- * where the retrieval service retrieves them from.
- */
-class LocalArtifactStagingLocation {
- /**
- * Create a new {@link LocalArtifactStagingLocation} rooted at the specified location, creating
- * any directories or subdirectories as necessary.
- */
- public static LocalArtifactStagingLocation createAt(File rootDirectory) {
- return new LocalArtifactStagingLocation(rootDirectory).createDirectories();
- }
-
- /**
- * Create a {@link LocalArtifactStagingLocation} for an existing directory. The directory must
- * contain a manifest and an artifact directory.
- */
- public static LocalArtifactStagingLocation forExistingDirectory(File rootDirectory) {
- return new LocalArtifactStagingLocation(rootDirectory).verifyExistence();
- }
-
- private final File rootDirectory;
- private final File artifactsDirectory;
-
- private LocalArtifactStagingLocation(File base) {
- this.rootDirectory = base;
- this.artifactsDirectory = new File(base, "artifacts");
- }
-
- private LocalArtifactStagingLocation createDirectories() {
- if (((rootDirectory.exists() && rootDirectory.isDirectory()) || rootDirectory.mkdirs())
- && rootDirectory.canWrite()) {
- checkState(
- ((artifactsDirectory.exists() && artifactsDirectory.isDirectory())
- || artifactsDirectory.mkdir())
- && artifactsDirectory.canWrite(),
- "Could not create artifact staging directory at %s",
- artifactsDirectory);
- } else {
- throw new IllegalStateException(
- String.format("Could not create staging directory structure at root %s", rootDirectory));
- }
- return this;
- }
-
- private LocalArtifactStagingLocation verifyExistence() {
- checkArgument(rootDirectory.exists(), "Nonexistent staging location root %s", rootDirectory);
- checkArgument(
- rootDirectory.isDirectory(), "Staging location %s is not a directory", rootDirectory);
- checkArgument(
- artifactsDirectory.exists(), "Nonexistent artifact directory %s", artifactsDirectory);
- checkArgument(
- artifactsDirectory.isDirectory(),
- "Artifact location %s is not a directory",
- artifactsDirectory);
- checkArgument(getManifestFile().exists(), "No Manifest in existing location %s", rootDirectory);
- return this;
- }
-
- /**
- * Returns the {@link File} which contains the artifact with the provided name.
- *
- * <p>The file may not exist.
- */
- public File getArtifactFile(String artifactName) {
- return new File(artifactsDirectory, artifactName);
- }
-
- /**
- * Returns the {@link File} which contains the {@link Manifest}.
- *
- * <p>The file may not exist.
- */
- public File getManifestFile() {
- return new File(rootDirectory, "MANIFEST");
- }
-
- /**
- * Returns the local location of this {@link LocalArtifactStagingLocation}.
- *
- * <p>This can be used to refer to the staging location when creating a retrieval service.
- */
- public String getRootPath() {
- try {
- return rootDirectory.getCanonicalPath();
- } catch (IOException e) {
- throw new IllegalStateException(e);
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactRetrievalService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactRetrievalService.java
deleted file mode 100644
index 41a3d8a..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactRetrievalService.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.artifact;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileChannel.MapMode;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactChunk;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.GetManifestResponse;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
-import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
-import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
-import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.stub.StreamObserver;
-
-/** An {@code ArtifactRetrievalService} which stages files to a local temp directory. */
-public class LocalFileSystemArtifactRetrievalService
- extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase
- implements ArtifactRetrievalService {
- private static final int DEFAULT_CHUNK_SIZE = 2 * 1024 * 1024;
-
- public static LocalFileSystemArtifactRetrievalService forRootDirectory(File base) {
- return new LocalFileSystemArtifactRetrievalService(base);
- }
-
- private final LocalArtifactStagingLocation location;
- private final Manifest manifest;
-
- private LocalFileSystemArtifactRetrievalService(File rootDirectory) {
- this.location = LocalArtifactStagingLocation.forExistingDirectory(rootDirectory);
- try (FileInputStream manifestStream = new FileInputStream(location.getManifestFile())) {
- this.manifest = ArtifactApi.Manifest.parseFrom(manifestStream);
- } catch (FileNotFoundException e) {
- throw new IllegalArgumentException(
- String.format(
- "No %s in root directory %s", Manifest.class.getSimpleName(), rootDirectory),
- e);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public final void getManifest(
- ArtifactApi.GetManifestRequest request,
- StreamObserver<GetManifestResponse> responseObserver) {
- try {
- responseObserver.onNext(GetManifestResponse.newBuilder().setManifest(manifest).build());
- responseObserver.onCompleted();
- } catch (Exception e) {
- responseObserver.onError(Status.INTERNAL.withCause(e).asException());
- }
- }
-
- /** Get the artifact with the provided name as a sequence of bytes. */
- private ByteBuffer getArtifact(String name) throws IOException {
- File artifact = location.getArtifactFile(name);
- if (!artifact.exists()) {
- throw new FileNotFoundException(String.format("No such artifact %s", name));
- }
- FileChannel input = new FileInputStream(artifact).getChannel();
- return input.map(MapMode.READ_ONLY, 0L, input.size());
- }
-
- @Override
- public void getArtifact(
- ArtifactApi.GetArtifactRequest request,
- StreamObserver<ArtifactApi.ArtifactChunk> responseObserver) {
- try {
- ByteBuffer artifact = getArtifact(request.getName());
- do {
- responseObserver.onNext(
- ArtifactChunk.newBuilder()
- .setData(
- ByteString.copyFrom(
- artifact, Math.min(artifact.remaining(), DEFAULT_CHUNK_SIZE)))
- .build());
- } while (artifact.hasRemaining());
- responseObserver.onCompleted();
- } catch (FileNotFoundException e) {
- responseObserver.onError(
- Status.INVALID_ARGUMENT
- .withDescription(String.format("No such artifact %s", request.getName()))
- .withCause(e)
- .asException());
- } catch (Exception e) {
- responseObserver.onError(
- Status.INTERNAL
- .withDescription(
- String.format("Could not retrieve artifact with name %s", request.getName()))
- .withCause(e)
- .asException());
- }
- }
-
- @Override
- public void close() throws Exception {}
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactStagerService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactStagerService.java
deleted file mode 100644
index 3df1f54..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/LocalFileSystemArtifactStagerService.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.artifact;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import javax.annotation.Nullable;
-import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
-import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc;
-import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.StatusException;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Throwables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An {@code ArtifactStagingService} which stages files to a local temp directory. TODO: refactor to
- * use staging session tokens
- */
-public class LocalFileSystemArtifactStagerService
- extends ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase implements FnService {
- private static final Logger LOG =
- LoggerFactory.getLogger(LocalFileSystemArtifactStagerService.class);
-
- public static LocalFileSystemArtifactStagerService forRootDirectory(File base) {
- return new LocalFileSystemArtifactStagerService(base);
- }
-
- private final LocalArtifactStagingLocation location;
-
- private LocalFileSystemArtifactStagerService(File stagingBase) {
- this.location = LocalArtifactStagingLocation.createAt(stagingBase);
- }
-
- @Override
- public StreamObserver<ArtifactApi.PutArtifactRequest> putArtifact(
- final StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver) {
- return new CreateAndWriteFileObserver(responseObserver);
- }
-
- @Override
- public void commitManifest(
- ArtifactApi.CommitManifestRequest request,
- StreamObserver<ArtifactApi.CommitManifestResponse> responseObserver) {
- try {
- commitManifestOrThrow(request, responseObserver);
- } catch (StatusRuntimeException e) {
- responseObserver.onError(e);
- LOG.error("Failed to commit Manifest {}", request.getManifest(), e);
- } catch (Exception e) {
- responseObserver.onError(
- Status.INTERNAL
- .withCause(e)
- .withDescription(Throwables.getStackTraceAsString(e))
- .asRuntimeException());
- LOG.error("Failed to commit Manifest {}", request.getManifest(), e);
- }
- }
-
- private void commitManifestOrThrow(
- ArtifactApi.CommitManifestRequest request,
- StreamObserver<ArtifactApi.CommitManifestResponse> responseObserver)
- throws IOException {
- Collection<ArtifactApi.ArtifactMetadata> missing = new ArrayList<>();
- for (ArtifactApi.ArtifactMetadata artifact : request.getManifest().getArtifactList()) {
- // TODO: Validate the checksums on the server side, to fail more aggressively if require
- if (!location.getArtifactFile(artifact.getName()).exists()) {
- missing.add(artifact);
- }
- }
- if (!missing.isEmpty()) {
- throw Status.INVALID_ARGUMENT
- .withDescription(
- String.format("Attempted to commit manifest with missing Artifacts: [%s]", missing))
- .asRuntimeException();
- }
- File mf = location.getManifestFile();
- checkState(mf.createNewFile(), "Could not create file to store manifest");
- try (OutputStream mfOut = new FileOutputStream(mf)) {
- request.getManifest().writeTo(mfOut);
- }
- responseObserver.onNext(
- ArtifactApi.CommitManifestResponse.newBuilder()
- .setRetrievalToken(location.getRootPath())
- .build());
- responseObserver.onCompleted();
- }
-
- @Override
- public void close() throws Exception {
- // TODO: Close all active staging calls, signalling errors to the caller.
- }
-
- @VisibleForTesting
- LocalArtifactStagingLocation getLocation() {
- return location;
- }
-
- private class CreateAndWriteFileObserver
- implements StreamObserver<ArtifactApi.PutArtifactRequest> {
- private final StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver;
- private FileWritingObserver writer;
-
- private CreateAndWriteFileObserver(
- StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver) {
- this.responseObserver = responseObserver;
- }
-
- @Override
- public void onNext(ArtifactApi.PutArtifactRequest value) {
- try {
- if (writer == null) {
- if (!value.getContentCase().equals(ArtifactApi.PutArtifactRequest.ContentCase.METADATA)) {
- throw Status.INVALID_ARGUMENT
- .withDescription(
- String.format(
- "Expected the first %s to contain the Artifact Name, got %s",
- ArtifactApi.PutArtifactRequest.class.getSimpleName(),
- value.getContentCase()))
- .asRuntimeException();
- }
- writer = createFile(value.getMetadata().getMetadata());
- } else {
- writer.onNext(value);
- }
- } catch (StatusRuntimeException e) {
- responseObserver.onError(e);
- } catch (Exception e) {
- responseObserver.onError(
- Status.INTERNAL
- .withCause(e)
- .withDescription(Throwables.getStackTraceAsString(e))
- .asRuntimeException());
- }
- }
-
- private FileWritingObserver createFile(ArtifactApi.ArtifactMetadata metadata)
- throws IOException {
- File destination = location.getArtifactFile(metadata.getName());
- if (!destination.createNewFile()) {
- throw Status.ALREADY_EXISTS
- .withDescription(String.format("Artifact with name %s already exists", metadata))
- .asRuntimeException();
- }
- return new FileWritingObserver(
- destination, new FileOutputStream(destination), responseObserver);
- }
-
- @Override
- public void onError(Throwable t) {
- if (writer != null) {
- writer.onError(t);
- } else {
- responseObserver.onCompleted();
- }
- }
-
- @Override
- public void onCompleted() {
- if (writer != null) {
- writer.onCompleted();
- } else {
- responseObserver.onCompleted();
- }
- }
- }
-
- private static class FileWritingObserver
- implements StreamObserver<ArtifactApi.PutArtifactRequest> {
- private final File destination;
- private final OutputStream target;
- private final StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver;
-
- private FileWritingObserver(
- File destination,
- OutputStream target,
- StreamObserver<ArtifactApi.PutArtifactResponse> responseObserver) {
- this.destination = destination;
- this.target = target;
- this.responseObserver = responseObserver;
- }
-
- @Override
- public void onNext(ArtifactApi.PutArtifactRequest value) {
- try {
- if (value.getData() == null) {
- StatusRuntimeException e =
- Status.INVALID_ARGUMENT
- .withDescription(
- String.format(
- "Expected all chunks in the current stream state to contain data, got %s",
- value.getContentCase()))
- .asRuntimeException();
- throw e;
- }
- value.getData().getData().writeTo(target);
- } catch (Exception e) {
- cleanedUp(e);
- }
- }
-
- @Override
- public void onError(Throwable t) {
- if (cleanedUp(null)) {
- responseObserver.onCompleted();
- }
- }
-
- @Override
- public void onCompleted() {
- try {
- target.close();
- } catch (IOException e) {
- LOG.error("Failed to complete writing file {}", destination, e);
- cleanedUp(e);
- return;
- }
- responseObserver.onNext(ArtifactApi.PutArtifactResponse.getDefaultInstance());
- responseObserver.onCompleted();
- }
-
- /**
- * Cleans up after the file writing failed exceptionally, due to an error either in the service
- * or sent from the client.
- *
- * @return false if an error was reported, true otherwise
- */
- private boolean cleanedUp(@Nullable Throwable whyFailed) {
- Throwable actual = whyFailed;
- try {
- target.close();
- if (!destination.delete()) {
- LOG.debug("Couldn't delete failed write at {}", destination);
- }
- } catch (IOException e) {
- if (whyFailed == null) {
- actual = e;
- } else {
- actual.addSuppressed(e);
- }
- LOG.error("Failed to clean up after writing file {}", destination, e);
- }
- if (actual != null) {
- if (actual instanceof StatusException || actual instanceof StatusRuntimeException) {
- responseObserver.onError(actual);
- } else {
- Status status =
- Status.INTERNAL
- .withCause(actual)
- .withDescription(Throwables.getStackTraceAsString(actual));
- responseObserver.onError(status.asException());
- }
- }
- return actual == null;
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/UnsupportedArtifactRetrievalService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/UnsupportedArtifactRetrievalService.java
deleted file mode 100644
index f0d56ba..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/UnsupportedArtifactRetrievalService.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.artifact;
-
-import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
-import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
-
-/**
- * An {@link ArtifactRetrievalService} which has not implemented any methods.
- *
- * <p>For use with an in-process SDK harness.
- */
-public class UnsupportedArtifactRetrievalService
- extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase
- implements ArtifactRetrievalService {
-
- public static ArtifactRetrievalService create() {
- return new UnsupportedArtifactRetrievalService();
- }
-
- private UnsupportedArtifactRetrievalService() {}
-
- @Override
- public void close() {
- // Do nothing.
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/package-info.java
deleted file mode 100644
index 1ec0da5..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/artifact/package-info.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/** Provides local implementations of the Artifact API services. */
-package org.apache.beam.runners.direct.portable.artifact;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/PreparingJob.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/PreparingJob.java
deleted file mode 100644
index 077a517..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/PreparingJob.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.job;
-
-import com.google.auto.value.AutoValue;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactStagingService;
-import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.Struct;
-
-/** A Job with a {@code prepare} call but no corresponding {@code run} call. */
-@AutoValue
-abstract class PreparingJob implements AutoCloseable {
- public static Builder builder() {
- return new AutoValue_PreparingJob.Builder();
- }
-
- abstract Pipeline getPipeline();
-
- abstract Struct getOptions();
-
- abstract String getStagingSessionToken();
-
- abstract GrpcFnServer<BeamFileSystemArtifactStagingService> getArtifactStagingServer();
-
- @Override
- public void close() throws Exception {
- getArtifactStagingServer().close();
- }
-
- @AutoValue.Builder
- public abstract static class Builder {
- abstract Builder setPipeline(Pipeline pipeline);
-
- abstract Builder setOptions(Struct options);
-
- abstract Builder setStagingSessionToken(String stagingSessionToken);
-
- abstract Builder setArtifactStagingServer(
- GrpcFnServer<BeamFileSystemArtifactStagingService> server);
-
- abstract PreparingJob build();
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobServer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobServer.java
deleted file mode 100644
index e2eac87..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobServer.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.job;
-
-import java.io.IOException;
-import java.nio.file.Paths;
-import java.util.Arrays;
-import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.ServerFactory;
-import org.kohsuke.args4j.CmdLineException;
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** A program that runs a {@link ReferenceRunnerJobService}. */
-public class ReferenceRunnerJobServer {
- private static final Logger LOG = LoggerFactory.getLogger(ReferenceRunnerJobServer.class);
- private final ServerConfiguration configuration;
- private GrpcFnServer<ReferenceRunnerJobService> server;
-
- private ReferenceRunnerJobServer(ServerConfiguration configuration) {
- this.configuration = configuration;
- }
-
- public static void main(String[] args) throws Exception {
- try {
- runServer(parseConfiguration(args));
- } catch (CmdLineException ignored) {
- }
- }
-
- private static ServerConfiguration parseConfiguration(String[] args) throws CmdLineException {
- ServerConfiguration configuration = new ServerConfiguration();
- CmdLineParser parser = new CmdLineParser(configuration);
- try {
- parser.parseArgument(args);
- } catch (CmdLineException e) {
- e.printStackTrace(System.err);
- printUsage(parser);
- throw e;
- }
- return configuration;
- }
-
- private static void printUsage(CmdLineParser parser) {
- System.err.println(
- String.format(
- "Usage: java %s arguments...", ReferenceRunnerJobService.class.getSimpleName()));
- parser.printUsage(System.err);
- System.err.println();
- }
-
- private static void runServer(ServerConfiguration configuration) throws Exception {
- ServerFactory serverFactory = ServerFactory.createDefault();
- ReferenceRunnerJobService.Configuration jobServiceConfig =
- createJobServiceConfig(configuration);
- ReferenceRunnerJobService service =
- ReferenceRunnerJobService.create(serverFactory, jobServiceConfig);
- try (GrpcFnServer<ReferenceRunnerJobService> server =
- createServer(configuration, serverFactory, service)) {
- System.out.println(
- String.format(
- "Started %s at %s",
- ReferenceRunnerJobService.class.getSimpleName(),
- server.getApiServiceDescriptor().getUrl()));
- server.getServer().awaitTermination();
- }
- System.out.println("Server shut down, exiting");
- }
-
- public static ReferenceRunnerJobServer fromParams(String[] args) {
- try {
- return new ReferenceRunnerJobServer(parseConfiguration(args));
- } catch (CmdLineException e) {
- throw new IllegalArgumentException(
- "Unable to parse command line arguments " + Arrays.asList(args), e);
- }
- }
-
- public String start() throws Exception {
- ServerFactory serverFactory = ServerFactory.createDefault();
- ReferenceRunnerJobService.Configuration jobServiceConfig =
- createJobServiceConfig(configuration);
- server =
- createServer(
- configuration,
- serverFactory,
- ReferenceRunnerJobService.create(serverFactory, jobServiceConfig));
-
- return server.getApiServiceDescriptor().getUrl();
- }
-
- public void stop() {
- if (server != null) {
- try {
- server.close();
- } catch (Exception e) {
- LOG.error("Unable to stop job server.", e);
- }
- }
- }
-
- private static GrpcFnServer<ReferenceRunnerJobService> createServer(
- ServerConfiguration configuration,
- ServerFactory serverFactory,
- ReferenceRunnerJobService service)
- throws IOException {
- if (configuration.port <= 0) {
- return GrpcFnServer.allocatePortAndCreateFor(service, serverFactory);
- }
- return GrpcFnServer.create(
- service,
- ApiServiceDescriptor.newBuilder().setUrl("localhost:" + configuration.port).build(),
- serverFactory);
- }
-
- /**
- * Helper function to fill out a {@code ReferenceRunnerJobService.Configuration Configuration}
- * object for {@code ReferenceRunnerJobService}.
- */
- private static ReferenceRunnerJobService.Configuration createJobServiceConfig(
- ServerConfiguration configuration) {
- ReferenceRunnerJobService.Configuration jobServiceConfig =
- new ReferenceRunnerJobService.Configuration();
- jobServiceConfig.artifactStagingPath = configuration.artifactStagingPath;
- jobServiceConfig.keepArtifacts = configuration.keepArtifacts;
- return jobServiceConfig;
- }
-
- /** Command-line options to configure the JobServer. */
- public static class ServerConfiguration {
- @Option(
- name = "-p",
- aliases = {"--port"},
- usage = "The local port to expose the server on. 0 to use a dynamic port. (Default: 8099)")
- private int port = 8099;
-
- @Option(name = "--artifacts-dir", usage = "The location to store staged artifact files")
- String artifactStagingPath =
- Paths.get(System.getProperty("java.io.tmpdir"), "beam-artifact-staging").toString();
-
- @Option(
- name = "--keep-artifacts",
- usage = "When enabled, do not delete staged artifacts when a job completes")
- boolean keepArtifacts;
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
deleted file mode 100644
index 07517af..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/ReferenceRunnerJobService.java
+++ /dev/null
@@ -1,308 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable.job;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.beam.model.jobmanagement.v1.JobApi;
-import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobState;
-import org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum;
-import org.apache.beam.model.jobmanagement.v1.JobApi.PrepareJobResponse;
-import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobRequest;
-import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobResponse;
-import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceImplBase;
-import org.apache.beam.runners.direct.portable.ReferenceRunner;
-import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.runners.fnexecution.GrpcFnServer;
-import org.apache.beam.runners.fnexecution.ServerFactory;
-import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactStagingService;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p13p1.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This JobService implements the grpc calls for running jobs by using the {@code ReferenceRunner}
- * as an engine.
- */
-public class ReferenceRunnerJobService extends JobServiceImplBase implements FnService {
-
- /** A configuration object for constructing the {@code ReferenceRunnerJobService}. */
- public static class Configuration {
- public String artifactStagingPath;
- public boolean keepArtifacts;
- }
-
- private static final Logger LOG = LoggerFactory.getLogger(ReferenceRunnerJobService.class);
- private static final int WAIT_MS = 1000;
-
- public static ReferenceRunnerJobService create(
- final ServerFactory serverFactory, Configuration configuration) {
- LOG.info("Starting {}", ReferenceRunnerJobService.class);
- return new ReferenceRunnerJobService(serverFactory, configuration);
- }
-
- private final ServerFactory serverFactory;
- private final Configuration configuration;
-
- private final ConcurrentMap<String, PreparingJob> unpreparedJobs;
- private final ConcurrentMap<String, ReferenceRunner> runningJobs;
- private final ConcurrentMap<String, JobState.Enum> jobStates;
- private final ExecutorService executor;
- private final ConcurrentLinkedQueue<GrpcFnServer<BeamFileSystemArtifactStagingService>>
- artifactStagingServices;
-
- private ReferenceRunnerJobService(ServerFactory serverFactory, Configuration configuration) {
- this.serverFactory = serverFactory;
- this.configuration = configuration;
- unpreparedJobs = new ConcurrentHashMap<>();
- runningJobs = new ConcurrentHashMap<>();
- jobStates = new ConcurrentHashMap<>();
- executor =
- Executors.newCachedThreadPool(
- new ThreadFactoryBuilder()
- .setDaemon(false)
- .setNameFormat("reference-runner-pipeline-%s")
- .build());
- artifactStagingServices = new ConcurrentLinkedQueue<>();
- }
-
- @Override
- public void prepare(
- JobApi.PrepareJobRequest request,
- StreamObserver<JobApi.PrepareJobResponse> responseObserver) {
- try {
- LOG.trace("{} {}", PrepareJobResponse.class.getSimpleName(), request);
-
- String preparationId = request.getJobName() + ThreadLocalRandom.current().nextInt();
- GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingService =
- createArtifactStagingService();
- artifactStagingServices.add(artifactStagingService);
- String stagingSessionToken =
- BeamFileSystemArtifactStagingService.generateStagingSessionToken(
- preparationId, configuration.artifactStagingPath);
- PreparingJob existingJob =
- unpreparedJobs.putIfAbsent(
- preparationId,
- PreparingJob.builder()
- .setArtifactStagingServer(artifactStagingService)
- .setPipeline(request.getPipeline())
- .setOptions(request.getPipelineOptions())
- .setStagingSessionToken(stagingSessionToken)
- .build());
- checkArgument(
- existingJob == null, "Unexpected existing job with preparation ID %s", preparationId);
-
- responseObserver.onNext(
- PrepareJobResponse.newBuilder()
- .setPreparationId(preparationId)
- .setArtifactStagingEndpoint(artifactStagingService.getApiServiceDescriptor())
- .setStagingSessionToken(stagingSessionToken)
- .build());
- responseObserver.onCompleted();
- } catch (Exception e) {
- LOG.error("Could not prepare job with name {}", request.getJobName(), e);
- responseObserver.onError(Status.INTERNAL.withCause(e).asException());
- }
- }
-
- private GrpcFnServer<BeamFileSystemArtifactStagingService> createArtifactStagingService()
- throws Exception {
- BeamFileSystemArtifactStagingService service = new BeamFileSystemArtifactStagingService();
- return GrpcFnServer.allocatePortAndCreateFor(service, serverFactory);
- }
-
- @Override
- @SuppressWarnings("FutureReturnValueIgnored") // Run API does not block on execution
- public void run(
- JobApi.RunJobRequest request, StreamObserver<JobApi.RunJobResponse> responseObserver) {
- try {
- LOG.trace("{} {}", RunJobRequest.class.getSimpleName(), request);
- String preparationId = request.getPreparationId();
- PreparingJob preparingJob = unpreparedJobs.get(preparationId);
- if (preparingJob == null) {
- responseObserver.onError(
- Status.INVALID_ARGUMENT
- .withDescription(String.format("Unknown Preparation Id %s", preparationId))
- .asException());
- return;
- }
- try {
- // Close any preparation-time only resources.
- preparingJob.close();
- } catch (Exception e) {
- responseObserver.onError(e);
- }
-
- ReferenceRunner runner =
- ReferenceRunner.forPipeline(
- preparingJob.getPipeline(), preparingJob.getOptions(), request.getRetrievalToken());
- String jobId = "job-" + Integer.toString(ThreadLocalRandom.current().nextInt());
- responseObserver.onNext(RunJobResponse.newBuilder().setJobId(jobId).build());
- responseObserver.onCompleted();
- runningJobs.put(jobId, runner);
- jobStates.putIfAbsent(jobId, Enum.RUNNING);
- executor.submit(
- () -> {
- try {
- jobStates.computeIfPresent(jobId, (id, status) -> Enum.RUNNING);
- runner.execute();
- jobStates.computeIfPresent(jobId, (id, status) -> Enum.DONE);
- } catch (Exception e) {
- jobStates.computeIfPresent(jobId, (id, status) -> Enum.FAILED);
- throw e;
- }
-
- // Delete artifacts after job is done.
- if (!configuration.keepArtifacts) {
- String stagingSessionToken = preparingJob.getStagingSessionToken();
- try {
- preparingJob
- .getArtifactStagingServer()
- .getService()
- .removeArtifacts(stagingSessionToken);
- } catch (Exception e) {
- LOG.error(
- "Failed to remove job staging directory for token {}: {}",
- stagingSessionToken,
- e);
- }
- }
- return null;
- });
- } catch (StatusRuntimeException e) {
- responseObserver.onError(e);
- } catch (Exception e) {
- responseObserver.onError(Status.INTERNAL.withCause(e).asException());
- }
- }
-
- @Override
- public void getState(
- GetJobStateRequest request, StreamObserver<GetJobStateResponse> responseObserver) {
- LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
- try {
- responseObserver.onNext(
- GetJobStateResponse.newBuilder()
- .setState(jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED))
- .build());
- responseObserver.onCompleted();
- } catch (Exception e) {
- String errMessage =
- String.format("Encountered Unexpected Exception for Invocation %s", request.getJobId());
- LOG.error(errMessage, e);
- responseObserver.onError(Status.INTERNAL.withCause(e).asException());
- }
- }
-
- @Override
- public void getStateStream(
- GetJobStateRequest request, StreamObserver<GetJobStateResponse> responseObserver) {
- LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
- String invocationId = request.getJobId();
- try {
- Thread.sleep(WAIT_MS);
- Enum state = jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED);
- responseObserver.onNext(GetJobStateResponse.newBuilder().setState(state).build());
- while (Enum.RUNNING.equals(state)) {
- Thread.sleep(WAIT_MS);
- state = jobStates.getOrDefault(request.getJobId(), Enum.UNRECOGNIZED);
- }
- responseObserver.onNext(GetJobStateResponse.newBuilder().setState(state).build());
- } catch (Exception e) {
- String errMessage =
- String.format("Encountered Unexpected Exception for Invocation %s", invocationId);
- LOG.error(errMessage, e);
- responseObserver.onError(Status.INTERNAL.withCause(e).asException());
- }
- responseObserver.onCompleted();
- }
-
- @Override
- public void describePipelineOptions(
- JobApi.DescribePipelineOptionsRequest request,
- StreamObserver<JobApi.DescribePipelineOptionsResponse> responseObserver) {
- LOG.trace("{} {}", JobApi.DescribePipelineOptionsRequest.class.getSimpleName(), request);
- try {
- JobApi.DescribePipelineOptionsResponse response =
- JobApi.DescribePipelineOptionsResponse.newBuilder()
- .addAllOptions(
- PipelineOptionsFactory.describe(PipelineOptionsFactory.getRegisteredOptions()))
- .build();
- responseObserver.onNext(response);
- responseObserver.onCompleted();
- } catch (Exception e) {
- LOG.error("Error describing pipeline options", e);
- responseObserver.onError(Status.INTERNAL.withCause(e).asException());
- }
- }
-
- @Override
- public void getMessageStream(
- JobMessagesRequest request, StreamObserver<JobMessagesResponse> responseObserver) {
- // Not implemented
- LOG.trace("{} {}", JobMessagesRequest.class.getSimpleName(), request);
- }
-
- @Override
- public void cancel(CancelJobRequest request, StreamObserver<CancelJobResponse> responseObserver) {
- LOG.trace("{} {}", CancelJobRequest.class.getSimpleName(), request);
- responseObserver.onError(
- Status.NOT_FOUND
- .withDescription(String.format("Unknown Job ID %s", request.getJobId()))
- .asException());
- }
-
- @Override
- public void close() throws Exception {
- for (PreparingJob preparingJob : ImmutableList.copyOf(unpreparedJobs.values())) {
- try {
- preparingJob.close();
- } catch (Exception e) {
- LOG.warn("Exception while closing preparing job {}", preparingJob);
- }
- }
- while (!artifactStagingServices.isEmpty()) {
- GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingService =
- artifactStagingServices.remove();
- try {
- artifactStagingService.close();
- } catch (Exception e) {
- LOG.error(
- "Unable to close staging sevice started on %s",
- artifactStagingService.getApiServiceDescriptor().getUrl(), e);
- }
- }
- }
-}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/package-info.java
deleted file mode 100644
index 9de085b..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/job/package-info.java
+++ /dev/null
@@ -1,23 +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.
- */
-
-/**
- * An execution engine for Beam Pipelines that uses the Java Runner harness and the Fn API to
- * execute.
- */
-package org.apache.beam.runners.direct.portable.job;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/package-info.java
deleted file mode 100644
index 1a51b2f..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/portable/package-info.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Defines the {@link org.apache.beam.sdk.options.PipelineOptions.DirectRunner} which executes both
- * Bounded and Unbounded {@code Pipelines} on the local machine.
- *
- * <p>See {@link org.apache.beam.sdk.runners} for more information about Pipeline Runners.
- */
-package org.apache.beam.runners.direct.portable;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
index 06c8781..4c4ba94 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
@@ -21,7 +21,6 @@ import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
import static org.junit.Assert.assertThat;
import java.util.Set;
-import org.apache.beam.runners.direct.portable.ExecutableGraphBuilder;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineRunner;
import org.apache.beam.sdk.metrics.MetricResults;
@@ -65,7 +64,6 @@ public class DirectRunnerApiSurfaceTest {
.pruningClass(DirectGraphs.class)
.pruningClass(
WatermarkManager.class /* TODO: BEAM-4237 Consider moving to local-java */)
- .pruningClass(ExecutableGraphBuilder.class)
.pruningPattern(
"org[.]apache[.]beam[.]runners[.]direct[.]portable.*"
/* TODO: BEAM-4237 reconsider package layout with the ReferenceRunner */ )
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
deleted file mode 100644
index 0236aef..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components.Builder;
-import org.apache.beam.runners.core.construction.Environments;
-import org.apache.beam.runners.core.construction.RehydratedComponents;
-import org.apache.beam.runners.core.construction.SdkComponents;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory;
-import org.apache.beam.runners.fnexecution.wire.WireCoders;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.fn.data.FnDataReceiver;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.windowing.FixedWindows;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.transforms.windowing.Window;
-import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link BundleFactoryOutputReceiverFactory}. */
-@RunWith(JUnit4.class)
-public class BundleFactoryOutputReceiverFactoryTest {
- private final BundleFactory bundleFactory = ImmutableListBundleFactory.create();
- private PCollectionNode fooPC;
- private PCollectionNode barPC;
- private RunnerApi.Components baseComponents;
-
- private OutputReceiverFactory factory;
- private Collection<UncommittedBundle<?>> outputBundles;
-
- @Before
- public void setup() throws IOException {
- Pipeline p = Pipeline.create();
- PCollection<String> foo =
- p.apply("createFoo", Create.of("1", "2", "3"))
- .apply("windowFoo", Window.into(FixedWindows.of(Duration.standardMinutes(5L))));
- PCollection<Integer> bar = p.apply("bar", Create.of(1, 2, 3));
-
- SdkComponents sdkComponents = SdkComponents.create();
- sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
- String fooId = sdkComponents.registerPCollection(foo);
- String barId = sdkComponents.registerPCollection(bar);
- baseComponents = sdkComponents.toComponents();
-
- fooPC = PipelineNode.pCollection(fooId, baseComponents.getPcollectionsOrThrow(fooId));
- barPC = PipelineNode.pCollection(barId, baseComponents.getPcollectionsOrThrow(barId));
-
- outputBundles = new ArrayList<>();
- factory =
- BundleFactoryOutputReceiverFactory.create(
- bundleFactory, baseComponents, outputBundles::add);
- }
-
- @Test
- public void addsBundlesToResult() {
- factory.create(fooPC.getId());
- factory.create(barPC.getId());
-
- assertThat(Iterables.size(outputBundles), equalTo(2));
-
- Collection<PCollectionNode> pcollections = new ArrayList<>();
- for (UncommittedBundle<?> bundle : outputBundles) {
- pcollections.add(bundle.getPCollection());
- }
- assertThat(pcollections, containsInAnyOrder(fooPC, barPC));
- }
-
- @Test
- public void receiverAddsElementsToBundle() throws Exception {
- FnDataReceiver<WindowedValue<byte[]>> receiver = factory.create(fooPC.getId());
-
- Builder builder = baseComponents.toBuilder();
- String sdkWireCoderId = WireCoders.addSdkWireCoder(fooPC, builder);
- Components components = builder.build();
-
- Coder<WindowedValue<String>> sdkCoder =
- (Coder<WindowedValue<String>>)
- RehydratedComponents.forComponents(components).getCoder(sdkWireCoderId);
- Coder<WindowedValue<byte[]>> runnerCoder =
- WireCoders.instantiateRunnerWireCoder(fooPC, components);
-
- WindowedValue<byte[]> firstElem =
- CoderUtils.decodeFromByteArray(
- runnerCoder,
- CoderUtils.encodeToByteArray(
- sdkCoder,
- WindowedValue.of(
- "1",
- new Instant(120),
- new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
- PaneInfo.NO_FIRING)));
- WindowedValue<byte[]> secondElem =
- CoderUtils.decodeFromByteArray(
- runnerCoder,
- CoderUtils.encodeToByteArray(
- sdkCoder,
- WindowedValue.of(
- "2",
- new Instant(240),
- new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
- PaneInfo.NO_FIRING)));
- receiver.accept(firstElem);
- receiver.accept(secondElem);
-
- CommittedBundle<?> output = getOnlyElement(outputBundles).commit(Instant.now());
- assertThat(output, containsInAnyOrder(firstElem, secondElem));
- }
-
- /**
- * Tests that if a {@link
- * org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode} is provided
- * multiple times, the returned {@link
- * org.apache.beam.runners.fnexecution.control.RemoteOutputReceiver} instances are independent.
- */
- @Test
- public void multipleInstancesOfPCollectionIndependent() throws Exception {
- FnDataReceiver<WindowedValue<byte[]>> firstReceiver = factory.create(fooPC.getId());
- FnDataReceiver<WindowedValue<byte[]>> secondReceiver = factory.create(fooPC.getId());
-
- Components.Builder builder = baseComponents.toBuilder();
- String sdkWireCoderId = WireCoders.addSdkWireCoder(fooPC, builder);
- Components components = builder.build();
-
- Coder<WindowedValue<String>> sdkCoder =
- (Coder<WindowedValue<String>>)
- RehydratedComponents.forComponents(components).getCoder(sdkWireCoderId);
-
- Coder<WindowedValue<byte[]>> runnerCoder =
- WireCoders.instantiateRunnerWireCoder(fooPC, components);
-
- WindowedValue<byte[]> firstElem =
- CoderUtils.decodeFromByteArray(
- runnerCoder,
- CoderUtils.encodeToByteArray(
- sdkCoder,
- WindowedValue.of(
- "1",
- new Instant(120),
- new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
- PaneInfo.NO_FIRING)));
- firstReceiver.accept(firstElem);
-
- WindowedValue<byte[]> secondElem =
- CoderUtils.decodeFromByteArray(
- runnerCoder,
- CoderUtils.encodeToByteArray(
- sdkCoder,
- WindowedValue.of(
- "2",
- new Instant(240),
- new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
- PaneInfo.NO_FIRING)));
- secondReceiver.accept(secondElem);
-
- Collection<WindowedValue<?>> outputs = new ArrayList<>();
- for (UncommittedBundle<?> uncommitted : outputBundles) {
- assertThat(uncommitted.getPCollection(), equalTo(fooPC));
- Iterable<? extends WindowedValue<?>> elements =
- uncommitted.commit(Instant.now()).getElements();
- Iterables.addAll(outputs, elements);
- assertThat(Iterables.size(elements), equalTo(1));
- }
- assertThat(outputs, containsInAnyOrder(firstElem, secondElem));
- }
-
- @Test
- public void differentPCollectionsIndependent() throws Exception {
- FnDataReceiver<WindowedValue<byte[]>> fooReceiver = factory.create(fooPC.getId());
-
- Components.Builder builder = baseComponents.toBuilder();
- String sdkWireCoderId = WireCoders.addSdkWireCoder(fooPC, builder);
- String barSdkWireCoderId = WireCoders.addSdkWireCoder(barPC, builder);
- Components components = builder.build();
-
- Coder<WindowedValue<String>> fooSdkCoder =
- (Coder<WindowedValue<String>>)
- RehydratedComponents.forComponents(components).getCoder(sdkWireCoderId);
- Coder<WindowedValue<byte[]>> fooRunnerCoder =
- WireCoders.instantiateRunnerWireCoder(fooPC, components);
-
- FnDataReceiver<WindowedValue<byte[]>> barReceiver = factory.create(barPC.getId());
- Coder<WindowedValue<Integer>> barSdkCoder =
- (Coder<WindowedValue<Integer>>)
- RehydratedComponents.forComponents(components).getCoder(barSdkWireCoderId);
- Coder<WindowedValue<byte[]>> barRunnerCoder =
- WireCoders.instantiateRunnerWireCoder(barPC, components);
-
- WindowedValue<byte[]> fooElem =
- CoderUtils.decodeFromByteArray(
- fooRunnerCoder,
- CoderUtils.encodeToByteArray(
- fooSdkCoder,
- WindowedValue.of(
- "1",
- new Instant(120),
- new IntervalWindow(new Instant(0), Duration.standardMinutes(5)),
- PaneInfo.NO_FIRING)));
- fooReceiver.accept(fooElem);
-
- WindowedValue<byte[]> barElem =
- CoderUtils.decodeFromByteArray(
- barRunnerCoder,
- CoderUtils.encodeToByteArray(
- barSdkCoder, WindowedValue.timestampedValueInGlobalWindow(2, new Instant(240))));
- barReceiver.accept(barElem);
-
- Collection<? super WindowedValue<?>> outputs = new ArrayList<>();
- for (UncommittedBundle<?> uncommitted : outputBundles) {
- WindowedValue<?> output = getOnlyElement(uncommitted.commit(Instant.now()).getElements());
- if (fooPC.equals(uncommitted.getPCollection())) {
- assertThat(output, equalTo(fooElem));
- } else if (barPC.equals(uncommitted.getPCollection())) {
- assertThat(output, equalTo(barElem));
- } else {
- fail(
- String.format(
- "Output %s should be either 'foo' or 'bar', got '%s",
- PCollection.class.getSimpleName(), uncommitted.getPCollection().getId()));
- }
- outputs.add(output);
- }
- assertThat(outputs, containsInAnyOrder(fooElem, barElem));
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CommittedResultTest.java
deleted file mode 100644
index 357efb3..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CommittedResultTest.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.junit.Assert.assertThat;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.List;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.IsBounded.Enum;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link CommittedResult}. */
-@RunWith(JUnit4.class)
-public class CommittedResultTest implements Serializable {
-
- @Rule
- public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
-
- private transient PCollectionNode created =
- PipelineNode.pCollection(
- "created", RunnerApi.PCollection.newBuilder().setUniqueName("created").build());
- private transient PTransformNode transform =
- PipelineNode.pTransform("foo", RunnerApi.PTransform.getDefaultInstance());
- private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-
- @Test
- public void getTransformExtractsFromResult() {
- CommittedResult<PTransformNode> result =
- CommittedResult.create(
- StepTransformResult.withoutHold(transform).build(),
- Optional.absent(),
- Collections.emptyList(),
- EnumSet.noneOf(OutputType.class));
-
- assertThat(result.getExecutable(), Matchers.equalTo(transform));
- }
-
- @Test
- public void getUncommittedElementsEqualInput() {
- CommittedBundle<Integer> bundle =
- bundleFactory
- .<Integer>createBundle(created)
- .add(WindowedValue.valueInGlobalWindow(2))
- .commit(Instant.now());
- CommittedResult<PTransformNode> result =
- CommittedResult.create(
- StepTransformResult.withoutHold(transform).build(),
- Optional.of(bundle),
- Collections.emptyList(),
- EnumSet.noneOf(OutputType.class));
-
- assertThat(result.getUnprocessedInputs().get(), Matchers.equalTo(bundle));
- }
-
- @Test
- public void getUncommittedElementsNull() {
- CommittedResult<PTransformNode> result =
- CommittedResult.create(
- StepTransformResult.withoutHold(transform).build(),
- Optional.absent(),
- Collections.emptyList(),
- EnumSet.noneOf(OutputType.class));
-
- assertThat(result.getUnprocessedInputs(), Matchers.equalTo(Optional.absent()));
- }
-
- @Test
- public void getOutputsEqualInput() {
- List<? extends CommittedBundle<Integer>> outputs =
- ImmutableList.of(
- bundleFactory
- .<Integer>createBundle(
- PipelineNode.pCollection(
- "bounded",
- RunnerApi.PCollection.newBuilder()
- .setUniqueName("bounded")
- .setIsBounded(Enum.BOUNDED)
- .build()))
- .commit(Instant.now()),
- bundleFactory
- .<Integer>createBundle(
- PipelineNode.pCollection(
- "unbounded",
- RunnerApi.PCollection.newBuilder()
- .setUniqueName("unbounded")
- .setIsBounded(Enum.UNBOUNDED)
- .build()))
- .commit(Instant.now()));
- CommittedResult<PTransformNode> result =
- CommittedResult.create(
- StepTransformResult.withoutHold(transform).build(),
- Optional.absent(),
- outputs,
- EnumSet.of(OutputType.BUNDLE, OutputType.PCOLLECTION_VIEW));
-
- assertThat(result.getOutputs(), Matchers.containsInAnyOrder(outputs.toArray()));
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternalsTest.java
deleted file mode 100644
index 125860a..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/CopyOnAccessInMemoryStateInternalsTest.java
+++ /dev/null
@@ -1,591 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.nullValue;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-
-import org.apache.beam.runners.core.StateNamespace;
-import org.apache.beam.runners.core.StateNamespaceForTest;
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.sdk.coders.CannotProvideCoderException;
-import org.apache.beam.sdk.coders.CoderRegistry;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.CombiningState;
-import org.apache.beam.sdk.state.GroupingState;
-import org.apache.beam.sdk.state.MapState;
-import org.apache.beam.sdk.state.SetState;
-import org.apache.beam.sdk.state.ValueState;
-import org.apache.beam.sdk.state.WatermarkHoldState;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.sdk.transforms.Sum;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Lists;
-import org.joda.time.Instant;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link CopyOnAccessInMemoryStateInternals}. */
-@RunWith(JUnit4.class)
-public class CopyOnAccessInMemoryStateInternalsTest {
-
- @Rule public final TestPipeline pipeline = TestPipeline.create();
- @Rule public ExpectedException thrown = ExpectedException.none();
- private String key = "foo";
-
- @Test
- public void testGetWithEmpty() {
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = internals.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
- assertThat(stringBag.read(), containsInAnyOrder("baz", "bar"));
-
- BagState<String> reReadStringBag = internals.state(namespace, bagTag);
- assertThat(reReadStringBag.read(), containsInAnyOrder("baz", "bar"));
- }
-
- @Test
- public void testGetWithAbsentInUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = internals.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
- assertThat(stringBag.read(), containsInAnyOrder("baz", "bar"));
-
- BagState<String> reReadVoidBag = internals.state(namespace, bagTag);
- assertThat(reReadVoidBag.read(), containsInAnyOrder("baz", "bar"));
-
- BagState<String> underlyingState = underlying.state(namespace, bagTag);
- assertThat(underlyingState.read(), emptyIterable());
- }
-
- /**
- * Tests that retrieving state with an underlying StateInternals with an existing value returns a
- * value that initially has equal value to the provided state but can be modified without
- * modifying the existing state.
- */
- @Test
- public void testGetWithPresentInUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<ValueState<String>> valueTag = StateTags.value("foo", StringUtf8Coder.of());
- ValueState<String> underlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.read(), nullValue(String.class));
-
- underlyingValue.write("bar");
- assertThat(underlyingValue.read(), equalTo("bar"));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
- ValueState<String> copyOnAccessState = internals.state(namespace, valueTag);
- assertThat(copyOnAccessState.read(), equalTo("bar"));
-
- copyOnAccessState.write("baz");
- assertThat(copyOnAccessState.read(), equalTo("baz"));
- assertThat(underlyingValue.read(), equalTo("bar"));
-
- ValueState<String> reReadUnderlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
- }
-
- @Test
- public void testBagStateWithUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<Integer>> valueTag = StateTags.bag("foo", VarIntCoder.of());
- BagState<Integer> underlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.read(), emptyIterable());
-
- underlyingValue.add(1);
- assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
- BagState<Integer> copyOnAccessState = internals.state(namespace, valueTag);
- assertThat(copyOnAccessState.read(), containsInAnyOrder(1));
-
- copyOnAccessState.add(4);
- assertThat(copyOnAccessState.read(), containsInAnyOrder(4, 1));
- assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
- BagState<Integer> reReadUnderlyingValue = underlying.state(namespace, valueTag);
- assertThat(
- Lists.newArrayList(underlyingValue.read()),
- equalTo(Lists.newArrayList(reReadUnderlyingValue.read())));
- }
-
- @Test
- public void testSetStateWithUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<SetState<Integer>> valueTag = StateTags.set("foo", VarIntCoder.of());
- SetState<Integer> underlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.read(), emptyIterable());
-
- underlyingValue.add(1);
- assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
- SetState<Integer> copyOnAccessState = internals.state(namespace, valueTag);
- assertThat(copyOnAccessState.read(), containsInAnyOrder(1));
-
- copyOnAccessState.add(4);
- assertThat(copyOnAccessState.read(), containsInAnyOrder(4, 1));
- assertThat(underlyingValue.read(), containsInAnyOrder(1));
-
- SetState<Integer> reReadUnderlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
- }
-
- @Test
- public void testMapStateWithUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<MapState<String, Integer>> valueTag =
- StateTags.map("foo", StringUtf8Coder.of(), VarIntCoder.of());
- MapState<String, Integer> underlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.entries().read(), emptyIterable());
-
- underlyingValue.put("hello", 1);
- assertThat(underlyingValue.get("hello").read(), equalTo(1));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
- MapState<String, Integer> copyOnAccessState = internals.state(namespace, valueTag);
- assertThat(copyOnAccessState.get("hello").read(), equalTo(1));
-
- copyOnAccessState.put("world", 4);
- assertThat(copyOnAccessState.get("hello").read(), equalTo(1));
- assertThat(copyOnAccessState.get("world").read(), equalTo(4));
- assertThat(underlyingValue.get("hello").read(), equalTo(1));
- assertNull(underlyingValue.get("world").read());
-
- MapState<String, Integer> reReadUnderlyingValue = underlying.state(namespace, valueTag);
- assertThat(underlyingValue.entries().read(), equalTo(reReadUnderlyingValue.entries().read()));
- }
-
- @Test
- public void testAccumulatorCombiningStateWithUnderlying() throws CannotProvideCoderException {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CombineFn<Long, long[], Long> sumLongFn = Sum.ofLongs();
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- CoderRegistry reg = pipeline.getCoderRegistry();
- StateTag<CombiningState<Long, long[], Long>> stateTag =
- StateTags.combiningValue(
- "summer", sumLongFn.getAccumulatorCoder(reg, reg.getCoder(Long.class)), sumLongFn);
- GroupingState<Long, Long> underlyingValue = underlying.state(namespace, stateTag);
- assertThat(underlyingValue.read(), equalTo(0L));
-
- underlyingValue.add(1L);
- assertThat(underlyingValue.read(), equalTo(1L));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
- GroupingState<Long, Long> copyOnAccessState = internals.state(namespace, stateTag);
- assertThat(copyOnAccessState.read(), equalTo(1L));
-
- copyOnAccessState.add(4L);
- assertThat(copyOnAccessState.read(), equalTo(5L));
- assertThat(underlyingValue.read(), equalTo(1L));
-
- GroupingState<Long, Long> reReadUnderlyingValue = underlying.state(namespace, stateTag);
- assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
- }
-
- @Test
- public void testWatermarkHoldStateWithUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- TimestampCombiner timestampCombiner = TimestampCombiner.EARLIEST;
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<WatermarkHoldState> stateTag =
- StateTags.watermarkStateInternal("wmstate", timestampCombiner);
- WatermarkHoldState underlyingValue = underlying.state(namespace, stateTag);
- assertThat(underlyingValue.read(), nullValue());
-
- underlyingValue.add(new Instant(250L));
- assertThat(underlyingValue.read(), equalTo(new Instant(250L)));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
- WatermarkHoldState copyOnAccessState = internals.state(namespace, stateTag);
- assertThat(copyOnAccessState.read(), equalTo(new Instant(250L)));
-
- copyOnAccessState.add(new Instant(100L));
- assertThat(copyOnAccessState.read(), equalTo(new Instant(100L)));
- assertThat(underlyingValue.read(), equalTo(new Instant(250L)));
-
- copyOnAccessState.add(new Instant(500L));
- assertThat(copyOnAccessState.read(), equalTo(new Instant(100L)));
-
- WatermarkHoldState reReadUnderlyingValue = underlying.state(namespace, stateTag);
- assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read()));
- }
-
- @Test
- public void testCommitWithoutUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = internals.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
- assertThat(stringBag.read(), containsInAnyOrder("baz", "bar"));
-
- internals.commit();
-
- BagState<String> reReadStringBag = internals.state(namespace, bagTag);
- assertThat(reReadStringBag.read(), containsInAnyOrder("baz", "bar"));
- assertThat(internals.isEmpty(), is(false));
- }
-
- @Test
- public void testCommitWithUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = underlying.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
-
- internals.commit();
- BagState<String> reReadStringBag = internals.state(namespace, bagTag);
- assertThat(reReadStringBag.read(), containsInAnyOrder("baz", "bar"));
-
- reReadStringBag.add("spam");
-
- BagState<String> underlyingState = underlying.state(namespace, bagTag);
- assertThat(underlyingState.read(), containsInAnyOrder("spam", "bar", "baz"));
- assertThat(underlyingState, is(theInstance(stringBag)));
- assertThat(internals.isEmpty(), is(false));
- }
-
- @Test
- public void testCommitWithClearedInUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CopyOnAccessInMemoryStateInternals<String> secondUnderlying =
- spy(CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying));
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, secondUnderlying);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = underlying.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
- stringBag.clear();
- // We should not read through the cleared bag
- secondUnderlying.commit();
-
- // Should not be visible
- stringBag.add("foo");
-
- internals.commit();
- BagState<String> internalsStringBag = internals.state(namespace, bagTag);
- assertThat(internalsStringBag.read(), emptyIterable());
- verify(secondUnderlying, never()).state(namespace, bagTag);
- assertThat(internals.isEmpty(), is(false));
- }
-
- @Test
- public void testCommitWithOverwrittenUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = underlying.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
-
- BagState<String> internalsState = internals.state(namespace, bagTag);
- internalsState.add("eggs");
- internalsState.add("ham");
- internalsState.add("0x00ff00");
- internalsState.add("&");
-
- internals.commit();
-
- BagState<String> reReadInternalState = internals.state(namespace, bagTag);
- assertThat(
- reReadInternalState.read(),
- containsInAnyOrder("bar", "baz", "0x00ff00", "eggs", "&", "ham"));
- BagState<String> reReadUnderlyingState = underlying.state(namespace, bagTag);
- assertThat(reReadUnderlyingState.read(), containsInAnyOrder("bar", "baz"));
- }
-
- @Test
- public void testCommitWithAddedUnderlying() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
- internals.commit();
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = underlying.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
-
- BagState<String> internalState = internals.state(namespace, bagTag);
- assertThat(internalState.read(), emptyIterable());
-
- BagState<String> reReadUnderlyingState = underlying.state(namespace, bagTag);
- assertThat(reReadUnderlyingState.read(), containsInAnyOrder("bar", "baz"));
- }
-
- @Test
- public void testCommitWithEmptyTableIsEmpty() {
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- internals.commit();
-
- assertThat(internals.isEmpty(), is(true));
- }
-
- @Test
- public void testCommitWithOnlyClearedValuesIsEmpty() {
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = internals.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("foo");
- stringBag.clear();
-
- internals.commit();
-
- assertThat(internals.isEmpty(), is(true));
- }
-
- @Test
- public void testCommitWithEmptyNewAndFullUnderlyingIsNotEmpty() {
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying);
-
- StateNamespace namespace = new StateNamespaceForTest("foo");
- StateTag<BagState<String>> bagTag = StateTags.bag("foo", StringUtf8Coder.of());
- BagState<String> stringBag = underlying.state(namespace, bagTag);
- assertThat(stringBag.read(), emptyIterable());
-
- stringBag.add("bar");
- stringBag.add("baz");
-
- internals.commit();
- assertThat(internals.isEmpty(), is(false));
- }
-
- @Test
- public void testGetEarliestWatermarkHoldAfterCommit() {
- BoundedWindow first =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(2048L);
- }
- };
- BoundedWindow second =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(689743L);
- }
- };
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying("foo", null);
-
- StateTag<WatermarkHoldState> firstHoldAddress =
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
- WatermarkHoldState firstHold =
- internals.state(StateNamespaces.window(null, first), firstHoldAddress);
- firstHold.add(new Instant(22L));
-
- StateTag<WatermarkHoldState> secondHoldAddress =
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
- WatermarkHoldState secondHold =
- internals.state(StateNamespaces.window(null, second), secondHoldAddress);
- secondHold.add(new Instant(2L));
-
- internals.commit();
- assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(2L)));
- }
-
- @Test
- public void testGetEarliestWatermarkHoldWithEarliestInUnderlyingTable() {
- BoundedWindow first =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(2048L);
- }
- };
- BoundedWindow second =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(689743L);
- }
- };
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying("foo", null);
- StateTag<WatermarkHoldState> firstHoldAddress =
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
- WatermarkHoldState firstHold =
- underlying.state(StateNamespaces.window(null, first), firstHoldAddress);
- firstHold.add(new Instant(22L));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying("foo", underlying.commit());
-
- StateTag<WatermarkHoldState> secondHoldAddress =
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
- WatermarkHoldState secondHold =
- internals.state(StateNamespaces.window(null, second), secondHoldAddress);
- secondHold.add(new Instant(244L));
-
- internals.commit();
- assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(22L)));
- }
-
- @Test
- public void testGetEarliestWatermarkHoldWithEarliestInNewTable() {
- BoundedWindow first =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(2048L);
- }
- };
- BoundedWindow second =
- new BoundedWindow() {
- @Override
- public Instant maxTimestamp() {
- return new Instant(689743L);
- }
- };
- CopyOnAccessInMemoryStateInternals<String> underlying =
- CopyOnAccessInMemoryStateInternals.withUnderlying("foo", null);
- StateTag<WatermarkHoldState> firstHoldAddress =
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
- WatermarkHoldState firstHold =
- underlying.state(StateNamespaces.window(null, first), firstHoldAddress);
- firstHold.add(new Instant(224L));
-
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying("foo", underlying.commit());
-
- StateTag<WatermarkHoldState> secondHoldAddress =
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST);
- WatermarkHoldState secondHold =
- internals.state(StateNamespaces.window(null, second), secondHoldAddress);
- secondHold.add(new Instant(24L));
-
- internals.commit();
- assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(24L)));
- }
-
- @Test
- public void testGetEarliestHoldBeforeCommit() {
- CopyOnAccessInMemoryStateInternals<String> internals =
- CopyOnAccessInMemoryStateInternals.withUnderlying(key, null);
-
- internals
- .state(
- StateNamespaces.global(),
- StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST))
- .add(new Instant(1234L));
-
- thrown.expect(IllegalStateException.class);
- thrown.expectMessage(CopyOnAccessInMemoryStateInternals.class.getSimpleName());
- thrown.expectMessage("Can't get the earliest watermark hold");
- thrown.expectMessage("before it is committed");
-
- internals.getEarliestWatermarkHold();
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectMetricsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectMetricsTest.java
deleted file mode 100644
index 59faca3..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectMetricsTest.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.sdk.metrics.MetricNameFilter.inNamespace;
-import static org.apache.beam.sdk.metrics.MetricResultsMatchers.attemptedMetricsResult;
-import static org.apache.beam.sdk.metrics.MetricResultsMatchers.committedMetricsResult;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.runners.core.metrics.DistributionData;
-import org.apache.beam.runners.core.metrics.GaugeData;
-import org.apache.beam.runners.core.metrics.MetricUpdates;
-import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
-import org.apache.beam.sdk.metrics.DistributionResult;
-import org.apache.beam.sdk.metrics.GaugeResult;
-import org.apache.beam.sdk.metrics.MetricKey;
-import org.apache.beam.sdk.metrics.MetricName;
-import org.apache.beam.sdk.metrics.MetricQueryResults;
-import org.apache.beam.sdk.metrics.MetricsFilter;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DirectMetrics}. */
-@RunWith(JUnit4.class)
-public class DirectMetricsTest {
-
- @Mock private CommittedBundle<Object> bundle1;
- @Mock private CommittedBundle<Object> bundle2;
-
- private static final MetricName NAME1 = MetricName.named("ns1", "name1");
- private static final MetricName NAME2 = MetricName.named("ns1", "name2");
- private static final MetricName NAME3 = MetricName.named("ns2", "name1");
- private static final MetricName NAME4 = MetricName.named("ns2", "name2");
-
- private DirectMetrics metrics = new DirectMetrics();
-
- @Before
- public void setUp() {
- MockitoAnnotations.initMocks(this);
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testApplyCommittedNoFilter() {
- metrics.commitLogical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("step1", NAME1), 5L),
- MetricUpdate.create(MetricKey.create("step1", NAME2), 8L)),
- ImmutableList.of(
- MetricUpdate.create(
- MetricKey.create("step1", NAME1), DistributionData.create(8, 2, 3, 5))),
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("step1", NAME4), GaugeData.create(15L)))));
- metrics.commitLogical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("step2", NAME1), 7L),
- MetricUpdate.create(MetricKey.create("step1", NAME2), 4L)),
- ImmutableList.of(
- MetricUpdate.create(
- MetricKey.create("step1", NAME1), DistributionData.create(4, 1, 4, 4))),
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("step1", NAME4), GaugeData.create(27L)))));
-
- MetricQueryResults results = metrics.allMetrics();
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- attemptedMetricsResult("ns1", "name1", "step1", 0L),
- attemptedMetricsResult("ns1", "name2", "step1", 0L),
- attemptedMetricsResult("ns1", "name1", "step2", 0L)));
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- committedMetricsResult("ns1", "name1", "step1", 5L),
- committedMetricsResult("ns1", "name2", "step1", 12L),
- committedMetricsResult("ns1", "name1", "step2", 7L)));
- assertThat(
- results.getDistributions(),
- contains(
- attemptedMetricsResult("ns1", "name1", "step1", DistributionResult.IDENTITY_ELEMENT)));
- assertThat(
- results.getDistributions(),
- contains(
- committedMetricsResult(
- "ns1", "name1", "step1", DistributionResult.create(12, 3, 3, 5))));
- assertThat(
- results.getGauges(),
- contains(attemptedMetricsResult("ns2", "name2", "step1", GaugeResult.empty())));
- assertThat(
- results.getGauges(),
- contains(
- committedMetricsResult(
- "ns2", "name2", "step1", GaugeResult.create(27L, Instant.now()))));
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testApplyAttemptedCountersQueryOneNamespace() {
- metrics.updatePhysical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("step1", NAME1), 5L),
- MetricUpdate.create(MetricKey.create("step1", NAME3), 8L)),
- ImmutableList.of(),
- ImmutableList.of()));
- metrics.updatePhysical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("step2", NAME1), 7L),
- MetricUpdate.create(MetricKey.create("step1", NAME3), 4L)),
- ImmutableList.of(),
- ImmutableList.of()));
-
- MetricQueryResults results =
- metrics.queryMetrics(MetricsFilter.builder().addNameFilter(inNamespace("ns1")).build());
-
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- attemptedMetricsResult("ns1", "name1", "step1", 5L),
- attemptedMetricsResult("ns1", "name1", "step2", 7L)));
-
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- committedMetricsResult("ns1", "name1", "step1", 0L),
- committedMetricsResult("ns1", "name1", "step2", 0L)));
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testApplyAttemptedQueryCompositeScope() {
- metrics.updatePhysical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("Outer1/Inner1", NAME1), 5L),
- MetricUpdate.create(MetricKey.create("Outer1/Inner2", NAME1), 8L)),
- ImmutableList.of(),
- ImmutableList.of()));
- metrics.updatePhysical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("Outer1/Inner1", NAME1), 12L),
- MetricUpdate.create(MetricKey.create("Outer2/Inner2", NAME1), 18L)),
- ImmutableList.of(),
- ImmutableList.of()));
-
- MetricQueryResults results =
- metrics.queryMetrics(MetricsFilter.builder().addStep("Outer1").build());
-
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- attemptedMetricsResult("ns1", "name1", "Outer1/Inner1", 12L),
- attemptedMetricsResult("ns1", "name1", "Outer1/Inner2", 8L)));
-
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- committedMetricsResult("ns1", "name1", "Outer1/Inner1", 0L),
- committedMetricsResult("ns1", "name1", "Outer1/Inner2", 0L)));
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testPartialScopeMatchingInMetricsQuery() {
- metrics.updatePhysical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner1", NAME1), 5L),
- MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner2", NAME1), 8L)),
- ImmutableList.of(),
- ImmutableList.of()));
- metrics.updatePhysical(
- bundle1,
- MetricUpdates.create(
- ImmutableList.of(
- MetricUpdate.create(MetricKey.create("Top2/Outer1/Inner1", NAME1), 12L),
- MetricUpdate.create(MetricKey.create("Top1/Outer2/Inner2", NAME1), 18L)),
- ImmutableList.of(),
- ImmutableList.of()));
-
- MetricQueryResults results =
- metrics.queryMetrics(MetricsFilter.builder().addStep("Top1/Outer1").build());
-
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner1", 5L),
- attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner2", 8L)));
-
- results = metrics.queryMetrics(MetricsFilter.builder().addStep("Inner2").build());
-
- assertThat(
- results.getCounters(),
- containsInAnyOrder(
- attemptedMetricsResult("ns1", "name1", "Top1/Outer1/Inner2", 8L),
- attemptedMetricsResult("ns1", "name1", "Top1/Outer2/Inner2", 18L)));
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTimerInternalsTest.java
deleted file mode 100644
index f5c73fd..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTimerInternalsTest.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder;
-import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DirectTimerInternals}. */
-@RunWith(JUnit4.class)
-public class DirectTimerInternalsTest {
- private MockClock clock;
- @Mock private TransformWatermarks watermarks;
-
- private TimerUpdateBuilder timerUpdateBuilder;
-
- private DirectTimerInternals internals;
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- clock = MockClock.fromInstant(new Instant(0));
-
- timerUpdateBuilder = TimerUpdate.builder(StructuralKey.of(1234, VarIntCoder.of()));
-
- internals = DirectTimerInternals.create(clock, watermarks, timerUpdateBuilder);
- }
-
- @Test
- public void setTimerAddsToBuilder() {
- TimerData eventTimer =
- TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
- TimerData processingTimer =
- TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
- TimerData synchronizedProcessingTimer =
- TimerData.of(
- StateNamespaces.global(),
- new Instant(98745632189L),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- internals.setTimer(eventTimer);
- internals.setTimer(processingTimer);
- internals.setTimer(synchronizedProcessingTimer);
-
- assertThat(
- internals.getTimerUpdate().getSetTimers(),
- containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
- }
-
- @Test
- public void deleteTimerDeletesOnBuilder() {
- TimerData eventTimer =
- TimerData.of(StateNamespaces.global(), new Instant(20145L), TimeDomain.EVENT_TIME);
- TimerData processingTimer =
- TimerData.of(StateNamespaces.global(), new Instant(125555555L), TimeDomain.PROCESSING_TIME);
- TimerData synchronizedProcessingTimer =
- TimerData.of(
- StateNamespaces.global(),
- new Instant(98745632189L),
- TimeDomain.SYNCHRONIZED_PROCESSING_TIME);
- internals.deleteTimer(eventTimer);
- internals.deleteTimer(processingTimer);
- internals.deleteTimer(synchronizedProcessingTimer);
-
- assertThat(
- internals.getTimerUpdate().getDeletedTimers(),
- containsInAnyOrder(eventTimer, synchronizedProcessingTimer, processingTimer));
- }
-
- @Test
- public void getProcessingTimeIsClockNow() {
- assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
- Instant oldProcessingTime = internals.currentProcessingTime();
-
- clock.advance(Duration.standardHours(12));
-
- assertThat(internals.currentProcessingTime(), equalTo(clock.now()));
- assertThat(
- internals.currentProcessingTime(),
- equalTo(oldProcessingTime.plus(Duration.standardHours(12))));
- }
-
- @Test
- public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() {
- when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L));
- assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L)));
- }
-
- @Test
- public void getInputWatermarkTimeUsesWatermarkTime() {
- when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L));
- assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L)));
- }
-
- @Test
- public void getOutputWatermarkTimeUsesWatermarkTime() {
- when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L));
- assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L)));
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTransformExecutorTest.java
deleted file mode 100644
index 0cae265..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/DirectTransformExecutorTest.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.nullValue;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.when;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.portable.CommittedResult.OutputType;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.MoreExecutors;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-
-/** Tests for {@link DirectTransformExecutor}. */
-@RunWith(JUnit4.class)
-public class DirectTransformExecutorTest {
- @Rule public ExpectedException thrown = ExpectedException.none();
- private final PCollectionNode created =
- PipelineNode.pCollection(
- "created", PCollection.newBuilder().setUniqueName("created").build());
-
- private final PTransformNode createdProducer =
- PipelineNode.pTransform(
- "create",
- PTransform.newBuilder().putOutputs("created", "created").setUniqueName("create").build());
- private final PTransformNode downstreamProducer =
- PipelineNode.pTransform(
- "downstream",
- PTransform.newBuilder().putInputs("input", "created").setUniqueName("create").build());
-
- private CountDownLatch evaluatorCompleted;
-
- private RegisteringCompletionCallback completionCallback;
- private TransformExecutorService transformEvaluationState;
- private BundleFactory bundleFactory;
- @Mock private DirectMetrics metrics;
- @Mock private EvaluationContext evaluationContext;
- @Mock private TransformEvaluatorRegistry registry;
-
- @Rule public TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
-
- bundleFactory = ImmutableListBundleFactory.create();
-
- transformEvaluationState =
- TransformExecutorServices.parallel(MoreExecutors.newDirectExecutorService());
-
- evaluatorCompleted = new CountDownLatch(1);
- completionCallback = new RegisteringCompletionCallback(evaluatorCompleted);
-
- PipelineNode.pCollection(
- "created", RunnerApi.PCollection.newBuilder().setUniqueName("created").build());
-
- when(evaluationContext.getMetrics()).thenReturn(metrics);
- }
-
- @Test
- public void callWithNullInputBundleFinishesBundleAndCompletes() throws Exception {
- final TransformResult<Object> result = StepTransformResult.withoutHold(createdProducer).build();
- final AtomicBoolean finishCalled = new AtomicBoolean(false);
- TransformEvaluator<Object> evaluator =
- new TransformEvaluator<Object>() {
- @Override
- public void processElement(WindowedValue<Object> element) throws Exception {
- throw new IllegalArgumentException("Shouldn't be called");
- }
-
- @Override
- public TransformResult<Object> finishBundle() throws Exception {
- finishCalled.set(true);
- return result;
- }
- };
-
- when(registry.forApplication(createdProducer, null)).thenReturn(evaluator);
-
- DirectTransformExecutor<Object> executor =
- new DirectTransformExecutor<>(
- evaluationContext,
- registry,
- null,
- createdProducer,
- completionCallback,
- transformEvaluationState);
- executor.run();
-
- assertThat(finishCalled.get(), is(true));
- assertThat(completionCallback.handledResult, Matchers.equalTo(result));
- assertThat(completionCallback.handledException, is(nullValue()));
- }
-
- @Test
- public void nullTransformEvaluatorTerminates() throws Exception {
- when(registry.forApplication(createdProducer, null)).thenReturn(null);
-
- DirectTransformExecutor<Object> executor =
- new DirectTransformExecutor<>(
- evaluationContext,
- registry,
- null,
- createdProducer,
- completionCallback,
- transformEvaluationState);
- executor.run();
-
- assertThat(completionCallback.handledResult, is(nullValue()));
- assertThat(completionCallback.handledEmpty, equalTo(true));
- assertThat(completionCallback.handledException, is(nullValue()));
- }
-
- @Test
- public void inputBundleProcessesEachElementFinishesAndCompletes() throws Exception {
- final TransformResult<String> result =
- StepTransformResult.<String>withoutHold(downstreamProducer).build();
- final Collection<WindowedValue<String>> elementsProcessed = new ArrayList<>();
- TransformEvaluator<String> evaluator =
- new TransformEvaluator<String>() {
- @Override
- public void processElement(WindowedValue<String> element) throws Exception {
- elementsProcessed.add(element);
- }
-
- @Override
- public TransformResult<String> finishBundle() throws Exception {
- return result;
- }
- };
-
- WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
- WindowedValue<String> spam = WindowedValue.valueInGlobalWindow("spam");
- WindowedValue<String> third = WindowedValue.valueInGlobalWindow("third");
- CommittedBundle<String> inputBundle =
- bundleFactory
- .<String>createBundle(created)
- .add(foo)
- .add(spam)
- .add(third)
- .commit(Instant.now());
- when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
-
- DirectTransformExecutor<String> executor =
- new DirectTransformExecutor<>(
- evaluationContext,
- registry,
- inputBundle,
- downstreamProducer,
- completionCallback,
- transformEvaluationState);
-
- Future<?> future = Executors.newSingleThreadExecutor().submit(executor);
-
- evaluatorCompleted.await();
- future.get();
-
- assertThat(elementsProcessed, containsInAnyOrder(spam, third, foo));
- assertThat(completionCallback.handledResult, Matchers.equalTo(result));
- assertThat(completionCallback.handledException, is(nullValue()));
- }
-
- @Test
- @SuppressWarnings("FutureReturnValueIgnored") // expected exception checked via completionCallback
- public void processElementThrowsExceptionCallsback() throws Exception {
- final TransformResult<String> result =
- StepTransformResult.<String>withoutHold(downstreamProducer).build();
- final Exception exception = new Exception();
- TransformEvaluator<String> evaluator =
- new TransformEvaluator<String>() {
- @Override
- public void processElement(WindowedValue<String> element) throws Exception {
- throw exception;
- }
-
- @Override
- public TransformResult<String> finishBundle() throws Exception {
- return result;
- }
- };
-
- WindowedValue<String> foo = WindowedValue.valueInGlobalWindow("foo");
- CommittedBundle<String> inputBundle =
- bundleFactory.<String>createBundle(created).add(foo).commit(Instant.now());
- when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
-
- DirectTransformExecutor<String> executor =
- new DirectTransformExecutor<>(
- evaluationContext,
- registry,
- inputBundle,
- downstreamProducer,
- completionCallback,
- transformEvaluationState);
- Executors.newSingleThreadExecutor().submit(executor);
-
- evaluatorCompleted.await();
-
- assertThat(completionCallback.handledResult, is(nullValue()));
- assertThat(completionCallback.handledException, Matchers.<Throwable>equalTo(exception));
- }
-
- @Test
- @SuppressWarnings("FutureReturnValueIgnored") // expected exception checked via completionCallback
- public void finishBundleThrowsExceptionCallsback() throws Exception {
- final Exception exception = new Exception();
- TransformEvaluator<String> evaluator =
- new TransformEvaluator<String>() {
- @Override
- public void processElement(WindowedValue<String> element) throws Exception {}
-
- @Override
- public TransformResult<String> finishBundle() throws Exception {
- throw exception;
- }
- };
-
- CommittedBundle<String> inputBundle =
- bundleFactory.<String>createBundle(created).commit(Instant.now());
- when(registry.<String>forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator);
-
- DirectTransformExecutor<String> executor =
- new DirectTransformExecutor<>(
- evaluationContext,
- registry,
- inputBundle,
- downstreamProducer,
- completionCallback,
- transformEvaluationState);
- Executors.newSingleThreadExecutor().submit(executor);
-
- evaluatorCompleted.await();
-
- assertThat(completionCallback.handledResult, is(nullValue()));
- assertThat(completionCallback.handledException, Matchers.<Throwable>equalTo(exception));
- }
-
- private static class RegisteringCompletionCallback implements CompletionCallback {
- private TransformResult<?> handledResult = null;
- private boolean handledEmpty = false;
- private Exception handledException = null;
- private final CountDownLatch onMethod;
-
- private RegisteringCompletionCallback(CountDownLatch onMethod) {
- this.onMethod = onMethod;
- }
-
- @Override
- public CommittedResult handleResult(CommittedBundle<?> inputBundle, TransformResult<?> result) {
- handledResult = result;
- onMethod.countDown();
- @SuppressWarnings("rawtypes")
- Iterable unprocessedElements =
- result.getUnprocessedElements() == null
- ? Collections.emptyList()
- : result.getUnprocessedElements();
-
- Optional<? extends CommittedBundle<?>> unprocessedBundle;
- if (inputBundle == null || Iterables.isEmpty(unprocessedElements)) {
- unprocessedBundle = Optional.absent();
- } else {
- unprocessedBundle =
- Optional.<CommittedBundle<?>>of(inputBundle.withElements(unprocessedElements));
- }
- return CommittedResult.create(
- result, unprocessedBundle, Collections.emptyList(), EnumSet.noneOf(OutputType.class));
- }
-
- @Override
- public void handleEmpty(PTransformNode transform) {
- handledEmpty = true;
- onMethod.countDown();
- }
-
- @Override
- public void handleException(CommittedBundle<?> inputBundle, Exception e) {
- handledException = e;
- onMethod.countDown();
- }
-
- @Override
- public void handleError(Error err) {
- throw err;
- }
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/EvaluationContextTest.java
deleted file mode 100644
index 46df904..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/EvaluationContextTest.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.hamcrest.Matchers.contains;
-import static org.hamcrest.Matchers.emptyIterable;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.junit.Assert.assertThat;
-
-import java.util.Collection;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.runners.core.StateNamespaces;
-import org.apache.beam.runners.core.StateTag;
-import org.apache.beam.runners.core.StateTags;
-import org.apache.beam.runners.core.TimerInternals.TimerData;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-import org.apache.beam.runners.direct.WatermarkManager.FiredTimers;
-import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate;
-import org.apache.beam.runners.local.StructuralKey;
-import org.apache.beam.sdk.coders.ByteArrayCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
-import org.apache.beam.sdk.state.BagState;
-import org.apache.beam.sdk.state.TimeDomain;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableSet;
-import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables;
-import org.hamcrest.Matchers;
-import org.joda.time.Instant;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link EvaluationContext}. */
-@RunWith(JUnit4.class)
-public class EvaluationContextTest {
- private EvaluationContext context;
-
- private PCollectionNode created;
- private PCollectionNode downstream;
-
- private ExecutableGraph<PTransformNode, PCollectionNode> graph;
-
- private PTransformNode createdProducer;
- private PTransformNode downstreamProducer;
- private PTransformNode unboundedProducer;
-
- @Before
- public void setup() {
- ExecutableGraphBuilder graphBuilder =
- ExecutableGraphBuilder.create()
- .addTransform("create", null, "created")
- .addTransform("downstream", "created", "downstream.out")
- .addTransform("unbounded", null, "unbounded.out");
-
- graph = graphBuilder.toGraph();
- created = graphBuilder.collectionNode("created");
- downstream = graphBuilder.collectionNode("downstream.out");
- createdProducer = graphBuilder.transformNode("create");
- downstreamProducer = graphBuilder.transformNode("downstream");
- unboundedProducer = graphBuilder.transformNode("unbounded");
-
- BundleFactory bundleFactory = ImmutableListBundleFactory.create();
- context = EvaluationContext.create(Instant::new, bundleFactory, graph, ImmutableSet.of());
- }
-
- @Test
- public void getExecutionContextSameStepSameKeyState() {
- StepStateAndTimers<String> fooContext =
- context.getStateAndTimers(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of()));
-
- StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- fooContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
- context.handleResult(
- ImmutableListBundleFactory.create()
- .createKeyedBundle(StructuralKey.of("foo", StringUtf8Coder.of()), created)
- .commit(Instant.now()),
- ImmutableList.of(),
- StepTransformResult.withoutHold(createdProducer)
- .withState(fooContext.stateInternals().commit())
- .build());
-
- StepStateAndTimers secondFooContext =
- context.getStateAndTimers(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of()));
- assertThat(
- secondFooContext.stateInternals().state(StateNamespaces.global(), intBag).read(),
- contains(1));
- }
-
- @Test
- public void getExecutionContextDifferentKeysIndependentState() {
- StepStateAndTimers fooContext =
- context.getStateAndTimers(createdProducer, StructuralKey.of("foo", StringUtf8Coder.of()));
-
- StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- fooContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
- StepStateAndTimers barContext =
- context.getStateAndTimers(createdProducer, StructuralKey.of("bar", StringUtf8Coder.of()));
- assertThat(barContext, not(equalTo(fooContext)));
- assertThat(
- barContext.stateInternals().state(StateNamespaces.global(), intBag).read(),
- emptyIterable());
- }
-
- @Test
- public void getExecutionContextDifferentStepsIndependentState() {
- StructuralKey<?> myKey = StructuralKey.of("foo", StringUtf8Coder.of());
- StepStateAndTimers fooContext = context.getStateAndTimers(createdProducer, myKey);
-
- StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- fooContext.stateInternals().state(StateNamespaces.global(), intBag).add(1);
-
- StepStateAndTimers barContext = context.getStateAndTimers(downstreamProducer, myKey);
- assertThat(
- barContext.stateInternals().state(StateNamespaces.global(), intBag).read(),
- emptyIterable());
- }
-
- @Test
- public void handleResultStoresState() {
- StructuralKey<?> myKey = StructuralKey.of("foo".getBytes(UTF_8), ByteArrayCoder.of());
- StepStateAndTimers fooContext = context.getStateAndTimers(downstreamProducer, myKey);
-
- StateTag<BagState<Integer>> intBag = StateTags.bag("myBag", VarIntCoder.of());
-
- CopyOnAccessInMemoryStateInternals<?> state = fooContext.stateInternals();
- BagState<Integer> bag = state.state(StateNamespaces.global(), intBag);
- bag.add(1);
- bag.add(2);
- bag.add(4);
-
- TransformResult<?> stateResult =
- StepTransformResult.withoutHold(downstreamProducer).withState(state).build();
-
- context.handleResult(
- context.createKeyedBundle(myKey, created).commit(Instant.now()),
- ImmutableList.of(),
- stateResult);
-
- StepStateAndTimers afterResultContext = context.getStateAndTimers(downstreamProducer, myKey);
-
- CopyOnAccessInMemoryStateInternals<?> afterResultState = afterResultContext.stateInternals();
- assertThat(afterResultState.state(StateNamespaces.global(), intBag).read(), contains(1, 2, 4));
- }
-
- @Test
- public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() throws Exception {
- final CountDownLatch callLatch = new CountDownLatch(1);
- Runnable callback = callLatch::countDown;
-
- // Should call back after the end of the global window
- context.scheduleAfterOutputWouldBeProduced(
- downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
-
- TransformResult<?> result =
- StepTransformResult.withHold(createdProducer, new Instant(0)).build();
-
- context.handleResult(null, ImmutableList.of(), result);
- // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit
- // will likely be flaky if this logic is broken
- assertThat(callLatch.await(500L, TimeUnit.MILLISECONDS), is(false));
-
- TransformResult<?> finishedResult = StepTransformResult.withoutHold(createdProducer).build();
- context.handleResult(null, ImmutableList.of(), finishedResult);
- context.forceRefresh();
- // Obtain the value via blocking call
- assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
- }
-
- @Test
- public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() throws Exception {
- TransformResult<?> finishedResult = StepTransformResult.withoutHold(createdProducer).build();
- context.handleResult(null, ImmutableList.of(), finishedResult);
-
- final CountDownLatch callLatch = new CountDownLatch(1);
- context.extractFiredTimers();
- Runnable callback = callLatch::countDown;
- context.scheduleAfterOutputWouldBeProduced(
- downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback);
- assertThat(callLatch.await(1, TimeUnit.SECONDS), is(true));
- }
-
- @Test
- public void extractFiredTimersExtractsTimers() {
- TransformResult<?> holdResult =
- StepTransformResult.withHold(createdProducer, new Instant(0)).build();
- context.handleResult(null, ImmutableList.of(), holdResult);
-
- StructuralKey<?> key = StructuralKey.of("foo".length(), VarIntCoder.of());
- TimerData toFire =
- TimerData.of(StateNamespaces.global(), new Instant(100L), TimeDomain.EVENT_TIME);
- TransformResult<?> timerResult =
- StepTransformResult.withoutHold(downstreamProducer)
- .withState(CopyOnAccessInMemoryStateInternals.withUnderlying(key, null))
- .withTimerUpdate(TimerUpdate.builder(key).setTimer(toFire).build())
- .build();
-
- // haven't added any timers, must be empty
- assertThat(context.extractFiredTimers(), emptyIterable());
- context.handleResult(
- context.createKeyedBundle(key, created).commit(Instant.now()),
- ImmutableList.of(),
- timerResult);
-
- // timer hasn't fired
- assertThat(context.extractFiredTimers(), emptyIterable());
-
- TransformResult<?> advanceResult = StepTransformResult.withoutHold(createdProducer).build();
- // Should cause the downstream timer to fire
- context.handleResult(null, ImmutableList.of(), advanceResult);
-
- Collection<FiredTimers<PTransformNode>> fired = context.extractFiredTimers();
- assertThat(Iterables.getOnlyElement(fired).getKey(), Matchers.equalTo(key));
-
- FiredTimers<PTransformNode> firedForKey = Iterables.getOnlyElement(fired);
- // Contains exclusively the fired timer
- assertThat(firedForKey.getTimers(), contains(toFire));
-
- // Don't reextract timers
- assertThat(context.extractFiredTimers(), emptyIterable());
- }
-
- @Test
- public void createKeyedBundleKeyed() {
- StructuralKey<String> key = StructuralKey.of("foo", StringUtf8Coder.of());
- CommittedBundle<KV<String, Integer>> keyedBundle =
- context
- .<String, KV<String, Integer>>createKeyedBundle(key, downstream)
- .commit(Instant.now());
- assertThat(keyedBundle.getKey(), Matchers.equalTo(key));
- }
-
- @Test
- public void isDoneWithUnboundedPCollection() {
- assertThat(context.isDone(unboundedProducer), is(false));
-
- context.handleResult(
- null, ImmutableList.of(), StepTransformResult.withoutHold(unboundedProducer).build());
- context.extractFiredTimers();
- assertThat(context.isDone(unboundedProducer), is(true));
- }
-
- @Test
- public void isDoneWithPartiallyDone() {
- assertThat(context.isDone(), is(false));
-
- UncommittedBundle<Integer> rootBundle = context.createBundle(created);
- rootBundle.add(WindowedValue.valueInGlobalWindow(1));
- CommittedResult handleResult =
- context.handleResult(
- null,
- ImmutableList.of(),
- StepTransformResult.<Integer>withoutHold(createdProducer)
- .addOutput(rootBundle)
- .build());
- @SuppressWarnings("unchecked")
- CommittedBundle<Integer> committedBundle =
- (CommittedBundle<Integer>) Iterables.getOnlyElement(handleResult.getOutputs());
- context.handleResult(
- null, ImmutableList.of(), StepTransformResult.withoutHold(unboundedProducer).build());
- assertThat(context.isDone(), is(false));
-
- for (PTransformNode consumers : graph.getPerElementConsumers(created)) {
- context.handleResult(
- committedBundle, ImmutableList.of(), StepTransformResult.withoutHold(consumers).build());
- }
- context.extractFiredTimers();
- assertThat(context.isDone(), is(true));
- }
-
- private static class TestBoundedWindow extends BoundedWindow {
- private final Instant ts;
-
- public TestBoundedWindow(Instant ts) {
- this.ts = ts;
- }
-
- @Override
- public Instant maxTimestamp() {
- return ts;
- }
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/ExecutableGraphBuilder.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/ExecutableGraphBuilder.java
deleted file mode 100644
index f0f5b70..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/ExecutableGraphBuilder.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import javax.annotation.Nullable;
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.runners.direct.ExecutableGraph;
-
-/**
- * A builder of simple {@link ExecutableGraph ExecutableGraphs} suitable for use in the portable
- * direct runner, to reduce verbosity of creating a graph with no payloads of any meaning.
- */
-public class ExecutableGraphBuilder {
- private final RunnerApi.Components.Builder components;
-
- private ExecutableGraphBuilder() {
- components = Components.newBuilder();
- }
-
- public static ExecutableGraphBuilder create() {
- return new ExecutableGraphBuilder();
- }
-
- public ExecutableGraphBuilder addTransform(
- String name, @Nullable String input, String... outputs) {
- PTransform.Builder pt = PTransform.newBuilder().setUniqueName(name);
- if (input != null) {
- pt = pt.putInputs("input", input);
- addPCollection(input);
- }
- for (String output : outputs) {
- pt = pt.putOutputs(output, output);
- addPCollection(output);
- }
- components.putTransforms(name, pt.build());
- return this;
- }
-
- private ExecutableGraphBuilder addPCollection(String name) {
- components.putPcollections(name, PCollection.newBuilder().setUniqueName(name).build());
- return this;
- }
-
- public PTransformNode transformNode(String name) {
- return PipelineNode.pTransform(name, components.getTransformsOrThrow(name));
- }
-
- public PCollectionNode collectionNode(String name) {
- return PipelineNode.pCollection(name, components.getPcollectionsOrThrow(name));
- }
-
- public ExecutableGraph<PTransformNode, PCollectionNode> toGraph() {
- return PortableGraph.forPipeline(
- Pipeline.newBuilder()
- .setComponents(components)
- .addAllRootTransformIds(components.getTransformsMap().keySet())
- .build());
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactoryTest.java
deleted file mode 100644
index e183895..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/FlattenEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.direct.portable;
-
-import static org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Iterables.getOnlyElement;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.runners.core.construction.PTransformTranslation;
-import org.apache.beam.runners.core.construction.graph.PipelineNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
-import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.util.WindowedValue;
-import org.joda.time.Instant;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Tests for {@link FlattenEvaluatorFactory}. */
-@RunWith(JUnit4.class)
-public class FlattenEvaluatorFactoryTest {
- private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
-
- @Test
- public void testFlattenInMemoryEvaluator() throws Exception {
- PCollectionNode left =
- PipelineNode.pCollection("left", PCollection.newBuilder().setUniqueName("left").build());
- PCollectionNode right =
- PipelineNode.pCollection("right", PCollection.newBuilder().setUniqueName("right").build());
- // Include a root node for a sane-looking graph
- PTransformNode source =
- PipelineNode.pTransform(
- "source",
- PTransform.newBuilder()
- .putOutputs("left", left.getId())
- .putOutputs("right", right.getId())
- .build());
-
- PCollectionNode flattened =
- PipelineNode.pCollection("flat", PCollection.newBuilder().setUniqueName("flat").build());
- PTransformNode flatten =
- PipelineNode.pTransform(
- "flatten",
- PTransform.newBuilder()
- .setUniqueName("flatten")
- .putInputs("left", left.getId())
- .putInputs("right", right.getId())
- .putOutputs("out", flattened.getId())
- .setSpec(
- FunctionSpec.newBuilder().setUrn(PTransformTranslation.FLATTEN_TRANSFORM_URN))
- .build());
-
- PortableGraph graph =
- PortableGraph.forPipeline(
- RunnerApi.Pipeline.newBuilder()
- .addRootTransformIds(source.getId())
- .addRootTransformIds(flatten.getId())
- .setComponents(
- RunnerApi.Components.newBuilder()
- .putTransforms(source.getId(), source.getTransform())
- .putPcollections(left.getId(), left.getPCollection())
- .putPcollections(right.getId(), right.getPCollection())
- .putTransforms(flatten.getId(), flatten.getTransform())
- .putPcollections(flattened.getId(), flattened.getPCollection()))
- .build());
-
- CommittedBundle<Integer> leftBundle =
- bundleFactory.<Integer>createBundle(left).commit(Instant.now());
- CommittedBundle<Integer> rightBundle =
- bundleFactory.<Integer>createBundle(right).commit(Instant.now());
-
- FlattenEvaluatorFactory factory = new FlattenEvaluatorFactory(graph, bundleFactory);
- TransformEvaluator<Integer> leftSideEvaluator = factory.forApplication(flatten, leftBundle);
- TransformEvaluator<Integer> rightSideEvaluator = factory.forApplication(flatten, rightBundle);
-
- leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1));
- rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1));
- leftSideEvaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)));
- leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING));
- rightSideEvaluator.processElement(
- WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING));
- rightSideEvaluator.processElement(
- WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)));
-
- TransformResult<Integer> rightSideResult = rightSideEvaluator.finishBundle();
- TransformResult<Integer> leftSideResult = leftSideEvaluator.finishBundle();
-
- assertThat(
- getOnlyElement(leftSideResult.getOutputBundles()).commit(Instant.now()),
- containsInAnyOrder(
- WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)),
- WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING),
- WindowedValue.valueInGlobalWindow(1)));
- assertThat(
- getOnlyElement(rightSideResult.getOutputBundles()).commit(Instant.now()),
- containsInAnyOrder(
- WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING),
- WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)),
- WindowedValue.valueInGlobalWindow(-1)));
- }
-}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
deleted file mode 100644
index 10db613..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
+++ /dev/null
@@ -1,247 +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
... 3153 lines suppressed ...