Re: Consulting ReadOnlyKeyValueStore from Processor can lead to deadlock

2018-06-05 Thread Matthias J. Sax
Created a Jira for each: - https://issues.apache.org/jira/browse/KAFKA-6998 - https://issues.apache.org/jira/browse/KAFKA-6999 -Matthias On 5/11/18 10:06 AM, Guozhang Wang wrote: > Hello Steven, thanks for pointing it out. I think both of the mentioned > issues worth be improving: > > 1. The

Re: Consulting ReadOnlyKeyValueStore from Processor can lead to deadlock

2018-05-11 Thread Guozhang Wang
Hello Steven, thanks for pointing it out. I think both of the mentioned issues worth be improving: 1. The read-write lock documentation for caching enabled stores. 2. When CACHE_MAX_BYTES_BUFFERING_CONFIG is set to 0, we should automatically disable the dummy caching layer in all stores as it is n

Re: Consulting ReadOnlyKeyValueStore from Processor can lead to deadlock

2018-05-10 Thread Ted Yu
bq. the docs and CachingKVS behavior could improve I would agree. Pointing out the usage of ReadWriteLock and mentioning the withCachingDisabled() method in doc would help other developers. On Thu, May 10, 2018 at 11:21 AM, Steven Schlansker < sschlans...@opentable.com> wrote: > > > On May 10,

Re: Consulting ReadOnlyKeyValueStore from Processor can lead to deadlock

2018-05-10 Thread Steven Schlansker
> On May 10, 2018, at 10:48 AM, Steven Schlansker > wrote: > > But it still remains -- when you go an read that ROKVS documentation, it sure > doesn't prepare you to this possibility! And, it's a little frustrating that > we have to have this 'caching' layer at all -- we already had to add >

Consulting ReadOnlyKeyValueStore from Processor can lead to deadlock

2018-05-10 Thread Steven Schlansker
Hello again fellow Kafkans, Yesterday we observed a production deadlock take down one of our instances. Upon digging, it's clear that our usage of Kafka is the proximate cause, but the danger of our approach is not clear at all just from the Javadocs. We have stream processors that read off an in