Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-10 Thread João Peixoto
That's exactly what I did and it worked ok... Still slow but bootstraps
within an acceptable amount of time.
On Tue, May 9, 2017 at 9:05 PM Sachin Mittal  wrote:

> What we do is start and instance and wait till it get all the partitions,
> then start second and so on.
> Hope this works as a workaround till they fix this in next release.
>
>
> On Wed, May 10, 2017 at 12:05 AM, João Peixoto 
> wrote:
>
> > Guozhang thanks a lot for that info, that is exactly what I'm observing
> it
> > seems.
> >
> > I'll keep an eye out.
> >
> > JP
> >
> > On Mon, May 8, 2017 at 3:17 PM Guozhang Wang  wrote:
> >
> > > Hello,
> > >
> > > Just to adds a few more pointers that there is a few improvements we
> have
> > > added in trunk and are considering to also piggy-back to a 0.10.2 in
> case
> > > we can have a 0.10.2.2 release, and one of them that would help with
> this
> > > case:
> > >
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 134%3A+Delay+initial+consumer+group+rebalance
> > >
> > > The key idea is that with many instances of the same app starting up at
> > the
> > > same time, in your case 5 * 5 = 25 threads, we can consider 1) reduce
> the
> > > latency of a single rebalance, 2) reduce the number of consecutive
> > > rebalances until all instances are up and running, and the above one is
> > > aimed for the second case. So I'd suggest taking a look at the app's
> logs
> > > and see if there are multiple rebalances triggered during the starting
> > up,
> > > and if yes the above fix may help the most.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Mon, May 8, 2017 at 7:41 AM, João Peixoto 
> > > wrote:
> > >
> > > > Thanks for the feedback. Here is additional information:
> > > >
> > > > * The stream instances are deployed on kubernetes through
> deployments.
> > I
> > > do
> > > > not know if they use emptyDir, hostPath or EBS
> > > > * The instances have 2 cores minimum
> > > >
> > > > Good advice on the state stores, I already had some of those
> > > > configurations, but for this issue in particular the state stores are
> > > > empty, since this happens when the kafka stream bootstraps for the
> > first
> > > > time.
> > > >
> > > >
> > > >
> > > > On Sat, May 6, 2017 at 7:31 AM Eno Thereska 
> > > > wrote:
> > > >
> > > > > Hi there,
> > > > >
> > > > > I wanted to add something: how many CPU cores does each of your
> > > > Kubernetes
> > > > > instance have? In 0.10.2.1 we noticed a regression in environments
> > > with 1
> > > > > core as described in https://issues.apache.org/
> > jira/browse/KAFKA-5174
> > > <
> > > > > https://issues.apache.org/jira/browse/KAFKA-5174>.
> > > > >
> > > > > If you have 1 core, the workaround is to change a config as
> described
> > > > here:
> > > > >
> > > > > http://docs.confluent.io/current/streams/upgrade-guide.
> > > > html#known-issues-and-workarounds
> > > > > <
> > > > > http://docs.confluent.io/current/streams/upgrade-guide.
> > > > html#known-issues-and-workarounds
> > > > > >
> > > > >
> > > > > Thanks
> > > > > Eno
> > > > >
> > > > >
> > > > > > On May 6, 2017, at 9:48 AM, Sachin Mittal 
> > > wrote:
> > > > > >
> > > > > > Note on few things.
> > > > > > Set changelog topic delete retention time to as less as possible
> if
> > > the
> > > > > > previous values for same key are not needed and can be safely
> > cleaned
> > > > up.
> > > > > > Set segment size and segment retention time also low so older
> > > segments
> > > > > can
> > > > > > be compacted and cleaned up.
> > > > > > Set delete ratio to be aggressive 0.01 so segments don't grow to
> > big.
> > > > > >
> > > > > > This way state stores would be created much faster.
> > > > > >
> > > > > > Also when using Windows smaller window size helps.
> > > > > >
> > > > > > Try not running many stream threads on single machine unless you
> > > have a
> > > > > > great hardware.
> > > > > >
> > > > > > Make sure a thread is not reading from many partitions. Make sure
> > > ratio
> > > > > of
> > > > > > partions to total threads is low.
> > > > > >
> > > > > > Hope this helps.
> > > > > >
> > > > > > Sachin
> > > > > >
> > > > > > On 6 May 2017 13:28, "Shimi Kiviti"  wrote:
> > > > > >
> > > > > >> This is very similar to issues that we see.
> > > > > >>
> > > > > >> Did you check the status of the consumer group? In my case it
> will
> > > be
> > > > in
> > > > > >> rebalancing most of the time. Once in a while it will show
> > consumers
> > > > and
> > > > > >> offsets but after a short time will go back to rebalancing.
> > > > > >>
> > > > > >> How much storage does your Kafka-streams use?
> > > > > >> Also, what is your k8s configuration?
> > > > > >> Deployment? Deployment with emptyDir, hostPath or EBS?
> > Statefulset?
> > > > > >>
> > > > > >> Thanks,
> > > > > >> Shimi
> > > > > >> On Sat, 6 May 2017 at 2:34 João Peixoto 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-09 Thread Sachin Mittal
What we do is start and instance and wait till it get all the partitions,
then start second and so on.
Hope this works as a workaround till they fix this in next release.


On Wed, May 10, 2017 at 12:05 AM, João Peixoto 
wrote:

> Guozhang thanks a lot for that info, that is exactly what I'm observing it
> seems.
>
> I'll keep an eye out.
>
> JP
>
> On Mon, May 8, 2017 at 3:17 PM Guozhang Wang  wrote:
>
> > Hello,
> >
> > Just to adds a few more pointers that there is a few improvements we have
> > added in trunk and are considering to also piggy-back to a 0.10.2 in case
> > we can have a 0.10.2.2 release, and one of them that would help with this
> > case:
> >
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 134%3A+Delay+initial+consumer+group+rebalance
> >
> > The key idea is that with many instances of the same app starting up at
> the
> > same time, in your case 5 * 5 = 25 threads, we can consider 1) reduce the
> > latency of a single rebalance, 2) reduce the number of consecutive
> > rebalances until all instances are up and running, and the above one is
> > aimed for the second case. So I'd suggest taking a look at the app's logs
> > and see if there are multiple rebalances triggered during the starting
> up,
> > and if yes the above fix may help the most.
> >
> >
> > Guozhang
> >
> >
> > On Mon, May 8, 2017 at 7:41 AM, João Peixoto 
> > wrote:
> >
> > > Thanks for the feedback. Here is additional information:
> > >
> > > * The stream instances are deployed on kubernetes through deployments.
> I
> > do
> > > not know if they use emptyDir, hostPath or EBS
> > > * The instances have 2 cores minimum
> > >
> > > Good advice on the state stores, I already had some of those
> > > configurations, but for this issue in particular the state stores are
> > > empty, since this happens when the kafka stream bootstraps for the
> first
> > > time.
> > >
> > >
> > >
> > > On Sat, May 6, 2017 at 7:31 AM Eno Thereska 
> > > wrote:
> > >
> > > > Hi there,
> > > >
> > > > I wanted to add something: how many CPU cores does each of your
> > > Kubernetes
> > > > instance have? In 0.10.2.1 we noticed a regression in environments
> > with 1
> > > > core as described in https://issues.apache.org/
> jira/browse/KAFKA-5174
> > <
> > > > https://issues.apache.org/jira/browse/KAFKA-5174>.
> > > >
> > > > If you have 1 core, the workaround is to change a config as described
> > > here:
> > > >
> > > > http://docs.confluent.io/current/streams/upgrade-guide.
> > > html#known-issues-and-workarounds
> > > > <
> > > > http://docs.confluent.io/current/streams/upgrade-guide.
> > > html#known-issues-and-workarounds
> > > > >
> > > >
> > > > Thanks
> > > > Eno
> > > >
> > > >
> > > > > On May 6, 2017, at 9:48 AM, Sachin Mittal 
> > wrote:
> > > > >
> > > > > Note on few things.
> > > > > Set changelog topic delete retention time to as less as possible if
> > the
> > > > > previous values for same key are not needed and can be safely
> cleaned
> > > up.
> > > > > Set segment size and segment retention time also low so older
> > segments
> > > > can
> > > > > be compacted and cleaned up.
> > > > > Set delete ratio to be aggressive 0.01 so segments don't grow to
> big.
> > > > >
> > > > > This way state stores would be created much faster.
> > > > >
> > > > > Also when using Windows smaller window size helps.
> > > > >
> > > > > Try not running many stream threads on single machine unless you
> > have a
> > > > > great hardware.
> > > > >
> > > > > Make sure a thread is not reading from many partitions. Make sure
> > ratio
> > > > of
> > > > > partions to total threads is low.
> > > > >
> > > > > Hope this helps.
> > > > >
> > > > > Sachin
> > > > >
> > > > > On 6 May 2017 13:28, "Shimi Kiviti"  wrote:
> > > > >
> > > > >> This is very similar to issues that we see.
> > > > >>
> > > > >> Did you check the status of the consumer group? In my case it will
> > be
> > > in
> > > > >> rebalancing most of the time. Once in a while it will show
> consumers
> > > and
> > > > >> offsets but after a short time will go back to rebalancing.
> > > > >>
> > > > >> How much storage does your Kafka-streams use?
> > > > >> Also, what is your k8s configuration?
> > > > >> Deployment? Deployment with emptyDir, hostPath or EBS?
> Statefulset?
> > > > >>
> > > > >> Thanks,
> > > > >> Shimi
> > > > >> On Sat, 6 May 2017 at 2:34 João Peixoto 
> > > > wrote:
> > > > >>
> > > > >>> After a while the instance started running.
> > > > >>>
> > > > >>> 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
> > > > >>> o.a.k.s.p.internals.StreamThread : stream-thread
> > > > >> [StreamThread-4]
> > > > >>> Committing task StreamTask 1_62
> > > > >>> (this is literally the next message)
> > > > >>> 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
> > > > >>> 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-09 Thread João Peixoto
Guozhang thanks a lot for that info, that is exactly what I'm observing it
seems.

I'll keep an eye out.

JP

On Mon, May 8, 2017 at 3:17 PM Guozhang Wang  wrote:

> Hello,
>
> Just to adds a few more pointers that there is a few improvements we have
> added in trunk and are considering to also piggy-back to a 0.10.2 in case
> we can have a 0.10.2.2 release, and one of them that would help with this
> case:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-134%3A+Delay+initial+consumer+group+rebalance
>
> The key idea is that with many instances of the same app starting up at the
> same time, in your case 5 * 5 = 25 threads, we can consider 1) reduce the
> latency of a single rebalance, 2) reduce the number of consecutive
> rebalances until all instances are up and running, and the above one is
> aimed for the second case. So I'd suggest taking a look at the app's logs
> and see if there are multiple rebalances triggered during the starting up,
> and if yes the above fix may help the most.
>
>
> Guozhang
>
>
> On Mon, May 8, 2017 at 7:41 AM, João Peixoto 
> wrote:
>
> > Thanks for the feedback. Here is additional information:
> >
> > * The stream instances are deployed on kubernetes through deployments. I
> do
> > not know if they use emptyDir, hostPath or EBS
> > * The instances have 2 cores minimum
> >
> > Good advice on the state stores, I already had some of those
> > configurations, but for this issue in particular the state stores are
> > empty, since this happens when the kafka stream bootstraps for the first
> > time.
> >
> >
> >
> > On Sat, May 6, 2017 at 7:31 AM Eno Thereska 
> > wrote:
> >
> > > Hi there,
> > >
> > > I wanted to add something: how many CPU cores does each of your
> > Kubernetes
> > > instance have? In 0.10.2.1 we noticed a regression in environments
> with 1
> > > core as described in https://issues.apache.org/jira/browse/KAFKA-5174
> <
> > > https://issues.apache.org/jira/browse/KAFKA-5174>.
> > >
> > > If you have 1 core, the workaround is to change a config as described
> > here:
> > >
> > > http://docs.confluent.io/current/streams/upgrade-guide.
> > html#known-issues-and-workarounds
> > > <
> > > http://docs.confluent.io/current/streams/upgrade-guide.
> > html#known-issues-and-workarounds
> > > >
> > >
> > > Thanks
> > > Eno
> > >
> > >
> > > > On May 6, 2017, at 9:48 AM, Sachin Mittal 
> wrote:
> > > >
> > > > Note on few things.
> > > > Set changelog topic delete retention time to as less as possible if
> the
> > > > previous values for same key are not needed and can be safely cleaned
> > up.
> > > > Set segment size and segment retention time also low so older
> segments
> > > can
> > > > be compacted and cleaned up.
> > > > Set delete ratio to be aggressive 0.01 so segments don't grow to big.
> > > >
> > > > This way state stores would be created much faster.
> > > >
> > > > Also when using Windows smaller window size helps.
> > > >
> > > > Try not running many stream threads on single machine unless you
> have a
> > > > great hardware.
> > > >
> > > > Make sure a thread is not reading from many partitions. Make sure
> ratio
> > > of
> > > > partions to total threads is low.
> > > >
> > > > Hope this helps.
> > > >
> > > > Sachin
> > > >
> > > > On 6 May 2017 13:28, "Shimi Kiviti"  wrote:
> > > >
> > > >> This is very similar to issues that we see.
> > > >>
> > > >> Did you check the status of the consumer group? In my case it will
> be
> > in
> > > >> rebalancing most of the time. Once in a while it will show consumers
> > and
> > > >> offsets but after a short time will go back to rebalancing.
> > > >>
> > > >> How much storage does your Kafka-streams use?
> > > >> Also, what is your k8s configuration?
> > > >> Deployment? Deployment with emptyDir, hostPath or EBS? Statefulset?
> > > >>
> > > >> Thanks,
> > > >> Shimi
> > > >> On Sat, 6 May 2017 at 2:34 João Peixoto 
> > > wrote:
> > > >>
> > > >>> After a while the instance started running.
> > > >>>
> > > >>> 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
> > > >>> o.a.k.s.p.internals.StreamThread : stream-thread
> > > >> [StreamThread-4]
> > > >>> Committing task StreamTask 1_62
> > > >>> (this is literally the next message)
> > > >>> 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
> > > >>> o.a.k.s.p.internals.StreamThread : stream-thread
> > > >> [StreamThread-4]
> > > >>> Committing all tasks because the commit interval 1ms has
> elapsed
> > > >>>
> > > >>> On Fri, May 5, 2017 at 3:48 PM João Peixoto <
> joao.harti...@gmail.com
> > >
> > > >>> wrote:
> > > >>>
> > >  Warning, long message
> > > 
> > >  *Problem*: Initializing a Kafka Stream is taking a lng time.
> > >  Currently at the 40 minute mark
> > > 
> > >  *Setup*:
> > >  2 co-partition topics with 100 partitions.
> > >  First topic contains a 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-08 Thread Guozhang Wang
Hello,

Just to adds a few more pointers that there is a few improvements we have
added in trunk and are considering to also piggy-back to a 0.10.2 in case
we can have a 0.10.2.2 release, and one of them that would help with this
case:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-134%3A+Delay+initial+consumer+group+rebalance

The key idea is that with many instances of the same app starting up at the
same time, in your case 5 * 5 = 25 threads, we can consider 1) reduce the
latency of a single rebalance, 2) reduce the number of consecutive
rebalances until all instances are up and running, and the above one is
aimed for the second case. So I'd suggest taking a look at the app's logs
and see if there are multiple rebalances triggered during the starting up,
and if yes the above fix may help the most.


Guozhang


On Mon, May 8, 2017 at 7:41 AM, João Peixoto 
wrote:

> Thanks for the feedback. Here is additional information:
>
> * The stream instances are deployed on kubernetes through deployments. I do
> not know if they use emptyDir, hostPath or EBS
> * The instances have 2 cores minimum
>
> Good advice on the state stores, I already had some of those
> configurations, but for this issue in particular the state stores are
> empty, since this happens when the kafka stream bootstraps for the first
> time.
>
>
>
> On Sat, May 6, 2017 at 7:31 AM Eno Thereska 
> wrote:
>
> > Hi there,
> >
> > I wanted to add something: how many CPU cores does each of your
> Kubernetes
> > instance have? In 0.10.2.1 we noticed a regression in environments with 1
> > core as described in https://issues.apache.org/jira/browse/KAFKA-5174 <
> > https://issues.apache.org/jira/browse/KAFKA-5174>.
> >
> > If you have 1 core, the workaround is to change a config as described
> here:
> >
> > http://docs.confluent.io/current/streams/upgrade-guide.
> html#known-issues-and-workarounds
> > <
> > http://docs.confluent.io/current/streams/upgrade-guide.
> html#known-issues-and-workarounds
> > >
> >
> > Thanks
> > Eno
> >
> >
> > > On May 6, 2017, at 9:48 AM, Sachin Mittal  wrote:
> > >
> > > Note on few things.
> > > Set changelog topic delete retention time to as less as possible if the
> > > previous values for same key are not needed and can be safely cleaned
> up.
> > > Set segment size and segment retention time also low so older segments
> > can
> > > be compacted and cleaned up.
> > > Set delete ratio to be aggressive 0.01 so segments don't grow to big.
> > >
> > > This way state stores would be created much faster.
> > >
> > > Also when using Windows smaller window size helps.
> > >
> > > Try not running many stream threads on single machine unless you have a
> > > great hardware.
> > >
> > > Make sure a thread is not reading from many partitions. Make sure ratio
> > of
> > > partions to total threads is low.
> > >
> > > Hope this helps.
> > >
> > > Sachin
> > >
> > > On 6 May 2017 13:28, "Shimi Kiviti"  wrote:
> > >
> > >> This is very similar to issues that we see.
> > >>
> > >> Did you check the status of the consumer group? In my case it will be
> in
> > >> rebalancing most of the time. Once in a while it will show consumers
> and
> > >> offsets but after a short time will go back to rebalancing.
> > >>
> > >> How much storage does your Kafka-streams use?
> > >> Also, what is your k8s configuration?
> > >> Deployment? Deployment with emptyDir, hostPath or EBS? Statefulset?
> > >>
> > >> Thanks,
> > >> Shimi
> > >> On Sat, 6 May 2017 at 2:34 João Peixoto 
> > wrote:
> > >>
> > >>> After a while the instance started running.
> > >>>
> > >>> 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
> > >>> o.a.k.s.p.internals.StreamThread : stream-thread
> > >> [StreamThread-4]
> > >>> Committing task StreamTask 1_62
> > >>> (this is literally the next message)
> > >>> 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
> > >>> o.a.k.s.p.internals.StreamThread : stream-thread
> > >> [StreamThread-4]
> > >>> Committing all tasks because the commit interval 1ms has elapsed
> > >>>
> > >>> On Fri, May 5, 2017 at 3:48 PM João Peixoto  >
> > >>> wrote:
> > >>>
> >  Warning, long message
> > 
> >  *Problem*: Initializing a Kafka Stream is taking a lng time.
> >  Currently at the 40 minute mark
> > 
> >  *Setup*:
> >  2 co-partition topics with 100 partitions.
> >  First topic contains a lot of messages in the order of hundreds of
> > >>> millions
> >  Second topic is a KTable and contains ~30k records
> > 
> >  Kafka cluster with 6 brokers running 0.10.1
> > 
> >  Kafka streams running on 0.10.2.1. 5 instances with 5 threads each.
> >  The instances are running on Kubernetes
> > 
> >  *Stream Configuration*:
> >  Properties props = new Properties();
> >  props.put(StreamsConfig.APPLICATION_ID_CONFIG, 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-08 Thread João Peixoto
Thanks for the feedback. Here is additional information:

* The stream instances are deployed on kubernetes through deployments. I do
not know if they use emptyDir, hostPath or EBS
* The instances have 2 cores minimum

Good advice on the state stores, I already had some of those
configurations, but for this issue in particular the state stores are
empty, since this happens when the kafka stream bootstraps for the first
time.



On Sat, May 6, 2017 at 7:31 AM Eno Thereska  wrote:

> Hi there,
>
> I wanted to add something: how many CPU cores does each of your Kubernetes
> instance have? In 0.10.2.1 we noticed a regression in environments with 1
> core as described in https://issues.apache.org/jira/browse/KAFKA-5174 <
> https://issues.apache.org/jira/browse/KAFKA-5174>.
>
> If you have 1 core, the workaround is to change a config as described here:
>
> http://docs.confluent.io/current/streams/upgrade-guide.html#known-issues-and-workarounds
> <
> http://docs.confluent.io/current/streams/upgrade-guide.html#known-issues-and-workarounds
> >
>
> Thanks
> Eno
>
>
> > On May 6, 2017, at 9:48 AM, Sachin Mittal  wrote:
> >
> > Note on few things.
> > Set changelog topic delete retention time to as less as possible if the
> > previous values for same key are not needed and can be safely cleaned up.
> > Set segment size and segment retention time also low so older segments
> can
> > be compacted and cleaned up.
> > Set delete ratio to be aggressive 0.01 so segments don't grow to big.
> >
> > This way state stores would be created much faster.
> >
> > Also when using Windows smaller window size helps.
> >
> > Try not running many stream threads on single machine unless you have a
> > great hardware.
> >
> > Make sure a thread is not reading from many partitions. Make sure ratio
> of
> > partions to total threads is low.
> >
> > Hope this helps.
> >
> > Sachin
> >
> > On 6 May 2017 13:28, "Shimi Kiviti"  wrote:
> >
> >> This is very similar to issues that we see.
> >>
> >> Did you check the status of the consumer group? In my case it will be in
> >> rebalancing most of the time. Once in a while it will show consumers and
> >> offsets but after a short time will go back to rebalancing.
> >>
> >> How much storage does your Kafka-streams use?
> >> Also, what is your k8s configuration?
> >> Deployment? Deployment with emptyDir, hostPath or EBS? Statefulset?
> >>
> >> Thanks,
> >> Shimi
> >> On Sat, 6 May 2017 at 2:34 João Peixoto 
> wrote:
> >>
> >>> After a while the instance started running.
> >>>
> >>> 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
> >>> o.a.k.s.p.internals.StreamThread : stream-thread
> >> [StreamThread-4]
> >>> Committing task StreamTask 1_62
> >>> (this is literally the next message)
> >>> 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
> >>> o.a.k.s.p.internals.StreamThread : stream-thread
> >> [StreamThread-4]
> >>> Committing all tasks because the commit interval 1ms has elapsed
> >>>
> >>> On Fri, May 5, 2017 at 3:48 PM João Peixoto 
> >>> wrote:
> >>>
>  Warning, long message
> 
>  *Problem*: Initializing a Kafka Stream is taking a lng time.
>  Currently at the 40 minute mark
> 
>  *Setup*:
>  2 co-partition topics with 100 partitions.
>  First topic contains a lot of messages in the order of hundreds of
> >>> millions
>  Second topic is a KTable and contains ~30k records
> 
>  Kafka cluster with 6 brokers running 0.10.1
> 
>  Kafka streams running on 0.10.2.1. 5 instances with 5 threads each.
>  The instances are running on Kubernetes
> 
>  *Stream Configuration*:
>  Properties props = new Properties();
>  props.put(StreamsConfig.APPLICATION_ID_CONFIG, streamName);
>  props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
>  props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG,
>  Serdes.String().getClass().getName());
>  props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG,
>  Serdes.ByteArray().getClass().getName());
>  props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1);
>  props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG");
>  props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 5);
>  props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
>  props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 500);
> 
>  *The events*:
>  I started 5 instances of my stream configuration at the same time.
> This
> >>> is
>  the first
>  time this configuration is running.
> 
>  2017-05-05 21:23:03.283  INFO 71 --- [   main]
>  o.a.k.s.p.internals.StreamThread : stream-thread
> >> [StreamThread-1]
>  Creating producer client
>  2017-05-05 21:23:03.415  INFO 71 --- [   main]
>  o.a.k.s.p.internals.StreamThread : stream-thread
> >> [StreamThread-1]
>  

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-06 Thread Eno Thereska
Hi there,

I wanted to add something: how many CPU cores does each of your Kubernetes 
instance have? In 0.10.2.1 we noticed a regression in environments with 1 core 
as described in https://issues.apache.org/jira/browse/KAFKA-5174 
. 

If you have 1 core, the workaround is to change a config as described here:
http://docs.confluent.io/current/streams/upgrade-guide.html#known-issues-and-workarounds
 


Thanks
Eno


> On May 6, 2017, at 9:48 AM, Sachin Mittal  wrote:
> 
> Note on few things.
> Set changelog topic delete retention time to as less as possible if the
> previous values for same key are not needed and can be safely cleaned up.
> Set segment size and segment retention time also low so older segments can
> be compacted and cleaned up.
> Set delete ratio to be aggressive 0.01 so segments don't grow to big.
> 
> This way state stores would be created much faster.
> 
> Also when using Windows smaller window size helps.
> 
> Try not running many stream threads on single machine unless you have a
> great hardware.
> 
> Make sure a thread is not reading from many partitions. Make sure ratio of
> partions to total threads is low.
> 
> Hope this helps.
> 
> Sachin
> 
> On 6 May 2017 13:28, "Shimi Kiviti"  wrote:
> 
>> This is very similar to issues that we see.
>> 
>> Did you check the status of the consumer group? In my case it will be in
>> rebalancing most of the time. Once in a while it will show consumers and
>> offsets but after a short time will go back to rebalancing.
>> 
>> How much storage does your Kafka-streams use?
>> Also, what is your k8s configuration?
>> Deployment? Deployment with emptyDir, hostPath or EBS? Statefulset?
>> 
>> Thanks,
>> Shimi
>> On Sat, 6 May 2017 at 2:34 João Peixoto  wrote:
>> 
>>> After a while the instance started running.
>>> 
>>> 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
>>> o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-4]
>>> Committing task StreamTask 1_62
>>> (this is literally the next message)
>>> 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
>>> o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-4]
>>> Committing all tasks because the commit interval 1ms has elapsed
>>> 
>>> On Fri, May 5, 2017 at 3:48 PM João Peixoto 
>>> wrote:
>>> 
 Warning, long message
 
 *Problem*: Initializing a Kafka Stream is taking a lng time.
 Currently at the 40 minute mark
 
 *Setup*:
 2 co-partition topics with 100 partitions.
 First topic contains a lot of messages in the order of hundreds of
>>> millions
 Second topic is a KTable and contains ~30k records
 
 Kafka cluster with 6 brokers running 0.10.1
 
 Kafka streams running on 0.10.2.1. 5 instances with 5 threads each.
 The instances are running on Kubernetes
 
 *Stream Configuration*:
 Properties props = new Properties();
 props.put(StreamsConfig.APPLICATION_ID_CONFIG, streamName);
 props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
 props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG,
 Serdes.String().getClass().getName());
 props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG,
 Serdes.ByteArray().getClass().getName());
 props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1);
 props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG");
 props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 5);
 props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 500);
 
 *The events*:
 I started 5 instances of my stream configuration at the same time. This
>>> is
 the first
 time this configuration is running.
 
 2017-05-05 21:23:03.283  INFO 71 --- [   main]
 o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-1]
 Creating producer client
 2017-05-05 21:23:03.415  INFO 71 --- [   main]
 o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-1]
 Creating consumer client
 2017-05-05 21:23:03.520  INFO 71 --- [   main]
 o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-1]
 Creating restore consumer client
 2017-05-05 21:23:03.528  INFO 71 --- [   main]
 o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-1]
 State transition from NOT_RUNNING to RUNNING.
 2017-05-05 21:23:03.531  INFO 71 --- [   main]
 o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-2]
 Creating producer client
 2017-05-05 21:23:03.564  INFO 71 --- [   main]
 o.a.k.s.p.internals.StreamThread : stream-thread
>> [StreamThread-2]
 Creating 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-06 Thread Sachin Mittal
Note on few things.
Set changelog topic delete retention time to as less as possible if the
previous values for same key are not needed and can be safely cleaned up.
Set segment size and segment retention time also low so older segments can
be compacted and cleaned up.
Set delete ratio to be aggressive 0.01 so segments don't grow to big.

This way state stores would be created much faster.

Also when using Windows smaller window size helps.

Try not running many stream threads on single machine unless you have a
great hardware.

Make sure a thread is not reading from many partitions. Make sure ratio of
partions to total threads is low.

Hope this helps.

Sachin

On 6 May 2017 13:28, "Shimi Kiviti"  wrote:

> This is very similar to issues that we see.
>
> Did you check the status of the consumer group? In my case it will be in
> rebalancing most of the time. Once in a while it will show consumers and
> offsets but after a short time will go back to rebalancing.
>
> How much storage does your Kafka-streams use?
> Also, what is your k8s configuration?
> Deployment? Deployment with emptyDir, hostPath or EBS? Statefulset?
>
> Thanks,
> Shimi
> On Sat, 6 May 2017 at 2:34 João Peixoto  wrote:
>
> > After a while the instance started running.
> >
> > 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
> > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-4]
> > Committing task StreamTask 1_62
> > (this is literally the next message)
> > 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
> > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-4]
> > Committing all tasks because the commit interval 1ms has elapsed
> >
> > On Fri, May 5, 2017 at 3:48 PM João Peixoto 
> > wrote:
> >
> > > Warning, long message
> > >
> > > *Problem*: Initializing a Kafka Stream is taking a lng time.
> > > Currently at the 40 minute mark
> > >
> > > *Setup*:
> > > 2 co-partition topics with 100 partitions.
> > > First topic contains a lot of messages in the order of hundreds of
> > millions
> > > Second topic is a KTable and contains ~30k records
> > >
> > > Kafka cluster with 6 brokers running 0.10.1
> > >
> > > Kafka streams running on 0.10.2.1. 5 instances with 5 threads each.
> > > The instances are running on Kubernetes
> > >
> > > *Stream Configuration*:
> > > Properties props = new Properties();
> > > props.put(StreamsConfig.APPLICATION_ID_CONFIG, streamName);
> > > props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
> > > props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG,
> > > Serdes.String().getClass().getName());
> > > props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG,
> > > Serdes.ByteArray().getClass().getName());
> > > props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1);
> > > props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG");
> > > props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 5);
> > > props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
> > > props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 500);
> > >
> > > *The events*:
> > > I started 5 instances of my stream configuration at the same time. This
> > is
> > > the first
> > > time this configuration is running.
> > >
> > > 2017-05-05 21:23:03.283  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-1]
> > > Creating producer client
> > > 2017-05-05 21:23:03.415  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-1]
> > > Creating consumer client
> > > 2017-05-05 21:23:03.520  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-1]
> > > Creating restore consumer client
> > > 2017-05-05 21:23:03.528  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-1]
> > > State transition from NOT_RUNNING to RUNNING.
> > > 2017-05-05 21:23:03.531  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-2]
> > > Creating producer client
> > > 2017-05-05 21:23:03.564  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-2]
> > > Creating consumer client
> > > 2017-05-05 21:23:03.569  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-2]
> > > Creating restore consumer client
> > > 2017-05-05 21:23:03.615  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-2]
> > > State transition from NOT_RUNNING to RUNNING.
> > > 2017-05-05 21:23:03.617  INFO 71 --- [   main]
> > > o.a.k.s.p.internals.StreamThread : stream-thread
> [StreamThread-3]
> > > Creating producer client
> > > 2017-05-05 21:23:03.621  INFO 71 --- [   main]
> > > 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-06 Thread Shimi Kiviti
This is very similar to issues that we see.

Did you check the status of the consumer group? In my case it will be in
rebalancing most of the time. Once in a while it will show consumers and
offsets but after a short time will go back to rebalancing.

How much storage does your Kafka-streams use?
Also, what is your k8s configuration?
Deployment? Deployment with emptyDir, hostPath or EBS? Statefulset?

Thanks,
Shimi
On Sat, 6 May 2017 at 2:34 João Peixoto  wrote:

> After a while the instance started running.
>
> 2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> Committing task StreamTask 1_62
> (this is literally the next message)
> 2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> Committing all tasks because the commit interval 1ms has elapsed
>
> On Fri, May 5, 2017 at 3:48 PM João Peixoto 
> wrote:
>
> > Warning, long message
> >
> > *Problem*: Initializing a Kafka Stream is taking a lng time.
> > Currently at the 40 minute mark
> >
> > *Setup*:
> > 2 co-partition topics with 100 partitions.
> > First topic contains a lot of messages in the order of hundreds of
> millions
> > Second topic is a KTable and contains ~30k records
> >
> > Kafka cluster with 6 brokers running 0.10.1
> >
> > Kafka streams running on 0.10.2.1. 5 instances with 5 threads each.
> > The instances are running on Kubernetes
> >
> > *Stream Configuration*:
> > Properties props = new Properties();
> > props.put(StreamsConfig.APPLICATION_ID_CONFIG, streamName);
> > props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
> > props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG,
> > Serdes.String().getClass().getName());
> > props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG,
> > Serdes.ByteArray().getClass().getName());
> > props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1);
> > props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG");
> > props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 5);
> > props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
> > props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 500);
> >
> > *The events*:
> > I started 5 instances of my stream configuration at the same time. This
> is
> > the first
> > time this configuration is running.
> >
> > 2017-05-05 21:23:03.283  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> > Creating producer client
> > 2017-05-05 21:23:03.415  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> > Creating consumer client
> > 2017-05-05 21:23:03.520  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> > Creating restore consumer client
> > 2017-05-05 21:23:03.528  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> > State transition from NOT_RUNNING to RUNNING.
> > 2017-05-05 21:23:03.531  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> > Creating producer client
> > 2017-05-05 21:23:03.564  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> > Creating consumer client
> > 2017-05-05 21:23:03.569  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> > Creating restore consumer client
> > 2017-05-05 21:23:03.615  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> > State transition from NOT_RUNNING to RUNNING.
> > 2017-05-05 21:23:03.617  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> > Creating producer client
> > 2017-05-05 21:23:03.621  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> > Creating consumer client
> > 2017-05-05 21:23:03.625  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> > Creating restore consumer client
> > 2017-05-05 21:23:03.628  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> > State transition from NOT_RUNNING to RUNNING.
> > 2017-05-05 21:23:03.629  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> > Creating producer client
> > 2017-05-05 21:23:03.632  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> > Creating consumer client
> > 2017-05-05 21:23:03.635  INFO 71 --- [   main]
> > o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> > Creating restore consumer 

Re: Large Kafka Streams deployment takes a long time to bootstrap

2017-05-05 Thread João Peixoto
After a while the instance started running.

2017-05-05 22:40:26.806  INFO 85 --- [ StreamThread-4]
o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
Committing task StreamTask 1_62
(this is literally the next message)
2017-05-05 23:13:27.820  INFO 85 --- [ StreamThread-4]
o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
Committing all tasks because the commit interval 1ms has elapsed

On Fri, May 5, 2017 at 3:48 PM João Peixoto  wrote:

> Warning, long message
>
> *Problem*: Initializing a Kafka Stream is taking a lng time.
> Currently at the 40 minute mark
>
> *Setup*:
> 2 co-partition topics with 100 partitions.
> First topic contains a lot of messages in the order of hundreds of millions
> Second topic is a KTable and contains ~30k records
>
> Kafka cluster with 6 brokers running 0.10.1
>
> Kafka streams running on 0.10.2.1. 5 instances with 5 threads each.
> The instances are running on Kubernetes
>
> *Stream Configuration*:
> Properties props = new Properties();
> props.put(StreamsConfig.APPLICATION_ID_CONFIG, streamName);
> props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ...);
> props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG,
> Serdes.String().getClass().getName());
> props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG,
> Serdes.ByteArray().getClass().getName());
> props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1);
> props.put(StreamsConfig.METRICS_RECORDING_LEVEL_CONFIG, "DEBUG");
> props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 5);
> props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
> props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 500);
>
> *The events*:
> I started 5 instances of my stream configuration at the same time. This is
> the first
> time this configuration is running.
>
> 2017-05-05 21:23:03.283  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> Creating producer client
> 2017-05-05 21:23:03.415  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> Creating consumer client
> 2017-05-05 21:23:03.520  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> Creating restore consumer client
> 2017-05-05 21:23:03.528  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-1]
> State transition from NOT_RUNNING to RUNNING.
> 2017-05-05 21:23:03.531  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> Creating producer client
> 2017-05-05 21:23:03.564  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> Creating consumer client
> 2017-05-05 21:23:03.569  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> Creating restore consumer client
> 2017-05-05 21:23:03.615  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-2]
> State transition from NOT_RUNNING to RUNNING.
> 2017-05-05 21:23:03.617  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> Creating producer client
> 2017-05-05 21:23:03.621  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> Creating consumer client
> 2017-05-05 21:23:03.625  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> Creating restore consumer client
> 2017-05-05 21:23:03.628  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-3]
> State transition from NOT_RUNNING to RUNNING.
> 2017-05-05 21:23:03.629  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> Creating producer client
> 2017-05-05 21:23:03.632  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> Creating consumer client
> 2017-05-05 21:23:03.635  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> Creating restore consumer client
> 2017-05-05 21:23:03.638  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-4]
> State transition from NOT_RUNNING to RUNNING.
> 2017-05-05 21:23:03.639  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-5]
> Creating producer client
> 2017-05-05 21:23:03.641  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-5]
> Creating consumer client
> 2017-05-05 21:23:03.644  INFO 71 --- [   main]
> o.a.k.s.p.internals.StreamThread : stream-thread [StreamThread-5]
> Creating restore consumer client
>