[
https://issues.apache.org/jira/browse/BEAM-2873?focusedWorklogId=176069&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-176069
]
ASF GitHub Bot logged work on BEAM-2873:
----------------------------------------
Author: ASF GitHub Bot
Created on: 17/Dec/18 15:19
Start Date: 17/Dec/18 15:19
Worklog Time Spent: 10m
Work Description: mxm closed pull request #4760: [BEAM-2873] Setting
number of shards for writes with runner determined sharding
URL: https://github.com/apache/beam/pull/4760
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 a5bfe70363e3..2a7ead19dab0 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
@@ -80,11 +80,6 @@ public void translate(Pipeline pipeline) {
optimizer.translate(pipeline);
TranslationMode translationMode = optimizer.getTranslationMode();
- pipeline.replaceAll(
- FlinkTransformOverrides.getDefaultOverrides(translationMode ==
TranslationMode.STREAMING));
-
- prepareFilesToStageForRemoteClusterExecution(options);
-
FlinkPipelineTranslator translator;
if (translationMode == TranslationMode.STREAMING) {
this.flinkStreamEnv =
@@ -98,6 +93,11 @@ public void translate(Pipeline pipeline) {
translator = new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
}
+ pipeline.replaceAll(
+ FlinkTransformOverrides.getDefaultOverrides(
+ translationMode == TranslationMode.STREAMING, options));
+ prepareFilesToStageForRemoteClusterExecution(options);
+
translator.translate(pipeline);
}
diff --git
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
index f800913ae6a3..823665ec5a31 100644
---
a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
+++
b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
@@ -17,14 +17,29 @@
*/
package org.apache.beam.runners.flink;
+import com.google.common.annotations.VisibleForTesting;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.runners.core.construction.PTransformReplacements;
import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.UnconsumedReads;
+import org.apache.beam.runners.core.construction.WriteFilesTranslation;
import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.FileBasedSink;
+import org.apache.beam.sdk.io.WriteFiles;
+import org.apache.beam.sdk.io.WriteFilesResult;
import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.runners.PTransformOverrideFactory;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TupleTag;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -153,4 +168,59 @@ boolean canTranslate(T transform,
FlinkStreamingTranslationContext context) {
return true;
}
}
+
+ @VisibleForTesting
+ static class StreamingShardedWriteFactory<UserT, DestinationT, OutputT>
+ implements PTransformOverrideFactory<
+ PCollection<UserT>, WriteFilesResult<DestinationT>,
+ WriteFiles<UserT, DestinationT, OutputT>> {
+ FlinkPipelineOptions options;
+
+ StreamingShardedWriteFactory(PipelineOptions options) {
+ this.options = options.as(FlinkPipelineOptions.class);
+ }
+
+ @Override
+ public PTransformReplacement<PCollection<UserT>,
WriteFilesResult<DestinationT>>
+ getReplacementTransform(
+ AppliedPTransform<
+ PCollection<UserT>, WriteFilesResult<DestinationT>,
+ WriteFiles<UserT, DestinationT, OutputT>>
+ transform) {
+ // By default, if numShards is not set WriteFiles will produce one file
per bundle. In
+ // streaming, there are large numbers of small bundles, resulting in
many tiny files.
+ // Instead we pick parallelism * 2 to ensure full parallelism, but
prevent too-many files.
+ Integer jobParallelism = options.getParallelism();
+
+ Preconditions.checkArgument(
+ jobParallelism > 0,
+ "Parallelism of a job should be greater than 0. Currently set: {}",
+ jobParallelism);
+ int numShards = jobParallelism * 2;
+
+ try {
+ List<PCollectionView<?>> sideInputs =
+ WriteFilesTranslation.getDynamicDestinationSideInputs(transform);
+ FileBasedSink sink = WriteFilesTranslation.getSink(transform);
+
+ @SuppressWarnings("unchecked")
+ WriteFiles<UserT, DestinationT, OutputT> replacement =
+ WriteFiles.to(sink).withSideInputs(sideInputs);
+ if (WriteFilesTranslation.isWindowedWrites(transform)) {
+ replacement = replacement.withWindowedWrites();
+ }
+ return PTransformReplacement.of(
+ PTransformReplacements.getSingletonMainInput(transform),
+ replacement.withNumShards(numShards));
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public Map<PValue, ReplacementOutput> mapOutputs(
+ Map<TupleTag<?>, PValue> outputs, WriteFilesResult<DestinationT>
newOutput) {
+ return Collections.emptyMap();
+ }
+ }
}
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 f0fd698637c0..890548993839 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
@@ -17,6 +17,8 @@
*/
package org.apache.beam.runners.flink;
+import static com.google.common.base.Preconditions.checkNotNull;
+
import com.google.common.collect.ImmutableList;
import java.util.List;
import org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems;
@@ -30,7 +32,8 @@
/** {@link PTransform} overrides for Flink runner. */
public class FlinkTransformOverrides {
- public static List<PTransformOverride> getDefaultOverrides(boolean
streaming) {
+ public static List<PTransformOverride> getDefaultOverrides(
+ boolean streaming, FlinkPipelineOptions options) {
ImmutableList.Builder<PTransformOverride> builder =
ImmutableList.builder();
builder
// TODO: [BEAM-5359] Support @RequiresStableInput on Flink runner
@@ -49,10 +52,16 @@
? new SplittableParDoViaKeyedWorkItems.OverrideFactory()
: new SplittableParDoNaiveBounded.OverrideFactory()));
if (streaming) {
- builder.add(
- PTransformOverride.of(
-
PTransformMatchers.urnEqualTo(PTransformTranslation.CREATE_VIEW_TRANSFORM_URN),
- new CreateStreamingFlinkView.Factory()));
+ builder
+ .add(
+ PTransformOverride.of(
+ PTransformMatchers.writeWithRunnerDeterminedSharding(),
+ new
FlinkStreamingPipelineTranslator.StreamingShardedWriteFactory(
+ checkNotNull(options))))
+ .add(
+ PTransformOverride.of(
+
PTransformMatchers.urnEqualTo(PTransformTranslation.CREATE_VIEW_TRANSFORM_URN),
+ new CreateStreamingFlinkView.Factory()));
}
return builder.build();
}
diff --git
a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
new file mode 100644
index 000000000000..eae5855d12ac
--- /dev/null
+++
b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+import java.util.Collections;
+import javax.annotation.Nullable;
+import
org.apache.beam.runners.flink.FlinkStreamingPipelineTranslator.StreamingShardedWriteFactory;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.io.DynamicFileDestinations;
+import org.apache.beam.sdk.io.FileBasedSink;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.WriteFiles;
+import org.apache.beam.sdk.io.WriteFilesResult;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.SerializableFunctions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/** Tests if overrides are properly applied. */
+public class FlinkTransformOverridesTest {
+
+ @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
+
+ @Test
+ public void testRunnerDeterminedSharding() {
+ FlinkPipelineOptions options =
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ options.setRunner(TestFlinkRunner.class);
+ options.setFlinkMaster("[auto]");
+ options.setParallelism(5);
+
+ TestPipeline p = TestPipeline.fromOptions(options);
+
+ StreamingShardedWriteFactory<Object, Void, Object> factory =
+ new StreamingShardedWriteFactory<>(p.getOptions());
+
+ WriteFiles<Object, Void, Object> original = WriteFiles.to(new
TestSink(tmpFolder.toString()));
+ @SuppressWarnings("unchecked")
+ PCollection<Object> objs = (PCollection)
p.apply(Create.empty(VoidCoder.of()));
+ AppliedPTransform<PCollection<Object>, WriteFilesResult<Void>,
WriteFiles<Object, Void, Object>>
+ originalApplication =
+ AppliedPTransform.of("writefiles", objs.expand(),
Collections.emptyMap(), original, p);
+
+ WriteFiles<Object, Void, Object> replacement =
+ (WriteFiles<Object, Void, Object>)
+
factory.getReplacementTransform(originalApplication).getTransform();
+
+ assertThat(replacement, not(equalTo((Object) original)));
+ assertThat(replacement.getNumShardsProvider().get(), is(10));
+ }
+
+ private static class TestSink extends FileBasedSink<Object, Void, Object> {
+ @Override
+ public void validate(PipelineOptions options) {}
+
+ private static final FilenamePolicy FILENAME_POLICY =
+ new FilenamePolicy() {
+ @Override
+ public ResourceId windowedFilename(
+ int shardNumber,
+ int numShards,
+ BoundedWindow window,
+ PaneInfo paneInfo,
+ OutputFileHints outputFileHints) {
+ throw new UnsupportedOperationException("should not be called");
+ }
+
+ @Nullable
+ @Override
+ public ResourceId unwindowedFilename(
+ int shardNumber, int numShards, OutputFileHints outputFileHints)
{
+ throw new UnsupportedOperationException("should not be called");
+ }
+ };
+
+ TestSink(String tmpFolder) {
+ super(
+
ValueProvider.StaticValueProvider.of(FileSystems.matchNewResource(tmpFolder,
true)),
+ DynamicFileDestinations.constant(FILENAME_POLICY,
SerializableFunctions.identity()));
+ }
+
+ @Override
+ public WriteOperation<Void, Object> createWriteOperation() {
+ throw new IllegalArgumentException("Should not be used");
+ }
+ }
+}
----------------------------------------------------------------
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: 176069)
Time Spent: 6h (was: 5h 50m)
> Detect number of shards for file sink in Flink Streaming Runner
> ---------------------------------------------------------------
>
> Key: BEAM-2873
> URL: https://issues.apache.org/jira/browse/BEAM-2873
> Project: Beam
> Issue Type: Improvement
> Components: runner-flink
> Reporter: Aljoscha Krettek
> Assignee: Dawid Wysakowicz
> Priority: Major
> Time Spent: 6h
> Remaining Estimate: 0h
>
> [~reuvenlax] mentioned that this is done for the Dataflow Runner and the
> default behaviour on Flink can be somewhat surprising for users.
> ML entry: https://www.mail-archive.com/[email protected]/msg02665.html:
> This is how the file sink has always worked in Beam. If no sharding is
> specified, then this means runner-determined sharding, and by default that is
> one file per bundle. If Flink has small bundles, then I suggest using the
> withNumShards method to explicitly pick the number of output shards.
> The Flink runner can detect that runner-determined sharding has been chosen,
> and override it with a specific number of shards. For example, the Dataflow
> streaming runner (which as you mentioned also has small bundles) detects this
> case and sets the number of out files shards based on the number of workers
> in the worker pool
> [Here|https://github.com/apache/beam/blob/9e6530adb00669b7cf0f01cb8b128be0a21fd721/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java#L354]
> is the code that does this; it should be quite simple to do something
> similar for Flink, and then there will be no need for users to explicitly
> call withNumShards themselves.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)