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

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

dguy closed pull request #4372: KAFKA-6412 Improve synchronization in 
CachingKeyValueStore methods
URL: https://github.com/apache/kafka/pull/4372
 
 
   

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/internals/CachingKeyValueStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
index f0669a4f6ee..9fff8ccca04 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
@@ -31,6 +31,9 @@
 
 import java.util.List;
 import java.util.Objects;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 class CachingKeyValueStore<K, V> extends WrappedStateStore.AbstractStateStore 
implements KeyValueStore<Bytes, byte[]>, CachedStateStore<K, V> {
 
@@ -44,6 +47,7 @@
     private InternalProcessorContext context;
     private StateSerdes<K, V> serdes;
     private Thread streamThread;
+    private ReadWriteLock lock = new ReentrantReadWriteLock();
 
     CachingKeyValueStore(final KeyValueStore<Bytes, byte[]> underlying,
                          final Serde<K> keySerde,
@@ -108,9 +112,14 @@ public void setFlushListener(final CacheFlushListener<K, 
V> flushListener,
     }
 
     @Override
-    public synchronized void flush() {
-        cache.flush(cacheName);
-        underlying.flush();
+    public void flush() {
+        lock.writeLock().lock();
+        try {
+            cache.flush(cacheName);
+            underlying.flush();
+        } finally {
+            lock.writeLock().unlock();
+        }
     }
 
     @Override
@@ -131,10 +140,21 @@ public boolean isOpen() {
     }
 
     @Override
-    public synchronized byte[] get(final Bytes key) {
+    public byte[] get(final Bytes key) {
         validateStoreOpen();
-        Objects.requireNonNull(key);
-        return getInternal(key);
+        Lock theLock;
+        if (Thread.currentThread().equals(streamThread)) {
+            theLock = lock.writeLock();
+        } else {
+            theLock = lock.readLock();
+        }
+        theLock.lock();
+        try {
+            Objects.requireNonNull(key);
+            return getInternal(key);
+        } finally {
+            theLock.unlock();
+        }
     }
 
     private byte[] getInternal(final Bytes key) {
@@ -176,50 +196,75 @@ public boolean isOpen() {
     }
 
     @Override
-    public synchronized long approximateNumEntries() {
+    public long approximateNumEntries() {
         validateStoreOpen();
-        return underlying.approximateNumEntries();
+        lock.readLock().lock();
+        try {
+            return underlying.approximateNumEntries();
+        } finally {
+            lock.readLock().unlock();
+        }
     }
 
     @Override
-    public synchronized void put(final Bytes key, final byte[] value) {
+    public void put(final Bytes key, final byte[] value) {
         Objects.requireNonNull(key, "key cannot be null");
         validateStoreOpen();
-        putInternal(key, value);
+        lock.writeLock().lock();
+        try {
+            putInternal(key, value);
+        } finally {
+            lock.writeLock().unlock();
+        }
     }
 
-    private synchronized void putInternal(final Bytes rawKey, final byte[] 
value) {
+    private void putInternal(final Bytes rawKey, final byte[] value) {
         Objects.requireNonNull(rawKey, "key cannot be null");
         cache.put(cacheName, rawKey, new LRUCacheEntry(value, true, 
context.offset(),
-                  context.timestamp(), context.partition(), context.topic()));
+              context.timestamp(), context.partition(), context.topic()));
     }
 
     @Override
-    public synchronized byte[] putIfAbsent(final Bytes key, final byte[] 
value) {
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
         Objects.requireNonNull(key, "key cannot be null");
         validateStoreOpen();
-        final byte[] v = getInternal(key);
-        if (v == null) {
-            putInternal(key, value);
+        lock.writeLock().lock();
+        try {
+            final byte[] v = getInternal(key);
+            if (v == null) {
+                putInternal(key, value);
+            }
+            return v;
+        } finally {
+            lock.writeLock().unlock();
         }
-        return v;
     }
 
     @Override
-    public synchronized void putAll(final List<KeyValue<Bytes, byte[]>> 
entries) {
-        for (KeyValue<Bytes, byte[]> entry : entries) {
-            put(entry.key, entry.value);
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        lock.writeLock().lock();
+        try {
+            for (KeyValue<Bytes, byte[]> entry : entries) {
+                put(entry.key, entry.value);
+            }
+        } finally {
+            lock.writeLock().unlock();
         }
     }
 
     @Override
-    public synchronized byte[] delete(final Bytes key) {
+    public byte[] delete(final Bytes key) {
         validateStoreOpen();
-        Objects.requireNonNull(key);
-        final byte[] v = getInternal(key);
-        cache.delete(cacheName, key);
-        underlying.delete(key);
-        return v;
+        lock.writeLock().lock();
+        try {
+            Objects.requireNonNull(key);
+            final byte[] v = getInternal(key);
+            cache.delete(cacheName, key);
+            underlying.delete(key);
+            return v;
+        } finally {
+            lock.writeLock().unlock();
+        }
     }
 
     KeyValueStore<Bytes, byte[]> underlying() {


 

----------------------------------------------------------------
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:
[email protected]


> Improve synchronization in CachingKeyValueStore methods
> -------------------------------------------------------
>
>                 Key: KAFKA-6412
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6412
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>            Reporter: Ted Yu
>             Fix For: 1.1.0
>
>         Attachments: 6412-jmh.v1.txt, k-6412.v1.txt
>
>
> Currently CachingKeyValueStore methods are synchronized at method level.
> It seems we can use read lock for getter and write lock for put / delete 
> methods.
> For getInternal(), if the underlying thread is streamThread, the 
> getInternal() may trigger eviction. This can be handled by obtaining write 
> lock at the beginning of the method for streamThread.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to