[
https://issues.apache.org/jira/browse/BEAM-4783?focusedWorklogId=142165&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-142165
]
ASF GitHub Bot logged work on BEAM-4783:
----------------------------------------
Author: ASF GitHub Bot
Created on: 07/Sep/18 12:36
Start Date: 07/Sep/18 12:36
Worklog Time Spent: 10m
Work Description: kyle-winkelman commented on a change in pull request
#6181: [BEAM-4783] Add bundleSize for splitting BoundedSources.
URL: https://github.com/apache/beam/pull/6181#discussion_r215943999
##########
File path:
runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
##########
@@ -80,35 +81,16 @@ public Bounded(
super(sc, NIL, JavaSparkContext$.MODULE$.fakeClassTag());
this.source = source;
this.options = options;
- // the input parallelism is determined by Spark's scheduler backend.
- // when running on YARN/SparkDeploy it's the result of max(totalCores,
2).
- // when running on Mesos it's 8.
- // when running local it's the total number of cores (local = 1,
local[N] = N,
- // local[*] = estimation of the machine's cores).
- // ** the configuration "spark.default.parallelism" takes precedence
over all of the above **
- this.numPartitions = sc.defaultParallelism();
- checkArgument(this.numPartitions > 0, "Number of partitions must be
greater than zero.");
+ this.bundleSize =
options.get().as(SparkPipelineOptions.class).getBundleSize();
+ checkArgument(this.bundleSize > 0, "Bundle size must be greater than
zero.");
this.stepName = stepName;
this.metricsAccum = MetricsAccumulator.getInstance();
}
- private static final long DEFAULT_BUNDLE_SIZE = 64L * 1024L * 1024L;
-
@Override
public Partition[] getPartitions() {
- long desiredSizeBytes = DEFAULT_BUNDLE_SIZE;
- try {
- desiredSizeBytes = source.getEstimatedSizeBytes(options.get()) /
numPartitions;
- } catch (Exception e) {
- LOG.warn(
- "Failed to get estimated bundle size for source {}, using default
bundle "
- + "size of {} bytes.",
- source,
- DEFAULT_BUNDLE_SIZE);
- }
try {
- List<? extends Source<T>> partitionedSources =
- source.split(desiredSizeBytes, options.get());
+ List<? extends Source<T>> partitionedSources =
source.split(bundleSize, options.get());
Review comment:
The knob based approach falls more in line with what Spark users would
expect. If they write native spark with an HDFS input like
`sc.textFile("hdfs:///input/war-and-peace.txt")`, they wouldn't expect the file
to be broken up differently depending on the number of cores they run with.
----------------------------------------------------------------
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: 142165)
Time Spent: 1h 20m (was: 1h 10m)
> Spark SourceRDD Not Designed With Dynamic Allocation In Mind
> ------------------------------------------------------------
>
> Key: BEAM-4783
> URL: https://issues.apache.org/jira/browse/BEAM-4783
> Project: Beam
> Issue Type: Improvement
> Components: runner-spark
> Affects Versions: 2.5.0
> Reporter: Kyle Winkelman
> Assignee: Jean-Baptiste Onofré
> Priority: Major
> Labels: newbie
> Time Spent: 1h 20m
> Remaining Estimate: 0h
>
> When the spark-runner is used along with the configuration
> spark.dynamicAllocation.enabled=true the SourceRDD does not detect this. It
> then falls back to the value calculated in this description:
> // when running on YARN/SparkDeploy it's the result of max(totalCores,
> 2).
> // when running on Mesos it's 8.
> // when running local it's the total number of cores (local = 1,
> local[N] = N,
> // local[*] = estimation of the machine's cores).
> // ** the configuration "spark.default.parallelism" takes precedence
> over all of the above **
> So in most cases this default is quite small. This is an issue when using a
> very large input file as it will only get split in half.
> I believe that when Dynamic Allocation is enable the SourceRDD should use the
> DEFAULT_BUNDLE_SIZE and possibly expose a SparkPipelineOptions that allows
> you to change this DEFAULT_BUNDLE_SIZE.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)