Skip to content

Commit 98c2ec6

Browse files
authored
[GEODE-10520] Security : Eliminate DirectBuffer Access to sun.nio.ch Internal Package (#7955)
* refactor: Replace internal JDK DirectBuffer with public API solution Replace sun.nio.ch.DirectBuffer usage with BufferAttachmentTracker, using only public Java APIs (WeakHashMap and ByteBuffer). Changes: - Created BufferAttachmentTracker: WeakHashMap-based tracker for slice-to-original buffer mappings, replacing internal DirectBuffer.attachment() access - Updated BufferPool: Modified slice creation to record mappings and simplified getPoolableBuffer() to use the tracker - Removed DirectBuffer wrapper: Deleted geode-unsafe DirectBuffer wrapper class - Updated MemberJvmOptions: Removed SUN_NIO_CH_EXPORT from required JVM options - Added comprehensive unit tests: BufferAttachmentTrackerTest validates all tracker functionality Benefits: - Eliminates one JVM module export requirement - Uses only public Java APIs - Maintains functionality with automatic memory cleanup via WeakHashMap - Fully backward compatible Testing: - All BufferPool tests pass - New BufferAttachmentTracker tests pass - Compilation successful * Add comprehensive documentation to BufferAttachmentTracker - Add detailed PMD suppression justification explaining thread-safety - Document why ConcurrentHashMap is safe for concurrent access - Explain lock-free operations and atomic guarantees - Add 7-line comment block explaining mutable static field design choice * Apply spotless formatting to BufferAttachmentTrackerTest * fix: Correct buffer pooling to prevent capacity issues in NioEngine - Fixed acquirePredefinedFixedBuffer() to return full-capacity buffers instead of modifying buffer limits before return - Added BufferAttachmentTracker.removeTracking() in releaseBuffer() to properly clean up slice-to-original mappings - Created non-slicing buffer acquisition methods for NioPlainEngine and NioSslEngine which require reusable full-capacity buffers - Separated buffer acquisition into two use cases: * Single-use sliced buffers (2-param acquireDirectBuffer) * Reusable full-capacity buffers (3-param acquireDirectBuffer) This fixes IllegalArgumentException 'newLimit > capacity' errors in distributed tests by ensuring pooled buffers maintain proper capacity. * Fix IndexOutOfBoundsException in BufferAttachmentTracker Replace ConcurrentHashMap with synchronized IdentityHashMap to avoid ByteBuffer.equals() issues. ByteBuffer uses content-based equality which can throw IndexOutOfBoundsException when buffer state (position/limit) changes after being used as a map key. IdentityHashMap uses object identity (==) which is safe and appropriate for tracking buffer relationships.
1 parent 919f915 commit 98c2ec6

File tree

6 files changed

+387
-123
lines changed

6 files changed

+387
-123
lines changed
Lines changed: 103 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,103 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
3+
* agreements. See the NOTICE file distributed with this work for additional information regarding
4+
* copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
5+
* "License"); you may not use this file except in compliance with the License. You may obtain a
6+
* copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software distributed under the License
11+
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
12+
* or implied. See the License for the specific language governing permissions and limitations under
13+
* the License.
14+
*/
15+
16+
package org.apache.geode.internal.net;
17+
18+
import java.nio.ByteBuffer;
19+
import java.util.Collections;
20+
import java.util.IdentityHashMap;
21+
import java.util.Map;
22+
23+
/**
24+
* Tracks the relationship between sliced ByteBuffers and their original parent buffers.
25+
* This replaces the need to access internal JDK implementation classes, using only
26+
* public Java APIs instead.
27+
*
28+
* When ByteBuffer.slice() is called, it creates a new buffer that shares content with
29+
* the original. We need to track this relationship so that when returning buffers to
30+
* the pool, we return the original pooled buffer, not the slice.
31+
*
32+
* This class uses IdentityHashMap (synchronized) which provides thread-safe access
33+
* using object identity rather than equals(). This is critical because ByteBuffer.equals()
34+
* compares buffer content and can throw IndexOutOfBoundsException if buffer position/limit
35+
* is modified after being used as a map key. Callers must explicitly call removeTracking()
36+
* to clean up entries when buffers are returned to the pool.
37+
*/
38+
class BufferAttachmentTracker {
39+
40+
/**
41+
* Maps sliced buffers to their original parent buffers using object identity.
42+
* Uses synchronized IdentityHashMap for thread-safe access without relying on
43+
* ByteBuffer.equals() or hashCode(), which can be problematic when buffer state changes.
44+
* Entries must be explicitly removed via removeTracking() to prevent memory leaks.
45+
*
46+
* Note: This static mutable field is intentionally designed for global buffer tracking
47+
* across the application. The PMD.StaticFieldsMustBeImmutable warning is suppressed
48+
* because:
49+
* 1. Mutable shared state is required to track buffer relationships across all threads
50+
* 2. IdentityHashMap uses object identity (==) avoiding equals()/hashCode() issues
51+
* 3. Collections.synchronizedMap provides thread-safe operations
52+
* 4. This is the most efficient design for this use case
53+
*/
54+
@SuppressWarnings("PMD.StaticFieldsMustBeImmutable")
55+
private static final Map<ByteBuffer, ByteBuffer> sliceToOriginal =
56+
Collections.synchronizedMap(new IdentityHashMap<>());
57+
58+
/**
59+
* Records that a slice buffer was created from an original buffer.
60+
*
61+
* @param slice the sliced ByteBuffer
62+
* @param original the original ByteBuffer that was sliced
63+
*/
64+
static void recordSlice(ByteBuffer slice, ByteBuffer original) {
65+
sliceToOriginal.put(slice, original);
66+
}
67+
68+
/**
69+
* Retrieves the original buffer for a given buffer, which may be a slice.
70+
* If the buffer is not a slice (not tracked), returns the buffer itself.
71+
*
72+
* @param buffer the buffer to look up, which may be a slice
73+
* @return the original pooled buffer, or the buffer itself if not a slice
74+
*/
75+
static ByteBuffer getOriginal(ByteBuffer buffer) {
76+
ByteBuffer original = sliceToOriginal.get(buffer);
77+
return original != null ? original : buffer;
78+
}
79+
80+
/**
81+
* Removes tracking for a buffer. Should be called when returning a buffer
82+
* to the pool to avoid memory leaks in the tracking map.
83+
*
84+
* @param buffer the buffer to stop tracking
85+
*/
86+
static void removeTracking(ByteBuffer buffer) {
87+
sliceToOriginal.remove(buffer);
88+
}
89+
90+
/**
91+
* For testing: returns the current size of the tracking map.
92+
*/
93+
static int getTrackingMapSize() {
94+
return sliceToOriginal.size();
95+
}
96+
97+
/**
98+
* For testing: clears all tracking entries.
99+
*/
100+
static void clearTracking() {
101+
sliceToOriginal.clear();
102+
}
103+
}

geode-core/src/main/java/org/apache/geode/internal/net/BufferPool.java

Lines changed: 48 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,11 @@
2222

2323
import org.jetbrains.annotations.NotNull;
2424

25-
import org.apache.geode.InternalGemFireException;
2625
import org.apache.geode.annotations.VisibleForTesting;
2726
import org.apache.geode.distributed.internal.DMStats;
2827
import org.apache.geode.distributed.internal.DistributionConfig;
2928
import org.apache.geode.internal.Assert;
3029
import org.apache.geode.internal.tcp.Connection;
31-
import org.apache.geode.unsafe.internal.sun.nio.ch.DirectBuffer;
3230
import org.apache.geode.util.internal.GeodeGlossary;
3331

3432
public class BufferPool {
@@ -111,8 +109,11 @@ private ByteBuffer acquireDirectBuffer(int size, boolean send) {
111109
result = acquireLargeBuffer(send, size);
112110
}
113111
if (result.capacity() > size) {
112+
ByteBuffer original = result;
114113
result.position(0).limit(size);
115114
result = result.slice();
115+
// Track the slice-to-original mapping to support buffer pool return
116+
BufferAttachmentTracker.recordSlice(result, original);
116117
}
117118
return result;
118119
}
@@ -159,19 +160,14 @@ private ByteBuffer acquirePredefinedFixedBuffer(boolean send, int size) {
159160
// it was garbage collected
160161
updateBufferStats(-defaultSize, ref.getSend(), true);
161162
} else {
163+
// Reset the buffer to full capacity - clear() resets position and sets limit to capacity
162164
bb.clear();
163-
if (defaultSize > size) {
164-
bb.limit(size);
165-
}
166165
return bb;
167166
}
168167
ref = bufferTempQueue.poll();
169168
}
170169
result = ByteBuffer.allocateDirect(defaultSize);
171170
updateBufferStats(defaultSize, send, true);
172-
if (defaultSize > size) {
173-
result.limit(size);
174-
}
175171
return result;
176172
}
177173

@@ -267,17 +263,51 @@ ByteBuffer expandWriteBufferIfNeeded(BufferType type, ByteBuffer existing,
267263
}
268264

269265
ByteBuffer acquireDirectBuffer(BufferPool.BufferType type, int capacity) {
266+
// This method is used by NioPlainEngine and NioSslEngine which need full-capacity buffers
267+
// that can be reused for multiple read/write operations. We should NOT create slices here.
270268
switch (type) {
271269
case UNTRACKED:
272270
return ByteBuffer.allocate(capacity);
273271
case TRACKED_SENDER:
274-
return acquireDirectSenderBuffer(capacity);
272+
return acquireDirectSenderBufferNonSliced(capacity);
275273
case TRACKED_RECEIVER:
276-
return acquireDirectReceiveBuffer(capacity);
274+
return acquireDirectReceiveBufferNonSliced(capacity);
277275
}
278276
throw new IllegalArgumentException("Unexpected buffer type " + type);
279277
}
280278

279+
/**
280+
* Acquire a direct sender buffer without slicing - returns a buffer with capacity >= requested
281+
* size
282+
*/
283+
private ByteBuffer acquireDirectSenderBufferNonSliced(int size) {
284+
if (!useDirectBuffers) {
285+
return ByteBuffer.allocate(size);
286+
}
287+
288+
if (size <= MEDIUM_BUFFER_SIZE) {
289+
return acquirePredefinedFixedBuffer(true, size);
290+
} else {
291+
return acquireLargeBuffer(true, size);
292+
}
293+
}
294+
295+
/**
296+
* Acquire a direct receive buffer without slicing - returns a buffer with capacity >= requested
297+
* size
298+
*/
299+
private ByteBuffer acquireDirectReceiveBufferNonSliced(int size) {
300+
if (!useDirectBuffers) {
301+
return ByteBuffer.allocate(size);
302+
}
303+
304+
if (size <= MEDIUM_BUFFER_SIZE) {
305+
return acquirePredefinedFixedBuffer(false, size);
306+
} else {
307+
return acquireLargeBuffer(false, size);
308+
}
309+
}
310+
281311
ByteBuffer acquireNonDirectBuffer(BufferPool.BufferType type, int capacity) {
282312
switch (type) {
283313
case UNTRACKED:
@@ -310,11 +340,13 @@ void releaseBuffer(BufferPool.BufferType type, @NotNull ByteBuffer buffer) {
310340
*/
311341
private void releaseBuffer(ByteBuffer buffer, boolean send) {
312342
if (buffer.isDirect()) {
313-
buffer = getPoolableBuffer(buffer);
314-
BBSoftReference bbRef = new BBSoftReference(buffer, send);
315-
if (buffer.capacity() <= SMALL_BUFFER_SIZE) {
343+
ByteBuffer original = getPoolableBuffer(buffer);
344+
// Clean up tracking for this buffer to prevent memory leaks
345+
BufferAttachmentTracker.removeTracking(buffer);
346+
BBSoftReference bbRef = new BBSoftReference(original, send);
347+
if (original.capacity() <= SMALL_BUFFER_SIZE) {
316348
bufferSmallQueue.offer(bbRef);
317-
} else if (buffer.capacity() <= MEDIUM_BUFFER_SIZE) {
349+
} else if (original.capacity() <= MEDIUM_BUFFER_SIZE) {
318350
bufferMiddleQueue.offer(bbRef);
319351
} else {
320352
bufferLargeQueue.offer(bbRef);
@@ -328,25 +360,14 @@ private void releaseBuffer(ByteBuffer buffer, boolean send) {
328360
* If we hand out a buffer that is larger than the requested size we create a
329361
* "slice" of the buffer having the requested capacity and hand that out instead.
330362
* When we put the buffer back in the pool we need to find the original, non-sliced,
331-
* buffer. This is held in DirectBuffer in its "attachment" field.
363+
* buffer. This is tracked using BufferAttachmentTracker.
332364
*
333365
* This method is visible for use in debugging and testing. For debugging, invoke this method if
334366
* you need to see the non-sliced buffer for some reason, such as logging its hashcode.
335367
*/
336368
@VisibleForTesting
337369
ByteBuffer getPoolableBuffer(final ByteBuffer buffer) {
338-
final Object attachment = DirectBuffer.attachment(buffer);
339-
340-
if (null == attachment) {
341-
return buffer;
342-
}
343-
344-
if (attachment instanceof ByteBuffer) {
345-
return (ByteBuffer) attachment;
346-
}
347-
348-
throw new InternalGemFireException("direct byte buffer attachment was not a byte buffer but a "
349-
+ attachment.getClass().getName());
370+
return BufferAttachmentTracker.getOriginal(buffer);
350371
}
351372

352373
/**

0 commit comments

Comments
 (0)