Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-18105 Implement buffer pooling with weak references #4263

Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -45,4 +45,9 @@ public interface ByteBufferPool {
* @param buffer a direct bytebuffer
*/
void putBuffer(ByteBuffer buffer);

/**
* Clear the buffer pool thus releasing all the buffers.
*/
default void release() { }
mukund-thakur marked this conversation as resolved.
Show resolved Hide resolved
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public final class ElasticByteBufferPool implements ByteBufferPool {
private static final class Key implements Comparable<Key> {
public class ElasticByteBufferPool implements ByteBufferPool {
protected static final class Key implements Comparable<Key> {
private final int capacity;
private final long insertionTime;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,155 @@
/**
* 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.hadoop.io;

import java.lang.ref.WeakReference;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.TreeMap;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.classification.VisibleForTesting;

/**
* Buffer pool implementation which uses weak references to store
* buffers in the pool, such that they are garbage collected when
* there are no references to the buffer during a gc run. This is
* important as direct buffers don't get garbage collected automatically
* during a gc run as they are not stored on heap memory.
* Also the buffers are stored in a tree map which helps in returning
* smallest buffer whose size is just greater than requested length.
* This is a thread safe implementation.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class WeakReferencedElasticByteBufferPool extends ElasticByteBufferPool {
mukund-thakur marked this conversation as resolved.
Show resolved Hide resolved

/**
* Map to store direct byte buffers of different sizes in the pool.
* Used tree map such that we can return next greater than capacity
* buffer if buffer with exact capacity is unavailable.
* This must be accessed in synchronized blocks.
*/
private final TreeMap<Key, WeakReference<ByteBuffer>> directBuffers =
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
new TreeMap<>();

/**
* Map to store heap based byte buffers of different sizes in the pool.
* Used tree map such that we can return next greater than capacity
* buffer if buffer with exact capacity is unavailable.
* This must be accessed in synchronized blocks.
*/
private final TreeMap<Key, WeakReference<ByteBuffer>> heapBuffers =
new TreeMap<>();

/**
* Method to get desired buffer tree.
* @param isDirect whether the buffer is heap based or direct.
* @return corresponding buffer tree.
*/
private TreeMap<Key, WeakReference<ByteBuffer>> getBufferTree(boolean isDirect) {
return isDirect
? directBuffers
: heapBuffers;
}

/**
* {@inheritDoc}
*
* @param direct whether we want a direct byte buffer or a heap one.
* @param length length of requested buffer.
* @return returns equal or next greater than capacity buffer from
* pool if already available and not garbage collected else creates
* a new buffer and return it.
*/
@Override
public synchronized ByteBuffer getBuffer(boolean direct, int length) {
TreeMap<Key, WeakReference<ByteBuffer>> buffersTree = getBufferTree(direct);

// Scan the entire tree and remove all weak null references.
buffersTree.entrySet().removeIf(next -> next.getValue().get() == null);

Map.Entry<Key, WeakReference<ByteBuffer>> entry =
buffersTree.ceilingEntry(new Key(length, 0));
// If there is no buffer present in the pool with desired size.
if (entry == null) {
return direct ? ByteBuffer.allocateDirect(length) :
ByteBuffer.allocate(length);
}
// buffer is available in the pool and not garbage collected.
WeakReference<ByteBuffer> bufferInPool = entry.getValue();
buffersTree.remove(entry.getKey());
ByteBuffer buffer = bufferInPool.get();
if (buffer != null) {
return buffer;
}
// buffer was in pool but already got garbage collected.
return direct
? ByteBuffer.allocateDirect(length)
: ByteBuffer.allocate(length);
}

/**
* Return buffer to the pool.
* @param buffer buffer to be returned.
*/
@Override
public synchronized void putBuffer(ByteBuffer buffer) {
buffer.clear();
TreeMap<Key, WeakReference<ByteBuffer>> buffersTree = getBufferTree(buffer.isDirect());
// Buffers are indexed by (capacity, time).
// If our key is not unique on the first try, we try again, since the
// time will be different. Since we use nanoseconds, it's pretty
// unlikely that we'll loop even once, unless the system clock has a
// poor granularity or multi-socket systems have clocks slightly out
// of sync.
while (true) {
Key keyToInsert = new Key(buffer.capacity(), System.nanoTime());
if (!buffersTree.containsKey(keyToInsert)) {
buffersTree.put(keyToInsert, new WeakReference<>(buffer));
return;
}
}
mukund-thakur marked this conversation as resolved.
Show resolved Hide resolved
}

/**
* Clear the buffer pool thus releasing all the buffers.
* The caller must remove all references of
* existing buffers before calling this method to avoid
* memory leaks.
*/
@Override
public synchronized void release() {
heapBuffers.clear();
directBuffers.clear();
}

/**
* Get current buffers count in the pool.
* @param isDirect whether we want to count the heap or direct buffers.
* @return count of buffers.
*/
@VisibleForTesting
public synchronized int getCurrentBuffersCount(boolean isDirect) {
return isDirect
? directBuffers.size()
: heapBuffers.size();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/**
* 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.hadoop.io;

import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;

import org.assertj.core.api.Assertions;
import org.junit.Test;

import org.apache.hadoop.test.HadoopTestBase;

import static org.apache.hadoop.test.LambdaTestUtils.intercept;

/**
* Non parameterized tests for {@code WeakReferencedElasticByteBufferPool}.
*/
public class TestMoreWeakReferencedElasticByteBufferPool
mukund-thakur marked this conversation as resolved.
Show resolved Hide resolved
extends HadoopTestBase {

@Test
public void testMixedBuffersInPool() {
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
ByteBuffer buffer1 = pool.getBuffer(true, 5);
ByteBuffer buffer2 = pool.getBuffer(true, 10);
ByteBuffer buffer3 = pool.getBuffer(false, 5);
ByteBuffer buffer4 = pool.getBuffer(false, 10);
ByteBuffer buffer5 = pool.getBuffer(true, 15);

assertBufferCounts(pool, 0, 0);
pool.putBuffer(buffer1);
pool.putBuffer(buffer2);
assertBufferCounts(pool, 2, 0);
pool.putBuffer(buffer3);
assertBufferCounts(pool, 2, 1);
pool.putBuffer(buffer5);
assertBufferCounts(pool, 3, 1);
pool.putBuffer(buffer4);
assertBufferCounts(pool, 3, 2);
pool.release();
assertBufferCounts(pool, 0, 0);

}

@Test
public void testUnexpectedBufferSizes() throws Exception {
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
ByteBuffer buffer1 = pool.getBuffer(true, 0);

// try writing a random byte in a 0 length buffer.
// Expected exception as buffer requested is of size 0.
intercept(BufferOverflowException.class,
() -> buffer1.put(new byte[1]));

// Expected IllegalArgumentException as negative length buffer is requested.
intercept(IllegalArgumentException.class,
() -> pool.getBuffer(true, -5));

// test returning null buffer to the pool.
intercept(NullPointerException.class,
() -> pool.putBuffer(null));
}

/**
* Utility method to assert counts of direct and heap buffers in
* the given buffer pool.
* @param pool buffer pool.
* @param numDirectBuffersExpected expected number of direct buffers.
* @param numHeapBuffersExpected expected number of heap buffers.
*/
private void assertBufferCounts(WeakReferencedElasticByteBufferPool pool,
mukund-thakur marked this conversation as resolved.
Show resolved Hide resolved
int numDirectBuffersExpected,
int numHeapBuffersExpected) {
Assertions.assertThat(pool.getCurrentBuffersCount(true))
.describedAs("Number of direct buffers in pool")
.isEqualTo(numDirectBuffersExpected);
Assertions.assertThat(pool.getCurrentBuffersCount(false))
.describedAs("Number of heap buffers in pool")
.isEqualTo(numHeapBuffersExpected);
}
}