You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "NateAdere (via GitHub)" <gi...@apache.org> on 2023/01/31 22:00:35 UTC

[GitHub] [cassandra] NateAdere opened a new pull request, #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

NateAdere opened a new pull request, #2122:
URL: https://github.com/apache/cassandra/pull/2122

   Used HeapByteBuffers and ThreadLocal to allocate a ByteBuffer for every thread. 
    * Reduced 20% memory allocation when doing mostly memory based queries
    * Reduced 4% memory allocation when doing disk-based queries
     
   ```
   patch by Natnael Adere; reviewed by <Reviewers> for CASSANDRA-#####
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/browse/CASSANDRA-18212)
   
   


-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1096199243


##########
test/unit/org/apache/cassandra/transport/DuplicateHeapBufferTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class DuplicateHeapBufferTest

Review Comment:
   spoke already, we should remove as you moved away from heap buffer copying



##########
src/java/org/apache/cassandra/utils/memory/MemoryUtil.java:
##########
@@ -34,19 +34,20 @@
 
     private static final Unsafe unsafe;
     private static final Class<?> DIRECT_BYTE_BUFFER_CLASS, RO_DIRECT_BYTE_BUFFER_CLASS;
-    private static final long DIRECT_BYTE_BUFFER_ADDRESS_OFFSET;

Review Comment:
   can you revert all changes?  as you migrated away from the heap buffer duplicate logic this logic is no longer needed to be changed



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -466,7 +475,7 @@ public static void writeValue(byte[] bytes, ByteBuf cb)
         cb.writeBytes(bytes);
     }
 
-    public static void writeValue(ByteBuffer bytes, ByteBuf cb)
+    public static void writeValu(ByteBuffer bytes, ByteBuf cb)

Review Comment:
   please revert



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -69,6 +69,15 @@ protected CharsetDecoder initialValue()
         }
     };
 
+    private final static FastThreadLocal<ByteBuffer> localBuffer = new FastThreadLocal<ByteBuffer>()

Review Comment:
   would be best to call it `localDirectBuffer`



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,37 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+        {
+            cb.writeBytes(MemoryUtil.duplicateHeapByteBuffer(bytes, getLocalBuffer()));
+        }
+    }
+
+    public static void writeValue(ByteBuffer src, ByteBuf dest)

Review Comment:
   should not copy/paste an existing method...
   
   You "should" add a new method that can write a `ByteBuffer` to a `ByteBuf` without allocating, then have the one line `cb.writeBytes(bytes.duplicate());` call that method instead.



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,37 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+        {
+            cb.writeBytes(MemoryUtil.duplicateHeapByteBuffer(bytes, getLocalBuffer()));
+        }
+    }
+
+    public static void writeValue(ByteBuffer src, ByteBuf dest)
+    {
+        if (src == null)
+        {
+            dest.writeInt(-1);
+            return;
+        }
+
+        int length = src.remaining();
+        dest.writeInt(length);
+
+        if (src.hasArray())
+        {
+            byte[] array = src.array();

Review Comment:
   can you add a comment explaining why we do this when direct duplicates the buffer?  You found a JDK bug, we should expose that to anyone reading so they know that there may be dragons to replicate



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -614,4 +653,9 @@ private static byte[] readRawBytes(ByteBuf cb, int length)
         return bytes;
     }
 
+    public static ByteBuffer getLocalBuffer()

Review Comment:
   also can you rename to `getLocalDirectBuffer`?  so callers are more clear 



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -614,4 +653,9 @@ private static byte[] readRawBytes(ByteBuf cb, int length)
         return bytes;
     }
 
+    public static ByteBuffer getLocalBuffer()

Review Comment:
   I would leave private for now



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] michaelsembwever commented on pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever commented on PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#issuecomment-1651289717

   Committed with https://github.com/apache/cassandra/commit/35f8da66f9b05d18b4177f8d2e1b86c772ad2221


-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1093540327


##########
test/unit/org/apache/cassandra/transport/DuplicateHeapBufferTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class DuplicateHeapBufferTest
+{
+    private static final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT;
+    private ByteBuf buf;
+
+    @Test
+    public void heapByteBuffer()
+    {
+        heapTest(50, false);
+    }
+    @Test
+    public void readOnlyHeapByteBuffer()
+    {
+        heapTest(50, true);
+    }
+
+    @Test
+    public void directByteBuffer()
+    {
+        directTest(50, false);
+    }
+    @Test
+    public void readOnlyDirectByteBuffer()
+    {
+        directTest(50, true);
+    }
+
+    @Test
+    public void CBUtilWriteValueTest()
+    {
+        ByteBuffer bb = ByteBuffer.allocate(50);
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        CBUtil.writeValue(bb, buf);
+        Assert.assertEquals(0, buf.readerIndex());
+        Assert.assertEquals(bb, CBUtil.readValue(buf));
+        Assert.assertEquals(buf.writerIndex(), buf.readerIndex());
+    }
+
+    private void directTest(int capacity, boolean readOnly)
+    {
+        ByteBuffer hollowBuffer = CBUtil.getLocalBuffer();
+        ByteBuffer bb = readOnly ? ByteBuffer.allocateDirect(capacity).asReadOnlyBuffer() : ByteBuffer.allocateDirect(capacity);
+
+        int size = bb.capacity();
+        buf = allocator.directBuffer(size);
+
+        try {
+            MemoryUtil.duplicateHeapByteBuffer(bb, hollowBuffer);
+        }
+        catch (Error e) {
+            Assert.assertEquals(new AssertionError().getClass(), e.getClass());

Review Comment:
   why do you need to create an instance of AssertionError to call getClass() on it? Does not `AssertionError.class.getClass()` do the job?



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1093536670


##########
test/unit/org/apache/cassandra/transport/DuplicateHeapBufferTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class DuplicateHeapBufferTest
+{
+    private static final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT;
+    private ByteBuf buf;
+
+    @Test
+    public void heapByteBuffer()
+    {
+        heapTest(50, false);
+    }
+    @Test
+    public void readOnlyHeapByteBuffer()
+    {
+        heapTest(50, true);
+    }
+
+    @Test
+    public void directByteBuffer()
+    {
+        directTest(50, false);
+    }
+    @Test
+    public void readOnlyDirectByteBuffer()
+    {
+        directTest(50, true);
+    }
+
+    @Test
+    public void CBUtilWriteValueTest()
+    {
+        ByteBuffer bb = ByteBuffer.allocate(50);
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        CBUtil.writeValue(bb, buf);
+        Assert.assertEquals(0, buf.readerIndex());
+        Assert.assertEquals(bb, CBUtil.readValue(buf));
+        Assert.assertEquals(buf.writerIndex(), buf.readerIndex());
+    }
+
+    private void directTest(int capacity, boolean readOnly)
+    {
+        ByteBuffer hollowBuffer = CBUtil.getLocalBuffer();
+        ByteBuffer bb = readOnly ? ByteBuffer.allocateDirect(capacity).asReadOnlyBuffer() : ByteBuffer.allocateDirect(capacity);
+
+        int size = bb.capacity();
+        buf = allocator.directBuffer(size);
+
+        try {

Review Comment:
   braces on new line in this whole try catch



##########
test/unit/org/apache/cassandra/transport/DuplicateHeapBufferTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class DuplicateHeapBufferTest
+{
+    private static final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT;
+    private ByteBuf buf;
+
+    @Test
+    public void heapByteBuffer()
+    {
+        heapTest(50, false);
+    }
+    @Test
+    public void readOnlyHeapByteBuffer()
+    {
+        heapTest(50, true);
+    }
+
+    @Test
+    public void directByteBuffer()
+    {
+        directTest(50, false);
+    }
+    @Test
+    public void readOnlyDirectByteBuffer()
+    {
+        directTest(50, true);
+    }
+
+    @Test
+    public void CBUtilWriteValueTest()
+    {
+        ByteBuffer bb = ByteBuffer.allocate(50);
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        CBUtil.writeValue(bb, buf);
+        Assert.assertEquals(0, buf.readerIndex());
+        Assert.assertEquals(bb, CBUtil.readValue(buf));
+        Assert.assertEquals(buf.writerIndex(), buf.readerIndex());
+    }
+
+    private void directTest(int capacity, boolean readOnly)
+    {
+        ByteBuffer hollowBuffer = CBUtil.getLocalBuffer();
+        ByteBuffer bb = readOnly ? ByteBuffer.allocateDirect(capacity).asReadOnlyBuffer() : ByteBuffer.allocateDirect(capacity);
+
+        int size = bb.capacity();
+        buf = allocator.directBuffer(size);
+
+        try {
+            MemoryUtil.duplicateHeapByteBuffer(bb, hollowBuffer);
+        }
+        catch (Error e) {
+            Assert.assertEquals(new AssertionError().getClass(), e.getClass());
+        }
+    }
+
+    private void heapTest(int capacity, boolean readOnly)
+    {
+        ByteBuffer hollowBuffer = CBUtil.getLocalBuffer();
+        ByteBuffer bb = readOnly ? ByteBuffer.allocate(capacity).asReadOnlyBuffer() : ByteBuffer.allocate(capacity);
+
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        try {

Review Comment:
   same



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1114799706


##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,37 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+        {
+            cb.writeBytes(MemoryUtil.duplicateHeapByteBuffer(bytes, getLocalBuffer()));
+        }
+    }
+
+    public static void writeValue(ByteBuffer src, ByteBuf dest)
+    {
+        if (src == null)
+        {
+            dest.writeInt(-1);
+            return;
+        }
+
+        int length = src.remaining();
+        dest.writeInt(length);
+
+        if (src.hasArray())
+        {
+            byte[] array = src.array();

Review Comment:
   can you flesh this out more?  Scott showed you the bug.
   
   it is also not an "improper" way, it was a bug in CMS



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1093542198


##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -46,7 +46,7 @@
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.TimeUUID;
 import org.apache.cassandra.utils.UUIDGen;
-
+import org.apache.cassandra.utils.memory.MemoryUtil;

Review Comment:
   should not be there some free line after imports and before the beginning of comments? This feels strange.



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -614,4 +623,9 @@ private static byte[] readRawBytes(ByteBuf cb, int length)
         return bytes;
     }
 
+    public static ByteBuffer getLocalBuffer()
+    {
+        return localBuffer.get();
+    }
+

Review Comment:
   empty line probably redundant?



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1114807288


##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,39 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            return handleCases(cases, rnd, offset, size);
         };
+    };
+
+    private enum BBCases { HEAP, READ_ONLY_HEAP, DIRECT, READ_ONLY_DIRECT }
+
+    private static ByteBuffer handleCases(Gen<BBCases> cases, RandomnessSource rnd, int offset, int size) {
+        switch (cases.generate(rnd))
+        {
+            case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+            case DIRECT:
+            {
+                ByteBuffer bb = directBufferFromSharedBlob(offset, size);
+                return bb;
+            }
+            case READ_ONLY_DIRECT:
+            {
+                ByteBuffer bb = directBufferFromSharedBlob(offset, size);
+                return bb.asReadOnlyBuffer();
+            }
+            default: throw new AssertionError("cann't wait for jdk 17!");

Review Comment:
   fix typo plz



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1100749972


##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -46,7 +46,7 @@
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.TimeUUID;
 import org.apache.cassandra.utils.UUIDGen;
-
+import org.apache.cassandra.utils.memory.MemoryUtil;

Review Comment:
   @NateAdere  why was this comment resolved?



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] NateAdere commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "NateAdere (via GitHub)" <gi...@apache.org>.
NateAdere commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1108944068


##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,36 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            return handleCases(cases, rnd, offset, size);
         };
+    };
+
+    private enum BBCases { HEAP, READ_ONLY_HEAP, DIRECT, READ_ONLY_DIRECT }
+
+    private static ByteBuffer handleCases(Gen<BBCases> cases, RandomnessSource rnd, int offset, int size) {
+        switch (cases.generate(rnd))
+        {
+            case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+            case DIRECT:
+            {
+                ByteBuffer bb = ByteBuffer.allocateDirect(size);
+                bb.put(LazySharedBlob.SHARED_BYTES, offset, size);
+                bb.flip();
+                return bb;

Review Comment:
   the new handleCases helps with readability so my plan is to keep that method and add "ByteBuffer bb = directBufferFromSharedBlob(offset, size);" to address the copy/paste code. Any opinions on removing or keeping the method?



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1100751739


##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,36 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            return handleCases(cases, rnd, offset, size);
         };
+    };
+
+    private enum BBCases { HEAP, READ_ONLY_HEAP, DIRECT, READ_ONLY_DIRECT }
+
+    private static ByteBuffer handleCases(Gen<BBCases> cases, RandomnessSource rnd, int offset, int size) {
+        switch (cases.generate(rnd))
+        {
+            case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+            case DIRECT:
+            {
+                ByteBuffer bb = ByteBuffer.allocateDirect(size);
+                bb.put(LazySharedBlob.SHARED_BYTES, offset, size);
+                bb.flip();
+                return bb;

Review Comment:
   when I said you should avoid copy/paste and make this a method, I mean 
   
   ```
   ByteBuffer bb = ByteBuffer.allocateDirect(size);
                    bb.put(LazySharedBlob.SHARED_BYTES, offset, size);
                    bb.flip();
   ```
   
   and not this new `handleCases`...



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,32 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+            addBytes(bytes, cb);
+    }
+
+    public static void addBytes(ByteBuffer src, ByteBuf dest)
+    {
+        if (src.remaining() == 0)
+            return;
+
+        int length = src.remaining();
+
+        //heap buffers are copied this way in order to avoid JVM crashing

Review Comment:
   please give more detail, this isn't enough for anyone to understand why this code is this way.  You should also move it within the `src.hasArray` block as that comment only makes sense there.



##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,36 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            return handleCases(cases, rnd, offset, size);
         };
+    };
+
+    private enum BBCases { HEAP, READ_ONLY_HEAP, DIRECT, READ_ONLY_DIRECT }
+
+    private static ByteBuffer handleCases(Gen<BBCases> cases, RandomnessSource rnd, int offset, int size) {
+        switch (cases.generate(rnd))
+        {
+            case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+            case DIRECT:
+            {
+                ByteBuffer bb = ByteBuffer.allocateDirect(size);
+                bb.put(LazySharedBlob.SHARED_BYTES, offset, size);
+                bb.flip();
+                return bb;
+            }
+            case READ_ONLY_DIRECT:
+            {
+                ByteBuffer bb = ByteBuffer.allocateDirect(size);
+                bb.put(LazySharedBlob.SHARED_BYTES, offset, size);
+                bb.flip();
+                return bb.asReadOnlyBuffer();
+            }
+            default: throw new AssertionError("cann't wait for jdk 17!");
+        }
     }
 
-    /**
+     /**

Review Comment:
   please revert



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1114804474


##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,39 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            return handleCases(cases, rnd, offset, size);
         };
+    };
+
+    private enum BBCases { HEAP, READ_ONLY_HEAP, DIRECT, READ_ONLY_DIRECT }
+
+    private static ByteBuffer handleCases(Gen<BBCases> cases, RandomnessSource rnd, int offset, int size) {
+        switch (cases.generate(rnd))
+        {
+            case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+            case DIRECT:
+            {
+                ByteBuffer bb = directBufferFromSharedBlob(offset, size);
+                return bb;
+            }

Review Comment:
   nit: cleanup, you can just do `return directBufferFromSharedBlob(offset, size);` which lets you get rid of 3 lines of code



##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,39 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            return handleCases(cases, rnd, offset, size);
         };
+    };
+
+    private enum BBCases { HEAP, READ_ONLY_HEAP, DIRECT, READ_ONLY_DIRECT }
+
+    private static ByteBuffer handleCases(Gen<BBCases> cases, RandomnessSource rnd, int offset, int size) {
+        switch (cases.generate(rnd))
+        {
+            case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+            case DIRECT:
+            {
+                ByteBuffer bb = directBufferFromSharedBlob(offset, size);
+                return bb;
+            }
+            case READ_ONLY_DIRECT:
+            {
+                ByteBuffer bb = directBufferFromSharedBlob(offset, size);
+                return bb.asReadOnlyBuffer();

Review Comment:
   nit: cleanup, you can just do `return directBufferFromSharedBlob(offset, size);` which lets you get rid of 3 lines of code



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1093539060


##########
test/unit/org/apache/cassandra/transport/DuplicateHeapBufferTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class DuplicateHeapBufferTest
+{
+    private static final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT;
+    private ByteBuf buf;
+
+    @Test
+    public void heapByteBuffer()
+    {
+        heapTest(50, false);
+    }
+    @Test
+    public void readOnlyHeapByteBuffer()
+    {
+        heapTest(50, true);
+    }
+
+    @Test
+    public void directByteBuffer()
+    {
+        directTest(50, false);
+    }
+    @Test
+    public void readOnlyDirectByteBuffer()
+    {
+        directTest(50, true);
+    }
+
+    @Test
+    public void CBUtilWriteValueTest()
+    {
+        ByteBuffer bb = ByteBuffer.allocate(50);
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        CBUtil.writeValue(bb, buf);
+        Assert.assertEquals(0, buf.readerIndex());

Review Comment:
   cant you import it statically? saves some time to read it 



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1093541170


##########
test/unit/org/apache/cassandra/transport/DuplicateHeapBufferTest.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import java.nio.ByteBuffer;
+import java.nio.ReadOnlyBufferException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class DuplicateHeapBufferTest
+{
+    private static final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT;
+    private ByteBuf buf;
+
+    @Test
+    public void heapByteBuffer()
+    {
+        heapTest(50, false);
+    }
+    @Test
+    public void readOnlyHeapByteBuffer()
+    {
+        heapTest(50, true);
+    }
+
+    @Test
+    public void directByteBuffer()
+    {
+        directTest(50, false);
+    }
+    @Test
+    public void readOnlyDirectByteBuffer()
+    {
+        directTest(50, true);
+    }
+
+    @Test
+    public void CBUtilWriteValueTest()
+    {
+        ByteBuffer bb = ByteBuffer.allocate(50);
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        CBUtil.writeValue(bb, buf);
+        Assert.assertEquals(0, buf.readerIndex());
+        Assert.assertEquals(bb, CBUtil.readValue(buf));
+        Assert.assertEquals(buf.writerIndex(), buf.readerIndex());
+    }
+
+    private void directTest(int capacity, boolean readOnly)
+    {
+        ByteBuffer hollowBuffer = CBUtil.getLocalBuffer();
+        ByteBuffer bb = readOnly ? ByteBuffer.allocateDirect(capacity).asReadOnlyBuffer() : ByteBuffer.allocateDirect(capacity);
+
+        int size = bb.capacity();
+        buf = allocator.directBuffer(size);
+
+        try {
+            MemoryUtil.duplicateHeapByteBuffer(bb, hollowBuffer);
+        }
+        catch (Error e) {
+            Assert.assertEquals(new AssertionError().getClass(), e.getClass());
+        }
+    }
+
+    private void heapTest(int capacity, boolean readOnly)
+    {
+        ByteBuffer hollowBuffer = CBUtil.getLocalBuffer();
+        ByteBuffer bb = readOnly ? ByteBuffer.allocate(capacity).asReadOnlyBuffer() : ByteBuffer.allocate(capacity);
+
+        int size = bb.capacity();
+        buf = allocator.heapBuffer(size);
+
+        try {
+            ByteBuffer temp = MemoryUtil.duplicateHeapByteBuffer(bb, hollowBuffer);
+            Assert.assertEquals(bb.position(), temp.position());
+            Assert.assertEquals(bb.limit(), temp.limit());
+            Assert.assertEquals(bb.capacity(), temp.capacity());
+            Assert.assertEquals(bb.array(), temp.array());
+        }
+        catch (Exception e){
+            Assert.assertEquals(new ReadOnlyBufferException().getClass(), e.getClass());

Review Comment:
   `ReadOnlyBufferException.class.getClass()` maybe?



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1099011365


##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,35 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+        {

Review Comment:
   remove the `{}`



##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -299,6 +299,16 @@ private static char[] createDNSDomainPartDomain()
     }
 
     public static Gen<ByteBuffer> bytes(int min, int max)
+    {
+        return bytes(min, max, SourceDSL.arbitrary().constant(BBCases.HEAP));
+    }
+
+    public static Gen<ByteBuffer> bytesAnyType(int min, int max)
+    {
+        return bytes(min, max, SourceDSL.arbitrary().enumValues(BBCases.class));
+    }
+
+    public static Gen<ByteBuffer> bytes(int min, int max, Gen<BBCases> cases)

Review Comment:
   have to make this private, or fix `BBCases`.  You can't have a public method expose a private class (the build should be failing right now...).
   
   If you choose to keep this method (I would not) then you should rename `BBCases` to be more clear to readers...



##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,35 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+        {
+            addBytes(bytes, cb);
+        }
+    }
+
+    public static void addBytes(ByteBuffer src, ByteBuf dest)
+    {
+        if (src == null)

Review Comment:
   why the null check?  at least in the single usage this doesn't allow null... I prefer if we remove and allow the NPE if a user provides null



##########
test/unit/org/apache/cassandra/transport/WriteBytesDuplicateTest.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.transport;
+
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.cassandra.utils.Generators;
+import org.assertj.core.api.Assertions;
+
+import static org.quicktheories.QuickTheory.qt;
+
+
+public class WriteBytesDuplicateTest

Review Comment:
   why `Duplicate`?  You are "writing"



##########
test/unit/org/apache/cassandra/utils/Generators.java:
##########
@@ -314,11 +324,32 @@ public static Gen<ByteBuffer> bytes(int min, int max)
             // to add more randomness, also shift offset in the array so the same size doesn't yield the same bytes
             int offset = (int) rnd.next(Constraint.between(0, MAX_BLOB_LENGTH - size));
 
-            return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+            switch (cases.generate(rnd))
+            {
+                case HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size);
+                case READ_ONLY_HEAP: return ByteBuffer.wrap(LazySharedBlob.SHARED_BYTES, offset, size).asReadOnlyBuffer();
+                case DIRECT:
+                {
+                    ByteBuffer bb = ByteBuffer.allocateDirect(size);
+                    bb.put(LazySharedBlob.SHARED_BYTES, offset, size);
+                    bb.flip();
+                    return bb;

Review Comment:
   can you make this a function?  best not to copy/paste



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] NateAdere commented on a diff in pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "NateAdere (via GitHub)" <gi...@apache.org>.
NateAdere commented on code in PR #2122:
URL: https://github.com/apache/cassandra/pull/2122#discussion_r1108937497


##########
src/java/org/apache/cassandra/transport/CBUtil.java:
##########
@@ -478,7 +487,37 @@ public static void writeValue(ByteBuffer bytes, ByteBuf cb)
         cb.writeInt(remaining);
 
         if (remaining > 0)
-            cb.writeBytes(bytes.duplicate());
+        {
+            cb.writeBytes(MemoryUtil.duplicateHeapByteBuffer(bytes, getLocalBuffer()));
+        }
+    }
+
+    public static void writeValue(ByteBuffer src, ByteBuf dest)
+    {
+        if (src == null)
+        {
+            dest.writeInt(-1);
+            return;
+        }
+
+        int length = src.remaining();
+        dest.writeInt(length);
+
+        if (src.hasArray())
+        {
+            byte[] array = src.array();

Review Comment:
   Is this comment descriptive enough?



-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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


[GitHub] [cassandra] michaelsembwever closed pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue

Posted by "michaelsembwever (via GitHub)" <gi...@apache.org>.
michaelsembwever closed pull request #2122: CASSANDRA-18212: Reduce memory allocations of calls to ByteBufer.duplicate() made in org.apache.cassandra.transport.CBUtil#writeValue
URL: https://github.com/apache/cassandra/pull/2122


-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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