You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jay Kreps (JIRA)" <ji...@apache.org> on 2016/09/01 18:52:22 UTC

[jira] [Comment Edited] (KAFKA-4113) Allow KTable bootstrap

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

Jay Kreps edited comment on KAFKA-4113 at 9/1/16 6:51 PM:
----------------------------------------------------------

Don't you get this naturally out of the message timestamps and the prioritization we already do? People say they want to  "fully populate" a table but i think this isn't true. Rather you want the table to be in the same state the associated streams would be in. To see the difference imagine a case where you have a job that is doing a stream-table join and say you lose all your materialized table state and have a job that is down for three hours (for whatever reason--maintenance or something). When it comes back up you don't actually want to catch all the way up on the table because if you do that you will be joining table data from now to stream data from three hours ago. Rather, what you want is to catch up the table to three hours ago and then keep the two roughly aligned so you are joining stream data from time X to the state of the table at time X.

But isn't this exactly what the time stamp prioritization does already?


was (Author: jkreps):
Don't you get this naturally out of the message timestamps and the prioritization we already do? People say they want to  "fully populate" a table but i think this isn't true. Rather you want the table to be in the same state the associated streams would be in. To see the difference imagine a case where you have a job that is doing a stream-table join and say you lose all your materialized table state and have a job that is down for two hours (for whatever reason--maintenance or something). When it comes back up you don't actually want to catch all the way up on the table because if you do that you will be joining table data from now to stream data from three hours ago. Rather, what you want is to catch up the table to three hours ago and then keep the two roughly aligned.

But isn't this exactly what the time stamp prioritization does already?

> Allow KTable bootstrap
> ----------------------
>
>                 Key: KAFKA-4113
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4113
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: streams
>            Reporter: Matthias J. Sax
>            Assignee: Guozhang Wang
>
> On the mailing list, there are multiple request about the possibility to "fully populate" a KTable before actual stream processing start.
> Even if it is somewhat difficult to define, when the initial populating phase should end, there are multiple possibilities:
> The main idea is, that there is a rarely updated topic that contains the data. Only after this topic got read completely and the KTable is ready, the application should start processing. This would indicate, that on startup, the current partition sizes must be fetched and stored, and after KTable got populated up to those offsets, stream processing can start.
> Other discussed ideas are:
> 1) an initial fixed time period for populating
> (it might be hard for a user to estimate the correct value)
> 2) an "idle" period, ie, if no update to a KTable for a certain time is
> done, we consider it as populated
> 3) a timestamp cut off point, ie, all records with an older timestamp
> belong to the initial populating phase
> The API change is not decided yet, and the API desing is part of this JIRA.
> One suggestion (for option (4)) was:
> {noformat}
> KTable table = builder.table("topic", 1000); // populate the table without reading any other topics until see one record with timestamp 1000.
> {noformat}



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