Hi Andreas,

Thanks for reporting. This sounds like a bug, but could also be a semantic 
thing. Couple of questions:

- which version of Kafka are you using?
- what is the nature of the failure of the threads, e.g., how come they have 
all crashed? If all threads crash, was there an exception they threw and it was 
caught? If all threads have crashed, would it be useful to still have the Kafka 
Streams instance running at all (e.g., I'd expect it to also crash or be 
terminated, in which case I don't see a value in providing a state()).

Eno

> On 16 May 2017, at 08:03, Andreas Gabrielsson 
> <andreas.gabriels...@klarna.com> wrote:
> 
> Hi All,
> 
> We recently implemented a health check for a Kafka Streams based application. 
> The health check is simply checking the state of Kafka Streams by calling 
> KafkaStreams.state(). It reports healthy if it’s not in PENDING_SHUTDOWN or 
> NOT_RUNNING states. 
> 
> We truly appreciate having the possibility to easily check the state of Kafka 
> Streams but to our surprise we noticed that KafkaStreams.state() returns 
> RUNNING even though all StreamThreads has crashed and reached NOT_RUNNING 
> state. Is this intended behaviour or is it a bug? Semantically it seems weird 
> to me that KafkaStreams would say it’s RUNNING when it is in fact not 
> consuming anything since all underlying working threads has crashed. 
> 
> If this is intended behaviour I would appreciate an explanation of why that 
> is the case. Also in that case, how could I determine if the consumption from 
> Kafka hasn’t crashed? 
> 
> If this is not intended behaviour, how fast could I expect it to be fixed? I 
> wouldn’t mind fixing it myself but I’m not sure if this is considered trivial 
> or big enough to require a JIRA. Also, if I would implement a fix I’d like 
> your input on what would be a reasonable solution. By just inspecting to code 
> I have an idea but I’m not sure I understand all the implication so I’d be 
> happy to hear your thoughts first. 
> 
> Thanks in advance,
> Andreas Gabrielsson

Reply via email to