From d7c3c2300e7f13df77f8762d4dd95e39c6c1969c Mon Sep 17 00:00:00 2001 From: Justin Marsh Date: Tue, 18 Nov 2025 22:55:14 -0800 Subject: [PATCH 1/3] Fix ByteBuf memory leak in PutOperation when operations are aborted When a PutOperation is aborted or fails before all data is processed by the ChunkFiller thread, channelReadBuf may still hold a reference to a ByteBuf that was read from the channel but not yet consumed. This causes a memory leak as the buffer is never released. Changes: - Add channelReadBuf.release() in cleanupChunks() to ensure the buffer is properly released when the operation completes or fails. - Remove synchronized modifier from PutChunk.fillFrom() as it's not needed with the ChunkFiller single threaded model - Add test case testPutOperationByteBufLeakOnAbort() to verify ByteBuf resources are properly released when operations are aborted mid-flight The fix ensures that even if the ChunkFiller thread hasn't processed all data from the channel when an operation completes/fails, the ByteBuf holding unprocessed data is properly released, preventing memory leaks. --- .../com/github/ambry/router/PutManager.java | 1 - .../com/github/ambry/router/PutOperation.java | 7 ++- .../ambry/router/NonBlockingRouterTest.java | 53 ++++++++++++++++++- 3 files changed, 58 insertions(+), 3 deletions(-) diff --git a/ambry-router/src/main/java/com/github/ambry/router/PutManager.java b/ambry-router/src/main/java/com/github/ambry/router/PutManager.java index 6a56a62e10..0fcb77e4d9 100644 --- a/ambry-router/src/main/java/com/github/ambry/router/PutManager.java +++ b/ambry-router/src/main/java/com/github/ambry/router/PutManager.java @@ -15,7 +15,6 @@ import com.github.ambry.account.Account; import com.github.ambry.account.AccountService; -import com.github.ambry.account.Container; import com.github.ambry.clustermap.ClusterMap; import com.github.ambry.clustermap.ClusterMapUtils; import com.github.ambry.commons.ByteBufferAsyncWritableChannel; diff --git a/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java b/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java index 5b814f6025..3924eff8a4 100644 --- a/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java +++ b/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java @@ -578,6 +578,11 @@ void setOperationCompleted() { */ public void cleanupChunks() { releaseDataForAllChunks(); + // At this point, if the channelReadBuf is not null it means it did not get fully read + // by the ChunkFiller in fillChunks and needs to be released. + if (channelReadBuf != null) { + channelReadBuf.release(); + } } /** @@ -1642,7 +1647,7 @@ void onFillComplete(boolean updateMetric) { * @param channelReadBuf the {@link ByteBuf} from which to read data. * @return the number of bytes transferred in this operation. */ - synchronized int fillFrom(ByteBuf channelReadBuf) { + int fillFrom(ByteBuf channelReadBuf) { int toWrite; ByteBuf slice; if (buf == null) { diff --git a/ambry-router/src/test/java/com/github/ambry/router/NonBlockingRouterTest.java b/ambry-router/src/test/java/com/github/ambry/router/NonBlockingRouterTest.java index 4a22af5f65..a9a0e2930f 100644 --- a/ambry-router/src/test/java/com/github/ambry/router/NonBlockingRouterTest.java +++ b/ambry-router/src/test/java/com/github/ambry/router/NonBlockingRouterTest.java @@ -14,6 +14,7 @@ package com.github.ambry.router; import com.codahale.metrics.MetricRegistry; +import com.github.ambry.utils.NettyByteBufLeakHelper; import com.github.ambry.account.Account; import com.github.ambry.account.Container; import com.github.ambry.clustermap.DataNodeId; @@ -22,6 +23,7 @@ import com.github.ambry.clustermap.PartitionId; import com.github.ambry.clustermap.ReplicaId; import com.github.ambry.commons.BlobId; +import com.github.ambry.commons.ByteBufReadableStreamChannel; import com.github.ambry.commons.ByteBufferReadableStreamChannel; import com.github.ambry.commons.Callback; import com.github.ambry.commons.LoggingNotificationSystem; @@ -37,7 +39,6 @@ import com.github.ambry.frontend.Operations; import com.github.ambry.messageformat.BlobProperties; import com.github.ambry.messageformat.MessageFormatRecord; -import com.github.ambry.named.NamedBlobRecord; import com.github.ambry.network.NetworkClient; import com.github.ambry.network.NetworkClientErrorCode; import com.github.ambry.network.NetworkClientFactory; @@ -97,6 +98,8 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; import javax.sql.DataSource; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import org.json.JSONObject; import org.junit.AfterClass; import org.junit.Assert; @@ -4550,4 +4553,52 @@ static void verifyRepairRequestRecordInDb(MysqlRepairRequestsDb db, BlobId blobI assertEquals(expectedRecord.getExpirationTimeMs(), record.getExpirationTimeMs()); } } + + /** + * Test for bytebuf memory leaks in PutOperation when operations are aborted in the middle of a put operation. + * This test verifies that PutOperation properly releases bytebuf when the operation completes/fails, even if + * the ChunkFiller thread hasn't processed some data yet. + */ + @Test + public void testPutOperationByteBufLeakOnAbort() throws Exception { + NettyByteBufLeakHelper testLeakHelper = new NettyByteBufLeakHelper(); + testLeakHelper.beforeTest(); + + Properties props = getNonBlockingRouterProperties(localDcName); + int chunkSize = 512; + props.setProperty("router.max.put.chunk.size.bytes", Integer.toString(chunkSize)); + setRouter(props, mockServerLayout, new LoggingNotificationSystem()); + + // Configure servers to succeed for first few chunks, then fail + List serverErrorList = new ArrayList<>(); + serverErrorList.add(ServerErrorCode.NoError); + serverErrorList.add(ServerErrorCode.NoError); + for (int i = 0; i < 100; i++) { + serverErrorList.add(ServerErrorCode.PartitionReadOnly); + } + mockServerLayout.getMockServers().forEach(server -> server.setServerErrors(serverErrorList)); + + // The first two will run normally, but 3+ will get ServerErrorCode.PartitionReadOnly + int blobSize = 100 * chunkSize; + byte[] blobData = new byte[blobSize]; + ThreadLocalRandom.current().nextBytes(blobData); + ByteBuf pooledBuf = PooledByteBufAllocator.DEFAULT.buffer(blobSize); + pooledBuf.writeBytes(blobData); + ByteBufReadableStreamChannel channel = new ByteBufReadableStreamChannel(pooledBuf); + + BlobProperties blobProperties = new BlobProperties(blobSize, "serviceId", "ownerId", "contentType", + false, Utils.Infinite_Time, Utils.getRandomShort(ThreadLocalRandom.current()), + Utils.getRandomShort(ThreadLocalRandom.current()), false, null, null, null); + + try { + router.putBlob(blobProperties, new byte[10], channel, PutBlobOptions.DEFAULT).get(); + } catch (ExecutionException e) { + // Expected for operations that hit error responses + } + // If there are leaks, it will be detected in NettyByteBufLeakHelper and fail the test. + // Should be called before router close as closing of the router shouldn't be required to prevent leaks. + testLeakHelper.afterTest(); + router.close(); + router = null; + } } From 2973bbda70b70bfb95ee6b1ea4f7dabb35f402e8 Mon Sep 17 00:00:00 2001 From: Justin Marsh Date: Tue, 25 Nov 2025 09:47:24 -0800 Subject: [PATCH 2/3] Go one step further and prevent use-after-free race condition After even more extensive review, we identified that not only is there a case where the read buf is leaked, it's also possible for the read buf to be used after free in rare cases where the memory is released by the network and GCed before it is retained by the ChunkFiller thread. Some of the memory leak in the previous commit is masking some of the use-after-free issues fixed in this commit. Given how intertwined they are, it wouldn't be safe to merge these changes separately. --- .../com/github/ambry/router/PutOperation.java | 52 +++-- .../github/ambry/router/PutOperationTest.java | 212 +++++++++++++++++- 2 files changed, 247 insertions(+), 17 deletions(-) diff --git a/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java b/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java index 3924eff8a4..34a8ef574f 100644 --- a/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java +++ b/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java @@ -57,6 +57,7 @@ import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCountUtil; import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; import java.security.GeneralSecurityException; import java.util.ArrayList; import java.util.Collections; @@ -574,14 +575,22 @@ void setOperationCompleted() { /** * Clean up the chunks to release any data buffer. This should be invoked when terminating the operation with - * an exception. + * an exception. This method also closes the chunkFillerChannel to fire any pending callbacks, ensuring the original + * buffer from the ReadableStreamChannel is properly released. Synchronized for memory visibility on channelReadBuf. + * The contract upheld by PutManager is that this method is called AT-MOST-ONCE. */ - public void cleanupChunks() { - releaseDataForAllChunks(); - // At this point, if the channelReadBuf is not null it means it did not get fully read - // by the ChunkFiller in fillChunks and needs to be released. - if (channelReadBuf != null) { - channelReadBuf.release(); + public synchronized void cleanupChunks() { + try { + releaseDataForAllChunks(); + } finally { + // Release the extra reference we retained when storing in channelReadBuf. + if (channelReadBuf != null) { + channelReadBuf.release(); + channelReadBuf = null; + } + // Close the chunkFillerChannel to fire any remaining callbacks in chunksAwaitingResolution. + // This ensures the original buffer (owned by the callback) is released and not leaked. + chunkFillerChannel.close(); } } @@ -688,6 +697,11 @@ void fillChunks() { // Attempt to fill a chunk if (channelReadBuf == null) { channelReadBuf = chunkFillerChannel.getNextByteBuf(0); + if (channelReadBuf != null) { + // Retain the buffer to protect against the channel callback releasing it + // while we still hold a reference we're processing. + channelReadBuf.retain(); + } } if (channelReadBuf != null) { if (channelReadBuf.readableBytes() > 0 && isChunkAwaitingResolution()) { @@ -712,8 +726,13 @@ void fillChunks() { routerCallback.onPollReady(); } if (!channelReadBuf.isReadable()) { - chunkFillerChannel.resolveOldestChunk(null); - channelReadBuf = null; + try { + chunkFillerChannel.resolveOldestChunk(null); + } finally { + // Release the reference we retained when storing getNextByteBuf, even if resolveOldestChunk throws. + channelReadBuf.release(); + channelReadBuf = null; + } } } } else { @@ -1101,16 +1120,19 @@ boolean isStitchOperation() { } /** - * Set the exception associated with this operation. - * First, if current operationException is null, directly set operationException as exception; - * Second, if operationException exists, compare ErrorCodes of exception and existing operation Exception depending - * on precedence level. An ErrorCode with a smaller precedence level overrides an ErrorCode with a larger precedence - * level. Update the operationException if necessary. - * @param exception the {@link RouterException} to possibly set. + * Set the exception associated with this operation and mark it complete. + * For {@link RouterException}: uses precedence-based replacement where lower precedence + * levels override higher ones. + * For {@link java.nio.channels.ClosedChannelException}: only set if no other exception has + * been set to avoid overwriting meaningful errors. + * For all others simply set the exception as we don't know what they are or how to classify them. + * @param exception the {@link Exception} to possibly set. */ void setOperationExceptionAndComplete(Exception exception) { if (exception instanceof RouterException) { RouterUtils.replaceOperationException(operationException, (RouterException) exception, this::getPrecedenceLevel); + } else if (exception instanceof ClosedChannelException) { + operationException.compareAndSet(null, exception); } else { operationException.set(exception); } diff --git a/ambry-router/src/test/java/com/github/ambry/router/PutOperationTest.java b/ambry-router/src/test/java/com/github/ambry/router/PutOperationTest.java index 30da69a40a..b401d8f828 100644 --- a/ambry-router/src/test/java/com/github/ambry/router/PutOperationTest.java +++ b/ambry-router/src/test/java/com/github/ambry/router/PutOperationTest.java @@ -18,6 +18,7 @@ import com.github.ambry.clustermap.MockPartitionId; import com.github.ambry.commons.BlobId; import com.github.ambry.commons.ByteBufReadableStreamChannel; +import com.github.ambry.commons.ByteBufferAsyncWritableChannel; import com.github.ambry.commons.ByteBufferReadableStreamChannel; import com.github.ambry.commons.LoggingNotificationSystem; import com.github.ambry.config.RouterConfig; @@ -42,8 +43,6 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; -import io.netty.buffer.UnpooledByteBufAllocator; -import io.netty.buffer.UnpooledHeapByteBuf; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; @@ -363,6 +362,215 @@ null, new RouterCallback(new MockNetworkClient(), new ArrayList<>()), null, null routerMetrics.metadataChunkCreationCount.getCount()); } + /** + * Test that verifies a server error (RouterException) is not overwritten by ClosedChannelException + * during cleanupChunks(). When cleanupChunks() closes chunkFillerChannel, the callback fires with + * ClosedChannelException, but this must not overwrite the original RouterException. + */ + @Test + public void testCleanupChunksDoesNotOverwriteOriginalException() throws Exception { + // Use max.in.mem.put.chunks=1 so callback remains pending until cleanupChunks() + Properties properties = createBasicRouterProperties(); + properties.setProperty("router.max.in.mem.put.chunks", "1"); + RouterConfig testRouterConfig = createRouterConfigFromProperties(properties); + + mockServer.setServerErrorForAllRequests(ServerErrorCode.UnknownError); + try { + BlobProperties blobProperties = new BlobProperties(-1, "serviceId", "memberId", "contentType", + false, Utils.Infinite_Time, Utils.getRandomShort(TestUtils.RANDOM), Utils.getRandomShort(TestUtils.RANDOM), + false, null, null, null); + byte[] userMetadata = new byte[10]; + + // Buffer larger than one chunk ensures callback remains pending until cleanupChunks() + int bufferSize = chunkSize * 2; + ByteBuf byteBuf = PooledByteBufAllocator.DEFAULT.heapBuffer(bufferSize); + byte[] content = new byte[bufferSize]; + random.nextBytes(content); + byteBuf.writeBytes(content); + ByteBufReadableStreamChannel channel = new ByteBufReadableStreamChannel(byteBuf); + + FutureResult future = new FutureResult<>(); + MockNetworkClient mockNetworkClient = new MockNetworkClient(); + List requestInfos = new ArrayList<>(); + requestRegistrationCallback.setRequestsToSend(requestInfos); + + PutOperation op = PutOperation.forUpload(testRouterConfig, routerMetrics, mockClusterMap, + new LoggingNotificationSystem(), new InMemAccountService(true, false), userMetadata, + channel, PutBlobOptions.DEFAULT, future, null, + new RouterCallback(mockNetworkClient, new ArrayList<>()), null, null, null, null, time, + blobProperties, MockClusterMap.DEFAULT_PARTITION_CLASS, quotaChargeCallback, compressionService); + + op.startOperation(); + + // Process until operation fails + int maxIterations = 100; + int iterations = 0; + while (!op.isOperationComplete() && iterations++ < maxIterations) { + op.fillChunks(); + requestInfos.clear(); + op.poll(requestRegistrationCallback); + + for (RequestInfo requestInfo : requestInfos) { + ResponseInfo responseInfo = getResponseInfo(requestInfo); + PutResponse putResponse = responseInfo.getError() == null + ? PutResponse.readFrom(new NettyByteBufDataInputStream(responseInfo.content())) + : null; + op.handleResponse(responseInfo, putResponse); + // release the request / response as is expected by callers + requestInfo.getRequest().release(); + responseInfo.release(); + } + } + + assertTrue("Operation should be complete", op.isOperationComplete()); + Exception exceptionBeforeCleanup = op.getOperationException(); + assertNotNull("Operation should have an exception", exceptionBeforeCleanup); + assertTrue("Exception should be RouterException", exceptionBeforeCleanup instanceof RouterException); + + // cleanupChunks() closes chunkFillerChannel, firing callback with ClosedChannelException + op.cleanupChunks(); + + // Verify the original RouterException was preserved + assertSame("Exception should be preserved after cleanup", exceptionBeforeCleanup, op.getOperationException()); + } finally { + mockServer.resetServerErrors(); + } + } + + /** + * Verifies channelReadBuf remains valid after the channel callback releases the original buffer. + * This tests the fix for a use-after-free race condition in PutOperation.fillChunks(). + */ + @Test + public void testChannelReadBufRemainsValidAfterChannelClose() throws Exception { + // Use max.in.mem.put.chunks=1 so fillChunks() exits with data remaining in channelReadBuf + Properties properties = createBasicRouterProperties(); + properties.setProperty("router.max.in.mem.put.chunks", "1"); + RouterConfig testRouterConfig = createRouterConfigFromProperties(properties); + + BlobProperties blobProperties = new BlobProperties(-1, "serviceId", "memberId", "contentType", + false, Utils.Infinite_Time,Utils.getRandomShort(TestUtils.RANDOM), Utils.getRandomShort(TestUtils.RANDOM), + false, null, null, null); + byte[] userMetadata = new byte[10]; + + // Buffer larger than one chunk leaves data in channelReadBuf after first fillChunks() + int bufferSize = chunkSize * 2; + ByteBuf byteBuf = PooledByteBufAllocator.DEFAULT.heapBuffer(bufferSize); + byte[] content = new byte[bufferSize]; + random.nextBytes(content); + byteBuf.writeBytes(content); + ByteBufReadableStreamChannel channel = new ByteBufReadableStreamChannel(byteBuf); + + FutureResult future = new FutureResult<>(); + MockNetworkClient mockNetworkClient = new MockNetworkClient(); + List requestInfos = new ArrayList<>(); + requestRegistrationCallback.setRequestsToSend(requestInfos); + + PutOperation op = PutOperation.forUpload(testRouterConfig, routerMetrics, mockClusterMap, + new LoggingNotificationSystem(), new InMemAccountService(true, false), userMetadata, + channel, PutBlobOptions.DEFAULT, future, null, + new RouterCallback(mockNetworkClient, new ArrayList<>()), null, null, null, null, time, + blobProperties, MockClusterMap.DEFAULT_PARTITION_CLASS, quotaChargeCallback, compressionService); + + op.startOperation(); + op.fillChunks(); + + // Get channelReadBuf via reflection to verify it still has remaining data and initial conditions are correct + ByteBuf channelReadBuf = (ByteBuf) FieldUtils.readField(op, "channelReadBuf", true); + assertNotNull("channelReadBuf should have the buffer", channelReadBuf); + assertTrue("channelReadBuf should have remaining bytes", channelReadBuf.readableBytes() > 0); + + // Poll and handle responses to complete the first chunk + op.poll(requestRegistrationCallback); + for (RequestInfo requestInfo : requestInfos) { + ResponseInfo responseInfo = getResponseInfo(requestInfo); + PutResponse putResponse = responseInfo.getError() == null + ? PutResponse.readFrom(new NettyByteBufDataInputStream(responseInfo.content())) + : null; + op.handleResponse(responseInfo, putResponse); + // release the request / response as is expected by callers + requestInfo.getRequest().release(); + responseInfo.release(); + } + + // Close chunkFillerChannel - this fires the callback which releases the original buffer + ByteBufferAsyncWritableChannel chunkFillerChannel = + (ByteBufferAsyncWritableChannel) FieldUtils.readField(op, "chunkFillerChannel", true); + chunkFillerChannel.close(); + + // Verify buffer is still valid after channel close. + ByteBuf bufAfterClose = (ByteBuf) FieldUtils.readField(op, "channelReadBuf", true); + assertNotNull("channelReadBuf should still hold a reference", bufAfterClose); + assertTrue("Buffer should still be valid (refCnt > 0) after channel close", bufAfterClose.refCnt() > 0); + assertTrue("Buffer should have readable bytes", bufAfterClose.readableBytes() > 0); + + op.cleanupChunks(); + + // Verify buffer is now released after cleanup. + assertEquals("Buffer should have refCnt=0 after cleanup", 0, bufAfterClose.refCnt()); + ByteBuf bufAfterCleanup = (ByteBuf) FieldUtils.readField(op, "channelReadBuf", true); + assertNull("channelReadBuf should be null after cleanup", bufAfterCleanup); + } + + /** + * Baseline test verifying a multi-chunk put completes without errors or memory leaks + * when using max.in.mem.put.chunks=1 exercising the channelReadBuf code path. + */ + @Test + public void testMultiChunkPutWithSingleInMemChunk() throws Exception { + // Use max.in.mem.put.chunks=1 to exercise the channelReadBuf retention logic + Properties properties = createBasicRouterProperties(); + properties.setProperty("router.max.in.mem.put.chunks", "1"); + RouterConfig testRouterConfig = createRouterConfigFromProperties(properties); + + BlobProperties blobProperties = new BlobProperties(-1, "serviceId", "memberId", "contentType", + false, Utils.Infinite_Time, Utils.getRandomShort(TestUtils.RANDOM), Utils.getRandomShort(TestUtils.RANDOM), + false, null, null, null); + byte[] userMetadata = new byte[10]; + + int bufferSize = chunkSize * 2; + ByteBuf byteBuf = PooledByteBufAllocator.DEFAULT.heapBuffer(bufferSize); + byte[] content = new byte[bufferSize]; + random.nextBytes(content); + byteBuf.writeBytes(content); + ByteBufReadableStreamChannel channel = new ByteBufReadableStreamChannel(byteBuf); + + FutureResult future = new FutureResult<>(); + MockNetworkClient mockNetworkClient = new MockNetworkClient(); + List requestInfos = new ArrayList<>(); + requestRegistrationCallback.setRequestsToSend(requestInfos); + + PutOperation op = PutOperation.forUpload(testRouterConfig, routerMetrics, mockClusterMap, + new LoggingNotificationSystem(), new InMemAccountService(true, false), userMetadata, + channel, PutBlobOptions.DEFAULT, future, null, + new RouterCallback(mockNetworkClient, new ArrayList<>()), null, null, null, null, time, + blobProperties, MockClusterMap.DEFAULT_PARTITION_CLASS, quotaChargeCallback, compressionService); + + op.startOperation(); + + // Process all chunks until operation is complete + while (!op.isOperationComplete()) { + op.fillChunks(); + requestInfos.clear(); + op.poll(requestRegistrationCallback); + + for (RequestInfo requestInfo : requestInfos) { + ResponseInfo responseInfo = getResponseInfo(requestInfo); + PutResponse putResponse = responseInfo.getError() == null + ? PutResponse.readFrom(new NettyByteBufDataInputStream(responseInfo.content())) + : null; + op.handleResponse(responseInfo, putResponse); + // release the request / response as is expected by callers + requestInfo.getRequest().release(); + responseInfo.release(); + } + } + + assertTrue("Operation should be complete", op.isOperationComplete()); + assertNull("Operation should have no exception: " + op.getOperationException(), op.getOperationException()); + assertEquals("Original buffer should be fully released", 0, byteBuf.refCnt()); + } + @Test public void testCRCSucceeds() throws Exception { final int successTarget = 2; From 39ad85d1bf1d2211f131503a4a67d953f38b463f Mon Sep 17 00:00:00 2001 From: Justin Marsh Date: Wed, 26 Nov 2025 11:29:04 -0800 Subject: [PATCH 3/3] Protect the operations in fillChunks from router errors. --- .../src/main/java/com/github/ambry/router/PutOperation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java b/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java index 34a8ef574f..6e5678c14e 100644 --- a/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java +++ b/ambry-router/src/main/java/com/github/ambry/router/PutOperation.java @@ -690,7 +690,7 @@ boolean isChunkFillingDone() { * chunkFillerChannel, if there is any. * @throws InterruptedException if the call to get a chunk from the chunkFillerChannel is interrupted. */ - void fillChunks() { + synchronized void fillChunks() { try { PutChunk chunkToFill; while (!isChunkFillingDone()) {