forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 56
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Implementation of recycling memory pool. (#72)
- Loading branch information
kun du
authored
Feb 12, 2020
1 parent
5069ba4
commit 42673dc
Showing
5 changed files
with
241 additions
and
5 deletions.
There are no files selected for viewing
114 changes: 114 additions & 0 deletions
114
clients/src/main/java/org/apache/kafka/common/memory/RecyclingMemoryPool.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,114 @@ | ||
/* | ||
* 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.kafka.common.memory; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.util.concurrent.LinkedBlockingQueue; | ||
import org.apache.kafka.common.metrics.Sensor; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
|
||
/** | ||
* An implementation of memory pool which recycles buffers of commonly used size. | ||
* This memory pool is useful if most of the requested buffers' size are within close size range. | ||
* In this case, instead of deallocate and reallocate the buffer, the memory pool will recycle the buffer for future use. | ||
*/ | ||
public class RecyclingMemoryPool implements MemoryPool { | ||
protected static final Logger log = LoggerFactory.getLogger(RecyclingMemoryPool.class); | ||
protected final int cacheableBufferSizeUpperThreshold; | ||
protected final int cacheableBufferSizeLowerThreshold; | ||
protected final LinkedBlockingQueue<ByteBuffer> bufferCache; | ||
protected final Sensor requestSensor; | ||
|
||
public RecyclingMemoryPool(int cacheableBufferSize, int bufferCacheCapacity, Sensor requestSensor) { | ||
if (bufferCacheCapacity <= 0 || cacheableBufferSize <= 0) { | ||
throw new IllegalArgumentException(String.format("Must provide a positive cacheable buffer size and buffer cache " + | ||
"capacity, provided %d and %d respectively.", cacheableBufferSize, bufferCacheCapacity)); | ||
} | ||
this.bufferCache = new LinkedBlockingQueue<>(bufferCacheCapacity); | ||
for (int i = 0; i < bufferCacheCapacity; i++) { | ||
bufferCache.offer(ByteBuffer.allocate(cacheableBufferSize)); | ||
} | ||
this.cacheableBufferSizeUpperThreshold = cacheableBufferSize; | ||
this.cacheableBufferSizeLowerThreshold = cacheableBufferSize / 2; | ||
this.requestSensor = requestSensor; | ||
} | ||
|
||
@Override | ||
public ByteBuffer tryAllocate(int sizeBytes) { | ||
if (sizeBytes < 1) { | ||
throw new IllegalArgumentException("requested size " + sizeBytes + "<=0"); | ||
} | ||
|
||
ByteBuffer allocated = null; | ||
if (sizeBytes > cacheableBufferSizeLowerThreshold && sizeBytes <= cacheableBufferSizeUpperThreshold) { | ||
allocated = bufferCache.poll(); | ||
} | ||
if (allocated != null) { | ||
allocated.limit(sizeBytes); | ||
} else { | ||
allocated = ByteBuffer.allocate(sizeBytes); | ||
} | ||
bufferToBeAllocated(allocated); | ||
return allocated; | ||
} | ||
|
||
@Override | ||
public void release(ByteBuffer previouslyAllocated) { | ||
if (previouslyAllocated == null) { | ||
throw new IllegalArgumentException("provided null buffer"); | ||
} | ||
if (previouslyAllocated.capacity() == cacheableBufferSizeUpperThreshold) { | ||
previouslyAllocated.clear(); | ||
bufferCache.offer(previouslyAllocated); | ||
} else { | ||
bufferToBeReleased(previouslyAllocated); | ||
} | ||
} | ||
|
||
//allows subclasses to do their own bookkeeping (and validation) _before_ memory is returned to client code. | ||
protected void bufferToBeAllocated(ByteBuffer justAllocated) { | ||
try { | ||
this.requestSensor.record(justAllocated.limit()); | ||
} catch (Exception e) { | ||
log.debug("failed to record size of allocated buffer"); | ||
} | ||
log.trace("allocated buffer of size {}", justAllocated.capacity()); | ||
} | ||
|
||
//allows subclasses to do their own bookkeeping (and validation) _before_ memory is marked as reclaimed. | ||
protected void bufferToBeReleased(ByteBuffer justReleased) { | ||
log.trace("released buffer of size {}", justReleased.capacity()); | ||
} | ||
|
||
@Override | ||
public long size() { | ||
return Long.MAX_VALUE; | ||
} | ||
|
||
@Override | ||
public long availableMemory() { | ||
return Long.MAX_VALUE; | ||
} | ||
|
||
@Override | ||
public boolean isOutOfMemory() { | ||
return false; | ||
} | ||
} |
110 changes: 110 additions & 0 deletions
110
clients/src/test/java/org/apache/kafka/common/memory/RecyclingMemoryPoolTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,110 @@ | ||
/* | ||
* 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.kafka.common.memory; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import org.apache.kafka.common.metrics.Metrics; | ||
import org.apache.kafka.common.metrics.Sensor; | ||
import org.junit.Assert; | ||
import org.junit.Test; | ||
|
||
|
||
public class RecyclingMemoryPoolTest { | ||
private static final int TWO_KILOBYTES = 2048; | ||
private static final int CACHEABLE_BUFFER_SIZE = 1024; | ||
private static final int BUFFER_CACHE_CAPACITY = 2; | ||
private static final Sensor ALLOCATE_SENSOR = new Metrics().sensor("allocate_sensor"); | ||
|
||
@Test(expected = IllegalArgumentException.class) | ||
public void testNegativeAllocation() { | ||
RecyclingMemoryPool memoryPool = new RecyclingMemoryPool(CACHEABLE_BUFFER_SIZE, BUFFER_CACHE_CAPACITY, ALLOCATE_SENSOR); | ||
memoryPool.tryAllocate(-1); | ||
} | ||
|
||
@Test(expected = IllegalArgumentException.class) | ||
public void testZeroAllocation() { | ||
RecyclingMemoryPool memoryPool = new RecyclingMemoryPool(CACHEABLE_BUFFER_SIZE, BUFFER_CACHE_CAPACITY, ALLOCATE_SENSOR); | ||
memoryPool.tryAllocate(0); | ||
} | ||
|
||
@Test(expected = IllegalArgumentException.class) | ||
public void testNullRelease() { | ||
RecyclingMemoryPool memoryPool = new RecyclingMemoryPool(CACHEABLE_BUFFER_SIZE, BUFFER_CACHE_CAPACITY, ALLOCATE_SENSOR); | ||
memoryPool.release(null); | ||
} | ||
|
||
@Test | ||
public void testAllocation() { | ||
RecyclingMemoryPool memoryPool = new RecyclingMemoryPool(CACHEABLE_BUFFER_SIZE, BUFFER_CACHE_CAPACITY, ALLOCATE_SENSOR); | ||
ByteBuffer buffer1 = memoryPool.tryAllocate(TWO_KILOBYTES); | ||
ByteBuffer buffer2 = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE); | ||
ByteBuffer buffer3 = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE * 2 / 3); | ||
ByteBuffer buffer4 = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE); | ||
|
||
memoryPool.release(buffer1); | ||
ByteBuffer reuse1 = memoryPool.tryAllocate(TWO_KILOBYTES); | ||
// Compare the references | ||
Assert.assertNotEquals(System.identityHashCode(reuse1), System.identityHashCode(buffer1)); | ||
|
||
memoryPool.release(buffer2); | ||
memoryPool.release(buffer3); | ||
memoryPool.release(buffer4); | ||
ByteBuffer reuse2 = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE); | ||
ByteBuffer reuse3 = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE * 2 / 3); | ||
ByteBuffer reuse4 = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE); | ||
|
||
Assert.assertEquals(System.identityHashCode(reuse2), System.identityHashCode(buffer2)); | ||
Assert.assertEquals(System.identityHashCode(reuse3), System.identityHashCode(buffer3)); | ||
Assert.assertNotEquals(System.identityHashCode(reuse4), System.identityHashCode(buffer4)); | ||
} | ||
|
||
@Test | ||
public void testMultiThreadAllocation() { | ||
RecyclingMemoryPool memoryPool = new RecyclingMemoryPool(CACHEABLE_BUFFER_SIZE, BUFFER_CACHE_CAPACITY, ALLOCATE_SENSOR); | ||
AtomicReference<Throwable> error = new AtomicReference<>(); | ||
List<Thread> processorThreads = new ArrayList<>(3); | ||
for (int i = 0; i < 3; i++) { | ||
processorThreads.add(new Thread(() -> { | ||
try { | ||
ByteBuffer buffer = memoryPool.tryAllocate(CACHEABLE_BUFFER_SIZE); | ||
Thread.sleep(1000); | ||
memoryPool.release(buffer); | ||
} catch (InterruptedException e) { | ||
error.compareAndSet(null, e); | ||
} | ||
})); | ||
} | ||
processorThreads.forEach(t -> { | ||
t.setDaemon(true); | ||
t.start(); | ||
}); | ||
processorThreads.forEach(t -> { | ||
try { | ||
t.join(30000); | ||
} catch (InterruptedException e) { | ||
error.compareAndSet(null, e); | ||
} | ||
}); | ||
|
||
Assert.assertNull(error.get()); | ||
Assert.assertEquals(memoryPool.bufferCache.size(), BUFFER_CACHE_CAPACITY); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters