[
https://issues.apache.org/jira/browse/BEAM-6245?focusedWorklogId=177037&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-177037
]
ASF GitHub Bot logged work on BEAM-6245:
----------------------------------------
Author: ASF GitHub Bot
Created on: 19/Dec/18 15:22
Start Date: 19/Dec/18 15:22
Worklog Time Spent: 10m
Work Description: mxm closed pull request #7296: [BEAM-6245] Add
integration test for FlinkTransformOverrides
URL: https://github.com/apache/beam/pull/7296
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
index 2a7ead19dab0..1a8ebc95816a 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -74,14 +74,11 @@ public void translate(Pipeline pipeline) {
this.flinkBatchEnv = null;
this.flinkStreamEnv = null;
- PipelineTranslationOptimizer optimizer =
- new PipelineTranslationOptimizer(TranslationMode.BATCH, options);
-
+ PipelineTranslationModeOptimizer optimizer = new
PipelineTranslationModeOptimizer(options);
optimizer.translate(pipeline);
- TranslationMode translationMode = optimizer.getTranslationMode();
FlinkPipelineTranslator translator;
- if (translationMode == TranslationMode.STREAMING) {
+ if (options.isStreaming()) {
this.flinkStreamEnv =
FlinkExecutionEnvironments.createStreamExecutionEnvironment(
options, options.getFilesToStage());
@@ -93,9 +90,7 @@ public void translate(Pipeline pipeline) {
translator = new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
}
- pipeline.replaceAll(
- FlinkTransformOverrides.getDefaultOverrides(
- translationMode == TranslationMode.STREAMING, options));
+ pipeline.replaceAll(FlinkTransformOverrides.getDefaultOverrides(options));
prepareFilesToStageForRemoteClusterExecution(options);
translator.translate(pipeline);
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java
index 890548993839..e7e1d51f1153 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkTransformOverrides.java
@@ -31,9 +31,8 @@
import org.apache.beam.sdk.transforms.PTransform;
/** {@link PTransform} overrides for Flink runner. */
-public class FlinkTransformOverrides {
- public static List<PTransformOverride> getDefaultOverrides(
- boolean streaming, FlinkPipelineOptions options) {
+class FlinkTransformOverrides {
+ static List<PTransformOverride> getDefaultOverrides(FlinkPipelineOptions
options) {
ImmutableList.Builder<PTransformOverride> builder =
ImmutableList.builder();
builder
// TODO: [BEAM-5359] Support @RequiresStableInput on Flink runner
@@ -48,10 +47,10 @@
.add(
PTransformOverride.of(
PTransformMatchers.urnEqualTo(PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN),
- streaming
+ options.isStreaming()
? new SplittableParDoViaKeyedWorkItems.OverrideFactory()
: new SplittableParDoNaiveBounded.OverrideFactory()));
- if (streaming) {
+ if (options.isStreaming()) {
builder
.add(
PTransformOverride.of(
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java
similarity index 79%
rename from
runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
rename to
runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java
index a832569bad2b..32ec4ca6229b 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationOptimizer.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java
@@ -26,30 +26,20 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-/** Traverses the Pipeline to determine the {@link TranslationMode} for this
pipeline. */
-class PipelineTranslationOptimizer extends FlinkPipelineTranslator {
-
- private static final Logger LOG =
LoggerFactory.getLogger(PipelineTranslationOptimizer.class);
+/**
+ * Traverses the Pipeline to determine the translation mode (i.e. streaming or
batch) for this
+ * pipeline.
+ */
+class PipelineTranslationModeOptimizer extends FlinkPipelineTranslator {
- private TranslationMode translationMode;
+ private static final Logger LOG =
LoggerFactory.getLogger(PipelineTranslationModeOptimizer.class);
private final FlinkPipelineOptions options;
- public PipelineTranslationOptimizer(TranslationMode defaultMode,
FlinkPipelineOptions options) {
- this.translationMode = defaultMode;
+ public PipelineTranslationModeOptimizer(FlinkPipelineOptions options) {
this.options = options;
}
- public TranslationMode getTranslationMode() {
-
- // override user-specified translation mode
- if (options.isStreaming()) {
- return TranslationMode.STREAMING;
- }
-
- return translationMode;
- }
-
@Override
public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node
node) {
return CompositeBehavior.ENTER_TRANSFORM;
@@ -64,7 +54,7 @@ public void visitPrimitiveTransform(TransformHierarchy.Node
node) {
if (hasUnboundedOutput(appliedPTransform)) {
Class<? extends PTransform> transformClass =
node.getTransform().getClass();
LOG.info("Found {}. Switching to streaming execution.", transformClass);
- translationMode = TranslationMode.STREAMING;
+ options.setStreaming(true);
}
}
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
deleted file mode 100644
index d01c27bf84da..000000000000
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/TranslationMode.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.beam.runners.flink;
-
-/** The translation mode of the Beam Pipeline. */
-enum TranslationMode {
-
- /** Uses the batch mode of Flink. */
- BATCH,
-
- /** Uses the streaming mode of Flink. */
- STREAMING
-}
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
index f7ea59c2ef68..4d7748463245 100644
---
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
@@ -24,6 +24,7 @@
import static org.hamcrest.core.Every.everyItem;
import static org.junit.Assert.assertThat;
+import com.google.common.collect.ImmutableList;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
@@ -32,6 +33,7 @@
import org.apache.beam.sdk.io.GenerateSequence;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.PTransformOverride;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.windowing.FixedWindows;
@@ -42,6 +44,8 @@
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
/** Tests for {@link FlinkPipelineExecutionEnvironment}. */
@RunWith(JUnit4.class)
@@ -109,6 +113,28 @@ public void
shouldNotPrepareFilesToStageWhenFlinkMasterIsSetToLocal() throws IOE
assertThat(options.getFilesToStage(), everyItem(not(matches(".*\\.jar"))));
}
+ @Test
+ public void shouldUseTransformOverrides() {
+ boolean[] testParameters = {true, false};
+ for (boolean streaming : testParameters) {
+ FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ options.setStreaming(streaming);
+ options.setRunner(FlinkRunner.class);
+ FlinkPipelineExecutionEnvironment flinkEnv = new
FlinkPipelineExecutionEnvironment(options);
+ Pipeline p = Mockito.spy(Pipeline.create(options));
+
+ flinkEnv.translate(p);
+
+ ArgumentCaptor<ImmutableList> captor =
ArgumentCaptor.forClass(ImmutableList.class);
+ Mockito.verify(p).replaceAll(captor.capture());
+ ImmutableList<PTransformOverride> overridesList = captor.getValue();
+
+ assertThat(overridesList.isEmpty(), is(false));
+ assertThat(
+ overridesList.size(),
is(FlinkTransformOverrides.getDefaultOverrides(options).size()));
+ }
+ }
+
private FlinkPipelineOptions testPreparingResourcesToStage(String
flinkMaster)
throws IOException {
Pipeline pipeline = Pipeline.create();
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizerTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizerTest.java
new file mode 100644
index 000000000000..2395c0dd4c37
--- /dev/null
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizerTest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.junit.Test;
+
+/**
+ * Traverses the Pipeline to determine the translation mode (i.e. streaming or
batch) for this
+ * pipeline.
+ */
+public class PipelineTranslationModeOptimizerTest {
+
+ @Test
+ public void testTranslationModeOverrideWithUnboundedSources() {
+ FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ options.setRunner(FlinkRunner.class);
+ options.setStreaming(false);
+
+ FlinkPipelineExecutionEnvironment flinkEnv = new
FlinkPipelineExecutionEnvironment(options);
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.apply(GenerateSequence.from(0));
+ flinkEnv.translate(pipeline);
+
+ assertThat(options.isStreaming(), is(true));
+ }
+
+ @Test
+ public void testTranslationModeNoOverrideWithoutUnboundedSources() {
+ boolean[] testArgs = new boolean[] {true, false};
+ for (boolean streaming : testArgs) {
+ FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ options.setRunner(FlinkRunner.class);
+ options.setStreaming(streaming);
+
+ FlinkPipelineExecutionEnvironment flinkEnv = new
FlinkPipelineExecutionEnvironment(options);
+ Pipeline pipeline = Pipeline.create(options);
+ flinkEnv.translate(pipeline);
+
+ assertThat(options.isStreaming(), is(streaming));
+ }
+ }
+}
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 177037)
Time Spent: 1.5h (was: 1h 20m)
> Add test for FlinkTransformOverrides
> ------------------------------------
>
> Key: BEAM-6245
> URL: https://issues.apache.org/jira/browse/BEAM-6245
> Project: Beam
> Issue Type: Test
> Components: runner-flink
> Reporter: Maximilian Michels
> Assignee: Maximilian Michels
> Priority: Minor
> Fix For: 2.10.0
>
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> We don't test that FlinkTransformOverrides are applied during pipeline
> translation.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)