Re: Minimizing global store restoration time

2019-03-01 Thread Guozhang Wang
KAFKA-6721 should not need a KIP since it is for refactoring the internal classes only for cleaner code. However 8023 which is aimed for having multi-thread support would not be covered by 6721. Guozhang On Fri, Mar 1, 2019 at 12:15 AM Patrik Kl

Re: Minimizing global store restoration time

2019-03-01 Thread Patrik Kleindl
Hi Guozhang I have created https://issues.apache.org/jira/browse/KAFKA-8023 and by accident found https://issues.apache.org/jira/browse/KAFKA-6721 which was what I was looking for at the beginning. Does this need a KIP? I can maybe help with the writeup but I am not sure I should help with the cod

Re: Minimizing global store restoration time

2019-02-28 Thread Guozhang Wang
Hi Taylor, 1) Yes we do allow users to have separate config values for global consumers / restore consumers via StreamsConfig#restoreConsumerPrefix and StreamsConfig#globalConsumerPrefix, as Patrik pointed out. 2) I think I agree with you that for global consumer, it is worth while to allow one t

Re: Minimizing global store restoration time

2019-02-27 Thread Taylor P
Hi Guozhang, Patrik, Yes, the global consumer setting is what needs to be changed for these settings. The restore consumer configs aren't used at all since a separate restore consumer is not initialized for global state store restoration - the global consumer is used. I think it would be an improv

Re: Minimizing global store restoration time

2019-02-21 Thread Patrik Kleindl
Hello Guozhang, thanks, that might help us too. Just to confirm, this depends on KTable/GlobalKTable usage, right? I did a test with streamsConfiguration.put(StreamsConfig.restoreConsumerPrefix(StreamsConfig.RECEIVE_BUFFER_CONFIG), 65536); streamsConfiguration.put(StreamsConfig.restoreConsumerPre

Re: Minimizing global store restoration time

2019-02-20 Thread Guozhang Wang
Hello Taylor, Sorry for the late reply! And thanks for the updated information. I'd recommend overriding some consumer configs via `StreamsConfig` (you can use the StreamsConfig#restoreConsumerPrefix for that) for the following props: 1) increase RECEIVE_BUFFER_CONFIG (64K may cause poll to retu

Re: Minimizing global store restoration time

2019-02-08 Thread Patrik Kleindl
Hi Taylor You are right, the parallel processing is not mentioned in this issue, if I remember correctly it was in the thread that lead to it as a possibility when changing to the restoration listeners. Best regards Patrik > Am 07.02.2019 um 00:47 schrieb Taylor P : > > Hi Patrik, > > I am no

Re: Minimizing global store restoration time

2019-02-07 Thread Taylor P
Hi Guozhang, It does not seem that RocksDB is the bottleneck because the global state restore listener logs show that the restore calls to globalConsumer#poll do not always return max.poll.records. A typical log looks like this: 21:20:22,530 Restored batch of 2000 records... 21:20:22,533 Restored

Re: Minimizing global store restoration time

2019-02-07 Thread Guozhang Wang
Hello Taylor, Thanks for reporting the issue. One thing I can think of is that in 2.0.1 RocksDB is already using batching-restorer to avoid restoring one record at a time, so with larger batches of records, better throughput can be achieved with given IOPS. On the other hand, since you mention co

Re: Minimizing global store restoration time

2019-02-06 Thread Taylor P
Hi Patrik, I am not sure that https://issues.apache.org/jira/browse/KAFKA-7380 will resolve this issue since our application is dependent on the global store being fully restored before the application can be considered healthy. It does not seem like KAFKA-7380 is aiming to address the nature of g

Re: Minimizing global store restoration time

2019-02-06 Thread Patrik Kleindl
Hi Taylor We are facing the same issue, although on a smaller scale. The main problem as you found is that the restoration is running sequentially, this should be addressed in https://issues.apache.org/jira/browse/KAFKA-7380, although there has been no progress lately. On the other hand you could

Minimizing global store restoration time

2019-02-05 Thread Taylor P
Hi, I am having issues with the global store taking a very long time to restore during startup of a Kafka Streams 2.0.1 application. The global store is backed by a RocksDB persistent store and is added to the Streams topology in the following manner: https://pastebin.com/raw/VJutDyYe The global s