spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Michael Armbrust (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-17812) More granular control of starting offsets
Date Tue, 11 Oct 2016 19:49:21 GMT

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

Michael Armbrust commented on SPARK-17812:
------------------------------------------

For the seeking back {{X}} offsets use case, I was interactively querying a stream and I wanted
*some* data, but not *all available data*.  I did not have specific offsets in mind, and under
the assumption that items are getting hashed across partitions, X offsets back is a very reasonable
proxy for time.  I agree actual time would be better.  However, since there is disagreement
on this case, I'd propose we break that out into its own ticket and focus on assign here.

I'm not sure I understand the concern with the {{startingOffsets}} option naming (which we
can still change, though, it would be nice to do so before a release happens).  It affects
which offsets will be included in the query and it only takes affect when the query is first
started.  [~ofirm], currently we support  (1) (though I wouldn't say *all* data as we are
limited by retention / compaction) and (2).  As you said, we can also support (3), though
this must be done after the fact by adding a predicate to the stream on the timestamp column.
 For performance it would be nice to push that down into Kafaka, but I'd split that optimization
into another ticket.

Regarding (4), I like the proposed JSON solution.  It would be nice if this was unified with
whatever format we decide to use in [SPARK-17829] so that you could easily pick up where another
query left off.  I'd also suggest we use {{-1}} and {{-2}} as special offsets for subscribing
to a topicpartition at the earliest or latests available offsets at query start time.

> More granular control of starting offsets
> -----------------------------------------
>
>                 Key: SPARK-17812
>                 URL: https://issues.apache.org/jira/browse/SPARK-17812
>             Project: Spark
>          Issue Type: Sub-task
>          Components: SQL
>            Reporter: Michael Armbrust
>
> 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 back {{X}} offsets in the stream from the moment the query starts
>  - seek to user specified offsets



--
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