[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-07-10 Thread Ashish Surana (JIRA)


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

Ashish Surana commented on KAFKA-6976:
--

Created below ticket to address this issue the right way:

https://issues.apache.org/jira/browse/KAFKA-7149: Reduce assignment data size 
to improve kafka streams scalability

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
> Attachments: kafkaStreamsDeadState.log
>
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-06-12 Thread Matthias J. Sax (JIRA)


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

Matthias J. Sax commented on KAFKA-6976:


Glad it works not. Can you do a PR to update the docs?

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
> Attachments: kafkaStreamsDeadState.log
>
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-06-12 Thread Ashish Surana (JIRA)


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

Ashish Surana commented on KAFKA-6976:
--

The issue is resolved after making some changes to broker settings:

*message.max.bytes=33554432*

*replica.fetch.max.bytes=33554432*

Here these are set to 32MB, but earlier these settings were set to 2MB for us. 
Like you explained, broker saves this assignment as one single message in 
__consumer_offset topics so the assignment size is also limited by these two 
message settings on kafka broker.

Assignment size is directly proportional to number of tasks (we observed that 
assignment roughly contains 11KB for each task). We had 512 partitions and 1 
replica, so entire assignment size is 11KB * 512(partitions) * 2(1active + 
1replica task)  ~= 12MB.

 

I think it would be a good idea to add this into documentation somewhere that 
assignment data size (or active tasks + replica tasks) is indirectly limited by 
these settings on the broker side.

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
> Attachments: kafkaStreamsDeadState.log
>
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-06-10 Thread Matthias J. Sax (JIRA)


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

Matthias J. Sax commented on KAFKA-6976:


Just cycling back to this (also saw, that you replied to KAFKA-6631). Seems 
that both are related.

I had a look into the logs, and Streams dies when the leader tries to send a 
sync-group request, ie, when it sends the computed assignment to the broker. 
The broker stores this assignment in the __consumer_offset topics and seems to 
fail with RecordTooLargeException.

Note, that `104857600` is 100MB (this comment is with regard to your comment on 
KAFKA-6631). Seems to be some network buffer you exceed – I am not very 
familiar with broker code, so I can't tell atm. Unclear to me what `1195725856` 
is – this is indeed larger than 1GB.

Have you tried to increase the message size and after that you get the second 
error? Or did the second error also occur originally?

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
> Attachments: kafkaStreamsDeadState.log
>
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-06-06 Thread Ashish Surana (JIRA)


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

Ashish Surana commented on KAFKA-6976:
--

One more broker side error:

2018-06-06 07:41:28,522 
[kafka-network-thread-1789583625-ListenerName(PLAINTEXT)-PLAINTEXT-2] WARN 
(Selector.java:377) - Unexpected error from /172.30.127.177; closing connection
org.apache.kafka.common.network.InvalidReceiveException: Invalid receive (size 
= 1195725856 larger than 104857600)
 at 
org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:91)
 at 
org.apache.kafka.common.network.NetworkReceive.readFrom(NetworkReceive.java:71)
 at org.apache.kafka.common.network.KafkaChannel.receive(KafkaChannel.java:169)
 at org.apache.kafka.common.network.KafkaChannel.read(KafkaChannel.java:150)
 at 
org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:355)
 at org.apache.kafka.common.network.Selector.poll(Selector.java:303)
 at kafka.network.Processor.poll(SocketServer.scala:494)
 at kafka.network.Processor.run(SocketServer.scala:432)
 at java.lang.Thread.run(Thread.java:745)

 

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
> Attachments: kafkaStreamsDeadState.log
>
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-06-06 Thread Ashish Surana (JIRA)


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

Ashish Surana commented on KAFKA-6976:
--

Observed some errors on broker side:

(a)

2018-06-06 05:07:56,179 [kafka-request-handler-1] ERROR (Logging.scala:99) - 
[Group Metadata Manager on Broker 1789583625]: Appending metadata message for 
group 2j2 generation 4 failed due to 
org.apache.kafka.common.errors.RecordTooLargeException, returning UNKNOWN error 
code to the client

(b)

2018-06-06 07:13:31,327 [kafka-request-handler-0] ERROR (Logging.scala:99) - 
[Group Metadata Manager on Broker 1789583625]: Appending metadata message for 
group 5j3 generation 12 failed due to 
org.apache.kafka.common.errors.RecordTooLargeException, returning UNKNOWN error 
code to the client

 

Observed client side error caught in UncaughtExceptionHandler

(a)

Unexpected error from SyncGroup: The server experienced an unexpected error 
when processing the request

 

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
> Attachments: kafkaStreamsDeadState.log
>
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-06-03 Thread Deepak Goyal (JIRA)


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

Deepak Goyal commented on KAFKA-6976:
-

[^kafkaStreamsDeadState.log] 
The logs attached are DEBUG level. At line number 64017, you'll be able to see 
the client going into dead state. Also, please note that logs are from an 
instance which served as an application leader for the Kafka-Streams.

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-05-31 Thread Matthias J. Sax (JIRA)


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

Matthias J. Sax commented on KAFKA-6976:


Thanks for reporting this: can you share the full logs at DEBUG level?

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6976) Kafka Streams instances going in to DEAD state

2018-05-31 Thread Deepak Goyal (JIRA)


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

Deepak Goyal commented on KAFKA-6976:
-

Furthermore, in the course of our application development, our application need 
consuming from four topics (each with 256 partitions). In this scenario, we 
could achieve a working cluster with only 60 machines. We have an approximate 
need of 400 machines in our cluster.

> Kafka Streams instances going in to DEAD state
> --
>
> Key: KAFKA-6976
> URL: https://issues.apache.org/jira/browse/KAFKA-6976
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Deepak Goyal
>Priority: Blocker
>
> We are using Kafka 0.10.2.0, Kafka-Streams 1.1.0. We have Kafka Cluster of 16 
> machines, and topic that is being consumed by Kafka Streams has 256 
> partitions. We spawned 400 machines of Kakfa Streams application. We see that 
> all of the StreamThreads go in to DEAD state.
> {quote}{{[2018-05-25 05:59:29,282] INFO stream-thread 
> [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] State transition 
> from PENDING_SHUTDOWN to DEAD 
> (org.apache.kafka.streams.processor.internals.StreamThread) [2018-05-25 
> 05:59:29,282] INFO stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] 
> State transition from REBALANCING to ERROR 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] WARN 
> stream-client [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7] All stream threads 
> have died. The instance will be in error state and should be closed. 
> (org.apache.kafka.streams.KafkaStreams) [2018-05-25 05:59:29,282] INFO 
> stream-thread [ksapp-19f923d7-5f9e-4137-b79f-ee20945a7dd7-StreamThread-1] 
> Shutdown complete 
> (org.apache.kafka.streams.processor.internals.StreamThread)}}
> {quote}
> Please note that when we only have 100 kafka-streams application machines, 
> things are working as expected. We see that instances are consuming messages 
> from topic.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)