[ https://issues.apache.org/jira/browse/BEAM-4783?focusedWorklogId=148801&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-148801 ]
ASF GitHub Bot logged work on BEAM-4783: ---------------------------------------- Author: ASF GitHub Bot Created on: 27/Sep/18 16:12 Start Date: 27/Sep/18 16:12 Worklog Time Spent: 10m Work Description: kyle-winkelman commented on issue #6181: [BEAM-4783] Add bundleSize for splitting BoundedSources. URL: https://github.com/apache/beam/pull/6181#issuecomment-425152159 I don't know how to proceed. I am convinced that in batch mode my proposal is the correct way to proceed. Another example of a silly interaction that occurs do to using defaultParallelism in SourceRDD is reading 2 different files. If one of the two files is a couple of orders of magnitude larger you will need to allocate enough resources to the job to read the larger file, lets say n cores, then the smaller file will get split into n pieces which will result in the smaller file being broken up into many very small bundles. The issue is I do not understand the repercussions this change will have on the streaming mode. Maybe we will need to have two different approaches to the groupBy logic, one for each mode. I am ok with this being experimental and only working if you supply the --bundleSize to the pipeline options. I would like an answer to the last question I asked to understand if in batch mode I can always use the new experimental groupByKeyOnlyDefaultPartitioner because I believe it will not cause a double shuffle in batch mode. Other than that I believe I need a code review and make sure everyone agrees with the approach. If this is not agreed upon I would hope someone could give me some advice on how to get the SparkRunner to work with dynamicAllocation. (Starting with 2 cores and spinning up more if the files are large and are split into more bundles.) ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 148801) Time Spent: 3h 10m (was: 3h) > 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: 3h 10m > 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)