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

ASF GitHub Bot commented on KAFKA-6999:
---------------------------------------

guozhangwang closed pull request #5351: KAFKA-6999: Document read-write lock 
usage of caching enabled stores
URL: https://github.com/apache/kafka/pull/5351
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java
index 06329802d3c..8725ebc35cb 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyKeyValueStore.java
@@ -23,6 +23,12 @@
  * Implementations should be thread-safe as concurrent reads and writes
  * are expected.
  *
+ * Please note that this contract defines the thread-safe read functionality 
only; it does not
+ * guarantee anything about whether the actual instance is writable by another 
thread, or
+ * whether it uses some locking mechanism under the hood. For this reason, 
making dependencies
+ * between the read and write operations on different StateStore instances can 
cause concurrency
+ * problems like deadlock.
+ *
  * @param <K> the key type
  * @param <V> the value type
  */


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Document read-write lock usage of caching enabled stores
> --------------------------------------------------------
>
>                 Key: KAFKA-6999
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6999
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>            Reporter: Matthias J. Sax
>            Priority: Minor
>
> From the mailing list
> {quote}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 incoming KStream, possibly 
> cross-reference some data from an auxiliary table via 
> ReadOnlyKeyValueStore.get()
>  
> This is done via custom logic rather than a direct KTable join because which 
> index is consulted may change depending on the shape of incoming data.
>  
> The ROKVS docs state,
>  * A key value store that only supports read operations.
>  * Implementations should be thread-safe as concurrent reads and writes
>  * are expected.
>  
> They do **not** indicate that the CachingKVS layer uses a ReadWriteLock. If 
> you have one CachingKVS flush a record cause a read from another CKVS, you 
> are suddenly vulnerable to classic lock order reversals! Surprise!
>  
> A partial stack trace highlighting the problem, with many uninteresting 
> frames removed, is inline at the bottom of this mail.
>  
> You could probably rightly point to us allowing a "observer" pattern to 
> callback from within streams processing code is dangerous. We might move this 
> off to an auxiliary thread to alleviate this problem.
>  
> But it still remains -- when you go an read that ROKVS documentation, it sure 
> doesn't prepare you to this possibility!
>  {quote}



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

Reply via email to