Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -387,6 +387,46 @@ public final class StreamStatisticNames {
public static final String BLOCKS_RELEASED
= "blocks_released";

/**
* Total number of prefetching operations executed.
*/
public static final String STREAM_READ_PREFETCH_OPERATIONS
= "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.
*
* 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.
*
* Value: {@value}.
*/
public static final String STREAM_READ_BLOCK_ACQUIRE_AND_READ
= "stream_read_block_acquire_read";

private StreamStatisticNames() {
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
*
Expand All @@ -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) {
Copy link
Contributor

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

Validate.checkPositiveInteger(size, "size");
Validate.checkPositiveInteger(bufferSize, "bufferSize");

this.size = size;
this.bufferSize = bufferSize;
this.allocated = new IdentityHashMap<BufferData, ByteBuffer>();
this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
this.pool = new BoundedResourcePool<ByteBuffer>(size) {
@Override
public ByteBuffer createNew() {
return ByteBuffer.allocate(bufferSize);
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
prefetchingStatistics.memoryAllocated(bufferSize);
return buffer;
}
};
}
Expand Down Expand Up @@ -236,11 +244,15 @@ public synchronized void close() {
}
}

this.pool.close();
this.pool = null;
int currentPoolSize = pool.numCreated();

pool.close();
pool = null;

allocated.clear();
allocated = null;

this.allocated.clear();
this.allocated = null;
prefetchingStatistics.memoryFreed(currentPoolSize * bufferSize);
}

// For debugging purposes.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -31,6 +33,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.fs.statistics.DurationTracker;

import static java.util.Objects.requireNonNull;

import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;

/**
Expand Down Expand Up @@ -70,33 +76,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);

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();
}

Expand Down Expand Up @@ -249,7 +259,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);
Expand Down Expand Up @@ -279,8 +289,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) {
prefetchingStatistics.executorAcquired(
Duration.between(taskQueuedStartTime, Instant.now()));
this.readBlock(
data,
true,
Expand All @@ -297,6 +309,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
}

BlockOperations.Operation op = null;
DurationTracker tracker = null;

synchronized (data) {
try {
Expand All @@ -318,6 +331,7 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
}

if (isPrefetch) {
tracker = prefetchingStatistics.prefetchOperationStarted();
op = this.ops.prefetch(data.getBlockNumber());
} else {
op = this.ops.getRead(data.getBlockNumber());
Expand All @@ -333,13 +347,25 @@ private void readBlock(BufferData data, boolean isPrefetch, BufferData.State...
} catch (Exception e) {
String message = String.format("error during readBlock(%s)", data.getBlockNumber());
LOG.error(message, e);

if (isPrefetch && tracker != null) {
tracker.failed();
}

this.numReadErrors.incrementAndGet();
data.setDone();
throw e;
} finally {
if (op != null) {
this.ops.end(op);
}

if (isPrefetch) {
prefetchingStatistics.prefetchOperationCompleted();
if (tracker != null) {
tracker.close();
}
}
}
}
}
Expand All @@ -350,16 +376,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);
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

}
Expand Down Expand Up @@ -420,14 +448,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) {
prefetchingStatistics.executorAcquired(
Duration.between(taskQueuedStartTime, Instant.now()));

if (this.closed) {
return;
}
Expand Down Expand Up @@ -493,7 +525,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 {
Expand All @@ -513,18 +545,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;
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.DurationTracker;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;

public interface PrefetchingStatistics extends IOStatisticsSource {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

needs javadocs i'm afraid


/**
* A prefetch operation has started.
* @return duration tracker
*/
DurationTracker prefetchOperationStarted();

/**
* A block has been saved to the file cache.
*/
void blockAddedToFileCache();

/**
* A block has been removed from the file cache.
*/
void blockRemovedFromFileCache();

/**
* A prefetch operation has completed.
*/
void prefetchOperationCompleted();

/**
* An executor has been acquired, either for prefetching or caching.
* @param timeInQueue time taken to acquire an executor.
*/
void executorAcquired(Duration timeInQueue);

/**
* A new buffer has been added to the buffer pool.
* @param size size of the new buffer
*/
void memoryAllocated(int size);

/**
* Previously allocated memory has been freed.
* @param size size of memory freed.
*/
void memoryFreed(int size);
}
Loading