You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/09/17 13:41:49 UTC

[GitHub] [cassandra] blerer commented on a change in pull request #737: CASSANDRA-15880 ThreadLocal removal on close() to prevent leaks

blerer commented on a change in pull request #737:
URL: https://github.com/apache/cassandra/pull/737#discussion_r490232796



##########
File path: src/java/org/apache/cassandra/db/commitlog/SimpleCachedBufferPool.java
##########
@@ -79,19 +91,20 @@ public ByteBuffer createBuffer(BufferType bufferType)
 
     public ByteBuffer getThreadLocalReusableBuffer(int size)
     {
-        ByteBuffer result = reusableBufferHolder.get();
-        if (result.capacity() < size || BufferType.typeOf(result) != preferredReusableBufferType)
+        ByteBuffer result = reusableBB.get();
+        if (result.capacity() < size)
         {
             FileUtils.clean(result);
             result = preferredReusableBufferType.allocate(size);
-            reusableBufferHolder.set(result);
+            reusableBB.set(result);
         }
         return result;
     }
 
     public void setPreferredReusableBufferType(BufferType type)
     {
         preferredReusableBufferType = type;
+        reusableBB = reusableBBHolder.get(preferredReusableBufferType);

Review comment:
       Should we not have `preferredReusableBufferType` and  `reusableBB` be `volatile` variables and the   `setPreferredReusableBufferType` method `synchronized` ? With the current implementation, it seems to me that, if a thread call `setPreferredReusableBufferType` and another `getThreadLocalReusableBuffer` the output of `getThreadLocalReusableBuffer` is unpredictable and the wrong buffer type can be added the `reusableBB` field.
   Am I missing something?

##########
File path: src/java/org/apache/cassandra/db/commitlog/SimpleCachedBufferPool.java
##########
@@ -107,6 +120,9 @@ public void releaseBuffer(ByteBuffer buffer)
     public void shutdown()
     {
         bufferPool.clear();
+        for (FastThreadLocal<ByteBuffer> bbHolder : reusableBBHolder.values())
+            bbHolder.remove();
+        reusableBB.remove();

Review comment:
       `remove` is called twice on the same `ThreadLocal` instance as the `reusableBB` is also in the `bbHolder` variable
   
   Moreover, the `remove` call will remove the buffers values associated to the thread calling the `shutdown` method and I do not believe that it is what we want. The goal of the approach is to reuse buffer. Calling `remove` will prevent the reuse on the next call forcing the buffer to be recreated.    

##########
File path: src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
##########
@@ -86,27 +87,21 @@ public Rebufferer instantiateRebufferer()
     public static class Standard extends CompressedChunkReader
     {
         // we read the raw compressed bytes into this buffer, then uncompressed them into the provided one.
-        private final ThreadLocal<ByteBuffer> compressedHolder;
+        private final SimpleCachedBufferPool reusableCompressBB;
+        private final int compressSize = getCompressSize();
 
         public Standard(ChannelProxy channel, CompressionMetadata metadata)
         {
             super(channel, metadata);
-            compressedHolder = ThreadLocal.withInitial(this::allocateBuffer);
+            reusableCompressBB = new SimpleCachedBufferPool(0, getCompressSize());

Review comment:
       We compute twice the compress size. I should do something like:
   ```
   compressSize = getCompressSize(metadata);
   reusableCompressBB = new SimpleCachedBufferPool(0, compressSize);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org