[ 
https://issues.apache.org/jira/browse/BEAM-2873?focusedWorklogId=174891&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-174891
 ]

ASF GitHub Bot logged work on BEAM-2873:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 13/Dec/18 14:13
            Start Date: 13/Dec/18 14:13
    Worklog Time Spent: 10m 
      Work Description: mxm commented on a change in pull request #4760: 
[BEAM-2873] Setting number of shards for writes with runner determined sharding
URL: https://github.com/apache/beam/pull/4760#discussion_r241412824
 
 

 ##########
 File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
 ##########
 @@ -0,0 +1,138 @@
+/*
+ * 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.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 testDefaultParallelism() {
+        FlinkPipelineOptions options = 
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+        options.setRunner(TestFlinkRunner.class);
+        options.setFlinkMaster("[auto]");
+
+        testStreamingWriteOverride(options, 2);
 
 Review comment:
   Would set the parallelism explicitly here on the options. This should fail 
in local mode where the default parallelism now depends on the number of 
available cores.

----------------------------------------------------------------
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: 174891)

> 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: 4.5h
>  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)

Reply via email to