You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jb...@apache.org on 2021/03/27 17:04:18 UTC

[geode] 01/01: GEODE-9081: Remove transient allocation.

This is an automated email from the ASF dual-hosted git repository.

jbarrett pushed a commit to branch feature/GEODE-9081-BufferPool-allocations
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 5e933bb38c9b3dbcb7e82f46d1726442531ed564
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Sat Mar 27 10:02:35 2021 -0700

    GEODE-9081: Remove transient allocation.
    
    * New geode-unsafe access to non-SDK API for DirectBuffer.
    * Use geode-unsafe to access DirectBuffer.attachment().
    * Removes latency of reflection.
    * Removes transient Object[] allocation in Method.invoke().
---
 .../org/apache/geode/internal/net/BufferPool.java  | 60 ++++++++--------------
 .../unsafe/internal/sun/nio/ch/DirectBuffer.java   | 36 +++++++++++++
 .../internal/sun/nio/ch/DirectBufferTest.java      | 47 +++++++++++++++++
 3 files changed, 103 insertions(+), 40 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/BufferPool.java b/geode-core/src/main/java/org/apache/geode/internal/net/BufferPool.java
index 74a2e4d..ae1cbf2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/net/BufferPool.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/BufferPool.java
@@ -12,15 +12,15 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+
 package org.apache.geode.internal.net;
 
 import java.lang.ref.SoftReference;
-import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.IdentityHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import org.apache.logging.log4j.Logger;
+import org.apache.geode.unsafe.internal.sun.nio.ch.DirectBuffer;
 
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.annotations.VisibleForTesting;
@@ -28,14 +28,10 @@ import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.tcp.Connection;
-import org.apache.geode.logging.internal.log4j.api.LogService;
 import org.apache.geode.util.internal.GeodeGlossary;
 
 public class BufferPool {
   private final DMStats stats;
-  private static final Logger logger = LogService.getLogger();
-
-  private Method parentOfSliceMethod;
 
   /**
    * Buffers may be acquired from the Buffers pool
@@ -325,40 +321,24 @@ public class BufferPool {
    * If we hand out a buffer that is larger than the requested size we create a
    * "slice" of the buffer having the requested capacity and hand that out instead.
    * When we put the buffer back in the pool we need to find the original, non-sliced,
-   * buffer. This is held in DirectBuffer in its "attachment" field, which is a public
-   * method, though DirectBuffer is package-private. This method is visible for use
-   * in debugging and testing. For debugging, invoke this method if you need to see
-   * the non-sliced buffer for some reason, such as logging its hashcode.
+   * buffer. This is held in DirectBuffer in its "attachment" field.
+   *
+   * This method is visible for use in debugging and testing. For debugging, invoke this method if
+   * you need to see the non-sliced buffer for some reason, such as logging its hashcode.
    */
   @VisibleForTesting
-  public ByteBuffer getPoolableBuffer(ByteBuffer buffer) {
-    if (!buffer.isDirect()) {
+  ByteBuffer getPoolableBuffer(final ByteBuffer buffer) {
+    final Object attachment = DirectBuffer.attachment(buffer);
+
+    if (null == attachment) {
       return buffer;
     }
-    ByteBuffer result = buffer;
-    if (parentOfSliceMethod == null) {
-      Class clazz = buffer.getClass();
-      try {
-        Method method = clazz.getMethod("attachment");
-        method.setAccessible(true);
-        parentOfSliceMethod = method;
-      } catch (Exception e) {
-        throw new InternalGemFireException("unable to retrieve underlying byte buffer", e);
-      }
-    }
-    try {
-      Object attachment = parentOfSliceMethod.invoke(buffer);
-      if (attachment instanceof ByteBuffer) {
-        result = (ByteBuffer) attachment;
-      } else if (attachment != null) {
-        throw new InternalGemFireException(
-            "direct byte buffer attachment was not a byte buffer but a " +
-                attachment.getClass().getName());
-      }
-    } catch (Exception e) {
-      throw new InternalGemFireException("unable to retrieve underlying byte buffer", e);
+
+    if (attachment instanceof ByteBuffer) {
+      return (ByteBuffer) attachment;
     }
-    return result;
+
+    throw new InternalGemFireException("direct byte buffer attachment was not a byte buffer but a " + attachment.getClass().getName());
   }
 
   /**
@@ -383,22 +363,22 @@ public class BufferPool {
 
     BBSoftReference(ByteBuffer bb, boolean send) {
       super(bb);
-      this.size = bb.capacity();
+      size = bb.capacity();
       this.send = send;
     }
 
     public int getSize() {
-      return this.size;
+      return size;
     }
 
     synchronized int consumeSize() {
-      int result = this.size;
-      this.size = 0;
+      int result = size;
+      size = 0;
       return result;
     }
 
     public boolean getSend() {
-      return this.send;
+      return send;
     }
 
     public ByteBuffer getBB() {
diff --git a/geode-unsafe/src/main/java/org/apache/geode/unsafe/internal/sun/nio/ch/DirectBuffer.java b/geode-unsafe/src/main/java/org/apache/geode/unsafe/internal/sun/nio/ch/DirectBuffer.java
new file mode 100644
index 0000000..dc894cf
--- /dev/null
+++ b/geode-unsafe/src/main/java/org/apache/geode/unsafe/internal/sun/nio/ch/DirectBuffer.java
@@ -0,0 +1,36 @@
+/*
+ * 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.geode.unsafe.internal.sun.nio.ch;
+
+/**
+ * Provides access to methods on non-SDK class {@link sun.nio.ch.DirectBuffer}.
+ */
+public interface DirectBuffer {
+
+  /**
+   * @see sun.nio.ch.DirectBuffer#attachment()
+   * @param object to get attachment for
+   * @return returns attachment if object is {@link sun.nio.ch.DirectBuffer} otherwise null.
+   */
+  static Object attachment(final Object object) {
+    if (object instanceof sun.nio.ch.DirectBuffer) {
+      return ((sun.nio.ch.DirectBuffer) object).attachment();
+    }
+
+    return null;
+  }
+
+}
diff --git a/geode-unsafe/src/test/java/org/apache/geode/unsafe/internal/sun/nio/ch/DirectBufferTest.java b/geode-unsafe/src/test/java/org/apache/geode/unsafe/internal/sun/nio/ch/DirectBufferTest.java
new file mode 100644
index 0000000..6ef0799
--- /dev/null
+++ b/geode-unsafe/src/test/java/org/apache/geode/unsafe/internal/sun/nio/ch/DirectBufferTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.geode.unsafe.internal.sun.nio.ch;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+public class DirectBufferTest {
+
+  @Test
+  public void attachmentIsNullForNonDirectBuffer() {
+    assertThat(DirectBuffer.attachment(null)).isNull();
+    assertThat(DirectBuffer.attachment(new Object())).isNull();
+    assertThat(DirectBuffer.attachment(ByteBuffer.allocate(1))).isNull();
+  }
+
+  @Test
+  public void attachmentIsNullForUnslicedDirectBuffer() {
+    assertThat(DirectBuffer.attachment(ByteBuffer.allocateDirect(1))).isNull();
+  }
+
+  @Test
+  public void attachmentIsRootBufferForDirectBufferSlice() {
+    final ByteBuffer root = ByteBuffer.allocateDirect(10);
+    final ByteBuffer slice = root.slice();
+
+    assertThat(DirectBuffer.attachment(slice)).isSameAs(root);
+  }
+
+}
\ No newline at end of file