Thanks Eno and Matthias for your feedback!

I've check KIP-95 and Matthias blog post (
https://www.confluent.io/blog/data-reprocessing-with-kafka-streams-resetting-a-streams-application/)
and I have a clearer idea on how stream internals work.

In a general use-case, following Application Reset Tool's procedure:
---

   1. for any specified input topic, it resets all offsets to zero
   2. for any specified intermediate topic, seeks to the end for all
   partitions
   3. for all internal topic
      1. resets all offsets to zero
      2. deletes the topic

---
But instead of resetting input topics to zero, resetting input topics to
offset by timestamp wouldn't be enough?

I will definitely take a look to StreamsResetter and give a try to support
this feature.


El lun., 30 ene. 2017 a las 1:43, Matthias J. Sax (<matth...@confluent.io>)
escribió:

> You can always built you own little tool similar to StreamsResetter.java
> to get this done. Ie, you set the committed offset "manually" based on
> timestamps before you start your application.
>
> But as Eno mentioned, you need to think carefully about what a
> consistent reset point would be because you cannot reset the
> application's state...
>
> If you start you application with an empty state, this might be less of
> an concern though and seems reasonable.
>
>
> -Matthias
>
> On 1/29/17 12:55 PM, Eno Thereska wrote:
> > Hi Jorge,
> >
> > This is currently not possible, but it is likely to be considered for
> discussion. One challenge is that, if you have multiple topics, it is
> difficult to rewind them all back to a consistent point in time. KIP-95,
> currently under discussion, is handling the slightly different issue, of
> stopping the consuming at a point in time:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-95%3A+Incremental+Batch+Processing+for+Kafka+Streams
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-95:+Incremental+Batch+Processing+for+Kafka+Streams
> >.
> >
> > Thanks
> > Eno
> >> On 29 Jan 2017, at 19:29, Jorge Esteban Quilcate Otoya <
> quilcate.jo...@gmail.com> wrote:
> >>
> >> Hi everyone,
> >>
> >> I was wondering if its possible to rewind consumers offset in Kafka
> Stream
> >> using timestamp as with `offsetsForTimes(Map<TopicPartition, Long>
> >> timestampsToSearch)` in KafkaConsumer.
> >>
> >> I know its possible to go back to `earliest` offset in topic or
> `latest`,
> >> but would be useful to go back using timestamp as with Consumer API do.
> >>
> >> Maybe is there an option to do this already and I'm missing something?
> >>
> >> Thanks in advance for your feedback!
> >>
> >> Jorge.
> >
> >
>
>

Reply via email to