You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Liechuan Ou (Jira)" <ji...@apache.org> on 2020/08/29 15:44:00 UTC
[jira] [Updated] (SPARK-32734) RDD actions in DStream.transfrom
delays batch submission
[ https://issues.apache.org/jira/browse/SPARK-32734?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Liechuan Ou updated SPARK-32734:
--------------------------------
Labels: pull-request-available (was: )
> RDD actions in DStream.transfrom delays batch submission
> --------------------------------------------------------
>
> Key: SPARK-32734
> URL: https://issues.apache.org/jira/browse/SPARK-32734
> Project: Spark
> Issue Type: Bug
> Components: DStreams
> Affects Versions: 3.0.0
> Reporter: Liechuan Ou
> Priority: Major
> Labels: pull-request-available
> Original Estimate: 168h
> Remaining Estimate: 168h
>
> h4. Issue
> Some spark applications have batch creation delay after running for some time. For instance, Batch 10:03 is submitted at 10:06. In spark UI, the latest batch doesn't match current time.
>
> ||Clock||BatchTime||
> |10:00|10:00|
> |10:02|10:01|
> |10:04|10:02|
> |10:06|10:03|
> h4. Investigation
> We observe such applications share a commonality that rdd actions exist in dstream.transfrom. Those actions will be executed in dstream.compute, which is called by JobGenerator. JobGenerator runs with a single thread event loop so any synchronized operations will block event processing.
> h4. Proposal
> delegate dstream.compute to JobSchduler
>
> {code:java}
> // class ForEachDStream
> override def generateJob(time: Time): Option[Job] = {
> val jobFunc = () => createRDDWithLocalProperties(time, displayInnerRDDOps) {
> parent.getOrCompute(time).foreach(rdd => foreachFunc(rdd, time))
> }
> Some(new Job(time, jobFunc))
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org