-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-18190. Adds iostats for prefetching #4458
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
Changes from 10 commits
0224784
d909226
6e037fc
e3684ec
ae352bb
0943890
8d9b327
bb40655
c121694
9f150cb
65f5ee7
becdd66
8a3f38d
1d710f1
57bbdc2
66fb591
62d1b8d
5834ea9
e556425
492b2ef
021c98a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,6 +31,8 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import static java.util.Objects.requireNonNull; | ||
|
|
||
| /** | ||
| * Manages a fixed pool of {@code ByteBuffer} instances. | ||
| * | ||
|
|
@@ -56,26 +58,32 @@ public class BufferPool implements Closeable { | |
| // Allows associating metadata to each buffer in the pool. | ||
| private Map<BufferData, ByteBuffer> allocated; | ||
|
|
||
| private PrefetchingStatistics prefetchingStatistics; | ||
|
|
||
| /** | ||
| * Initializes a new instance of the {@code BufferPool} class. | ||
| * | ||
| * @param size number of buffer in this pool. | ||
| * @param bufferSize size in bytes of each buffer. | ||
| * @param prefetchingStatistics statistics for this stream. | ||
| * | ||
| * @throws IllegalArgumentException if size is zero or negative. | ||
| * @throws IllegalArgumentException if bufferSize is zero or negative. | ||
| */ | ||
| public BufferPool(int size, int bufferSize) { | ||
| public BufferPool(int size, int bufferSize, PrefetchingStatistics prefetchingStatistics) { | ||
| Validate.checkPositiveInteger(size, "size"); | ||
| Validate.checkPositiveInteger(bufferSize, "bufferSize"); | ||
|
|
||
| this.size = size; | ||
| this.bufferSize = bufferSize; | ||
| this.allocated = new IdentityHashMap<BufferData, ByteBuffer>(); | ||
| this.prefetchingStatistics = requireNonNull(prefetchingStatistics); | ||
ahmarsuhail marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| this.pool = new BoundedResourcePool<ByteBuffer>(size) { | ||
| @Override | ||
| public ByteBuffer createNew() { | ||
| return ByteBuffer.allocate(bufferSize); | ||
| ByteBuffer buffer = ByteBuffer.allocate(bufferSize); | ||
| prefetchingStatistics.memoryAllocated(bufferSize); | ||
| return buffer; | ||
| } | ||
| }; | ||
| } | ||
|
|
@@ -236,11 +244,15 @@ public synchronized void close() { | |
| } | ||
| } | ||
|
|
||
| int currentPoolSize = this.pool.numCreated(); | ||
|
||
|
|
||
| this.pool.close(); | ||
| this.pool = null; | ||
|
|
||
| this.allocated.clear(); | ||
| this.allocated = null; | ||
|
|
||
| this.prefetchingStatistics.memoryFreed(currentPoolSize * bufferSize); | ||
| } | ||
|
|
||
| // For debugging purposes. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,8 @@ | |
|
|
||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.time.Duration; | ||
| import java.time.Instant; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
@@ -31,6 +33,8 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import static java.util.Objects.requireNonNull; | ||
|
|
||
| /** | ||
| * Provides read access to the underlying file one block at a time. | ||
| * Improve read performance by prefetching and locall caching blocks. | ||
|
|
@@ -68,33 +72,37 @@ public abstract class CachingBlockManager extends BlockManager { | |
| // Once set to true, any further caching requests will be ignored. | ||
| private final AtomicBoolean cachingDisabled; | ||
|
|
||
| private final PrefetchingStatistics prefetchingStatistics; | ||
|
|
||
| /** | ||
| * Constructs an instance of a {@code CachingBlockManager}. | ||
| * | ||
| * @param futurePool asynchronous tasks are performed in this pool. | ||
| * @param blockData information about each block of the underlying file. | ||
| * @param bufferPoolSize size of the in-memory cache in terms of number of blocks. | ||
| * @param prefetchingStatistics statistics for this stream. | ||
| * | ||
| * @throws IllegalArgumentException if futurePool is null. | ||
| * @throws IllegalArgumentException if bufferPoolSize is zero or negative. | ||
| */ | ||
| public CachingBlockManager( | ||
| ExecutorServiceFuturePool futurePool, | ||
| BlockData blockData, | ||
| int bufferPoolSize) { | ||
| int bufferPoolSize, | ||
| PrefetchingStatistics prefetchingStatistics) { | ||
| super(blockData); | ||
|
|
||
| Validate.checkNotNull(futurePool, "futurePool"); | ||
| Validate.checkPositiveInteger(bufferPoolSize, "bufferPoolSize"); | ||
|
|
||
| this.futurePool = futurePool; | ||
| this.futurePool = requireNonNull(futurePool); | ||
| this.bufferPoolSize = bufferPoolSize; | ||
| this.numCachingErrors = new AtomicInteger(); | ||
| this.numReadErrors = new AtomicInteger(); | ||
| this.cachingDisabled = new AtomicBoolean(); | ||
| this.prefetchingStatistics = requireNonNull(prefetchingStatistics); | ||
ahmarsuhail marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| if (this.getBlockData().getFileSize() > 0) { | ||
| this.bufferPool = new BufferPool(bufferPoolSize, this.getBlockData().getBlockSize()); | ||
| this.bufferPool = new BufferPool(bufferPoolSize, this.getBlockData().getBlockSize(), | ||
| this.prefetchingStatistics); | ||
| this.cache = this.createCache(); | ||
| } | ||
|
|
||
|
|
@@ -247,7 +255,7 @@ public void requestPrefetch(int blockNumber) { | |
| } | ||
|
|
||
| BlockOperations.Operation op = this.ops.requestPrefetch(blockNumber); | ||
| PrefetchTask prefetchTask = new PrefetchTask(data, this); | ||
| PrefetchTask prefetchTask = new PrefetchTask(data, this, Instant.now()); | ||
| Future<Void> prefetchFuture = this.futurePool.executeFunction(prefetchTask); | ||
| data.setPrefetch(prefetchFuture); | ||
| this.ops.end(op); | ||
|
|
@@ -277,8 +285,10 @@ private void read(BufferData data) throws IOException { | |
| } | ||
| } | ||
|
|
||
| private void prefetch(BufferData data) throws IOException { | ||
| private void prefetch(BufferData data, Instant taskQueuedStartTime) throws IOException { | ||
| synchronized (data) { | ||
| this.prefetchingStatistics.executorAcquired( | ||
| Duration.between(taskQueuedStartTime, Instant.now())); | ||
| this.readBlock( | ||
| data, | ||
| true, | ||
|
|
@@ -316,6 +326,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... | |
| } | ||
|
|
||
| if (isPrefetch) { | ||
| this.prefetchingStatistics.prefetchOperationStarted(); | ||
| op = this.ops.prefetch(data.getBlockNumber()); | ||
| } else { | ||
| op = this.ops.getRead(data.getBlockNumber()); | ||
|
|
@@ -328,6 +339,10 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... | |
| this.read(buffer, offset, size); | ||
| buffer.flip(); | ||
| data.setReady(expectedState); | ||
|
|
||
| if(isPrefetch) { | ||
|
||
| this.prefetchingStatistics.prefetchOperationCompleted(); | ||
|
||
| } | ||
| } catch (Exception e) { | ||
| String message = String.format("error during readBlock(%s)", data.getBlockNumber()); | ||
| LOG.error(message, e); | ||
|
|
@@ -348,16 +363,18 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... | |
| private static class PrefetchTask implements Supplier<Void> { | ||
| private final BufferData data; | ||
| private final CachingBlockManager blockManager; | ||
| private final Instant taskQueuedStartTime; | ||
|
|
||
| PrefetchTask(BufferData data, CachingBlockManager blockManager) { | ||
| PrefetchTask(BufferData data, CachingBlockManager blockManager, Instant taskQueuedStartTime) { | ||
| this.data = data; | ||
| this.blockManager = blockManager; | ||
| this.taskQueuedStartTime = taskQueuedStartTime; | ||
| } | ||
|
|
||
| @Override | ||
| public Void get() { | ||
| try { | ||
| this.blockManager.prefetch(data); | ||
| this.blockManager.prefetch(data, taskQueuedStartTime); | ||
| } catch (Exception e) { | ||
| LOG.error("error during prefetch", e); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i worry about this. seen problems with transient network failures and abfs where the logs are flooded with many parallel stack traces before it recovers. |
||
| } | ||
|
|
@@ -418,14 +435,18 @@ public void requestCaching(BufferData data) { | |
| blockFuture = cf; | ||
| } | ||
|
|
||
| CachePutTask task = new CachePutTask(data, blockFuture, this); | ||
| CachePutTask task = new CachePutTask(data, blockFuture, this, Instant.now()); | ||
| Future<Void> actionFuture = this.futurePool.executeFunction(task); | ||
| data.setCaching(actionFuture); | ||
| this.ops.end(op); | ||
| } | ||
| } | ||
|
|
||
| private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture) { | ||
| private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture, | ||
| Instant taskQueuedStartTime) { | ||
| this.prefetchingStatistics.executorAcquired( | ||
| Duration.between(taskQueuedStartTime, Instant.now())); | ||
|
|
||
| if (this.closed) { | ||
| return; | ||
| } | ||
|
|
@@ -491,7 +512,7 @@ private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture) { | |
| } | ||
|
|
||
| protected BlockCache createCache() { | ||
| return new SingleFilePerBlockCache(); | ||
| return new SingleFilePerBlockCache(prefetchingStatistics); | ||
| } | ||
|
|
||
| protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException { | ||
|
|
@@ -511,18 +532,22 @@ private static class CachePutTask implements Supplier<Void> { | |
| // Block manager that manages this block. | ||
| private final CachingBlockManager blockManager; | ||
|
|
||
| private final Instant taskQueuedStartTime; | ||
|
|
||
| CachePutTask( | ||
| BufferData data, | ||
| Future<Void> blockFuture, | ||
| CachingBlockManager blockManager) { | ||
| CachingBlockManager blockManager, | ||
| Instant taskQueuedStartTime) { | ||
| this.data = data; | ||
| this.blockFuture = blockFuture; | ||
| this.blockManager = blockManager; | ||
| this.taskQueuedStartTime = taskQueuedStartTime; | ||
| } | ||
|
|
||
| @Override | ||
| public Void get() { | ||
| this.blockManager.addToCacheAndRelease(this.data, this.blockFuture); | ||
| this.blockManager.addToCacheAndRelease(this.data, this.blockFuture, taskQueuedStartTime); | ||
| return null; | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,41 @@ | ||
| /* | ||
| * 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.fs.common; | ||
|
|
||
| import java.time.Duration; | ||
|
|
||
| import org.apache.hadoop.fs.statistics.IOStatisticsSource; | ||
|
|
||
| public interface PrefetchingStatistics extends IOStatisticsSource { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. needs javadocs i'm afraid |
||
|
|
||
| void prefetchOperationStarted(); | ||
|
|
||
| void blockAddedToFileCache(); | ||
|
|
||
| void blockRemovedFromFileCache(); | ||
|
|
||
| void prefetchOperationCompleted(); | ||
|
|
||
| void executorAcquired(Duration timeInQueue); | ||
|
|
||
| void memoryAllocated(int size); | ||
|
|
||
| void memoryFreed(int size); | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -674,6 +674,46 @@ private Constants() { | |
| public static final String STREAM_READ_GAUGE_INPUT_POLICY = | ||
| "stream_read_gauge_input_policy"; | ||
|
|
||
| /** | ||
| * Total number of prefetching operations executed. | ||
| */ | ||
| public static final String STREAM_READ_PREFETCH_OPERATIONS | ||
steveloughran marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| = "stream_read_prefetch_operations"; | ||
|
|
||
| /** | ||
| * Total number of block in disk cache. | ||
| */ | ||
| public static final String STREAM_READ_BLOCKS_IN_FILE_CACHE | ||
| = "stream_read_blocks_in_cache"; | ||
|
|
||
| /** | ||
| * Total number of active prefetch operations. | ||
| */ | ||
| public static final String STREAM_READ_ACTIVE_PREFETCH_OPERATIONS | ||
| = "stream_read_active_prefetch_operations"; | ||
|
|
||
| /** | ||
| * Total bytes of memory in use by this input stream. | ||
| */ | ||
| public static final String STREAM_READ_ACTIVE_MEMORY_IN_USE | ||
| = "stream_read_active_memory_in_use"; | ||
|
|
||
| /** | ||
| * count/duration of reading a remote block. | ||
| * IO. | ||
| * Value: {@value}. | ||
| */ | ||
| public static final String STREAM_READ_REMOTE_BLOCK_READ | ||
| = "stream_read_block_read"; | ||
|
|
||
| /** | ||
| * count/duration of acquiring a buffer and reading to it | ||
| * IO. | ||
| * Value: {@value}. | ||
| */ | ||
| public static final String STREAM_READ_BLOCK_ACQUIRE_AND_READ | ||
| = "stream_read_block_read"; | ||
|
|
||
| @InterfaceAudience.Private | ||
| @InterfaceStability.Unstable | ||
| public static final String S3_CLIENT_FACTORY_IMPL = | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we are going to have to rename this class; org.apache.hadoop.io.ByteBufferPool has long existed and it will only cause confusion. not in this PR, but soon