[ https://issues.apache.org/jira/browse/SPARK-24036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16452576#comment-16452576 ]
Jose Torres commented on SPARK-24036: ------------------------------------- The broader Spark community is of course always welcome to help. The work here is generally split into three components: * Supporting single partition aggregates. I have a substantially complete prototype of this in [https://github.com/jose-torres/spark/pull/13] - it doesn't really involve design as much as removing a very silly hack I put in earlier. * Extending support to make continuous queries with multiple partitions run. My experimentation suggests that this only requires making ShuffleExchangeExec not cache its RDD in continuous mode, but I haven't strongly verified this. * Making the multiple partition aggregates truly continuous. ShuffleExchangeExec will of course insert a stage boundary, which means that latency will end up being bound by the checkpoint interval. What we need to do is create a new kind of shuffle for continuous processing which is non-blocking (cc [~liweisheng]). There are two possibilities here which I haven't evaluated in detail: ** Reuse the existing shuffle infrastructure, optimizing for latency later if needed. ** Just write RPC endpoints on both ends tossing rows around, optimizing for throughput later if needed. (I'm leaning towards this one.) If you're interested in working on some of this, I can prioritize a design for that third part. > Stateful operators in continuous processing > ------------------------------------------- > > Key: SPARK-24036 > URL: https://issues.apache.org/jira/browse/SPARK-24036 > Project: Spark > Issue Type: Improvement > Components: Structured Streaming > Affects Versions: 2.4.0 > Reporter: Jose Torres > Priority: Major > > The first iteration of continuous processing in Spark 2.3 does not work with > stateful operators. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org