[
https://issues.apache.org/jira/browse/BEAM-6077?focusedWorklogId=171953&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-171953
]
ASF GitHub Bot logged work on BEAM-6077:
----------------------------------------
Author: ASF GitHub Bot
Created on: 04/Dec/18 17:02
Start Date: 04/Dec/18 17:02
Worklog Time Spent: 10m
Work Description: mxm closed pull request #7128: [BEAM-6077] If
available, use max_parallelism for splitting unbounded source
URL: https://github.com/apache/beam/pull/7128
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/FlinkExecutionEnvironments.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
index ddacf3f1ed13..55c386584957 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
@@ -152,6 +152,9 @@ static StreamExecutionEnvironment
createStreamExecutionEnvironment(
determineParallelism(
options.getParallelism(), flinkStreamEnv.getParallelism(),
flinkConfigDir);
flinkStreamEnv.setParallelism(parallelism);
+ if (options.getMaxParallelism() > 0) {
+ flinkStreamEnv.setMaxParallelism(options.getMaxParallelism());
+ }
// set parallelism in the options (required by some execution code)
options.setParallelism(parallelism);
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index 234b457530a2..d8fa063527f3 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -73,6 +73,14 @@
void setParallelism(Integer value);
+ @Description(
+ "The pipeline wide maximum degree of parallelism to be used. The maximum
parallelism specifies the upper limit "
+ + "for dynamic scaling and the number of key groups used for
partitioned state.")
+ @Default.Integer(-1)
+ Integer getMaxParallelism();
+
+ void setMaxParallelism(Integer value);
+
@Description(
"The interval in milliseconds at which to trigger checkpoints of the
running pipeline. "
+ "Default: No checkpointing.")
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
index b2ecd4cdf4dd..a9860a346711 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
@@ -21,6 +21,7 @@
import static
org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN;
import com.google.auto.service.AutoService;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import java.io.IOException;
@@ -197,12 +198,13 @@ public void translateNode(
String fullName = getCurrentTransformName(context);
try {
+ int parallelism =
+ context.getExecutionEnvironment().getMaxParallelism() > 0
+ ? context.getExecutionEnvironment().getMaxParallelism()
+ : context.getExecutionEnvironment().getParallelism();
UnboundedSourceWrapper<T, ?> sourceWrapper =
new UnboundedSourceWrapper<>(
- fullName,
- context.getPipelineOptions(),
- rawSource,
- context.getExecutionEnvironment().getParallelism());
+ fullName, context.getPipelineOptions(), rawSource,
parallelism);
nonDedupSource =
context
.getExecutionEnvironment()
@@ -300,13 +302,14 @@ public void translateNode(
UnboundedSource<T, ?> adaptedRawSource = new
BoundedToUnboundedSourceAdapter<>(rawSource);
DataStream<WindowedValue<T>> source;
try {
+ int parallelism =
+ context.getExecutionEnvironment().getMaxParallelism() > 0
+ ? context.getExecutionEnvironment().getMaxParallelism()
+ : context.getExecutionEnvironment().getParallelism();
UnboundedSourceWrapperNoValueWithRecordId<T, ?> sourceWrapper =
new UnboundedSourceWrapperNoValueWithRecordId<>(
new UnboundedSourceWrapper<>(
- fullName,
- context.getPipelineOptions(),
- adaptedRawSource,
- context.getExecutionEnvironment().getParallelism()));
+ fullName, context.getPipelineOptions(), adaptedRawSource,
parallelism));
source =
context
.getExecutionEnvironment()
@@ -1272,7 +1275,7 @@ public String
getUrn(CreateStreamingFlinkView.CreateFlinkPCollectionView<?, ?> t
* Wrapper for {@link UnboundedSourceWrapper}, which simplifies output type,
namely, removes
* {@link ValueWithRecordId}.
*/
- private static class UnboundedSourceWrapperNoValueWithRecordId<
+ static class UnboundedSourceWrapperNoValueWithRecordId<
OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark>
extends RichParallelSourceFunction<WindowedValue<OutputT>>
implements ProcessingTimeCallback,
@@ -1282,6 +1285,11 @@ public String
getUrn(CreateStreamingFlinkView.CreateFlinkPCollectionView<?, ?> t
private final UnboundedSourceWrapper<OutputT, CheckpointMarkT>
unboundedSourceWrapper;
+ @VisibleForTesting
+ UnboundedSourceWrapper<OutputT, CheckpointMarkT> getUnderlyingSource() {
+ return unboundedSourceWrapper;
+ }
+
private UnboundedSourceWrapperNoValueWithRecordId(
UnboundedSourceWrapper<OutputT, CheckpointMarkT>
unboundedSourceWrapper) {
this.unboundedSourceWrapper = unboundedSourceWrapper;
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
index 74c595a9465d..b6095cbc23e0 100644
---
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
@@ -66,6 +66,20 @@ public void shouldSetParallelismStreaming() {
assertThat(sev.getParallelism(), is(42));
}
+ @Test
+ public void shouldSetMaxParallelismStreaming() {
+ FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ options.setRunner(TestFlinkRunner.class);
+ options.setMaxParallelism(42);
+
+ StreamExecutionEnvironment sev =
+ FlinkExecutionEnvironments.createStreamExecutionEnvironment(
+ options, Collections.emptyList());
+
+ assertThat(options.getMaxParallelism(), is(42));
+ assertThat(sev.getMaxParallelism(), is(42));
+ }
+
@Test
public void shouldInferParallelismFromEnvironmentBatch() throws IOException {
String flinkConfDir = extractFlinkConfig();
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java
new file mode 100644
index 000000000000..8c9eb111c891
--- /dev/null
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.core.construction.PTransformTranslation;
+import
org.apache.beam.runners.flink.FlinkStreamingTransformTranslators.UnboundedSourceWrapperNoValueWithRecordId;
+import
org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.SourceTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.junit.Test;
+
+/** Tests for Flink streaming transform translators. */
+public class FlinkStreamingTransformTranslatorsTest {
+
+ @Test
+ public void readSourceTranslatorBoundedWithMaxParallelism() {
+
+ final int maxParallelism = 6;
+ final int parallelism = 2;
+
+ Read.Bounded transform = Read.from(new TestBoundedSource(maxParallelism));
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(parallelism);
+ env.setMaxParallelism(maxParallelism);
+
+ StreamTransformation<?> sourceTransform =
+ applyReadSourceTransform(transform, PCollection.IsBounded.BOUNDED,
env);
+
+ UnboundedSourceWrapperNoValueWithRecordId source =
+ (UnboundedSourceWrapperNoValueWithRecordId)
+ ((SourceTransformation<?>)
sourceTransform).getOperator().getUserFunction();
+
+ assertEquals(maxParallelism,
source.getUnderlyingSource().getSplitSources().size());
+ }
+
+ @Test
+ public void readSourceTranslatorBoundedWithoutMaxParallelism() {
+
+ final int parallelism = 2;
+
+ Read.Bounded transform = Read.from(new TestBoundedSource(parallelism));
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(parallelism);
+
+ StreamTransformation<?> sourceTransform =
+ applyReadSourceTransform(transform, PCollection.IsBounded.BOUNDED,
env);
+
+ UnboundedSourceWrapperNoValueWithRecordId source =
+ (UnboundedSourceWrapperNoValueWithRecordId)
+ ((SourceTransformation<?>)
sourceTransform).getOperator().getUserFunction();
+
+ assertEquals(parallelism,
source.getUnderlyingSource().getSplitSources().size());
+ }
+
+ @Test
+ public void readSourceTranslatorUnboundedWithMaxParallelism() {
+
+ final int maxParallelism = 6;
+ final int parallelism = 2;
+
+ Read.Unbounded transform = Read.from(new TestUnboundedSource());
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(parallelism);
+ env.setMaxParallelism(maxParallelism);
+
+ StreamTransformation<?> sourceTransform =
+ applyReadSourceTransform(transform, PCollection.IsBounded.UNBOUNDED,
env);
+
+ UnboundedSourceWrapper source =
+ (UnboundedSourceWrapper)
+ ((SourceTransformation) ((OneInputTransformation)
sourceTransform).getInput())
+ .getOperator()
+ .getUserFunction();
+
+ assertEquals(maxParallelism, source.getSplitSources().size());
+ }
+
+ @Test
+ public void readSourceTranslatorUnboundedWithoutMaxParallelism() {
+
+ final int parallelism = 2;
+
+ Read.Unbounded transform = Read.from(new TestUnboundedSource());
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(parallelism);
+
+ StreamTransformation<?> sourceTransform =
+ applyReadSourceTransform(transform, PCollection.IsBounded.UNBOUNDED,
env);
+
+ UnboundedSourceWrapper source =
+ (UnboundedSourceWrapper)
+ ((SourceTransformation) ((OneInputTransformation)
sourceTransform).getInput())
+ .getOperator()
+ .getUserFunction();
+
+ assertEquals(parallelism, source.getSplitSources().size());
+ }
+
+ private StreamTransformation<?> applyReadSourceTransform(
+ PTransform<?, ?> transform, PCollection.IsBounded isBounded,
StreamExecutionEnvironment env) {
+
+ FlinkStreamingPipelineTranslator.StreamTransformTranslator<PTransform<?,
?>> translator =
+ getReadSourceTranslator();
+ FlinkStreamingTranslationContext ctx =
+ new FlinkStreamingTranslationContext(env,
PipelineOptionsFactory.create());
+
+ Pipeline pipeline = Pipeline.create();
+ PCollection<String> pc =
+ PCollection.createPrimitiveOutputInternal(
+ pipeline, WindowingStrategy.globalDefault(), isBounded,
StringUtf8Coder.of());
+ pc.setName("output");
+
+ Map<TupleTag<?>, PValue> outputs = new HashMap<>();
+ outputs.put(new TupleTag<>(), pc);
+ AppliedPTransform<?, ?, ?> appliedTransform =
+ AppliedPTransform.of(
+ "test-transform", Collections.emptyMap(), outputs, transform,
Pipeline.create());
+
+ ctx.setCurrentTransform(appliedTransform);
+ translator.translateNode(transform, ctx);
+
+ return ctx.getInputDataStream(pc).getTransformation();
+ }
+
+ @SuppressWarnings("unchecked")
+ private
FlinkStreamingPipelineTranslator.StreamTransformTranslator<PTransform<?, ?>>
+ getReadSourceTranslator() {
+ PTransformTranslation.RawPTransform<?, ?> t =
mock(PTransformTranslation.RawPTransform.class);
+ when(t.getUrn()).thenReturn(PTransformTranslation.READ_TRANSFORM_URN);
+ return
(FlinkStreamingPipelineTranslator.StreamTransformTranslator<PTransform<?, ?>>)
+ FlinkStreamingTransformTranslators.getTranslator(t);
+ }
+
+ /** {@link BoundedSource} for testing purposes of read transform
translators. */
+ private static class TestBoundedSource extends BoundedSource<String> {
+
+ private final int bytes;
+
+ private TestBoundedSource(int bytes) {
+ this.bytes = bytes;
+ }
+
+ @Override
+ public List<? extends BoundedSource<String>> split(
+ long desiredBundleSizeBytes, PipelineOptions options) throws Exception
{
+ List<BoundedSource<String>> splits = new ArrayList<>();
+ long remaining = bytes;
+ while (remaining > 0) {
+ remaining -= desiredBundleSizeBytes;
+ splits.add(this);
+ }
+ return splits;
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws
Exception {
+ return bytes;
+ }
+
+ @Override
+ public BoundedReader<String> createReader(PipelineOptions options) throws
IOException {
+ return null;
+ }
+
+ @Override
+ public Coder<String> getOutputCoder() {
+ return StringUtf8Coder.of();
+ }
+ }
+
+ /** {@link UnboundedSource} for testing purposes of read transform
translators. */
+ private static class TestUnboundedSource
+ extends UnboundedSource<String, UnboundedSource.CheckpointMark> {
+
+ @Override
+ public List<? extends UnboundedSource<String, CheckpointMark>> split(
+ int desiredNumSplits, PipelineOptions options) throws Exception {
+ List<UnboundedSource<String, CheckpointMark>> splits = new ArrayList<>();
+ for (int i = 0; i < desiredNumSplits; i++) {
+ splits.add(this);
+ }
+ return splits;
+ }
+
+ @Override
+ public UnboundedReader<String> createReader(
+ PipelineOptions options, @Nullable CheckpointMark checkpointMark)
throws IOException {
+ return null;
+ }
+
+ @Override
+ public Coder<CheckpointMark> getCheckpointMarkCoder() {
+ return null;
+ }
+ }
+}
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
index 257501fd6586..095342fb9854 100644
---
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java
@@ -70,6 +70,7 @@
public void testDefaults() {
FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
assertThat(options.getParallelism(), is(-1));
+ assertThat(options.getMaxParallelism(), is(-1));
assertThat(options.getFlinkMaster(), is("[auto]"));
assertThat(options.getFilesToStage(), is(nullValue()));
assertThat(options.getLatencyTrackingInterval(), is(0L));
----------------------------------------------------------------
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: 171953)
Time Spent: 1h 40m (was: 1.5h)
> Make UnboundedSource state rescale friendly
> -------------------------------------------
>
> Key: BEAM-6077
> URL: https://issues.apache.org/jira/browse/BEAM-6077
> Project: Beam
> Issue Type: Improvement
> Components: runner-flink
> Affects Versions: 2.8.0
> Reporter: Jozef Vilcek
> Assignee: Jozef Vilcek
> Priority: Major
> Fix For: 2.10.0
>
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
> FlinkRunner's UnboundedSourceWrapper currently does not rescale well it's
> state when job changes parallelism.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)