spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Cody Koeninger (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-17812) More granular control of starting offsets (assign)
Date Fri, 14 Oct 2016 01:59:20 GMT

    [ https://issues.apache.org/jira/browse/SPARK-17812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15573843#comment-15573843
] 

Cody Koeninger commented on SPARK-17812:
----------------------------------------

So I think this is what we're agreed on:

Mutually exclusive subscription options (only assign is new to this ticket)
{noformat}
.option("subscribe","topicFoo,topicBar")
.option("subscribePattern","topic.*")
.option("assign","""{"topicfoo": [0, 1],"topicbar": [0, 1]}""")
{noformat}

where assign can only be specified that way, no inline offsets

Single starting position option with three mutually exclusive types of value
{noformat}
.option("startingOffsets", "earliest" | "latest" | """{"topicFoo": {"0": 1234, "1": -2}, "topicBar":{"0":
-1}}""")
{noformat}

startingOffsets with json fails if any topicpartition in the assignments doesn't have an offset.

Sound right?

I'll go ahead and start on it.  I'm assuming I should try to reuse some of the existing catalyst
Jackson stuff and keep in mind a format that's potentially usable by the checkpoints as well?

I don't think earliest / latest is too unclear as long as there's a way to get to the other
knobs that auto.offset.reset (should have) provided. Punting the tunability of new partitions
to another ticket sounds good.  


> More granular control of starting offsets (assign)
> --------------------------------------------------
>
>                 Key: SPARK-17812
>                 URL: https://issues.apache.org/jira/browse/SPARK-17812
>             Project: Spark
>          Issue Type: Sub-task
>          Components: SQL
>            Reporter: Michael Armbrust
>            Assignee: Cody Koeninger
>
> Right now you can only run a Streaming Query starting from either the earliest or latests
offsets available at the moment the query is started.  Sometimes this is a lot of data.  It
would be nice to be able to do the following:
>  - seek to user specified offsets for manually specified topicpartitions



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org


Mime
View raw message