[ https://issues.apache.org/jira/browse/SPARK-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15554027#comment-15554027 ]
Michael Armbrust commented on SPARK-15406: ------------------------------------------ It would be helpful if you could enumerate problems (other than the name) of structured streaming checkpoints, as I think we have avoided many of the pitfalls by not serializing the job itself. I do know that there are a few places where we are still using java serialization that should be replaced with something like JSON (in particular the offset log). This would only be an issue across Spark versions, and supporting upgrading there will require some careful thought. I do think we should support this though. Regarding that KIP, it will be great when that is available. It should be straightforward to construct an appropriate sequence number using a combination of of partitionId, version and row number. In general I think that some combination of that triple should be sufficient to update downstream datasources with exactly-once semantics. > Structured streaming support for consuming from Kafka > ----------------------------------------------------- > > Key: SPARK-15406 > URL: https://issues.apache.org/jira/browse/SPARK-15406 > Project: Spark > Issue Type: New Feature > Reporter: Cody Koeninger > > This is the parent JIRA to track all the work for the building a Kafka source > for Structured Streaming. Here is the design doc for an initial version of > the Kafka Source. > https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing > ================== Old description ========================= > Structured streaming doesn't have support for kafka yet. I personally feel > like time based indexing would make for a much better interface, but it's > been pushed back to kafka 0.10.1 > https://cwiki.apache.org/confluence/display/KAFKA/KIP-33+-+Add+a+time+based+log+index -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org