HADOOP-18028. High performance S3A input stream (#4752)

This is the the preview release of the HADOOP-18028 S3A performance input stream.
It is still stabilizing, but ready to test.

Contains

HADOOP-18028. High performance S3A input stream (#4109)
	Contributed by Bhalchandra Pandit.

HADOOP-18180. Replace use of twitter util-core with java futures (#4115)
	Contributed by PJ Fanning.

HADOOP-18177. Document prefetching architecture. (#4205)
	Contributed by Ahmar Suhail

HADOOP-18175. fix test failures with prefetching s3a input stream (#4212)
 Contributed by Monthon Klongklaew

HADOOP-18231.  S3A prefetching: fix failing tests & drain stream async.  (#4386)

	* adds in new test for prefetching input stream
	* creates streamStats before opening stream
	* updates numBlocks calculation method
	* fixes ITestS3AOpenCost.testOpenFileLongerLength
	* drains stream async
	* fixes failing unit test

	Contributed by Ahmar Suhail

HADOOP-18254. Disable S3A prefetching by default. (#4469)
	Contributed by Ahmar Suhail

HADOOP-18190. Collect IOStatistics during S3A prefetching (#4458)

	This adds iOStatisticsConnection to the S3PrefetchingInputStream class, with
	new statistic names in StreamStatistics.

	This stream is not (yet) IOStatisticsContext aware.

	Contributed by Ahmar Suhail

HADOOP-18379 rebase feature/HADOOP-18028-s3a-prefetch to trunk
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching
	Contributed by Steve Loughran
This commit is contained in:
Steve Loughran 2022-08-18 13:53:06 +01:00 committed by GitHub
parent cd72f7e042
commit 682931a6ac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
61 changed files with 9534 additions and 23 deletions

View File

@ -0,0 +1,70 @@
/*
* 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.impl.prefetch;
import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
/**
* Provides functionality necessary for caching blocks of data read from FileSystem.
*/
public interface BlockCache extends Closeable {
/**
* Indicates whether the given block is in this cache.
*
* @param blockNumber the id of the given block.
* @return true if the given block is in this cache, false otherwise.
*/
boolean containsBlock(int blockNumber);
/**
* Gets the blocks in this cache.
*
* @return the blocks in this cache.
*/
Iterable<Integer> blocks();
/**
* Gets the number of blocks in this cache.
*
* @return the number of blocks in this cache.
*/
int size();
/**
* Gets the block having the given {@code blockNumber}.
*
* @param blockNumber the id of the desired block.
* @param buffer contents of the desired block are copied to this buffer.
* @throws IOException if there is an error reading the given block.
*/
void get(int blockNumber, ByteBuffer buffer) throws IOException;
/**
* Puts the given block in this cache.
*
* @param blockNumber the id of the given block.
* @param buffer contents of the given block to be added to this cache.
* @throws IOException if there is an error writing the given block.
*/
void put(int blockNumber, ByteBuffer buffer) throws IOException;
}

View File

@ -0,0 +1,250 @@
/*
* 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.impl.prefetch;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkWithinRange;
/**
* Holds information about blocks of data in a file.
*/
public final class BlockData {
// State of each block of data.
enum State {
/** Data is not yet ready to be read from this block (still being prefetched). */
NOT_READY,
/** A read of this block has been enqueued in the prefetch queue. */
QUEUED,
/** A read of this block has been enqueued in the prefetch queue. */
READY,
/** This block has been cached in the local disk cache. */
CACHED
}
/**
* State of all blocks in a file.
*/
private State[] state;
/**
* The size of a file.
*/
private final long fileSize;
/**
* The file is divided into blocks of this size.
*/
private final int blockSize;
/**
* The file has these many blocks.
*/
private final int numBlocks;
/**
* Constructs an instance of {@link BlockData}.
* @param fileSize the size of a file.
* @param blockSize the file is divided into blocks of this size.
* @throws IllegalArgumentException if fileSize is negative.
* @throws IllegalArgumentException if blockSize is negative.
* @throws IllegalArgumentException if blockSize is zero or negative.
*/
public BlockData(long fileSize, int blockSize) {
checkNotNegative(fileSize, "fileSize");
if (fileSize == 0) {
checkNotNegative(blockSize, "blockSize");
} else {
checkPositiveInteger(blockSize, "blockSize");
}
this.fileSize = fileSize;
this.blockSize = blockSize;
this.numBlocks =
(fileSize == 0)
? 0
: ((int) (fileSize / blockSize)) + (fileSize % blockSize > 0
? 1
: 0);
this.state = new State[this.numBlocks];
for (int b = 0; b < this.numBlocks; b++) {
setState(b, State.NOT_READY);
}
}
/**
* Gets the size of each block.
* @return the size of each block.
*/
public int getBlockSize() {
return blockSize;
}
/**
* Gets the size of the associated file.
* @return the size of the associated file.
*/
public long getFileSize() {
return fileSize;
}
/**
* Gets the number of blocks in the associated file.
* @return the number of blocks in the associated file.
*/
public int getNumBlocks() {
return numBlocks;
}
/**
* Indicates whether the given block is the last block in the associated file.
* @param blockNumber the id of the desired block.
* @return true if the given block is the last block in the associated file, false otherwise.
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public boolean isLastBlock(int blockNumber) {
if (fileSize == 0) {
return false;
}
throwIfInvalidBlockNumber(blockNumber);
return blockNumber == (numBlocks - 1);
}
/**
* Gets the id of the block that contains the given absolute offset.
* @param offset the absolute offset to check.
* @return the id of the block that contains the given absolute offset.
* @throws IllegalArgumentException if offset is invalid.
*/
public int getBlockNumber(long offset) {
throwIfInvalidOffset(offset);
return (int) (offset / blockSize);
}
/**
* Gets the size of the given block.
* @param blockNumber the id of the desired block.
* @return the size of the given block.
*/
public int getSize(int blockNumber) {
if (fileSize == 0) {
return 0;
}
if (isLastBlock(blockNumber)) {
return (int) (fileSize - (((long) blockSize) * (numBlocks - 1)));
} else {
return blockSize;
}
}
/**
* Indicates whether the given absolute offset is valid.
* @param offset absolute offset in the file..
* @return true if the given absolute offset is valid, false otherwise.
*/
public boolean isValidOffset(long offset) {
return (offset >= 0) && (offset < fileSize);
}
/**
* Gets the start offset of the given block.
* @param blockNumber the id of the given block.
* @return the start offset of the given block.
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public long getStartOffset(int blockNumber) {
throwIfInvalidBlockNumber(blockNumber);
return blockNumber * (long) blockSize;
}
/**
* Gets the relative offset corresponding to the given block and the absolute offset.
* @param blockNumber the id of the given block.
* @param offset absolute offset in the file.
* @return the relative offset corresponding to the given block and the absolute offset.
* @throws IllegalArgumentException if either blockNumber or offset is invalid.
*/
public int getRelativeOffset(int blockNumber, long offset) {
throwIfInvalidOffset(offset);
return (int) (offset - getStartOffset(blockNumber));
}
/**
* Gets the state of the given block.
* @param blockNumber the id of the given block.
* @return the state of the given block.
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public State getState(int blockNumber) {
throwIfInvalidBlockNumber(blockNumber);
return state[blockNumber];
}
/**
* Sets the state of the given block to the given value.
* @param blockNumber the id of the given block.
* @param blockState the target state.
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public void setState(int blockNumber, State blockState) {
throwIfInvalidBlockNumber(blockNumber);
state[blockNumber] = blockState;
}
// Debug helper.
public String getStateString() {
StringBuilder sb = new StringBuilder();
int blockNumber = 0;
while (blockNumber < numBlocks) {
State tstate = getState(blockNumber);
int endBlockNumber = blockNumber;
while ((endBlockNumber < numBlocks) && (getState(endBlockNumber)
== tstate)) {
endBlockNumber++;
}
sb.append(
String.format("[%03d ~ %03d] %s%n", blockNumber, endBlockNumber - 1,
tstate));
blockNumber = endBlockNumber;
}
return sb.toString();
}
private void throwIfInvalidBlockNumber(int blockNumber) {
checkWithinRange(blockNumber, "blockNumber", 0, numBlocks - 1);
}
private void throwIfInvalidOffset(long offset) {
checkWithinRange(offset, "offset", 0, fileSize - 1);
}
}

View File

@ -0,0 +1,145 @@
/*
* 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.impl.prefetch;
import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
/**
* Provides read access to the underlying file one block at a time.
*
* This class is the simplest form of a {@code BlockManager} that does
* perform prefetching or caching.
*/
public abstract class BlockManager implements Closeable {
/**
* Information about each block of the underlying file.
*/
private final BlockData blockData;
/**
* Constructs an instance of {@code BlockManager}.
*
* @param blockData information about each block of the underlying file.
*
* @throws IllegalArgumentException if blockData is null.
*/
public BlockManager(BlockData blockData) {
checkNotNull(blockData, "blockData");
this.blockData = blockData;
}
/**
* Gets block data information.
*
* @return instance of {@code BlockData}.
*/
public BlockData getBlockData() {
return blockData;
}
/**
* Gets the block having the given {@code blockNumber}.
*
* The entire block is read into memory and returned as a {@code BufferData}.
* The blocks are treated as a limited resource and must be released when
* one is done reading them.
*
* @param blockNumber the number of the block to be read and returned.
* @return {@code BufferData} having data from the given block.
*
* @throws IOException if there an error reading the given block.
* @throws IllegalArgumentException if blockNumber is negative.
*/
public BufferData get(int blockNumber) throws IOException {
checkNotNegative(blockNumber, "blockNumber");
int size = blockData.getSize(blockNumber);
ByteBuffer buffer = ByteBuffer.allocate(size);
long startOffset = blockData.getStartOffset(blockNumber);
read(buffer, startOffset, size);
buffer.flip();
return new BufferData(blockNumber, buffer);
}
/**
* Reads into the given {@code buffer} {@code size} bytes from the underlying file
* starting at {@code startOffset}.
*
* @param buffer the buffer to read data in to.
* @param startOffset the offset at which reading starts.
* @param size the number bytes to read.
* @return number of bytes read.
* @throws IOException if there an error reading the given block.
*/
public abstract int read(ByteBuffer buffer, long startOffset, int size) throws IOException;
/**
* Releases resources allocated to the given block.
*
* @param data the {@code BufferData} to release.
*
* @throws IllegalArgumentException if data is null.
*/
public void release(BufferData data) {
checkNotNull(data, "data");
// Do nothing because we allocate a new buffer each time.
}
/**
* Requests optional prefetching of the given block.
*
* @param blockNumber the id of the block to prefetch.
*
* @throws IllegalArgumentException if blockNumber is negative.
*/
public void requestPrefetch(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
// Do nothing because we do not support prefetches.
}
/**
* Requests cancellation of any previously issued prefetch requests.
*/
public void cancelPrefetches() {
// Do nothing because we do not support prefetches.
}
/**
* Requests that the given block should be copied to the cache. Optional operation.
*
* @param data the {@code BufferData} instance to optionally cache.
*/
public void requestCaching(BufferData data) {
// Do nothing because we do not support caching.
}
@Override
public void close() {
}
}

View File

@ -0,0 +1,425 @@
/*
* 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.impl.prefetch;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.DoubleSummaryStatistics;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
/**
* Block level operations performed on a file.
* This class is meant to be used by {@code BlockManager}.
* It is separated out in its own file due to its size.
*
* This class is used for debugging/logging. Calls to this class
* can be safely removed without affecting the overall operation.
*/
public final class BlockOperations {
private static final Logger LOG = LoggerFactory.getLogger(BlockOperations.class);
/**
* Operation kind.
*/
public enum Kind {
UNKNOWN("??", "unknown", false),
CANCEL_PREFETCHES("CP", "cancelPrefetches", false),
CLOSE("CX", "close", false),
CACHE_PUT("C+", "putC", true),
GET_CACHED("GC", "getCached", true),
GET_PREFETCHED("GP", "getPrefetched", true),
GET_READ("GR", "getRead", true),
PREFETCH("PF", "prefetch", true),
RELEASE("RL", "release", true),
REQUEST_CACHING("RC", "requestCaching", true),
REQUEST_PREFETCH("RP", "requestPrefetch", true);
private String shortName;
private String name;
private boolean hasBlock;
Kind(String shortName, String name, boolean hasBlock) {
this.shortName = shortName;
this.name = name;
this.hasBlock = hasBlock;
}
private static Map<String, Kind> shortNameToKind = new HashMap<>();
public static Kind fromShortName(String shortName) {
if (shortNameToKind.isEmpty()) {
for (Kind kind : Kind.values()) {
shortNameToKind.put(kind.shortName, kind);
}
}
return shortNameToKind.get(shortName);
}
}
public static class Operation {
private final Kind kind;
private final int blockNumber;
private final long timestamp;
public Operation(Kind kind, int blockNumber) {
this.kind = kind;
this.blockNumber = blockNumber;
this.timestamp = System.nanoTime();
}
public Kind getKind() {
return kind;
}
public int getBlockNumber() {
return blockNumber;
}
public long getTimestamp() {
return timestamp;
}
public void getSummary(StringBuilder sb) {
if (kind.hasBlock) {
sb.append(String.format("%s(%d)", kind.shortName, blockNumber));
} else {
sb.append(String.format("%s", kind.shortName));
}
}
public String getDebugInfo() {
if (kind.hasBlock) {
return String.format("--- %s(%d)", kind.name, blockNumber);
} else {
return String.format("... %s()", kind.name);
}
}
}
public static class End extends Operation {
private Operation op;
public End(Operation op) {
super(op.kind, op.blockNumber);
this.op = op;
}
@Override
public void getSummary(StringBuilder sb) {
sb.append("E");
super.getSummary(sb);
}
@Override
public String getDebugInfo() {
return "***" + super.getDebugInfo().substring(3);
}
public double duration() {
return (getTimestamp() - op.getTimestamp()) / 1e9;
}
}
private ArrayList<Operation> ops;
private boolean debugMode;
public BlockOperations() {
this.ops = new ArrayList<>();
}
public synchronized void setDebug(boolean state) {
debugMode = state;
}
private synchronized Operation add(Operation op) {
if (debugMode) {
LOG.info(op.getDebugInfo());
}
ops.add(op);
return op;
}
public Operation getPrefetched(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.GET_PREFETCHED, blockNumber));
}
public Operation getCached(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.GET_CACHED, blockNumber));
}
public Operation getRead(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.GET_READ, blockNumber));
}
public Operation release(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.RELEASE, blockNumber));
}
public Operation requestPrefetch(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.REQUEST_PREFETCH, blockNumber));
}
public Operation prefetch(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.PREFETCH, blockNumber));
}
public Operation cancelPrefetches() {
return add(new Operation(Kind.CANCEL_PREFETCHES, -1));
}
public Operation close() {
return add(new Operation(Kind.CLOSE, -1));
}
public Operation requestCaching(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.REQUEST_CACHING, blockNumber));
}
public Operation addToCache(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
return add(new Operation(Kind.CACHE_PUT, blockNumber));
}
public Operation end(Operation op) {
return add(new End(op));
}
private static void append(StringBuilder sb, String format, Object... args) {
sb.append(String.format(format, args));
}
public synchronized String getSummary(boolean showDebugInfo) {
StringBuilder sb = new StringBuilder();
for (Operation op : ops) {
if (op != null) {
if (showDebugInfo) {
sb.append(op.getDebugInfo());
sb.append("\n");
} else {
op.getSummary(sb);
sb.append(";");
}
}
}
sb.append("\n");
getDurationInfo(sb);
return sb.toString();
}
public synchronized void getDurationInfo(StringBuilder sb) {
Map<Kind, DoubleSummaryStatistics> durations = new HashMap<>();
for (Operation op : ops) {
if (op instanceof End) {
End endOp = (End) op;
DoubleSummaryStatistics stats = durations.get(endOp.getKind());
if (stats == null) {
stats = new DoubleSummaryStatistics();
durations.put(endOp.getKind(), stats);
}
stats.accept(endOp.duration());
}
}
List<Kind> kinds = Arrays.asList(
Kind.GET_CACHED,
Kind.GET_PREFETCHED,
Kind.GET_READ,
Kind.CACHE_PUT,
Kind.PREFETCH,
Kind.REQUEST_CACHING,
Kind.REQUEST_PREFETCH,
Kind.CANCEL_PREFETCHES,
Kind.RELEASE,
Kind.CLOSE
);
for (Kind kind : kinds) {
append(sb, "%-18s : ", kind);
DoubleSummaryStatistics stats = durations.get(kind);
if (stats == null) {
append(sb, "--\n");
} else {
append(
sb,
"#ops = %3d, total = %5.1f, min: %3.1f, avg: %3.1f, max: %3.1f\n",
stats.getCount(),
stats.getSum(),
stats.getMin(),
stats.getAverage(),
stats.getMax());
}
}
}
public synchronized void analyze(StringBuilder sb) {
Map<Integer, List<Operation>> blockOps = new HashMap<>();
// Group-by block number.
for (Operation op : ops) {
if (op.blockNumber < 0) {
continue;
}
List<Operation> perBlockOps;
if (!blockOps.containsKey(op.blockNumber)) {
perBlockOps = new ArrayList<>();
blockOps.put(op.blockNumber, perBlockOps);
}
perBlockOps = blockOps.get(op.blockNumber);
perBlockOps.add(op);
}
List<Integer> prefetchedNotUsed = new ArrayList<>();
List<Integer> cachedNotUsed = new ArrayList<>();
for (Map.Entry<Integer, List<Operation>> entry : blockOps.entrySet()) {
Integer blockNumber = entry.getKey();
List<Operation> perBlockOps = entry.getValue();
Map<Kind, Integer> kindCounts = new HashMap<>();
Map<Kind, Integer> endKindCounts = new HashMap<>();
for (Operation op : perBlockOps) {
if (op instanceof End) {
int endCount = endKindCounts.getOrDefault(op.kind, 0) + 1;
endKindCounts.put(op.kind, endCount);
} else {
int count = kindCounts.getOrDefault(op.kind, 0) + 1;
kindCounts.put(op.kind, count);
}
}
for (Kind kind : kindCounts.keySet()) {
int count = kindCounts.getOrDefault(kind, 0);
int endCount = endKindCounts.getOrDefault(kind, 0);
if (count != endCount) {
append(sb, "[%d] %s : #ops(%d) != #end-ops(%d)\n", blockNumber, kind, count, endCount);
}
if (count > 1) {
append(sb, "[%d] %s = %d\n", blockNumber, kind, count);
}
}
int prefetchCount = kindCounts.getOrDefault(Kind.PREFETCH, 0);
int getPrefetchedCount = kindCounts.getOrDefault(Kind.GET_PREFETCHED, 0);
if ((prefetchCount > 0) && (getPrefetchedCount < prefetchCount)) {
prefetchedNotUsed.add(blockNumber);
}
int cacheCount = kindCounts.getOrDefault(Kind.CACHE_PUT, 0);
int getCachedCount = kindCounts.getOrDefault(Kind.GET_CACHED, 0);
if ((cacheCount > 0) && (getCachedCount < cacheCount)) {
cachedNotUsed.add(blockNumber);
}
}
if (!prefetchedNotUsed.isEmpty()) {
append(sb, "Prefetched but not used: %s\n", getIntList(prefetchedNotUsed));
}
if (!cachedNotUsed.isEmpty()) {
append(sb, "Cached but not used: %s\n", getIntList(cachedNotUsed));
}
}
private static String getIntList(Iterable<Integer> nums) {
List<String> numList = new ArrayList<>();
for (Integer n : nums) {
numList.add(n.toString());
}
return String.join(", ", numList);
}
public static BlockOperations fromSummary(String summary) {
BlockOperations ops = new BlockOperations();
ops.setDebug(true);
Pattern blockOpPattern = Pattern.compile("([A-Z+]+)(\\(([0-9]+)?\\))?");
String[] tokens = summary.split(";");
for (String token : tokens) {
Matcher matcher = blockOpPattern.matcher(token);
if (!matcher.matches()) {
String message = String.format("Unknown summary format: %s", token);
throw new IllegalArgumentException(message);
}
String shortName = matcher.group(1);
String blockNumberStr = matcher.group(3);
int blockNumber = (blockNumberStr == null) ? -1 : Integer.parseInt(blockNumberStr);
Kind kind = Kind.fromShortName(shortName);
Kind endKind = null;
if (kind == null) {
if (shortName.charAt(0) == 'E') {
endKind = Kind.fromShortName(shortName.substring(1));
}
}
if (kind == null && endKind == null) {
String message = String.format("Unknown short name: %s (token = %s)", shortName, token);
throw new IllegalArgumentException(message);
}
if (kind != null) {
ops.add(new Operation(kind, blockNumber));
} else {
Operation op = null;
for (int i = ops.ops.size() - 1; i >= 0; i--) {
op = ops.ops.get(i);
if ((op.blockNumber == blockNumber) && (op.kind == endKind) && !(op instanceof End)) {
ops.add(new End(op));
break;
}
}
if (op == null) {
LOG.warn("Start op not found: {}({})", endKind, blockNumber);
}
}
}
return ops;
}
}

View File

@ -0,0 +1,195 @@
/*
* 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.impl.prefetch;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
/**
* Manages a fixed pool of resources.
*
* Avoids creating a new resource if a previously created instance is already available.
*/
public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
/**
* The size of this pool. Fixed at creation time.
*/
private final int size;
/**
* Items currently available in the pool.
*/
private ArrayBlockingQueue<T> items;
/**
* Items that have been created so far (regardless of whether they are currently available).
*/
private Set<T> createdItems;
/**
* Constructs a resource pool of the given size.
*
* @param size the size of this pool. Cannot be changed post creation.
*
* @throws IllegalArgumentException if size is zero or negative.
*/
public BoundedResourcePool(int size) {
Validate.checkPositiveInteger(size, "size");
this.size = size;
this.items = new ArrayBlockingQueue<>(size);
// The created items are identified based on their object reference.
this.createdItems = Collections.newSetFromMap(new IdentityHashMap<T, Boolean>());
}
/**
* Acquires a resource blocking if necessary until one becomes available.
*/
@Override
public T acquire() {
return this.acquireHelper(true);
}
/**
* Acquires a resource blocking if one is immediately available. Otherwise returns null.
*/
@Override
public T tryAcquire() {
return this.acquireHelper(false);
}
/**
* Releases a previously acquired resource.
*
* @throws IllegalArgumentException if item is null.
*/
@Override
public void release(T item) {
checkNotNull(item, "item");
synchronized (createdItems) {
if (!createdItems.contains(item)) {
throw new IllegalArgumentException("This item is not a part of this pool");
}
}
// Return if this item was released earlier.
// We cannot use items.contains() because that check is not based on reference equality.
for (T entry : items) {
if (entry == item) {
return;
}
}
try {
items.put(item);
} catch (InterruptedException e) {
throw new IllegalStateException("release() should never block", e);
}
}
@Override
public synchronized void close() {
for (T item : createdItems) {
close(item);
}
items.clear();
items = null;
createdItems.clear();
createdItems = null;
}
/**
* Derived classes may implement a way to cleanup each item.
*/
@Override
protected synchronized void close(T item) {
// Do nothing in this class. Allow overriding classes to take any cleanup action.
}
/**
* Number of items created so far. Mostly for testing purposes.
* @return the count.
*/
public int numCreated() {
synchronized (createdItems) {
return createdItems.size();
}
}
/**
* Number of items available to be acquired. Mostly for testing purposes.
* @return the number available.
*/
public synchronized int numAvailable() {
return (size - numCreated()) + items.size();
}
// For debugging purposes.
@Override
public synchronized String toString() {
return String.format(
"size = %d, #created = %d, #in-queue = %d, #available = %d",
size, numCreated(), items.size(), numAvailable());
}
/**
* Derived classes must implement a way to create an instance of a resource.
*/
protected abstract T createNew();
private T acquireHelper(boolean canBlock) {
// Prefer reusing an item if one is available.
// That avoids unnecessarily creating new instances.
T result = items.poll();
if (result != null) {
return result;
}
synchronized (createdItems) {
// Create a new instance if allowed by the capacity of this pool.
if (createdItems.size() < size) {
T item = createNew();
createdItems.add(item);
return item;
}
}
if (canBlock) {
try {
// Block for an instance to be available.
return items.take();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
}
} else {
return null;
}
}
}

View File

@ -0,0 +1,319 @@
/*
* 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.impl.prefetch;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Future;
import java.util.zip.CRC32;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Holds the state of a ByteBuffer that is in use by {@code CachingBlockManager}.
*
* This class is not meant to be of general use. It exists into its own file due to its size.
* We use the term block and buffer interchangeably in this file because one buffer
* holds exactly one block of data.
*
* Holding all of the state associated with a block allows us to validate and control
* state transitions in a synchronized fashion.
*/
public final class BufferData {
private static final Logger LOG = LoggerFactory.getLogger(BufferData.class);
public enum State {
/**
* Unknown / invalid state.
*/
UNKNOWN,
/**
* Buffer has been acquired but has no data.
*/
BLANK,
/**
* This block is being prefetched.
*/
PREFETCHING,
/**
* This block is being added to the local cache.
*/
CACHING,
/**
* This block has data and is ready to be read.
*/
READY,
/**
* This block is no longer in-use and should not be used once in this state.
*/
DONE
}
/**
* Number of the block associated with this buffer.
*/
private final int blockNumber;
/**
* The buffer associated with this block.
*/
private ByteBuffer buffer;
/**
* Current state of this block.
*/
private volatile State state;
/**
* Future of the action being performed on this block (eg, prefetching or caching).
*/
private Future<Void> action;
/**
* Checksum of the buffer contents once in READY state.
*/
private long checksum = 0;
/**
* Constructs an instances of this class.
*
* @param blockNumber Number of the block associated with this buffer.
* @param buffer The buffer associated with this block.
*
* @throws IllegalArgumentException if blockNumber is negative.
* @throws IllegalArgumentException if buffer is null.
*/
public BufferData(int blockNumber, ByteBuffer buffer) {
Validate.checkNotNegative(blockNumber, "blockNumber");
Validate.checkNotNull(buffer, "buffer");
this.blockNumber = blockNumber;
this.buffer = buffer;
this.state = State.BLANK;
}
/**
* Gets the id of this block.
*
* @return the id of this block.
*/
public int getBlockNumber() {
return this.blockNumber;
}
/**
* Gets the buffer associated with this block.
*
* @return the buffer associated with this block.
*/
public ByteBuffer getBuffer() {
return this.buffer;
}
/**
* Gets the state of this block.
*
* @return the state of this block.
*/
public State getState() {
return this.state;
}
/**
* Gets the checksum of data in this block.
*
* @return the checksum of data in this block.
*/
public long getChecksum() {
return this.checksum;
}
/**
* Computes CRC32 checksum of the given buffer's contents.
*
* @param buffer the buffer whose content's checksum is to be computed.
* @return the computed checksum.
*/
public static long getChecksum(ByteBuffer buffer) {
ByteBuffer tempBuffer = buffer.duplicate();
tempBuffer.rewind();
CRC32 crc32 = new CRC32();
crc32.update(tempBuffer);
return crc32.getValue();
}
public synchronized Future<Void> getActionFuture() {
return this.action;
}
/**
* Indicates that a prefetch operation is in progress.
*
* @param actionFuture the {@code Future} of a prefetch action.
*
* @throws IllegalArgumentException if actionFuture is null.
*/
public synchronized void setPrefetch(Future<Void> actionFuture) {
Validate.checkNotNull(actionFuture, "actionFuture");
this.updateState(State.PREFETCHING, State.BLANK);
this.action = actionFuture;
}
/**
* Indicates that a caching operation is in progress.
*
* @param actionFuture the {@code Future} of a caching action.
*
* @throws IllegalArgumentException if actionFuture is null.
*/
public synchronized void setCaching(Future<Void> actionFuture) {
Validate.checkNotNull(actionFuture, "actionFuture");
this.throwIfStateIncorrect(State.PREFETCHING, State.READY);
this.state = State.CACHING;
this.action = actionFuture;
}
/**
* Marks the completion of reading data into the buffer.
* The buffer cannot be modified once in this state.
*
* @param expectedCurrentState the collection of states from which transition to READY is allowed.
*/
public synchronized void setReady(State... expectedCurrentState) {
if (this.checksum != 0) {
throw new IllegalStateException("Checksum cannot be changed once set");
}
this.buffer = this.buffer.asReadOnlyBuffer();
this.checksum = getChecksum(this.buffer);
this.buffer.rewind();
this.updateState(State.READY, expectedCurrentState);
}
/**
* Indicates that this block is no longer of use and can be reclaimed.
*/
public synchronized void setDone() {
if (this.checksum != 0) {
if (getChecksum(this.buffer) != this.checksum) {
throw new IllegalStateException("checksum changed after setReady()");
}
}
this.state = State.DONE;
this.action = null;
}
/**
* Updates the current state to the specified value.
* Asserts that the current state is as expected.
* @param newState the state to transition to.
* @param expectedCurrentState the collection of states from which
* transition to {@code newState} is allowed.
*
* @throws IllegalArgumentException if newState is null.
* @throws IllegalArgumentException if expectedCurrentState is null.
*/
public synchronized void updateState(State newState,
State... expectedCurrentState) {
Validate.checkNotNull(newState, "newState");
Validate.checkNotNull(expectedCurrentState, "expectedCurrentState");
this.throwIfStateIncorrect(expectedCurrentState);
this.state = newState;
}
/**
* Helper that asserts the current state is one of the expected values.
*
* @param states the collection of allowed states.
*
* @throws IllegalArgumentException if states is null.
*/
public void throwIfStateIncorrect(State... states) {
Validate.checkNotNull(states, "states");
if (this.stateEqualsOneOf(states)) {
return;
}
List<String> statesStr = new ArrayList<String>();
for (State s : states) {
statesStr.add(s.toString());
}
String message = String.format(
"Expected buffer state to be '%s' but found: %s",
String.join(" or ", statesStr), this);
throw new IllegalStateException(message);
}
public boolean stateEqualsOneOf(State... states) {
State currentState = this.state;
for (State s : states) {
if (currentState == s) {
return true;
}
}
return false;
}
public String toString() {
return String.format(
"[%03d] id: %03d, %s: buf: %s, checksum: %d, future: %s",
this.blockNumber,
System.identityHashCode(this),
this.state,
this.getBufferStr(this.buffer),
this.checksum,
this.getFutureStr(this.action));
}
private String getFutureStr(Future<Void> f) {
if (f == null) {
return "--";
} else {
return this.action.isDone() ? "done" : "not done";
}
}
private String getBufferStr(ByteBuffer buf) {
if (buf == null) {
return "--";
} else {
return String.format(
"(id = %d, pos = %d, lim = %d)",
System.identityHashCode(buf),
buf.position(), buf.limit());
}
}
}

View File

@ -0,0 +1,323 @@
/*
* 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.impl.prefetch;
import java.io.Closeable;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Future;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkState;
import static org.apache.hadoop.util.Preconditions.checkArgument;
import static org.apache.hadoop.util.Preconditions.checkNotNull;
/**
* Manages a fixed pool of {@code ByteBuffer} instances.
* <p>
* Avoids creating a new buffer if a previously created buffer is already available.
*/
public class BufferPool implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(BufferPool.class);
/**
* Max number of buffers in this pool.
*/
private final int size;
/**
* Size in bytes of each buffer.
*/
private final int bufferSize;
/*
Invariants for internal state.
-- a buffer is either in this.pool or in this.allocated
-- transition between this.pool <==> this.allocated must be atomic
-- only one buffer allocated for a given blockNumber
*/
/**
* Underlying bounded resource pool.
*/
private BoundedResourcePool<ByteBuffer> pool;
/**
* Allows associating metadata to each buffer in the pool.
*/
private Map<BufferData, ByteBuffer> allocated;
/**
* Prefetching stats.
*/
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,
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);
this.pool = new BoundedResourcePool<ByteBuffer>(size) {
@Override
public ByteBuffer createNew() {
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
prefetchingStatistics.memoryAllocated(bufferSize);
return buffer;
}
};
}
/**
* Gets a list of all blocks in this pool.
* @return a list of all blocks in this pool.
*/
public List<BufferData> getAll() {
synchronized (allocated) {
return Collections.unmodifiableList(new ArrayList<>(allocated.keySet()));
}
}
/**
* Acquires a {@code ByteBuffer}; blocking if necessary until one becomes available.
* @param blockNumber the id of the block to acquire.
* @return the acquired block's {@code BufferData}.
*/
public synchronized BufferData acquire(int blockNumber) {
BufferData data;
final int maxRetryDelayMs = 600 * 1000;
final int statusUpdateDelayMs = 120 * 1000;
Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs);
do {
if (retryer.updateStatus()) {
if (LOG.isDebugEnabled()) {
LOG.debug("waiting to acquire block: {}", blockNumber);
LOG.debug("state = {}", this);
}
releaseReadyBlock(blockNumber);
}
data = tryAcquire(blockNumber);
}
while ((data == null) && retryer.continueRetry());
if (data != null) {
return data;
} else {
String message =
String.format("Wait failed for acquire(%d)", blockNumber);
throw new IllegalStateException(message);
}
}
/**
* Acquires a buffer if one is immediately available. Otherwise returns null.
* @param blockNumber the id of the block to try acquire.
* @return the acquired block's {@code BufferData} or null.
*/
public synchronized BufferData tryAcquire(int blockNumber) {
return acquireHelper(blockNumber, false);
}
private synchronized BufferData acquireHelper(int blockNumber,
boolean canBlock) {
checkNotNegative(blockNumber, "blockNumber");
releaseDoneBlocks();
BufferData data = find(blockNumber);
if (data != null) {
return data;
}
ByteBuffer buffer = canBlock ? pool.acquire() : pool.tryAcquire();
if (buffer == null) {
return null;
}
buffer.clear();
data = new BufferData(blockNumber, buffer.duplicate());
synchronized (allocated) {
checkState(find(blockNumber) == null, "buffer data already exists");
allocated.put(data, buffer);
}
return data;
}
/**
* Releases resources for any blocks marked as 'done'.
*/
private synchronized void releaseDoneBlocks() {
for (BufferData data : getAll()) {
if (data.stateEqualsOneOf(BufferData.State.DONE)) {
release(data);
}
}
}
/**
* If no blocks were released after calling releaseDoneBlocks() a few times,
* we may end up waiting forever. To avoid that situation, we try releasing
* a 'ready' block farthest away from the given block.
*/
private synchronized void releaseReadyBlock(int blockNumber) {
BufferData releaseTarget = null;
for (BufferData data : getAll()) {
if (data.stateEqualsOneOf(BufferData.State.READY)) {
if (releaseTarget == null) {
releaseTarget = data;
} else {
if (distance(data, blockNumber) > distance(releaseTarget,
blockNumber)) {
releaseTarget = data;
}
}
}
}
if (releaseTarget != null) {
LOG.warn("releasing 'ready' block: {}", releaseTarget);
releaseTarget.setDone();
}
}
private int distance(BufferData data, int blockNumber) {
return Math.abs(data.getBlockNumber() - blockNumber);
}
/**
* Releases a previously acquired resource.
* @param data the {@code BufferData} instance to release.
* @throws IllegalArgumentException if data is null.
* @throws IllegalArgumentException if data cannot be released due to its state.
*/
public synchronized void release(BufferData data) {
checkNotNull(data, "data");
synchronized (data) {
checkArgument(
canRelease(data),
String.format("Unable to release buffer: %s", data));
ByteBuffer buffer = allocated.get(data);
if (buffer == null) {
// Likely released earlier.
return;
}
buffer.clear();
pool.release(buffer);
allocated.remove(data);
}
releaseDoneBlocks();
}
@Override
public synchronized void close() {
for (BufferData data : getAll()) {
Future<Void> actionFuture = data.getActionFuture();
if (actionFuture != null) {
actionFuture.cancel(true);
}
}
int currentPoolSize = pool.numCreated();
pool.close();
pool = null;
allocated.clear();
allocated = null;
prefetchingStatistics.memoryFreed(currentPoolSize * bufferSize);
}
// For debugging purposes.
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(pool.toString());
sb.append("\n");
List<BufferData> allData = new ArrayList<>(getAll());
Collections.sort(allData,
(d1, d2) -> d1.getBlockNumber() - d2.getBlockNumber());
for (BufferData data : allData) {
sb.append(data.toString());
sb.append("\n");
}
return sb.toString();
}
// Number of ByteBuffers created so far.
public synchronized int numCreated() {
return pool.numCreated();
}
// Number of ByteBuffers available to be acquired.
public synchronized int numAvailable() {
releaseDoneBlocks();
return pool.numAvailable();
}
private BufferData find(int blockNumber) {
synchronized (allocated) {
for (BufferData data : allocated.keySet()) {
if ((data.getBlockNumber() == blockNumber)
&& !data.stateEqualsOneOf(BufferData.State.DONE)) {
return data;
}
}
}
return null;
}
private boolean canRelease(BufferData data) {
return data.stateEqualsOneOf(
BufferData.State.DONE,
BufferData.State.READY);
}
}

View File

@ -0,0 +1,638 @@
/*
* 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.impl.prefetch;
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;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
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.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Provides read access to the underlying file one block at a time.
* Improve read performance by prefetching and locall caching blocks.
*/
public abstract class CachingBlockManager extends BlockManager {
private static final Logger LOG = LoggerFactory.getLogger(CachingBlockManager.class);
private static final int TIMEOUT_MINUTES = 60;
/**
* Asynchronous tasks are performed in this pool.
*/
private final ExecutorServiceFuturePool futurePool;
/**
* Pool of shared ByteBuffer instances.
*/
private BufferPool bufferPool;
/**
* Size of the in-memory cache in terms of number of blocks.
* Total memory consumption is up to bufferPoolSize * blockSize.
*/
private final int bufferPoolSize;
/**
* Local block cache.
*/
private BlockCache cache;
/**
* Error counts. For testing purposes.
*/
private final AtomicInteger numCachingErrors;
private final AtomicInteger numReadErrors;
/**
* Operations performed by this block manager.
*/
private final BlockOperations ops;
private boolean closed;
/**
* If a single caching operation takes more than this time (in seconds),
* we disable caching to prevent further perf degradation due to caching.
*/
private static final int SLOW_CACHING_THRESHOLD = 5;
/**
* 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 bufferPoolSize is zero or negative.
*/
public CachingBlockManager(
ExecutorServiceFuturePool futurePool,
BlockData blockData,
int bufferPoolSize,
PrefetchingStatistics prefetchingStatistics) {
super(blockData);
Validate.checkPositiveInteger(bufferPoolSize, "bufferPoolSize");
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.prefetchingStatistics);
this.cache = this.createCache();
}
this.ops = new BlockOperations();
this.ops.setDebug(false);
}
/**
* Gets the block having the given {@code blockNumber}.
*
* @throws IllegalArgumentException if blockNumber is negative.
*/
@Override
public BufferData get(int blockNumber) throws IOException {
checkNotNegative(blockNumber, "blockNumber");
BufferData data;
final int maxRetryDelayMs = bufferPoolSize * 120 * 1000;
final int statusUpdateDelayMs = 120 * 1000;
Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs);
boolean done;
do {
if (closed) {
throw new IOException("this stream is already closed");
}
data = bufferPool.acquire(blockNumber);
done = getInternal(data);
if (retryer.updateStatus()) {
LOG.warn("waiting to get block: {}", blockNumber);
LOG.info("state = {}", this.toString());
}
}
while (!done && retryer.continueRetry());
if (done) {
return data;
} else {
String message = String.format("Wait failed for get(%d)", blockNumber);
throw new IllegalStateException(message);
}
}
private boolean getInternal(BufferData data) throws IOException {
Validate.checkNotNull(data, "data");
// Opportunistic check without locking.
if (data.stateEqualsOneOf(
BufferData.State.PREFETCHING,
BufferData.State.CACHING,
BufferData.State.DONE)) {
return false;
}
synchronized (data) {
// Reconfirm state after locking.
if (data.stateEqualsOneOf(
BufferData.State.PREFETCHING,
BufferData.State.CACHING,
BufferData.State.DONE)) {
return false;
}
int blockNumber = data.getBlockNumber();
if (data.getState() == BufferData.State.READY) {
BlockOperations.Operation op = ops.getPrefetched(blockNumber);
ops.end(op);
return true;
}
data.throwIfStateIncorrect(BufferData.State.BLANK);
read(data);
return true;
}
}
/**
* Releases resources allocated to the given block.
*
* @throws IllegalArgumentException if data is null.
*/
@Override
public void release(BufferData data) {
if (closed) {
return;
}
Validate.checkNotNull(data, "data");
BlockOperations.Operation op = ops.release(data.getBlockNumber());
bufferPool.release(data);
ops.end(op);
}
@Override
public synchronized void close() {
if (closed) {
return;
}
closed = true;
final BlockOperations.Operation op = ops.close();
// Cancel any prefetches in progress.
cancelPrefetches();
cleanupWithLogger(LOG, cache);
ops.end(op);
LOG.info(ops.getSummary(false));
bufferPool.close();
bufferPool = null;
}
/**
* Requests optional prefetching of the given block.
* The block is prefetched only if we can acquire a free buffer.
*
* @throws IllegalArgumentException if blockNumber is negative.
*/
@Override
public void requestPrefetch(int blockNumber) {
checkNotNegative(blockNumber, "blockNumber");
if (closed) {
return;
}
// We initiate a prefetch only if we can acquire a buffer from the shared pool.
BufferData data = bufferPool.tryAcquire(blockNumber);
if (data == null) {
return;
}
// Opportunistic check without locking.
if (!data.stateEqualsOneOf(BufferData.State.BLANK)) {
// The block is ready or being prefetched/cached.
return;
}
synchronized (data) {
// Reconfirm state after locking.
if (!data.stateEqualsOneOf(BufferData.State.BLANK)) {
// The block is ready or being prefetched/cached.
return;
}
BlockOperations.Operation op = ops.requestPrefetch(blockNumber);
PrefetchTask prefetchTask = new PrefetchTask(data, this, Instant.now());
Future<Void> prefetchFuture = futurePool.executeFunction(prefetchTask);
data.setPrefetch(prefetchFuture);
ops.end(op);
}
}
/**
* Requests cancellation of any previously issued prefetch requests.
*/
@Override
public void cancelPrefetches() {
BlockOperations.Operation op = ops.cancelPrefetches();
for (BufferData data : bufferPool.getAll()) {
// We add blocks being prefetched to the local cache so that the prefetch is not wasted.
if (data.stateEqualsOneOf(BufferData.State.PREFETCHING, BufferData.State.READY)) {
requestCaching(data);
}
}
ops.end(op);
}
private void read(BufferData data) throws IOException {
synchronized (data) {
readBlock(data, false, BufferData.State.BLANK);
}
}
private void prefetch(BufferData data, Instant taskQueuedStartTime) throws IOException {
synchronized (data) {
prefetchingStatistics.executorAcquired(
Duration.between(taskQueuedStartTime, Instant.now()));
readBlock(
data,
true,
BufferData.State.PREFETCHING,
BufferData.State.CACHING);
}
}
private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... expectedState)
throws IOException {
if (closed) {
return;
}
BlockOperations.Operation op = null;
DurationTracker tracker = null;
synchronized (data) {
try {
if (data.stateEqualsOneOf(BufferData.State.DONE, BufferData.State.READY)) {
// DONE : Block was released, likely due to caching being disabled on slow perf.
// READY : Block was already fetched by another thread. No need to re-read.
return;
}
data.throwIfStateIncorrect(expectedState);
int blockNumber = data.getBlockNumber();
// Prefer reading from cache over reading from network.
if (cache.containsBlock(blockNumber)) {
op = ops.getCached(blockNumber);
cache.get(blockNumber, data.getBuffer());
data.setReady(expectedState);
return;
}
if (isPrefetch) {
tracker = prefetchingStatistics.prefetchOperationStarted();
op = ops.prefetch(data.getBlockNumber());
} else {
op = ops.getRead(data.getBlockNumber());
}
long offset = getBlockData().getStartOffset(data.getBlockNumber());
int size = getBlockData().getSize(data.getBlockNumber());
ByteBuffer buffer = data.getBuffer();
buffer.clear();
read(buffer, offset, size);
buffer.flip();
data.setReady(expectedState);
} catch (Exception e) {
String message = String.format("error during readBlock(%s)", data.getBlockNumber());
LOG.error(message, e);
if (isPrefetch && tracker != null) {
tracker.failed();
}
numReadErrors.incrementAndGet();
data.setDone();
throw e;
} finally {
if (op != null) {
ops.end(op);
}
if (isPrefetch) {
prefetchingStatistics.prefetchOperationCompleted();
if (tracker != null) {
tracker.close();
}
}
}
}
}
/**
* Read task that is submitted to the future pool.
*/
private static class PrefetchTask implements Supplier<Void> {
private final BufferData data;
private final CachingBlockManager blockManager;
private final Instant taskQueuedStartTime;
PrefetchTask(BufferData data, CachingBlockManager blockManager, Instant taskQueuedStartTime) {
this.data = data;
this.blockManager = blockManager;
this.taskQueuedStartTime = taskQueuedStartTime;
}
@Override
public Void get() {
try {
blockManager.prefetch(data, taskQueuedStartTime);
} catch (Exception e) {
LOG.error("error during prefetch", e);
}
return null;
}
}
private static final BufferData.State[] EXPECTED_STATE_AT_CACHING =
new BufferData.State[] {
BufferData.State.PREFETCHING, BufferData.State.READY
};
/**
* Requests that the given block should be copied to the local cache.
* The block must not be accessed by the caller after calling this method
* because it will released asynchronously relative to the caller.
*
* @throws IllegalArgumentException if data is null.
*/
@Override
public void requestCaching(BufferData data) {
if (closed) {
return;
}
if (cachingDisabled.get()) {
data.setDone();
return;
}
Validate.checkNotNull(data, "data");
// Opportunistic check without locking.
if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) {
return;
}
synchronized (data) {
// Reconfirm state after locking.
if (!data.stateEqualsOneOf(EXPECTED_STATE_AT_CACHING)) {
return;
}
if (cache.containsBlock(data.getBlockNumber())) {
data.setDone();
return;
}
BufferData.State state = data.getState();
BlockOperations.Operation op = ops.requestCaching(data.getBlockNumber());
Future<Void> blockFuture;
if (state == BufferData.State.PREFETCHING) {
blockFuture = data.getActionFuture();
} else {
CompletableFuture<Void> cf = new CompletableFuture<>();
cf.complete(null);
blockFuture = cf;
}
CachePutTask task = new CachePutTask(data, blockFuture, this, Instant.now());
Future<Void> actionFuture = futurePool.executeFunction(task);
data.setCaching(actionFuture);
ops.end(op);
}
}
private void addToCacheAndRelease(BufferData data, Future<Void> blockFuture,
Instant taskQueuedStartTime) {
prefetchingStatistics.executorAcquired(
Duration.between(taskQueuedStartTime, Instant.now()));
if (closed) {
return;
}
if (cachingDisabled.get()) {
data.setDone();
return;
}
try {
blockFuture.get(TIMEOUT_MINUTES, TimeUnit.MINUTES);
if (data.stateEqualsOneOf(BufferData.State.DONE)) {
// There was an error during prefetch.
return;
}
} catch (Exception e) {
LOG.error("error waiting on blockFuture: {}", data, e);
data.setDone();
return;
}
if (cachingDisabled.get()) {
data.setDone();
return;
}
BlockOperations.Operation op = null;
synchronized (data) {
try {
if (data.stateEqualsOneOf(BufferData.State.DONE)) {
return;
}
if (cache.containsBlock(data.getBlockNumber())) {
data.setDone();
return;
}
op = ops.addToCache(data.getBlockNumber());
ByteBuffer buffer = data.getBuffer().duplicate();
buffer.rewind();
cachePut(data.getBlockNumber(), buffer);
data.setDone();
} catch (Exception e) {
numCachingErrors.incrementAndGet();
String message = String.format("error adding block to cache after wait: %s", data);
LOG.error(message, e);
data.setDone();
}
if (op != null) {
BlockOperations.End endOp = (BlockOperations.End) ops.end(op);
if (endOp.duration() > SLOW_CACHING_THRESHOLD) {
if (!cachingDisabled.getAndSet(true)) {
String message = String.format(
"Caching disabled because of slow operation (%.1f sec)", endOp.duration());
LOG.warn(message);
}
}
}
}
}
protected BlockCache createCache() {
return new SingleFilePerBlockCache(prefetchingStatistics);
}
protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException {
if (closed) {
return;
}
cache.put(blockNumber, buffer);
}
private static class CachePutTask implements Supplier<Void> {
private final BufferData data;
// Block being asynchronously fetched.
private final Future<Void> blockFuture;
// Block manager that manages this block.
private final CachingBlockManager blockManager;
private final Instant taskQueuedStartTime;
CachePutTask(
BufferData data,
Future<Void> blockFuture,
CachingBlockManager blockManager,
Instant taskQueuedStartTime) {
this.data = data;
this.blockFuture = blockFuture;
this.blockManager = blockManager;
this.taskQueuedStartTime = taskQueuedStartTime;
}
@Override
public Void get() {
blockManager.addToCacheAndRelease(data, blockFuture, taskQueuedStartTime);
return null;
}
}
/**
* Number of ByteBuffers available to be acquired.
*
* @return the number of available buffers.
*/
public int numAvailable() {
return bufferPool.numAvailable();
}
/**
* Number of caching operations completed.
*
* @return the number of cached buffers.
*/
public int numCached() {
return cache.size();
}
/**
* Number of errors encountered when caching.
*
* @return the number of errors encountered when caching.
*/
public int numCachingErrors() {
return numCachingErrors.get();
}
/**
* Number of errors encountered when reading.
*
* @return the number of errors encountered when reading.
*/
public int numReadErrors() {
return numReadErrors.get();
}
BufferData getData(int blockNumber) {
return bufferPool.tryAcquire(blockNumber);
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("cache(");
sb.append(cache.toString());
sb.append("); ");
sb.append("pool: ");
sb.append(bufferPool.toString());
return sb.toString();
}
}

View File

@ -0,0 +1,80 @@
/*
* 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.impl.prefetch;
import java.time.Duration;
import org.apache.hadoop.fs.statistics.DurationTracker;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
/**
* Empty implementation of the prefetching statistics interface.
*/
public final class EmptyPrefetchingStatistics
implements PrefetchingStatistics {
private static final EmptyPrefetchingStatistics
EMPTY_PREFETCHING_STATISTICS =
new EmptyPrefetchingStatistics();
private EmptyPrefetchingStatistics() {
}
public static EmptyPrefetchingStatistics getInstance() {
return EMPTY_PREFETCHING_STATISTICS;
}
@Override
public DurationTracker prefetchOperationStarted() {
return stubDurationTracker();
}
@Override
public void blockAddedToFileCache() {
}
@Override
public void blockRemovedFromFileCache() {
}
@Override
public void prefetchOperationCompleted() {
}
@Override
public void executorAcquired(Duration timeInQueue) {
}
@Override
public void memoryAllocated(int size) {
}
@Override
public void memoryFreed(int size) {
}
}

View File

@ -0,0 +1,70 @@
/*
* 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.impl.prefetch;
import java.util.Locale;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.function.Supplier;
/**
* A FuturePool implementation backed by a java.util.concurrent.ExecutorService.
*
* If a piece of work has started, it cannot (currently) be cancelled.
*
* This class is a simplified version of <code>com.twitter:util-core_2.11</code>
* ExecutorServiceFuturePool designed to avoid depending on that Scala library.
* One problem with using a Scala library is that many downstream projects
* (eg Apache Spark) use Scala, and they might want to use a different version of Scala
* from the version that Hadoop chooses to use.
*
*/
public class ExecutorServiceFuturePool {
private ExecutorService executor;
public ExecutorServiceFuturePool(ExecutorService executor) {
this.executor = executor;
}
/**
* @param f function to run in future on executor pool
* @return future
* @throws java.util.concurrent.RejectedExecutionException can be thrown
* @throws NullPointerException if f param is null
*/
public Future<Void> executeFunction(final Supplier<Void> f) {
return executor.submit(f::get);
}
/**
* @param r runnable to run in future on executor pool
* @return future
* @throws java.util.concurrent.RejectedExecutionException can be thrown
* @throws NullPointerException if r param is null
*/
@SuppressWarnings("unchecked")
public Future<Void> executeRunnable(final Runnable r) {
return (Future<Void>) executor.submit(r::run);
}
public String toString() {
return String.format(Locale.ROOT, "ExecutorServiceFuturePool(executor=%s)", executor);
}
}

View File

@ -0,0 +1,301 @@
/*
* 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.impl.prefetch;
import java.nio.ByteBuffer;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkState;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkWithinRange;
/**
* Provides functionality related to tracking the position within a file.
*
* The file is accessed through an in memory buffer. The absolute position within
* the file is the sum of start offset of the buffer within the file and the relative
* offset of the current access location within the buffer.
*
* A file is made up of equal sized blocks. The last block may be of a smaller size.
* The size of a buffer associated with this file is typically the same as block size.
*/
public final class FilePosition {
/**
* Holds block based information about a file.
*/
private BlockData blockData;
/**
* Information about the buffer in use.
*/
private BufferData data;
/**
* Provides access to the underlying file.
*/
private ByteBuffer buffer;
/**
* Start offset of the buffer relative to the start of a file.
*/
private long bufferStartOffset;
/**
* Offset where reading starts relative to the start of a file.
*/
private long readStartOffset;
// Read stats after a seek (mostly for debugging use).
private int numSingleByteReads;
private int numBytesRead;
private int numBufferReads;
/**
* Constructs an instance of {@link FilePosition}.
*
* @param fileSize size of the associated file.
* @param blockSize size of each block within the file.
*
* @throws IllegalArgumentException if fileSize is negative.
* @throws IllegalArgumentException if blockSize is zero or negative.
*/
public FilePosition(long fileSize, int blockSize) {
checkNotNegative(fileSize, "fileSize");
if (fileSize == 0) {
checkNotNegative(blockSize, "blockSize");
} else {
checkPositiveInteger(blockSize, "blockSize");
}
this.blockData = new BlockData(fileSize, blockSize);
// The position is valid only when a valid buffer is associated with this file.
this.invalidate();
}
/**
* Associates a buffer with this file.
*
* @param bufferData the buffer associated with this file.
* @param startOffset Start offset of the buffer relative to the start of a file.
* @param readOffset Offset where reading starts relative to the start of a file.
*
* @throws IllegalArgumentException if bufferData is null.
* @throws IllegalArgumentException if startOffset is negative.
* @throws IllegalArgumentException if readOffset is negative.
* @throws IllegalArgumentException if readOffset is outside the range [startOffset, buffer end].
*/
public void setData(BufferData bufferData,
long startOffset,
long readOffset) {
checkNotNull(bufferData, "bufferData");
checkNotNegative(startOffset, "startOffset");
checkNotNegative(readOffset, "readOffset");
checkWithinRange(
readOffset,
"readOffset",
startOffset,
startOffset + bufferData.getBuffer().limit() - 1);
data = bufferData;
buffer = bufferData.getBuffer().duplicate();
bufferStartOffset = startOffset;
readStartOffset = readOffset;
setAbsolute(readOffset);
resetReadStats();
}
public ByteBuffer buffer() {
throwIfInvalidBuffer();
return buffer;
}
public BufferData data() {
throwIfInvalidBuffer();
return data;
}
/**
* Gets the current absolute position within this file.
*
* @return the current absolute position within this file.
*/
public long absolute() {
throwIfInvalidBuffer();
return bufferStartOffset + relative();
}
/**
* If the given {@code pos} lies within the current buffer, updates the current position to
* the specified value and returns true; otherwise returns false without changing the position.
*
* @param pos the absolute position to change the current position to if possible.
* @return true if the given current position was updated, false otherwise.
*/
public boolean setAbsolute(long pos) {
if (isValid() && isWithinCurrentBuffer(pos)) {
int relativePos = (int) (pos - bufferStartOffset);
buffer.position(relativePos);
return true;
} else {
return false;
}
}
/**
* Gets the current position within this file relative to the start of the associated buffer.
*
* @return the current position within this file relative to the start of the associated buffer.
*/
public int relative() {
throwIfInvalidBuffer();
return buffer.position();
}
/**
* Determines whether the given absolute position lies within the current buffer.
*
* @param pos the position to check.
* @return true if the given absolute position lies within the current buffer, false otherwise.
*/
public boolean isWithinCurrentBuffer(long pos) {
throwIfInvalidBuffer();
long bufferEndOffset = bufferStartOffset + buffer.limit() - 1;
return (pos >= bufferStartOffset) && (pos <= bufferEndOffset);
}
/**
* Gets the id of the current block.
*
* @return the id of the current block.
*/
public int blockNumber() {
throwIfInvalidBuffer();
return blockData.getBlockNumber(bufferStartOffset);
}
/**
* Determines whether the current block is the last block in this file.
*
* @return true if the current block is the last block in this file, false otherwise.
*/
public boolean isLastBlock() {
return blockData.isLastBlock(blockNumber());
}
/**
* Determines if the current position is valid.
*
* @return true if the current position is valid, false otherwise.
*/
public boolean isValid() {
return buffer != null;
}
/**
* Marks the current position as invalid.
*/
public void invalidate() {
buffer = null;
bufferStartOffset = -1;
data = null;
}
/**
* Gets the start of the current block's absolute offset.
*
* @return the start of the current block's absolute offset.
*/
public long bufferStartOffset() {
throwIfInvalidBuffer();
return bufferStartOffset;
}
/**
* Determines whether the current buffer has been fully read.
*
* @return true if the current buffer has been fully read, false otherwise.
*/
public boolean bufferFullyRead() {
throwIfInvalidBuffer();
return (bufferStartOffset == readStartOffset)
&& (relative() == buffer.limit())
&& (numBytesRead == buffer.limit());
}
public void incrementBytesRead(int n) {
numBytesRead += n;
if (n == 1) {
numSingleByteReads++;
} else {
numBufferReads++;
}
}
public int numBytesRead() {
return numBytesRead;
}
public int numSingleByteReads() {
return numSingleByteReads;
}
public int numBufferReads() {
return numBufferReads;
}
private void resetReadStats() {
numBytesRead = 0;
numSingleByteReads = 0;
numBufferReads = 0;
}
public String toString() {
StringBuilder sb = new StringBuilder();
if (buffer == null) {
sb.append("currentBuffer = null");
} else {
int pos = buffer.position();
int val;
if (pos >= buffer.limit()) {
val = -1;
} else {
val = buffer.get(pos);
}
String currentBufferState =
String.format("%d at pos: %d, lim: %d", val, pos, buffer.limit());
sb.append(String.format(
"block: %d, pos: %d (CBuf: %s)%n",
blockNumber(), absolute(),
currentBufferState));
sb.append("\n");
}
return sb.toString();
}
private void throwIfInvalidBuffer() {
checkState(buffer != null, "'buffer' must not be null");
}
}

View File

@ -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.impl.prefetch;
import java.time.Duration;
import org.apache.hadoop.fs.statistics.DurationTracker;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
public interface PrefetchingStatistics extends IOStatisticsSource {
/**
* 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);
}

View File

@ -0,0 +1,71 @@
/*
* 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.impl.prefetch;
import java.io.Closeable;
/**
* Manages a fixed pool of resources.
*
* Avoids creating a new resource if a previously created instance is already available.
*/
public abstract class ResourcePool<T> implements Closeable {
/**
* Acquires a resource blocking if necessary until one becomes available.
*
* @return the acquired resource instance.
*/
public abstract T acquire();
/**
* Acquires a resource blocking if one is immediately available. Otherwise returns null.
* @return the acquired resource instance (if immediately available) or null.
*/
public abstract T tryAcquire();
/**
* Releases a previously acquired resource.
*
* @param item the resource to release.
*/
public abstract void release(T item);
@Override
public void close() {
}
/**
* Derived classes may implement a way to cleanup each item.
*
* @param item the resource to close.
*/
protected void close(T item) {
// Do nothing in this class. Allow overriding classes to take any cleanup action.
}
/**
* Derived classes must implement a way to create an instance of a resource.
*
* @return the created instance.
*/
protected abstract T createNew();
}

View File

@ -0,0 +1,93 @@
/*
* 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.impl.prefetch;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkGreater;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
/**
* Provides retry related functionality.
*/
public class Retryer {
/* Maximum amount of delay (in ms) before retry fails. */
private int maxDelay;
/* Per retry delay (in ms). */
private int perRetryDelay;
/**
* The time interval (in ms) at which status update would be made.
*/
private int statusUpdateInterval;
/* Current delay. */
private int delay;
/**
* Initializes a new instance of the {@code Retryer} class.
*
* @param perRetryDelay per retry delay (in ms).
* @param maxDelay maximum amount of delay (in ms) before retry fails.
* @param statusUpdateInterval time interval (in ms) at which status update would be made.
*
* @throws IllegalArgumentException if perRetryDelay is zero or negative.
* @throws IllegalArgumentException if maxDelay is less than or equal to perRetryDelay.
* @throws IllegalArgumentException if statusUpdateInterval is zero or negative.
*/
public Retryer(int perRetryDelay, int maxDelay, int statusUpdateInterval) {
checkPositiveInteger(perRetryDelay, "perRetryDelay");
checkGreater(maxDelay, "maxDelay", perRetryDelay, "perRetryDelay");
checkPositiveInteger(statusUpdateInterval, "statusUpdateInterval");
this.perRetryDelay = perRetryDelay;
this.maxDelay = maxDelay;
this.statusUpdateInterval = statusUpdateInterval;
}
/**
* Returns true if retrying should continue, false otherwise.
*
* @return true if the caller should retry, false otherwise.
*/
public boolean continueRetry() {
if (this.delay >= this.maxDelay) {
return false;
}
try {
Thread.sleep(this.perRetryDelay);
} catch (InterruptedException e) {
// Ignore the exception as required by the semantic of this class;
}
this.delay += this.perRetryDelay;
return true;
}
/**
* Returns true if status update interval has been reached.
*
* @return true if status update interval has been reached.
*/
public boolean updateStatus() {
return (this.delay > 0) && this.delay % this.statusUpdateInterval == 0;
}
}

View File

@ -0,0 +1,354 @@
/*
* 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.impl.prefetch;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;
import java.nio.file.Files;
import java.nio.file.OpenOption;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.nio.file.attribute.FileAttribute;
import java.nio.file.attribute.PosixFilePermission;
import java.nio.file.attribute.PosixFilePermissions;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
/**
* Provides functionality necessary for caching blocks of data read from FileSystem.
* Each cache block is stored on the local disk as a separate file.
*/
public class SingleFilePerBlockCache implements BlockCache {
private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class);
/**
* Blocks stored in this cache.
*/
private final Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
/**
* Number of times a block was read from this cache.
* Used for determining cache utilization factor.
*/
private int numGets = 0;
private boolean closed;
private final PrefetchingStatistics prefetchingStatistics;
/**
* Cache entry.
* Each block is stored as a separate file.
*/
private static final class Entry {
private final int blockNumber;
private final Path path;
private final int size;
private final long checksum;
Entry(int blockNumber, Path path, int size, long checksum) {
this.blockNumber = blockNumber;
this.path = path;
this.size = size;
this.checksum = checksum;
}
@Override
public String toString() {
return String.format(
"([%03d] %s: size = %d, checksum = %d)",
blockNumber, path, size, checksum);
}
}
/**
* Constructs an instance of a {@code SingleFilePerBlockCache}.
*
* @param prefetchingStatistics statistics for this stream.
*/
public SingleFilePerBlockCache(PrefetchingStatistics prefetchingStatistics) {
this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
}
/**
* Indicates whether the given block is in this cache.
*/
@Override
public boolean containsBlock(int blockNumber) {
return blocks.containsKey(blockNumber);
}
/**
* Gets the blocks in this cache.
*/
@Override
public Iterable<Integer> blocks() {
return Collections.unmodifiableList(new ArrayList<>(blocks.keySet()));
}
/**
* Gets the number of blocks in this cache.
*/
@Override
public int size() {
return blocks.size();
}
/**
* Gets the block having the given {@code blockNumber}.
*
* @throws IllegalArgumentException if buffer is null.
*/
@Override
public void get(int blockNumber, ByteBuffer buffer) throws IOException {
if (closed) {
return;
}
checkNotNull(buffer, "buffer");
Entry entry = getEntry(blockNumber);
buffer.clear();
readFile(entry.path, buffer);
buffer.rewind();
validateEntry(entry, buffer);
}
protected int readFile(Path path, ByteBuffer buffer) throws IOException {
int numBytesRead = 0;
int numBytes;
FileChannel channel = FileChannel.open(path, StandardOpenOption.READ);
while ((numBytes = channel.read(buffer)) > 0) {
numBytesRead += numBytes;
}
buffer.limit(buffer.position());
channel.close();
return numBytesRead;
}
private Entry getEntry(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
Entry entry = blocks.get(blockNumber);
if (entry == null) {
throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
}
numGets++;
return entry;
}
/**
* Puts the given block in this cache.
*
* @throws IllegalArgumentException if buffer is null.
* @throws IllegalArgumentException if buffer.limit() is zero or negative.
*/
@Override
public void put(int blockNumber, ByteBuffer buffer) throws IOException {
if (closed) {
return;
}
checkNotNull(buffer, "buffer");
if (blocks.containsKey(blockNumber)) {
Entry entry = blocks.get(blockNumber);
validateEntry(entry, buffer);
return;
}
Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
Path blockFilePath = getCacheFilePath();
long size = Files.size(blockFilePath);
if (size != 0) {
String message =
String.format("[%d] temp file already has data. %s (%d)",
blockNumber, blockFilePath, size);
throw new IllegalStateException(message);
}
writeFile(blockFilePath, buffer);
prefetchingStatistics.blockAddedToFileCache();
long checksum = BufferData.getChecksum(buffer);
Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum);
blocks.put(blockNumber, entry);
}
private static final Set<? extends OpenOption> CREATE_OPTIONS =
EnumSet.of(StandardOpenOption.WRITE,
StandardOpenOption.CREATE,
StandardOpenOption.TRUNCATE_EXISTING);
protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
buffer.rewind();
WritableByteChannel writeChannel = Files.newByteChannel(path, CREATE_OPTIONS);
while (buffer.hasRemaining()) {
writeChannel.write(buffer);
}
writeChannel.close();
}
protected Path getCacheFilePath() throws IOException {
return getTempFilePath();
}
@Override
public void close() throws IOException {
if (closed) {
return;
}
closed = true;
LOG.info(getStats());
int numFilesDeleted = 0;
for (Entry entry : blocks.values()) {
try {
Files.deleteIfExists(entry.path);
prefetchingStatistics.blockRemovedFromFileCache();
numFilesDeleted++;
} catch (IOException e) {
// Ignore while closing so that we can delete as many cache files as possible.
}
}
if (numFilesDeleted > 0) {
LOG.info("Deleted {} cache files", numFilesDeleted);
}
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("stats: ");
sb.append(getStats());
sb.append(", blocks:[");
sb.append(getIntList(blocks()));
sb.append("]");
return sb.toString();
}
private void validateEntry(Entry entry, ByteBuffer buffer) {
if (entry.size != buffer.limit()) {
String message = String.format(
"[%d] entry.size(%d) != buffer.limit(%d)",
entry.blockNumber, entry.size, buffer.limit());
throw new IllegalStateException(message);
}
long checksum = BufferData.getChecksum(buffer);
if (entry.checksum != checksum) {
String message = String.format(
"[%d] entry.checksum(%d) != buffer checksum(%d)",
entry.blockNumber, entry.checksum, checksum);
throw new IllegalStateException(message);
}
}
/**
* Produces a human readable list of blocks for the purpose of logging.
* This method minimizes the length of returned list by converting
* a contiguous list of blocks into a range.
* for example,
* 1, 3, 4, 5, 6, 8 becomes 1, 3~6, 8
*/
private String getIntList(Iterable<Integer> nums) {
List<String> numList = new ArrayList<>();
List<Integer> numbers = new ArrayList<Integer>();
for (Integer n : nums) {
numbers.add(n);
}
Collections.sort(numbers);
int index = 0;
while (index < numbers.size()) {
int start = numbers.get(index);
int prev = start;
int end = start;
while ((++index < numbers.size()) && ((end = numbers.get(index)) == prev + 1)) {
prev = end;
}
if (start == prev) {
numList.add(Integer.toString(start));
} else {
numList.add(String.format("%d~%d", start, prev));
}
}
return String.join(", ", numList);
}
private String getStats() {
StringBuilder sb = new StringBuilder();
sb.append(String.format(
"#entries = %d, #gets = %d",
blocks.size(), numGets));
return sb.toString();
}
private static final String CACHE_FILE_PREFIX = "fs-cache-";
public static boolean isCacheSpaceAvailable(long fileSize) {
try {
Path cacheFilePath = getTempFilePath();
long freeSpace = new File(cacheFilePath.toString()).getUsableSpace();
LOG.info("fileSize = {}, freeSpace = {}", fileSize, freeSpace);
Files.deleteIfExists(cacheFilePath);
return fileSize < freeSpace;
} catch (IOException e) {
LOG.error("isCacheSpaceAvailable", e);
return false;
}
}
// The suffix (file extension) of each serialized index file.
private static final String BINARY_FILE_SUFFIX = ".bin";
// File attributes attached to any intermediate temporary file created during index creation.
private static final FileAttribute<Set<PosixFilePermission>> TEMP_FILE_ATTRS =
PosixFilePermissions.asFileAttribute(EnumSet.of(PosixFilePermission.OWNER_READ,
PosixFilePermission.OWNER_WRITE));
private static Path getTempFilePath() throws IOException {
return Files.createTempFile(
CACHE_FILE_PREFIX,
BINARY_FILE_SUFFIX,
TEMP_FILE_ATTRS
);
}
}

View File

@ -0,0 +1,399 @@
/*
* 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.impl.prefetch;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Collection;
import static org.apache.hadoop.util.Preconditions.checkArgument;
/**
* A superset of Validate class in Apache commons lang3.
* <p>
* It provides consistent message strings for frequently encountered checks.
* That simplifies callers because they have to supply only the name of the argument
* that failed a check instead of having to supply the entire message.
*/
public final class Validate {
private Validate() {
}
/**
* Validates that the given reference argument is not null.
* @param obj the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNull(Object obj, String argName) {
checkArgument(obj != null, "'%s' must not be null.", argName);
}
/**
* Validates that the given integer argument is not zero or negative.
* @param value the argument value to validate
* @param argName the name of the argument being validated.
*/
public static void checkPositiveInteger(long value, String argName) {
checkArgument(value > 0, "'%s' must be a positive integer.", argName);
}
/**
* Validates that the given integer argument is not negative.
* @param value the argument value to validate
* @param argName the name of the argument being validated.
*/
public static void checkNotNegative(long value, String argName) {
checkArgument(value >= 0, "'%s' must not be negative.", argName);
}
/**
* Validates that the expression (that checks a required field is present) is true.
* @param isPresent indicates whether the given argument is present.
* @param argName the name of the argument being validated.
*/
public static void checkRequired(boolean isPresent, String argName) {
checkArgument(isPresent, "'%s' is required.", argName);
}
/**
* Validates that the expression (that checks a field is valid) is true.
* @param isValid indicates whether the given argument is valid.
* @param argName the name of the argument being validated.
*/
public static void checkValid(boolean isValid, String argName) {
checkArgument(isValid, "'%s' is invalid.", argName);
}
/**
* Validates that the expression (that checks a field is valid) is true.
* @param isValid indicates whether the given argument is valid.
* @param argName the name of the argument being validated.
* @param validValues the list of values that are allowed.
*/
public static void checkValid(boolean isValid,
String argName,
String validValues) {
checkArgument(isValid, "'%s' is invalid. Valid values are: %s.", argName,
validValues);
}
/**
* Validates that the given string is not null and has non-zero length.
* @param arg the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(String arg, String argName) {
checkNotNull(arg, argName);
checkArgument(
!arg.isEmpty(),
"'%s' must not be empty.",
argName);
}
/**
* Validates that the given array is not null and has at least one element.
* @param <T> the type of array's elements.
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static <T> void checkNotNullAndNotEmpty(T[] array, String argName) {
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(byte[] array, String argName) {
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(short[] array, String argName) {
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(int[] array, String argName) {
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(long[] array, String argName) {
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given buffer is not null and has non-zero capacity.
* @param <T> the type of iterable's elements.
* @param iter the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static <T> void checkNotNullAndNotEmpty(Iterable<T> iter,
String argName) {
checkNotNull(iter, argName);
int minNumElements = iter.iterator().hasNext() ? 1 : 0;
checkNotEmpty(minNumElements, argName);
}
/**
* Validates that the given set is not null and has an exact number of items.
* @param <T> the type of collection's elements.
* @param collection the argument reference to validate.
* @param numElements the expected number of elements in the collection.
* @param argName the name of the argument being validated.
*/
public static <T> void checkNotNullAndNumberOfElements(
Collection<T> collection, int numElements, String argName) {
checkNotNull(collection, argName);
checkArgument(
collection.size() == numElements,
"Number of elements in '%s' must be exactly %s, %s given.",
argName,
numElements,
collection.size()
);
}
/**
* Validates that the given two values are equal.
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
* @param value2Name the name of the second argument.
*/
public static void checkValuesEqual(
long value1,
String value1Name,
long value2,
String value2Name) {
checkArgument(
value1 == value2,
"'%s' (%s) must equal '%s' (%s).",
value1Name,
value1,
value2Name,
value2);
}
/**
* Validates that the first value is an integer multiple of the second value.
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
* @param value2Name the name of the second argument.
*/
public static void checkIntegerMultiple(
long value1,
String value1Name,
long value2,
String value2Name) {
checkArgument(
(value1 % value2) == 0,
"'%s' (%s) must be an integer multiple of '%s' (%s).",
value1Name,
value1,
value2Name,
value2);
}
/**
* Validates that the first value is greater than the second value.
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
* @param value2Name the name of the second argument.
*/
public static void checkGreater(
long value1,
String value1Name,
long value2,
String value2Name) {
checkArgument(
value1 > value2,
"'%s' (%s) must be greater than '%s' (%s).",
value1Name,
value1,
value2Name,
value2);
}
/**
* Validates that the first value is greater than or equal to the second value.
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
* @param value2Name the name of the second argument.
*/
public static void checkGreaterOrEqual(
long value1,
String value1Name,
long value2,
String value2Name) {
checkArgument(
value1 >= value2,
"'%s' (%s) must be greater than or equal to '%s' (%s).",
value1Name,
value1,
value2Name,
value2);
}
/**
* Validates that the first value is less than or equal to the second value.
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
* @param value2Name the name of the second argument.
*/
public static void checkLessOrEqual(
long value1,
String value1Name,
long value2,
String value2Name) {
checkArgument(
value1 <= value2,
"'%s' (%s) must be less than or equal to '%s' (%s).",
value1Name,
value1,
value2Name,
value2);
}
/**
* Validates that the given value is within the given range of values.
* @param value the value to check.
* @param valueName the name of the argument.
* @param minValueInclusive inclusive lower limit for the value.
* @param maxValueInclusive inclusive upper limit for the value.
*/
public static void checkWithinRange(
long value,
String valueName,
long minValueInclusive,
long maxValueInclusive) {
checkArgument(
(value >= minValueInclusive) && (value <= maxValueInclusive),
"'%s' (%s) must be within the range [%s, %s].",
valueName,
value,
minValueInclusive,
maxValueInclusive);
}
/**
* Validates that the given value is within the given range of values.
* @param value the value to check.
* @param valueName the name of the argument.
* @param minValueInclusive inclusive lower limit for the value.
* @param maxValueInclusive inclusive upper limit for the value.
*/
public static void checkWithinRange(
double value,
String valueName,
double minValueInclusive,
double maxValueInclusive) {
checkArgument(
(value >= minValueInclusive) && (value <= maxValueInclusive),
"'%s' (%s) must be within the range [%s, %s].",
valueName,
value,
minValueInclusive,
maxValueInclusive);
}
/**
* Validates that the given path exists.
* @param path the path to check.
* @param argName the name of the argument being validated.
*/
public static void checkPathExists(Path path, String argName) {
checkNotNull(path, argName);
checkArgument(Files.exists(path), "Path %s (%s) does not exist.", argName,
path);
}
/**
* Validates that the given path exists and is a directory.
* @param path the path to check.
* @param argName the name of the argument being validated.
*/
public static void checkPathExistsAsDir(Path path, String argName) {
checkPathExists(path, argName);
checkArgument(
Files.isDirectory(path),
"Path %s (%s) must point to a directory.",
argName,
path);
}
/**
* Validates that the given path exists and is a file.
* @param path the path to check.
* @param argName the name of the argument being validated.
*/
public static void checkPathExistsAsFile(Path path, String argName) {
checkPathExists(path, argName);
checkArgument(Files.isRegularFile(path),
"Path %s (%s) must point to a file.", argName, path);
}
/**
* Check state.
* @param expression expression which must hold.
* @param format format string
* @param args arguments for the error string
* @throws IllegalStateException if the state is not valid.
*/
public static void checkState(boolean expression,
String format,
Object... args) {
if (!expression) {
throw new IllegalStateException(String.format(format, args));
}
}
private static void checkNotEmpty(int arraySize, String argName) {
checkArgument(
arraySize > 0,
"'%s' must have at least one element.",
argName);
}
}

View File

@ -0,0 +1,28 @@
/*
* 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.
*/
/**
* block caching for use in object store clients.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.impl.prefetch;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -415,6 +415,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() {
}

View File

@ -0,0 +1,63 @@
/*
* 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.impl.prefetch;
import org.apache.hadoop.test.LambdaTestUtils;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
public final class ExceptionAsserts {
private ExceptionAsserts() {
}
/**
* Asserts that the given code throws an exception of the given type
* and that the exception message contains the given sub-message.
*
* Usage:
*
* ExceptionAsserts.assertThrows(
* IllegalArgumentException.class,
* "'nullArg' must not be null",
* () -> Preconditions.checkNotNull(null, "nullArg"));
*
* Note: JUnit 5 has similar functionality but it will be a long time before
* we move to that framework because of significant differences and lack of
* backward compatibility for some JUnit rules.
*/
public static <E extends Exception> void assertThrows(
Class<E> expectedExceptionClass,
String partialMessage,
LambdaTestUtils.VoidCallable code) throws Exception {
intercept(expectedExceptionClass, partialMessage, code);
}
public static <E extends Exception> void assertThrows(
Class<E> expectedExceptionClass,
LambdaTestUtils.VoidCallable code) throws Exception {
intercept(expectedExceptionClass, code);
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.impl.prefetch;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Frequently used test data items.
*/
public final class SampleDataForTests {
private SampleDataForTests() {
}
// Array data.
public static final Object[] NULL_ARRAY = null;
public static final Object[] EMPTY_ARRAY = new Object[0];
public static final Object[] NON_EMPTY_ARRAY = new Object[1];
public static final byte[] NULL_BYTE_ARRAY = null;
public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
public static final byte[] NON_EMPTY_BYTE_ARRAY = new byte[1];
public static final short[] NULL_SHORT_ARRAY = null;
public static final short[] EMPTY_SHORT_ARRAY = new short[0];
public static final short[] NON_EMPTY_SHORT_ARRAY = new short[1];
public static final int[] NULL_INT_ARRAY = null;
public static final int[] EMPTY_INT_ARRAY = new int[0];
public static final int[] NON_EMPTY_INT_ARRAY = new int[1];
public static final long[] NULL_LONG_ARRAY = null;
public static final long[] EMPTY_LONG_ARRAY = new long[0];
public static final long[] NON_EMPTY_LONG_ARRAY = new long[1];
public static final List<Object> NULL_LIST = null;
public static final List<Object> EMPTY_LIST = new ArrayList<Object>();
public static final List<Object> VALID_LIST = Arrays.asList(new Object[1]);
}

View File

@ -0,0 +1,98 @@
/*
* 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.impl.prefetch;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertTrue;
public class TestBlockCache extends AbstractHadoopTestBase {
private static final int BUFFER_SIZE = 16;
@Test
public void testArgChecks() throws Exception {
// Should not throw.
BlockCache cache =
new SingleFilePerBlockCache(EmptyPrefetchingStatistics.getInstance());
ByteBuffer buffer = ByteBuffer.allocate(16);
// Verify it throws correctly.
intercept(IllegalArgumentException.class, "'buffer' must not be null",
() -> cache.put(42, null));
intercept(NullPointerException.class, null,
() -> new SingleFilePerBlockCache(null));
}
@Test
public void testPutAndGet() throws Exception {
BlockCache cache =
new SingleFilePerBlockCache(EmptyPrefetchingStatistics.getInstance());
ByteBuffer buffer1 = ByteBuffer.allocate(BUFFER_SIZE);
for (byte i = 0; i < BUFFER_SIZE; i++) {
buffer1.put(i);
}
assertEquals(0, cache.size());
assertFalse(cache.containsBlock(0));
cache.put(0, buffer1);
assertEquals(1, cache.size());
assertTrue(cache.containsBlock(0));
ByteBuffer buffer2 = ByteBuffer.allocate(BUFFER_SIZE);
cache.get(0, buffer2);
assertNotSame(buffer1, buffer2);
assertBuffersEqual(buffer1, buffer2);
assertEquals(1, cache.size());
assertFalse(cache.containsBlock(1));
cache.put(1, buffer1);
assertEquals(2, cache.size());
assertTrue(cache.containsBlock(1));
ByteBuffer buffer3 = ByteBuffer.allocate(BUFFER_SIZE);
cache.get(1, buffer3);
assertNotSame(buffer1, buffer3);
assertBuffersEqual(buffer1, buffer3);
}
private void assertBuffersEqual(ByteBuffer buffer1, ByteBuffer buffer2) {
assertNotNull(buffer1);
assertNotNull(buffer2);
assertEquals(buffer1.limit(), buffer2.limit());
assertEquals(BUFFER_SIZE, buffer1.limit());
for (int i = 0; i < BUFFER_SIZE; i++) {
assertEquals(buffer1.get(i), buffer2.get(i));
}
}
}

View File

@ -0,0 +1,159 @@
/*
* 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.impl.prefetch;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class TestBlockData extends AbstractHadoopTestBase {
@Test
public void testArgChecks() throws Exception {
// Should not throw.
new BlockData(10, 5);
new BlockData(5, 10);
new BlockData(0, 10);
// Verify it throws correctly.
intercept(IllegalArgumentException.class, "'fileSize' must not be negative",
() -> new BlockData(-1, 2));
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new BlockData(10, 0));
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new BlockData(10, -2));
intercept(IllegalArgumentException.class,
"'blockNumber' (-1) must be within the range [0, 3]",
() -> new BlockData(10, 3).isLastBlock(
-1));
intercept(IllegalArgumentException.class,
"'blockNumber' (11) must be within the range [0, 3]",
() -> new BlockData(10, 3).isLastBlock(
11));
}
@Test
public void testComputedFields() throws Exception {
testComputedFieldsHelper(0, 10);
testComputedFieldsHelper(1, 10);
testComputedFieldsHelper(10, 1);
testComputedFieldsHelper(10, 2);
testComputedFieldsHelper(10, 3);
}
private void testComputedFieldsHelper(long fileSize, int blockSize)
throws Exception {
BlockData bd = new BlockData(fileSize, blockSize);
if (fileSize == 0) {
assertFalse(bd.isLastBlock(0));
assertFalse(bd.isLastBlock(1));
assertFalse(bd.isValidOffset(0));
assertEquals(0, bd.getSize(0));
assertEquals("", bd.getStateString());
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'offset' (0) must be within the range [0, -1]",
() -> bd.getBlockNumber(0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'blockNumber' (0) must be within the range [0, -1]",
() -> bd.getStartOffset(0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'offset' (0) must be within the range [0, -1]",
() -> bd.getRelativeOffset(0, 0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'blockNumber' (0) must be within the range [0, -1]",
() -> bd.getState(0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'blockNumber' (0) must be within the range [0, -1]",
() -> bd.setState(0, BlockData.State.READY));
return;
}
assertEquals(fileSize, bd.getFileSize());
assertEquals(blockSize, bd.getBlockSize());
int expectedNumBlocks = (int) (fileSize / blockSize);
if (fileSize % blockSize > 0) {
expectedNumBlocks++;
}
assertEquals(expectedNumBlocks, bd.getNumBlocks());
int lastBlockNumber = expectedNumBlocks - 1;
for (int b = 0; b < lastBlockNumber; b++) {
assertFalse(bd.isLastBlock(b));
assertEquals(blockSize, bd.getSize(b));
}
assertTrue(bd.isLastBlock(lastBlockNumber));
int lastBlockSize = (int) (fileSize - blockSize * (expectedNumBlocks - 1));
assertEquals(lastBlockSize, bd.getSize(lastBlockNumber));
// Offset related methods.
for (long offset = 0; offset < fileSize; offset++) {
int expectedBlockNumber = (int) (offset / blockSize);
assertEquals(expectedBlockNumber, bd.getBlockNumber(offset));
for (int b = 0; b < expectedNumBlocks - 1; b++) {
long expectedStartOffset = b * blockSize;
assertEquals(expectedStartOffset, bd.getStartOffset(b));
int expectedRelativeOffset = (int) (offset - expectedStartOffset);
assertEquals(expectedRelativeOffset, bd.getRelativeOffset(b, offset));
}
}
// State methods.
for (int b = 0; b < expectedNumBlocks; b++) {
assertEquals(b * blockSize, bd.getStartOffset(b));
assertEquals(BlockData.State.NOT_READY, bd.getState(b));
bd.setState(b, BlockData.State.QUEUED);
assertEquals(BlockData.State.QUEUED, bd.getState(b));
bd.setState(b, BlockData.State.READY);
assertEquals(BlockData.State.READY, bd.getState(b));
bd.setState(b, BlockData.State.CACHED);
assertEquals(BlockData.State.CACHED, bd.getState(b));
}
}
}

View File

@ -0,0 +1,106 @@
/*
* 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.impl.prefetch;
import java.lang.reflect.Method;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertTrue;
public class TestBlockOperations extends AbstractHadoopTestBase {
@Test
public void testArgChecks() throws Exception {
// Should not throw.
BlockOperations ops = new BlockOperations();
// Verify it throws correctly.
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.getPrefetched(-1));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.getCached(-1));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.getRead(-1));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.release(-1));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.requestPrefetch(-1));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.requestCaching(-1));
}
@Test
public void testGetSummary() throws Exception {
verifySummary("getPrefetched", "GP");
verifySummary("getCached", "GC");
verifySummary("getRead", "GR");
verifySummary("release", "RL");
verifySummary("requestPrefetch", "RP");
verifySummary("prefetch", "PF");
verifySummary("requestCaching", "RC");
verifySummary("addToCache", "C+");
verifySummaryNoArg("cancelPrefetches", "CP");
verifySummaryNoArg("close", "CX");
}
private void verifySummary(String methodName, String shortName)
throws Exception {
int blockNumber = 42;
BlockOperations ops = new BlockOperations();
Method method = ops.getClass().getDeclaredMethod(methodName, int.class);
BlockOperations.Operation op =
(BlockOperations.Operation) method.invoke(ops, blockNumber);
ops.end(op);
String summary = ops.getSummary(false);
String opSummary = String.format("%s(%d)", shortName, blockNumber);
String expectedSummary = String.format("%s;E%s;", opSummary, opSummary);
assertTrue(summary.startsWith(expectedSummary));
}
private void verifySummaryNoArg(String methodName, String shortName)
throws Exception {
BlockOperations ops = new BlockOperations();
Method method = ops.getClass().getDeclaredMethod(methodName);
BlockOperations.Operation op =
(BlockOperations.Operation) method.invoke(ops);
ops.end(op);
String summary = ops.getSummary(false);
String expectedSummary = String.format("%s;E%s;", shortName, shortName);
assertTrue(summary.startsWith(expectedSummary));
}
}

View File

@ -0,0 +1,147 @@
/*
* 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.impl.prefetch;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Set;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
public class TestBoundedResourcePool extends AbstractHadoopTestBase {
static class BufferPool extends BoundedResourcePool<ByteBuffer> {
BufferPool(int size) {
super(size);
}
@Override
protected ByteBuffer createNew() {
return ByteBuffer.allocate(10);
}
}
@Test
public void testArgChecks() throws Exception {
// Should not throw.
BufferPool pool = new BufferPool(5);
// Verify it throws correctly.
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(-1));
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(0));
intercept(IllegalArgumentException.class, "'item' must not be null",
() -> pool.release(null));
intercept(IllegalArgumentException.class,
"This item is not a part of this pool",
() -> pool.release(ByteBuffer.allocate(4)));
}
@Test
public void testAcquireReleaseSingle() {
final int numBuffers = 5;
BufferPool pool = new BufferPool(numBuffers);
assertEquals(0, pool.numCreated());
assertEquals(numBuffers, pool.numAvailable());
ByteBuffer buffer1 = pool.acquire();
assertNotNull(buffer1);
assertEquals(1, pool.numCreated());
assertEquals(numBuffers - 1, pool.numAvailable());
// Release and immediately reacquire => should not end up creating new buffer.
pool.release(buffer1);
assertEquals(1, pool.numCreated());
ByteBuffer buffer2 = pool.acquire();
assertNotNull(buffer2);
assertSame(buffer1, buffer2);
assertEquals(1, pool.numCreated());
}
@Test
public void testAcquireReleaseMultiple() {
final int numBuffers = 5;
BufferPool pool = new BufferPool(numBuffers);
Set<ByteBuffer> buffers =
Collections.newSetFromMap(new IdentityHashMap<ByteBuffer, Boolean>());
assertEquals(0, pool.numCreated());
// Acquire all one by one.
for (int i = 0; i < numBuffers; i++) {
assertEquals(numBuffers - i, pool.numAvailable());
ByteBuffer buffer = pool.acquire();
assertNotNull(buffer);
assertFalse(buffers.contains(buffer));
buffers.add(buffer);
assertEquals(i + 1, pool.numCreated());
}
assertEquals(numBuffers, pool.numCreated());
assertEquals(0, pool.numAvailable());
int releaseCount = 0;
// Release all one by one.
for (ByteBuffer buffer : buffers) {
assertEquals(releaseCount, pool.numAvailable());
releaseCount++;
pool.release(buffer);
assertEquals(releaseCount, pool.numAvailable());
// Releasing the same buffer again should not have any ill effect.
pool.release(buffer);
assertEquals(releaseCount, pool.numAvailable());
pool.release(buffer);
assertEquals(releaseCount, pool.numAvailable());
}
// Acquire all one by one again to ensure that they are the same ones we got earlier.
for (int i = 0; i < numBuffers; i++) {
ByteBuffer buffer = pool.acquire();
assertTrue(buffers.contains(buffer));
}
assertEquals(numBuffers, pool.numCreated());
assertEquals(0, pool.numAvailable());
}
}

View File

@ -0,0 +1,244 @@
/*
* 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.impl.prefetch;
import java.nio.ByteBuffer;
import java.nio.ReadOnlyBufferException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertSame;
public class TestBufferData extends AbstractHadoopTestBase {
@Test
public void testArgChecks() throws Exception {
// Should not throw.
ByteBuffer buffer = ByteBuffer.allocate(1);
BufferData data = new BufferData(1, buffer);
// Verify it throws correctly.
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> new BufferData(-1, buffer));
intercept(IllegalArgumentException.class, "'buffer' must not be null",
() -> new BufferData(1, null));
intercept(IllegalArgumentException.class, "'actionFuture' must not be null",
() -> data.setPrefetch(null));
intercept(IllegalArgumentException.class, "'actionFuture' must not be null",
() -> data.setCaching(null));
intercept(IllegalArgumentException.class, "'states' must not be null",
() -> data.throwIfStateIncorrect((BufferData.State[]) null));
intercept(IllegalStateException.class,
"Expected buffer state to be 'READY or CACHING' but found",
() -> data.throwIfStateIncorrect(BufferData.State.READY,
BufferData.State.CACHING));
}
@Test
public void testValidStateUpdates() {
ByteBuffer buffer = ByteBuffer.allocate(1);
BufferData data = new BufferData(1, buffer);
assertEquals(BufferData.State.BLANK, data.getState());
CompletableFuture<Void> actionFuture = new CompletableFuture<>();
actionFuture.complete(null);
data.setPrefetch(actionFuture);
assertEquals(BufferData.State.PREFETCHING, data.getState());
assertNotNull(data.getActionFuture());
assertSame(actionFuture, data.getActionFuture());
CompletableFuture<Void> actionFuture2 = new CompletableFuture<>();
data.setCaching(actionFuture2);
assertEquals(BufferData.State.CACHING, data.getState());
assertNotNull(data.getActionFuture());
assertSame(actionFuture2, data.getActionFuture());
assertNotSame(actionFuture, actionFuture2);
List<BufferData.State> states = Arrays.asList(
BufferData.State.BLANK,
BufferData.State.PREFETCHING,
BufferData.State.CACHING,
BufferData.State.READY
);
BufferData data2 = new BufferData(1, buffer);
BufferData.State prevState = null;
for (BufferData.State state : states) {
if (prevState != null) {
assertEquals(prevState, data2.getState());
data2.updateState(state, prevState);
assertEquals(state, data2.getState());
}
prevState = state;
}
}
@Test
public void testInvalidStateUpdates() throws Exception {
CompletableFuture<Void> actionFuture = new CompletableFuture<>();
actionFuture.complete(null);
testInvalidStateUpdatesHelper(
(d) -> d.setPrefetch(actionFuture),
BufferData.State.BLANK,
BufferData.State.READY);
testInvalidStateUpdatesHelper(
(d) -> d.setCaching(actionFuture),
BufferData.State.PREFETCHING,
BufferData.State.READY);
}
@Test
public void testSetReady() throws Exception {
byte[] bytes1 = new byte[5];
initBytes(bytes1);
ByteBuffer buffer = ByteBuffer.allocate(10);
buffer.put(bytes1);
buffer.limit(bytes1.length);
BufferData data = new BufferData(1, buffer);
assertNotEquals(BufferData.State.READY, data.getState());
assertEquals(0, data.getChecksum());
data.setReady(BufferData.State.BLANK);
assertEquals(BufferData.State.READY, data.getState());
assertNotEquals(0, data.getChecksum());
// Verify that buffer cannot be modified once in READY state.
ExceptionAsserts.assertThrows(
ReadOnlyBufferException.class,
null,
() -> data.getBuffer().put(bytes1));
// Verify that buffer cannot be set to READY state more than once.
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"Checksum cannot be changed once set",
() -> data.setReady(BufferData.State.BLANK));
// Verify that we detect post READY buffer modification.
buffer.array()[2] = (byte) 42;
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"checksum changed after setReady()",
() -> data.setDone());
}
@Test
public void testChecksum() {
byte[] bytes1 = new byte[5];
byte[] bytes2 = new byte[10];
initBytes(bytes1);
initBytes(bytes2);
ByteBuffer buffer1 = ByteBuffer.wrap(bytes1);
ByteBuffer buffer2 = ByteBuffer.wrap(bytes2);
buffer2.limit(bytes1.length);
long checksum1 = BufferData.getChecksum(buffer1);
long checksum2 = BufferData.getChecksum(buffer2);
assertEquals(checksum1, checksum2);
}
private void initBytes(byte[] bytes) {
for (int i = 0; i < bytes.length; i++) {
bytes[i] = (byte) i;
}
}
@FunctionalInterface
public interface StateChanger {
void run(BufferData data) throws Exception;
}
private void testInvalidStateUpdatesHelper(
StateChanger changeState,
BufferData.State... validFromState) throws Exception {
ByteBuffer buffer = ByteBuffer.allocate(1);
BufferData data = new BufferData(1, buffer);
data.updateState(validFromState[0], BufferData.State.BLANK);
List<BufferData.State> states = this.getStatesExcept(validFromState);
BufferData.State prevState = validFromState[0];
String expectedMessage =
String.format("Expected buffer state to be '%s", validFromState[0]);
for (BufferData.State s : states) {
data.updateState(s, prevState);
ExceptionAsserts.assertThrows(
IllegalStateException.class,
expectedMessage,
() -> changeState.run(data));
assertEquals(s, data.getState());
prevState = s;
}
}
static final List<BufferData.State> ALL_STATES = Arrays.asList(
BufferData.State.UNKNOWN,
BufferData.State.BLANK,
BufferData.State.PREFETCHING,
BufferData.State.CACHING,
BufferData.State.READY
);
private List<BufferData.State> getStatesExcept(BufferData.State... states) {
List<BufferData.State> result = new ArrayList<>();
for (BufferData.State s : ALL_STATES) {
boolean found = false;
for (BufferData.State ss : states) {
if (s == ss) {
found = true;
}
}
if (!found) {
result.add(s);
}
}
return result;
}
}

View File

@ -0,0 +1,158 @@
/*
* 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.impl.prefetch;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
public class TestBufferPool extends AbstractHadoopTestBase {
private static final int POOL_SIZE = 2;
private static final int BUFFER_SIZE = 10;
private final PrefetchingStatistics statistics =
EmptyPrefetchingStatistics.getInstance();
@Test
public void testArgChecks() throws Exception {
// Should not throw.
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
// Verify it throws correctly.
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(0, 10, statistics));
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(-1, 10, statistics));
intercept(IllegalArgumentException.class,
"'bufferSize' must be a positive integer",
() -> new BufferPool(10, 0, statistics));
intercept(IllegalArgumentException.class,
"'bufferSize' must be a positive integer",
() -> new BufferPool(1, -10, statistics));
intercept(NullPointerException.class,
() -> new BufferPool(1, 10, null));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> pool.acquire(-1));
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> pool.tryAcquire(-1));
intercept(NullPointerException.class, "data",
() -> pool.release((BufferData) null));
}
@Test
public void testGetAndRelease() {
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
assertInitialState(pool, POOL_SIZE);
int count = 0;
for (BufferData data : pool.getAll()) {
count++;
}
assertEquals(0, count);
BufferData data1 = this.acquire(pool, 1);
BufferData data2 = this.acquire(pool, 2);
BufferData data3 = pool.tryAcquire(3);
assertNull(data3);
count = 0;
for (BufferData data : pool.getAll()) {
count++;
}
assertEquals(2, count);
assertEquals(2, pool.numCreated());
assertEquals(0, pool.numAvailable());
data1.updateState(BufferData.State.READY, BufferData.State.BLANK);
pool.release(data1);
assertEquals(2, pool.numCreated());
assertEquals(1, pool.numAvailable());
data2.updateState(BufferData.State.READY, BufferData.State.BLANK);
pool.release(data2);
assertEquals(2, pool.numCreated());
assertEquals(2, pool.numAvailable());
}
@Test
public void testRelease() throws Exception {
testReleaseHelper(BufferData.State.BLANK, true);
testReleaseHelper(BufferData.State.PREFETCHING, true);
testReleaseHelper(BufferData.State.CACHING, true);
testReleaseHelper(BufferData.State.READY, false);
}
private void testReleaseHelper(BufferData.State stateBeforeRelease,
boolean expectThrow)
throws Exception {
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
assertInitialState(pool, POOL_SIZE);
BufferData data = this.acquire(pool, 1);
data.updateState(stateBeforeRelease, BufferData.State.BLANK);
if (expectThrow) {
intercept(IllegalArgumentException.class, "Unable to release buffer",
() -> pool.release(data));
} else {
pool.release(data);
}
}
private BufferData acquire(BufferPool pool, int blockNumber) {
BufferData data = pool.acquire(blockNumber);
assertNotNull(data);
assertSame(data, pool.acquire(blockNumber));
assertEquals(blockNumber, data.getBlockNumber());
return data;
}
private void assertInitialState(BufferPool pool, int poolSize) {
assertEquals(poolSize, pool.numAvailable());
assertEquals(0, pool.numCreated());
}
}

View File

@ -0,0 +1,98 @@
/*
* 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.impl.prefetch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
import static org.junit.Assert.assertTrue;
public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
private ExecutorService executorService;
@Before
public void setUp() {
executorService = Executors.newFixedThreadPool(3);
}
@After
public void tearDown() {
if (executorService != null) {
executorService.shutdownNow();
}
}
@Test
public void testRunnableSucceeds() throws Exception {
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
Future<Void> future =
futurePool.executeRunnable(() -> atomicBoolean.set(true));
future.get(30, TimeUnit.SECONDS);
assertTrue("atomicBoolean set to true?", atomicBoolean.get());
}
@Test
public void testSupplierSucceeds() throws Exception {
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
Future<Void> future = futurePool.executeFunction(() -> {
atomicBoolean.set(true);
return null;
});
future.get(30, TimeUnit.SECONDS);
assertTrue("atomicBoolean set to true?", atomicBoolean.get());
}
@Test
public void testRunnableFails() throws Exception {
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
Future<Void> future = futurePool.executeRunnable(() -> {
throw new IllegalStateException("deliberate");
});
interceptFuture(IllegalStateException.class, "deliberate", 30,
TimeUnit.SECONDS, future);
}
@Test
public void testSupplierFails() throws Exception {
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
Future<Void> future = futurePool.executeFunction(() -> {
throw new IllegalStateException("deliberate");
});
interceptFuture(IllegalStateException.class, "deliberate", 30,
TimeUnit.SECONDS, future);
}
}

View File

@ -0,0 +1,195 @@
/*
* 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.impl.prefetch;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class TestFilePosition extends AbstractHadoopTestBase {
@Test
public void testArgChecks() throws Exception {
ByteBuffer buffer = ByteBuffer.allocate(10);
BufferData data = new BufferData(0, buffer);
// Should not throw.
new FilePosition(0, 0);
new FilePosition(0, 5);
new FilePosition(10, 5);
new FilePosition(5, 10);
new FilePosition(10, 5).setData(data, 3, 4);
// Verify it throws correctly.
intercept(IllegalArgumentException.class, "'fileSize' must not be negative",
() -> new FilePosition(-1, 2));
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new FilePosition(1, 0));
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new FilePosition(1, -1));
FilePosition pos = new FilePosition(10, 3);
// Verify that we cannot obtain buffer properties without setting buffer.
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.buffer());
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.absolute());
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.isWithinCurrentBuffer(2));
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.blockNumber());
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.isLastBlock());
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.bufferFullyRead());
// Verify that we cannot set invalid buffer parameters.
intercept(IllegalArgumentException.class, "'bufferData' must not be null",
() -> pos.setData(null, 4, 4));
intercept(IllegalArgumentException.class,
"'startOffset' must not be negative", () -> pos.setData(data, -4, 4));
intercept(IllegalArgumentException.class,
"'readOffset' must not be negative", () -> pos.setData(data, 4, -4));
intercept(IllegalArgumentException.class,
"'readOffset' must not be negative", () -> pos.setData(data, 4, -4));
intercept(IllegalArgumentException.class,
"'readOffset' (15) must be within the range [4, 13]",
() -> pos.setData(data, 4, 15));
intercept(IllegalArgumentException.class,
"'readOffset' (3) must be within the range [4, 13]",
() -> pos.setData(data, 4, 3));
}
@Test
public void testValidity() {
int bufferSize = 8;
long fileSize = 100;
long bufferStartOffset = 7;
long readStartOffset = 9;
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
BufferData data = new BufferData(0, buffer);
FilePosition pos = new FilePosition(fileSize, bufferSize);
assertFalse(pos.isValid());
pos.setData(data, bufferStartOffset, readStartOffset);
assertTrue(pos.isValid());
pos.invalidate();
assertFalse(pos.isValid());
}
@Test
public void testOffsets() {
int bufferSize = 8;
long fileSize = 100;
long bufferStartOffset = 7;
long readStartOffset = 9;
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
BufferData data = new BufferData(0, buffer);
FilePosition pos = new FilePosition(fileSize, bufferSize);
pos.setData(data, bufferStartOffset, readStartOffset);
assertTrue(pos.isValid());
assertEquals(readStartOffset, pos.absolute());
assertEquals(readStartOffset - bufferStartOffset, pos.relative());
assertTrue(pos.isWithinCurrentBuffer(8));
assertFalse(pos.isWithinCurrentBuffer(6));
assertFalse(pos.isWithinCurrentBuffer(1));
int expectedBlockNumber = (int) (bufferStartOffset / bufferSize);
assertEquals(expectedBlockNumber, pos.blockNumber());
assertFalse(pos.isLastBlock());
pos.setData(data, fileSize - 3, fileSize - 2);
assertTrue(pos.isLastBlock());
}
@Test
public void testBufferStats() {
int bufferSize = 8;
long fileSize = 100;
long bufferStartOffset = 7;
long readStartOffset = 9;
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
BufferData data = new BufferData(0, buffer);
FilePosition pos = new FilePosition(fileSize, bufferSize);
pos.setData(data, bufferStartOffset, readStartOffset);
assertTrue(pos.isValid());
assertEquals(bufferStartOffset, pos.bufferStartOffset());
assertEquals(0, pos.numBytesRead());
assertEquals(0, pos.numSingleByteReads());
assertEquals(0, pos.numBufferReads());
pos.incrementBytesRead(1);
pos.incrementBytesRead(1);
pos.incrementBytesRead(1);
pos.incrementBytesRead(5);
pos.incrementBytesRead(51);
assertEquals(59, pos.numBytesRead());
assertEquals(3, pos.numSingleByteReads());
assertEquals(2, pos.numBufferReads());
assertFalse(pos.bufferFullyRead());
pos.setData(data, bufferStartOffset, bufferStartOffset);
assertTrue(pos.isValid());
assertEquals(0, pos.numBytesRead());
assertEquals(0, pos.numSingleByteReads());
assertEquals(0, pos.numBufferReads());
for (int i = 0; i < bufferSize; i++) {
pos.buffer().get();
pos.incrementBytesRead(1);
}
assertTrue(pos.bufferFullyRead());
}
}

View File

@ -0,0 +1,80 @@
/*
* 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.impl.prefetch;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class TestRetryer extends AbstractHadoopTestBase {
@Test
public void testArgChecks() throws Exception {
// Should not throw.
new Retryer(10, 50, 500);
// Verify it throws correctly.
intercept(IllegalArgumentException.class,
"'perRetryDelay' must be a positive integer",
() -> new Retryer(-1, 50, 500));
intercept(IllegalArgumentException.class,
"'perRetryDelay' must be a positive integer",
() -> new Retryer(0, 50, 500));
intercept(IllegalArgumentException.class,
"'maxDelay' (5) must be greater than 'perRetryDelay' (10)",
() -> new Retryer(10, 5, 500));
intercept(IllegalArgumentException.class,
"'statusUpdateInterval' must be a positive integer",
() -> new Retryer(10, 50, -1));
intercept(IllegalArgumentException.class,
"'statusUpdateInterval' must be a positive integer",
() -> new Retryer(10, 50, 0));
}
@Test
public void testRetry() {
int perRetryDelay = 1;
int statusUpdateInterval = 3;
int maxDelay = 10;
Retryer retryer =
new Retryer(perRetryDelay, maxDelay, statusUpdateInterval);
for (int t = 1; t <= maxDelay; t++) {
assertTrue(retryer.continueRetry());
if (t % statusUpdateInterval == 0) {
assertTrue(retryer.updateStatus());
} else {
assertFalse(retryer.updateStatus());
}
}
assertFalse(retryer.continueRetry());
}
}

View File

@ -0,0 +1,341 @@
/*
* 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.impl.prefetch;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Arrays;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_BYTE_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_INT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_LIST;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_LONG_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_SHORT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_BYTE_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_INT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_LONG_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_SHORT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_BYTE_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_INT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_LIST;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_LONG_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_SHORT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.VALID_LIST;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
public class TestValidate extends AbstractHadoopTestBase {
@Test
public void testCheckNotNull() throws Exception {
String nonNullArg = "nonNullArg";
String nullArg = null;
// Should not throw.
Validate.checkNotNull(nonNullArg, "nonNullArg");
// Verify it throws.
intercept(IllegalArgumentException.class, "'nullArg' must not be null",
() -> Validate.checkNotNull(nullArg, "nullArg"));
}
@Test
public void testCheckPositiveInteger() throws Exception {
int positiveArg = 1;
int zero = 0;
int negativeArg = -1;
// Should not throw.
checkPositiveInteger(positiveArg, "positiveArg");
// Verify it throws.
intercept(IllegalArgumentException.class,
"'negativeArg' must be a positive integer",
() -> checkPositiveInteger(negativeArg, "negativeArg"));
intercept(IllegalArgumentException.class,
"'zero' must be a positive integer",
() -> checkPositiveInteger(zero, "zero"));
}
@Test
public void testCheckNotNegative() throws Exception {
int positiveArg = 1;
int zero = 0;
int negativeArg = -1;
// Should not throw.
Validate.checkNotNegative(zero, "zeroArg");
Validate.checkNotNegative(positiveArg, "positiveArg");
// Verify it throws.
intercept(IllegalArgumentException.class,
"'negativeArg' must not be negative",
() -> Validate.checkNotNegative(negativeArg, "negativeArg"));
}
@Test
public void testCheckRequired() throws Exception {
// Should not throw.
Validate.checkRequired(true, "arg");
// Verify it throws.
intercept(IllegalArgumentException.class, "'arg' is required",
() -> Validate.checkRequired(false, "arg"));
}
@Test
public void testCheckValid() throws Exception {
// Should not throw.
Validate.checkValid(true, "arg");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'arg' is invalid",
() -> Validate.checkValid(false, "arg"));
}
@Test
public void testCheckValidWithValues() throws Exception {
String validValues = "foo, bar";
// Should not throw.
Validate.checkValid(true, "arg", validValues);
// Verify it throws.
intercept(IllegalArgumentException.class,
"'arg' is invalid. Valid values are: foo, bar",
() -> Validate.checkValid(false, "arg", validValues));
}
@Test
public void testCheckNotNullAndNotEmpty() throws Exception {
// Should not throw.
Validate.checkNotNullAndNotEmpty(NON_EMPTY_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_BYTE_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_SHORT_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_INT_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_LONG_ARRAY, "array");
// Verify it throws.
intercept(IllegalArgumentException.class, "'string' must not be empty",
() -> Validate.checkNotNullAndNotEmpty("", "string"));
intercept(IllegalArgumentException.class, "'array' must not be null", () ->
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_ARRAY,
"array"));
intercept(IllegalArgumentException.class,
"'array' must have at least one element", () ->
Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_ARRAY,
"array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(NULL_BYTE_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(EMPTY_BYTE_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(NULL_SHORT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(EMPTY_SHORT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(NULL_INT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(EMPTY_INT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(NULL_LONG_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(EMPTY_LONG_ARRAY, "array"));
}
@Test
public void testCheckListNotNullAndNotEmpty() throws Exception {
// Should not throw.
Validate.checkNotNullAndNotEmpty(VALID_LIST, "list");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'list' must not be null",
() -> Validate.checkNotNullAndNotEmpty(NULL_LIST, "list"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'list' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(EMPTY_LIST, "list"));
}
@Test
public void testCheckNotNullAndNumberOfElements() throws Exception {
// Should not throw.
Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2, 3), 3, "arg");
// Verify it throws.
intercept(IllegalArgumentException.class, "'arg' must not be null",
() -> Validate.checkNotNullAndNumberOfElements(null, 3, "arg"));
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"Number of elements in 'arg' must be exactly 3, 2 given.",
() -> Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2), 3,
"arg")
);
}
@Test
public void testCheckValuesEqual() throws Exception {
// Should not throw.
Validate.checkValuesEqual(1, "arg1", 1, "arg2");
// Verify it throws.
intercept(IllegalArgumentException.class,
"'arg1' (1) must equal 'arg2' (2)",
() -> Validate.checkValuesEqual(1, "arg1", 2, "arg2"));
}
@Test
public void testCheckIntegerMultiple() throws Exception {
// Should not throw.
Validate.checkIntegerMultiple(10, "arg1", 5, "arg2");
// Verify it throws.
intercept(IllegalArgumentException.class,
"'arg1' (10) must be an integer multiple of 'arg2' (3)",
() -> Validate.checkIntegerMultiple(10, "arg1", 3, "arg2"));
}
@Test
public void testCheckGreater() throws Exception {
// Should not throw.
Validate.checkGreater(10, "arg1", 5, "arg2");
// Verify it throws.
intercept(IllegalArgumentException.class,
"'arg1' (5) must be greater than 'arg2' (10)",
() -> Validate.checkGreater(5, "arg1", 10, "arg2"));
}
@Test
public void testCheckGreaterOrEqual() throws Exception {
// Should not throw.
Validate.checkGreaterOrEqual(10, "arg1", 5, "arg2");
// Verify it throws.
intercept(IllegalArgumentException.class,
"'arg1' (5) must be greater than or equal to 'arg2' (10)",
() -> Validate.checkGreaterOrEqual(5, "arg1", 10, "arg2"));
}
@Test
public void testCheckWithinRange() throws Exception {
// Should not throw.
Validate.checkWithinRange(10, "arg", 5, 15);
Validate.checkWithinRange(10.0, "arg", 5.0, 15.0);
// Verify it throws.
intercept(IllegalArgumentException.class,
"'arg' (5) must be within the range [10, 20]",
() -> Validate.checkWithinRange(5, "arg", 10, 20));
intercept(IllegalArgumentException.class,
"'arg' (5.0) must be within the range [10.0, 20.0]",
() -> Validate.checkWithinRange(5.0, "arg", 10.0, 20.0));
}
@Test
public void testCheckPathExists() throws Exception {
Path tempFile = Files.createTempFile("foo", "bar");
Path tempDir = tempFile.getParent();
Path notFound = Paths.get("<not-found>");
// Should not throw.
Validate.checkPathExists(tempFile, "tempFile");
Validate.checkPathExists(tempDir, "tempDir");
// Verify it throws.
intercept(IllegalArgumentException.class, "'nullArg' must not be null",
() -> Validate.checkPathExists(null, "nullArg"));
intercept(IllegalArgumentException.class,
"Path notFound (<not-found>) does not exist",
() -> Validate.checkPathExists(notFound, "notFound"));
intercept(IllegalArgumentException.class, "must point to a directory",
() -> Validate.checkPathExistsAsDir(tempFile, "tempFile"));
intercept(IllegalArgumentException.class, "must point to a file",
() -> Validate.checkPathExistsAsFile(tempDir, "tempDir"));
}
}

View File

@ -1203,4 +1203,29 @@ public final class Constants {
* Default maximum read size in bytes during vectored reads : {@value}.
*/
public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M
/**
* Controls whether the prefetching input stream is enabled.
*/
public static final String PREFETCH_ENABLED_KEY = "fs.s3a.prefetch.enabled";
/**
* Default option as to whether the prefetching input stream is enabled.
*/
public static final boolean PREFETCH_ENABLED_DEFAULT = false;
// If the default values are used, each file opened for reading will consume
// 64 MB of heap space (8 blocks x 8 MB each).
/**
* The size of a single prefetched block in number of bytes.
*/
public static final String PREFETCH_BLOCK_SIZE_KEY = "fs.s3a.prefetch.block.size";
public static final int PREFETCH_BLOCK_DEFAULT_SIZE = 8 * 1024 * 1024;
/**
* Maximum number of blocks prefetched at any given time.
*/
public static final String PREFETCH_BLOCK_COUNT_KEY = "fs.s3a.prefetch.block.count";
public static final int PREFETCH_BLOCK_DEFAULT_COUNT = 8;
}

View File

@ -81,6 +81,7 @@ import com.amazonaws.services.s3.transfer.model.CopyResult;
import com.amazonaws.services.s3.transfer.model.UploadResult;
import com.amazonaws.event.ProgressListener;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -126,6 +127,7 @@ import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
import org.apache.hadoop.fs.statistics.DurationTracker;
@ -291,6 +293,19 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
private TransferManager transfers;
private ExecutorService boundedThreadPool;
private ThreadPoolExecutor unboundedThreadPool;
// S3 reads are prefetched asynchronously using this future pool.
private ExecutorServiceFuturePool futurePool;
// If true, the prefetching input stream is used for reads.
private boolean prefetchEnabled;
// Size in bytes of a single prefetch block.
private int prefetchBlockSize;
// Size of prefetch queue (in number of blocks).
private int prefetchBlockCount;
private int executorCapacity;
private long multiPartThreshold;
public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
@ -497,6 +512,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
longBytesOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1);
enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true);
this.prefetchEnabled = conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
this.prefetchBlockSize = intOption(
conf, PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE, PREFETCH_BLOCK_DEFAULT_SIZE);
this.prefetchBlockCount =
intOption(conf, PREFETCH_BLOCK_COUNT_KEY, PREFETCH_BLOCK_DEFAULT_COUNT, 1);
initThreadPools(conf);
int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION);
@ -602,11 +623,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
// amazon client exception: stop all services then throw the translation
cleanupWithLogger(LOG, span);
stopAllServices();
if (this.futurePool != null) {
this.futurePool = null;
}
throw translateException("initializing ", new Path(name), e);
} catch (IOException | RuntimeException e) {
// other exceptions: stop the services.
cleanupWithLogger(LOG, span);
stopAllServices();
if (this.futurePool != null) {
this.futurePool = null;
}
throw e;
}
}
@ -727,9 +754,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1);
long keepAliveTime = longOption(conf, KEEPALIVE_TIME,
DEFAULT_KEEPALIVE_TIME, 0);
int numPrefetchThreads = this.prefetchEnabled ? this.prefetchBlockCount : 0;
boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
maxThreads,
maxThreads + totalTasks,
maxThreads + totalTasks + numPrefetchThreads,
keepAliveTime, TimeUnit.SECONDS,
name + "-bounded");
unboundedThreadPool = new ThreadPoolExecutor(
@ -741,6 +770,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
unboundedThreadPool.allowCoreThreadTimeOut(true);
executorCapacity = intOption(conf,
EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1);
if (this.prefetchEnabled) {
this.futurePool = new ExecutorServiceFuturePool(boundedThreadPool);
}
}
/**
@ -1498,13 +1530,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
auditSpan);
fileInformation.applyOptions(readContext);
LOG.debug("Opening '{}'", readContext);
return new FSDataInputStream(
new S3AInputStream(
readContext.build(),
createObjectAttributes(path, fileStatus),
createInputStreamCallbacks(auditSpan),
if (this.prefetchEnabled) {
return new FSDataInputStream(
new S3APrefetchingInputStream(
readContext.build(),
createObjectAttributes(path, fileStatus),
createInputStreamCallbacks(auditSpan),
inputStreamStats));
} else {
return new FSDataInputStream(
new S3AInputStream(
readContext.build(),
createObjectAttributes(path, fileStatus),
createInputStreamCallbacks(auditSpan),
inputStreamStats,
unboundedThreadPool));
}
}
/**
@ -1590,7 +1632,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
statisticsContext,
fileStatus,
vectoredIOContext,
IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator())
IOStatisticsContext.getCurrentIOStatisticsContext().getAggregator(),
futurePool,
prefetchBlockSize,
prefetchBlockCount)
.withAuditSpan(auditSpan);
openFileHelper.applyDefaultOptions(roc);
return roc.build();
@ -3706,12 +3751,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
}
protected CopyFromLocalOperation.CopyFromLocalOperationCallbacks
createCopyFromLocalCallbacks() throws IOException {
createCopyFromLocalCallbacks() throws IOException {
LocalFileSystem local = getLocal(getConf());
return new CopyFromLocalCallbacksImpl(local);
}
protected class CopyFromLocalCallbacksImpl implements
protected final class CopyFromLocalCallbacksImpl implements
CopyFromLocalOperation.CopyFromLocalOperationCallbacks {
private final LocalFileSystem local;

View File

@ -845,11 +845,19 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
StreamStatisticNames.STREAM_READ_VERSION_MISMATCHES)
.withGauges(STREAM_READ_GAUGE_INPUT_POLICY)
.withGauges(STREAM_READ_GAUGE_INPUT_POLICY,
STREAM_READ_BLOCKS_IN_FILE_CACHE.getSymbol(),
STREAM_READ_ACTIVE_PREFETCH_OPERATIONS.getSymbol(),
STREAM_READ_ACTIVE_MEMORY_IN_USE.getSymbol()
)
.withDurationTracking(ACTION_HTTP_GET_REQUEST,
ACTION_EXECUTOR_ACQUIRED,
StoreStatisticNames.ACTION_FILE_OPENED,
StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED)
StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED,
StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS,
StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ,
StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ)
.build();
setIOStatistics(st);
aborted = st.getCounterReference(
@ -918,6 +926,18 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
return incCounter(name, value);
}
/**
* Increment the Statistic gauge and the local IOStatistics
* equivalent.
* @param statistic statistic
* @param v value.
* @return local IOStatistic value
*/
private long incAllGauges(Statistic statistic, long v) {
incrementGauge(statistic, v);
return incGauge(statistic.getSymbol(), v);
}
/**
* {@inheritDoc}.
* Increments the number of seek operations,
@ -1046,6 +1066,12 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
bytesDiscardedInVectoredIO.addAndGet(discarded);
}
@Override
public void executorAcquired(Duration timeInQueue) {
// update the duration fields in the IOStatistics.
localIOStatistics().addTimedOperation(ACTION_EXECUTOR_ACQUIRED, timeInQueue);
}
/**
* {@code close()} merges the stream statistics into the filesystem's
* instrumentation instance.
@ -1310,6 +1336,37 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
? StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED
: StreamStatisticNames.STREAM_READ_REMOTE_STREAM_DRAINED);
}
@Override
public DurationTracker prefetchOperationStarted() {
incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, 1);
return trackDuration(StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS);
}
@Override
public void blockAddedToFileCache() {
incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, 1);
}
@Override
public void blockRemovedFromFileCache() {
incAllGauges(STREAM_READ_BLOCKS_IN_FILE_CACHE, -1);
}
@Override
public void prefetchOperationCompleted() {
incAllGauges(STREAM_READ_ACTIVE_PREFETCH_OPERATIONS, -1);
}
@Override
public void memoryAllocated(int size) {
incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, size);
}
@Override
public void memoryFreed(int size) {
incAllGauges(STREAM_READ_ACTIVE_MEMORY_IN_USE, -size);
}
}
/**

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
@ -74,6 +75,15 @@ public class S3AReadOpContext extends S3AOpContext {
/** Thread-level IOStatistics aggregator. **/
private final IOStatisticsAggregator ioStatisticsAggregator;
// S3 reads are prefetched asynchronously using this future pool.
private ExecutorServiceFuturePool futurePool;
// Size in bytes of a single prefetch block.
private final int prefetchBlockSize;
// Size of prefetch queue (in number of blocks).
private final int prefetchBlockCount;
/**
* Instantiate.
* @param path path of read
@ -83,6 +93,9 @@ public class S3AReadOpContext extends S3AOpContext {
* @param dstFileStatus target file status
* @param vectoredIOContext context for vectored read operation.
* @param ioStatisticsAggregator IOStatistics aggregator for each thread.
* @param futurePool the ExecutorServiceFuturePool instance used by async prefetches.
* @param prefetchBlockSize the size (in number of bytes) of each prefetched block.
* @param prefetchBlockCount maximum number of prefetched blocks.
*/
public S3AReadOpContext(
final Path path,
@ -91,12 +104,23 @@ public class S3AReadOpContext extends S3AOpContext {
S3AStatisticsContext instrumentation,
FileStatus dstFileStatus,
VectoredIOContext vectoredIOContext,
IOStatisticsAggregator ioStatisticsAggregator) {
IOStatisticsAggregator ioStatisticsAggregator,
ExecutorServiceFuturePool futurePool,
int prefetchBlockSize,
int prefetchBlockCount) {
super(invoker, stats, instrumentation,
dstFileStatus);
this.path = requireNonNull(path);
this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext");
this.ioStatisticsAggregator = ioStatisticsAggregator;
this.futurePool = futurePool;
Preconditions.checkArgument(
prefetchBlockSize > 0, "invalid prefetchBlockSize %d", prefetchBlockSize);
this.prefetchBlockSize = prefetchBlockSize;
Preconditions.checkArgument(
prefetchBlockCount > 0, "invalid prefetchBlockCount %d", prefetchBlockCount);
this.prefetchBlockCount = prefetchBlockCount;
}
/**
@ -232,6 +256,33 @@ public class S3AReadOpContext extends S3AOpContext {
return ioStatisticsAggregator;
}
/**
* Gets the {@code ExecutorServiceFuturePool} used for asynchronous prefetches.
*
* @return the {@code ExecutorServiceFuturePool} used for asynchronous prefetches.
*/
public ExecutorServiceFuturePool getFuturePool() {
return this.futurePool;
}
/**
* Gets the size in bytes of a single prefetch block.
*
* @return the size in bytes of a single prefetch block.
*/
public int getPrefetchBlockSize() {
return this.prefetchBlockSize;
}
/**
* Gets the size of prefetch queue (in number of blocks).
*
* @return the size of prefetch queue (in number of blocks).
*/
public int getPrefetchBlockCount() {
return this.prefetchBlockCount;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(

View File

@ -378,6 +378,18 @@ public enum Statistic {
StreamStatisticNames.STREAM_READ_TOTAL_BYTES,
"Total count of bytes read from an input stream",
TYPE_COUNTER),
STREAM_READ_BLOCKS_IN_FILE_CACHE(
StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE,
"Gauge of blocks in disk cache",
TYPE_GAUGE),
STREAM_READ_ACTIVE_PREFETCH_OPERATIONS(
StreamStatisticNames.STREAM_READ_ACTIVE_PREFETCH_OPERATIONS,
"Gauge of active prefetches",
TYPE_GAUGE),
STREAM_READ_ACTIVE_MEMORY_IN_USE(
StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE,
"Gauge of active memory in use",
TYPE_GAUGE),
/* Stream Write statistics */

View File

@ -0,0 +1,78 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BlockManager;
import org.apache.hadoop.fs.impl.prefetch.Validate;
/**
* Provides read access to S3 file one block at a time.
*
* A naive implementation of a {@code BlockManager} that provides no prefetching or caching.
* Useful baseline for comparing performance difference against {@code S3ACachingBlockManager}.
*/
public class S3ABlockManager extends BlockManager {
/**
* Reader that reads from S3 file.
*/
private final S3ARemoteObjectReader reader;
/**
* Constructs an instance of {@code S3ABlockManager}.
*
* @param reader a reader that reads from S3 file.
* @param blockData information about each block of the S3 file.
*
* @throws IllegalArgumentException if reader is null.
* @throws IllegalArgumentException if blockData is null.
*/
public S3ABlockManager(S3ARemoteObjectReader reader, BlockData blockData) {
super(blockData);
Validate.checkNotNull(reader, "reader");
this.reader = reader;
}
/**
* Reads into the given {@code buffer} {@code size} bytes from the underlying file
* starting at {@code startOffset}.
*
* @param buffer the buffer to read data in to.
* @param startOffset the offset at which reading starts.
* @param size the number bytes to read.
* @return number of bytes read.
*/
@Override
public int read(ByteBuffer buffer, long startOffset, int size)
throws IOException {
return reader.read(buffer, startOffset, size);
}
@Override
public void close() {
reader.close();
}
}

View File

@ -0,0 +1,96 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.CachingBlockManager;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
/**
* Provides access to S3 file one block at a time.
*/
public class S3ACachingBlockManager extends CachingBlockManager {
private static final Logger LOG = LoggerFactory.getLogger(
S3ACachingBlockManager.class);
/**
* Reader that reads from S3 file.
*/
private final S3ARemoteObjectReader reader;
/**
* Constructs an instance of a {@code S3ACachingBlockManager}.
*
* @param futurePool asynchronous tasks are performed in this pool.
* @param reader reader that reads from S3 file.
* @param blockData information about each block of the S3 file.
* @param bufferPoolSize size of the in-memory cache in terms of number of blocks.
* @param streamStatistics statistics for this stream.
*
* @throws IllegalArgumentException if reader is null.
*/
public S3ACachingBlockManager(
ExecutorServiceFuturePool futurePool,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize,
S3AInputStreamStatistics streamStatistics) {
super(futurePool, blockData, bufferPoolSize, streamStatistics);
Validate.checkNotNull(reader, "reader");
this.reader = reader;
}
protected S3ARemoteObjectReader getReader() {
return this.reader;
}
/**
* Reads into the given {@code buffer} {@code size} bytes from the underlying file
* starting at {@code startOffset}.
*
* @param buffer the buffer to read data in to.
* @param startOffset the offset at which reading starts.
* @param size the number bytes to read.
* @return number of bytes read.
*/
@Override
public int read(ByteBuffer buffer, long startOffset, int size)
throws IOException {
return this.reader.read(buffer, startOffset, size);
}
@Override
public synchronized void close() {
this.reader.close();
super.close();
}
}

View File

@ -0,0 +1,214 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BlockManager;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BLOCK_ACQUIRE_AND_READ;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
/**
* Provides an {@code InputStream} that allows reading from an S3 file.
* Prefetched blocks are cached to local disk if a seek away from the
* current block is issued.
*/
public class S3ACachingInputStream extends S3ARemoteInputStream {
private static final Logger LOG = LoggerFactory.getLogger(
S3ACachingInputStream.class);
/**
* Number of blocks queued for prefching.
*/
private final int numBlocksToPrefetch;
private final BlockManager blockManager;
/**
* Initializes a new instance of the {@code S3ACachingInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
* @param client callbacks used for interacting with the underlying S3 client.
* @param streamStatistics statistics for this stream.
*
* @throws IllegalArgumentException if context is null.
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3ACachingInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics) {
super(context, s3Attributes, client, streamStatistics);
this.numBlocksToPrefetch = this.getContext().getPrefetchBlockCount();
int bufferPoolSize = this.numBlocksToPrefetch + 1;
this.blockManager = this.createBlockManager(
this.getContext().getFuturePool(),
this.getReader(),
this.getBlockData(),
bufferPoolSize);
int fileSize = (int) s3Attributes.getLen();
LOG.debug("Created caching input stream for {} (size = {})", this.getName(),
fileSize);
}
/**
* Moves the current read position so that the next read will occur at {@code pos}.
*
* @param pos the next read will take place at this position.
*
* @throws IllegalArgumentException if pos is outside of the range [0, file size].
*/
@Override
public void seek(long pos) throws IOException {
throwIfClosed();
throwIfInvalidSeek(pos);
// The call to setAbsolute() returns true if the target position is valid and
// within the current block. Therefore, no additional work is needed when we get back true.
if (!getFilePosition().setAbsolute(pos)) {
LOG.info("seek({})", getOffsetStr(pos));
// We could be here in two cases:
// -- the target position is invalid:
// We ignore this case here as the next read will return an error.
// -- it is valid but outside of the current block.
if (getFilePosition().isValid()) {
// There are two cases to consider:
// -- the seek was issued after this buffer was fully read.
// In this case, it is very unlikely that this buffer will be needed again;
// therefore we release the buffer without caching.
// -- if we are jumping out of the buffer before reading it completely then
// we will likely need this buffer again (as observed empirically for Parquet)
// therefore we issue an async request to cache this buffer.
if (!getFilePosition().bufferFullyRead()) {
blockManager.requestCaching(getFilePosition().data());
} else {
blockManager.release(getFilePosition().data());
}
getFilePosition().invalidate();
blockManager.cancelPrefetches();
}
setSeekTargetPos(pos);
}
}
@Override
public void close() throws IOException {
// Close the BlockManager first, cancelling active prefetches,
// deleting cached files and freeing memory used by buffer pool.
blockManager.close();
super.close();
LOG.info("closed: {}", getName());
}
@Override
protected boolean ensureCurrentBuffer() throws IOException {
if (isClosed()) {
return false;
}
if (getFilePosition().isValid() && getFilePosition()
.buffer()
.hasRemaining()) {
return true;
}
long readPos;
int prefetchCount;
if (getFilePosition().isValid()) {
// A sequential read results in a prefetch.
readPos = getFilePosition().absolute();
prefetchCount = numBlocksToPrefetch;
} else {
// A seek invalidates the current position.
// We prefetch only 1 block immediately after a seek operation.
readPos = getSeekTargetPos();
prefetchCount = 1;
}
if (!getBlockData().isValidOffset(readPos)) {
return false;
}
if (getFilePosition().isValid()) {
if (getFilePosition().bufferFullyRead()) {
blockManager.release(getFilePosition().data());
} else {
blockManager.requestCaching(getFilePosition().data());
}
}
int toBlockNumber = getBlockData().getBlockNumber(readPos);
long startOffset = getBlockData().getStartOffset(toBlockNumber);
for (int i = 1; i <= prefetchCount; i++) {
int b = toBlockNumber + i;
if (b < getBlockData().getNumBlocks()) {
blockManager.requestPrefetch(b);
}
}
BufferData data = invokeTrackingDuration(
getS3AStreamStatistics()
.trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ),
() -> blockManager.get(toBlockNumber));
getFilePosition().setData(data, startOffset, readPos);
return true;
}
@Override
public String toString() {
if (isClosed()) {
return "closed";
}
StringBuilder sb = new StringBuilder();
sb.append(String.format("fpos = (%s)%n", getFilePosition()));
sb.append(blockManager.toString());
return sb.toString();
}
protected BlockManager createBlockManager(
ExecutorServiceFuturePool futurePool,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize) {
return new S3ACachingBlockManager(futurePool, reader, blockData,
bufferPoolSize,
getS3AStreamStatistics());
}
}

View File

@ -0,0 +1,102 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
/**
* Provides an {@code InputStream} that allows reading from an S3 file.
* The entire file is read into memory before reads can begin.
*
* Use of this class is recommended only for small files that can fit
* entirely in memory.
*/
public class S3AInMemoryInputStream extends S3ARemoteInputStream {
private static final Logger LOG = LoggerFactory.getLogger(
S3AInMemoryInputStream.class);
private ByteBuffer buffer;
/**
* Initializes a new instance of the {@code S3AInMemoryInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
* @param client callbacks used for interacting with the underlying S3 client.
* @param streamStatistics statistics for this stream.
*
* @throws IllegalArgumentException if context is null.
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3AInMemoryInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics) {
super(context, s3Attributes, client, streamStatistics);
int fileSize = (int) s3Attributes.getLen();
this.buffer = ByteBuffer.allocate(fileSize);
LOG.debug("Created in-memory input stream for {} (size = {})",
getName(), fileSize);
}
/**
* Ensures that a non-empty valid buffer is available for immediate reading.
* It returns true when at least one such buffer is available for reading.
* It returns false on reaching the end of the stream.
*
* @return true if at least one such buffer is available for reading, false otherwise.
*/
@Override
protected boolean ensureCurrentBuffer() throws IOException {
if (isClosed()) {
return false;
}
if (getBlockData().getFileSize() == 0) {
return false;
}
if (!getFilePosition().isValid()) {
buffer.clear();
int numBytesRead =
getReader().read(buffer, 0, buffer.capacity());
if (numBytesRead <= 0) {
return false;
}
BufferData data = new BufferData(0, buffer);
getFilePosition().setData(data, 0, getSeekTargetPos());
}
return getFilePosition().buffer().hasRemaining();
}
}

View File

@ -0,0 +1,260 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
/**
* Enhanced {@code InputStream} for reading from S3.
*
* This implementation provides improved read throughput by asynchronously prefetching
* blocks of configurable size from the underlying S3 file.
*/
public class S3APrefetchingInputStream
extends FSInputStream
implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(
S3APrefetchingInputStream.class);
/**
* Underlying input stream used for reading S3 file.
*/
private S3ARemoteInputStream inputStream;
/**
* Initializes a new instance of the {@code S3APrefetchingInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
* @param client callbacks used for interacting with the underlying S3 client.
* @param streamStatistics statistics for this stream.
*
* @throws IllegalArgumentException if context is null.
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3APrefetchingInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics) {
Validate.checkNotNull(context, "context");
Validate.checkNotNull(s3Attributes, "s3Attributes");
Validate.checkNotNullAndNotEmpty(s3Attributes.getBucket(),
"s3Attributes.getBucket()");
Validate.checkNotNullAndNotEmpty(s3Attributes.getKey(),
"s3Attributes.getKey()");
Validate.checkNotNegative(s3Attributes.getLen(), "s3Attributes.getLen()");
Validate.checkNotNull(client, "client");
Validate.checkNotNull(streamStatistics, "streamStatistics");
long fileSize = s3Attributes.getLen();
if (fileSize <= context.getPrefetchBlockSize()) {
LOG.debug("Creating in memory input stream for {}", context.getPath());
this.inputStream = new S3AInMemoryInputStream(
context,
s3Attributes,
client,
streamStatistics);
} else {
LOG.debug("Creating in caching input stream for {}", context.getPath());
this.inputStream = new S3ACachingInputStream(
context,
s3Attributes,
client,
streamStatistics);
}
}
/**
* Returns the number of bytes available for reading without blocking.
*
* @return the number of bytes available for reading without blocking.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized int available() throws IOException {
throwIfClosed();
return inputStream.available();
}
/**
* Gets the current position.
*
* @return the current position.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized long getPos() throws IOException {
return isClosed() ? 0 : inputStream.getPos();
}
/**
* Reads and returns one byte from this stream.
*
* @return the next byte from this stream.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized int read() throws IOException {
throwIfClosed();
return inputStream.read();
}
/**
* Reads up to {@code len} bytes from this stream and copies them into
* the given {@code buffer} starting at the given {@code offset}.
* Returns the number of bytes actually copied in to the given buffer.
*
* @param buffer the buffer to copy data into.
* @param offset data is copied starting at this offset.
* @param len max number of bytes to copy.
* @return the number of bytes actually copied in to the given buffer.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized int read(byte[] buffer, int offset, int len)
throws IOException {
throwIfClosed();
return inputStream.read(buffer, offset, len);
}
/**
* Closes this stream and releases all acquired resources.
*
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized void close() throws IOException {
if (inputStream != null) {
inputStream.close();
inputStream = null;
super.close();
}
}
/**
* Updates internal data such that the next read will take place at the given {@code pos}.
*
* @param pos new read position.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized void seek(long pos) throws IOException {
throwIfClosed();
inputStream.seek(pos);
}
/**
* Sets the number of bytes to read ahead each time.
*
* @param readahead the number of bytes to read ahead each time..
*/
@Override
public synchronized void setReadahead(Long readahead) {
if (!isClosed()) {
inputStream.setReadahead(readahead);
}
}
/**
* Indicates whether the given {@code capability} is supported by this stream.
*
* @param capability the capability to check.
* @return true if the given {@code capability} is supported by this stream, false otherwise.
*/
@Override
public boolean hasCapability(String capability) {
if (!isClosed()) {
return inputStream.hasCapability(capability);
}
return false;
}
/**
* Access the input stream statistics.
* This is for internal testing and may be removed without warning.
* @return the statistics for this input stream
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public S3AInputStreamStatistics getS3AStreamStatistics() {
if (isClosed()) {
return null;
}
return inputStream.getS3AStreamStatistics();
}
/**
* Gets the internal IO statistics.
*
* @return the internal IO statistics.
*/
@Override
public IOStatistics getIOStatistics() {
if (isClosed()) {
return null;
}
return inputStream.getIOStatistics();
}
protected boolean isClosed() {
return inputStream == null;
}
protected void throwIfClosed() throws IOException {
if (isClosed()) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
}
// Unsupported functions.
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
throwIfClosed();
return false;
}
@Override
public boolean markSupported() {
return false;
}
}

View File

@ -0,0 +1,475 @@
/*
* 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.s3a.prefetch;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.FilePosition;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
import static java.util.Objects.requireNonNull;
/**
* Provides an {@link InputStream} that allows reading from an S3 file.
*/
public abstract class S3ARemoteInputStream
extends InputStream
implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(
S3ARemoteInputStream.class);
/**
* The S3 file read by this instance.
*/
private S3ARemoteObject remoteObject;
/**
* Reading of S3 file takes place through this reader.
*/
private S3ARemoteObjectReader reader;
/**
* Name of this stream. Used only for logging.
*/
private final String name;
/**
* Indicates whether the stream has been closed.
*/
private volatile boolean closed;
/**
* Current position within the file.
*/
private FilePosition fpos;
/** The target of the most recent seek operation. */
private long seekTargetPos;
/** Information about each block of the mapped S3 file. */
private BlockData blockData;
/** Read-specific operation context. */
private S3AReadOpContext context;
/** Attributes of the S3 object being read. */
private S3ObjectAttributes s3Attributes;
/** Callbacks used for interacting with the underlying S3 client. */
private S3AInputStream.InputStreamCallbacks client;
/** Used for reporting input stream access statistics. */
private final S3AInputStreamStatistics streamStatistics;
private S3AInputPolicy inputPolicy;
private final ChangeTracker changeTracker;
private final IOStatistics ioStatistics;
/**
* Initializes a new instance of the {@code S3ARemoteInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
* @param client callbacks used for interacting with the underlying S3 client.
* @param streamStatistics statistics for this stream.
*
* @throws IllegalArgumentException if context is null.
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3ARemoteInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics) {
this.context = requireNonNull(context);
this.s3Attributes = requireNonNull(s3Attributes);
this.client = requireNonNull(client);
this.streamStatistics = requireNonNull(streamStatistics);
this.ioStatistics = streamStatistics.getIOStatistics();
this.name = S3ARemoteObject.getPath(s3Attributes);
this.changeTracker = new ChangeTracker(
this.name,
context.getChangeDetectionPolicy(),
this.streamStatistics.getChangeTrackerStatistics(),
s3Attributes);
setInputPolicy(context.getInputPolicy());
setReadahead(context.getReadahead());
long fileSize = s3Attributes.getLen();
int bufferSize = context.getPrefetchBlockSize();
this.blockData = new BlockData(fileSize, bufferSize);
this.fpos = new FilePosition(fileSize, bufferSize);
this.remoteObject = getS3File();
this.reader = new S3ARemoteObjectReader(remoteObject);
this.seekTargetPos = 0;
}
/**
* Gets the internal IO statistics.
*
* @return the internal IO statistics.
*/
@Override
public IOStatistics getIOStatistics() {
return ioStatistics;
}
/**
* Access the input stream statistics.
* This is for internal testing and may be removed without warning.
* @return the statistics for this input stream
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public S3AInputStreamStatistics getS3AStreamStatistics() {
return streamStatistics;
}
/**
* Sets the number of bytes to read ahead each time.
*
* @param readahead the number of bytes to read ahead each time..
*/
@Override
public synchronized void setReadahead(Long readahead) {
// We support read head by prefetching therefore we ignore the supplied value.
if (readahead != null) {
Validate.checkNotNegative(readahead, "readahead");
}
}
/**
* Indicates whether the given {@code capability} is supported by this stream.
*
* @param capability the capability to check.
* @return true if the given {@code capability} is supported by this stream, false otherwise.
*/
@Override
public boolean hasCapability(String capability) {
return capability.equalsIgnoreCase(StreamCapabilities.IOSTATISTICS)
|| capability.equalsIgnoreCase(StreamCapabilities.READAHEAD);
}
/**
* Set/update the input policy of the stream.
* This updates the stream statistics.
* @param inputPolicy new input policy.
*/
private void setInputPolicy(S3AInputPolicy inputPolicy) {
this.inputPolicy = inputPolicy;
streamStatistics.inputPolicySet(inputPolicy.ordinal());
}
/**
* Returns the number of bytes that can read from this stream without blocking.
*/
@Override
public int available() throws IOException {
throwIfClosed();
if (!ensureCurrentBuffer()) {
return 0;
}
return fpos.buffer().remaining();
}
/**
* Gets the current position.
*
* @return the current position.
* @throws IOException if there is an IO error during this operation.
*/
public long getPos() throws IOException {
throwIfClosed();
if (fpos.isValid()) {
return fpos.absolute();
} else {
return seekTargetPos;
}
}
/**
* Moves the current read position so that the next read will occur at {@code pos}.
*
* @param pos the absolute position to seek to.
* @throws IOException if there an error during this operation.
*
* @throws IllegalArgumentException if pos is outside of the range [0, file size].
*/
public void seek(long pos) throws IOException {
throwIfClosed();
throwIfInvalidSeek(pos);
if (!fpos.setAbsolute(pos)) {
fpos.invalidate();
seekTargetPos = pos;
}
}
/**
* Ensures that a non-empty valid buffer is available for immediate reading.
* It returns true when at least one such buffer is available for reading.
* It returns false on reaching the end of the stream.
*
* @return true if at least one such buffer is available for reading, false otherwise.
* @throws IOException if there is an IO error during this operation.
*/
protected abstract boolean ensureCurrentBuffer() throws IOException;
@Override
public int read() throws IOException {
throwIfClosed();
if (remoteObject.size() == 0
|| seekTargetPos >= remoteObject.size()) {
return -1;
}
if (!ensureCurrentBuffer()) {
return -1;
}
incrementBytesRead(1);
return fpos.buffer().get() & 0xff;
}
/**
* Reads bytes from this stream and copies them into
* the given {@code buffer} starting at the beginning (offset 0).
* Returns the number of bytes actually copied in to the given buffer.
*
* @param buffer the buffer to copy data into.
* @return the number of bytes actually copied in to the given buffer.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public int read(byte[] buffer) throws IOException {
return read(buffer, 0, buffer.length);
}
/**
* Reads up to {@code len} bytes from this stream and copies them into
* the given {@code buffer} starting at the given {@code offset}.
* Returns the number of bytes actually copied in to the given buffer.
*
* @param buffer the buffer to copy data into.
* @param offset data is copied starting at this offset.
* @param len max number of bytes to copy.
* @return the number of bytes actually copied in to the given buffer.
* @throws IOException if there is an IO error during this operation.
*/
@Override
public int read(byte[] buffer, int offset, int len) throws IOException {
throwIfClosed();
if (len == 0) {
return 0;
}
if (remoteObject.size() == 0
|| seekTargetPos >= remoteObject.size()) {
return -1;
}
if (!ensureCurrentBuffer()) {
return -1;
}
int numBytesRead = 0;
int numBytesRemaining = len;
while (numBytesRemaining > 0) {
if (!ensureCurrentBuffer()) {
break;
}
ByteBuffer buf = fpos.buffer();
int bytesToRead = Math.min(numBytesRemaining, buf.remaining());
buf.get(buffer, offset, bytesToRead);
incrementBytesRead(bytesToRead);
offset += bytesToRead;
numBytesRemaining -= bytesToRead;
numBytesRead += bytesToRead;
}
return numBytesRead;
}
protected S3ARemoteObject getFile() {
return remoteObject;
}
protected S3ARemoteObjectReader getReader() {
return reader;
}
protected S3ObjectAttributes getS3ObjectAttributes() {
return s3Attributes;
}
protected FilePosition getFilePosition() {
return fpos;
}
protected String getName() {
return name;
}
protected boolean isClosed() {
return closed;
}
protected long getSeekTargetPos() {
return seekTargetPos;
}
protected void setSeekTargetPos(long pos) {
seekTargetPos = pos;
}
protected BlockData getBlockData() {
return blockData;
}
protected S3AReadOpContext getContext() {
return context;
}
private void incrementBytesRead(int bytesRead) {
if (bytesRead > 0) {
streamStatistics.bytesRead(bytesRead);
if (getContext().getStats() != null) {
getContext().getStats().incrementBytesRead(bytesRead);
}
fpos.incrementBytesRead(bytesRead);
}
}
protected S3ARemoteObject getS3File() {
return new S3ARemoteObject(
context,
s3Attributes,
client,
streamStatistics,
changeTracker
);
}
protected String getOffsetStr(long offset) {
int blockNumber = -1;
if (blockData.isValidOffset(offset)) {
blockNumber = blockData.getBlockNumber(offset);
}
return String.format("%d:%d", blockNumber, offset);
}
/**
* Closes this stream and releases all acquired resources.
*
* @throws IOException if there is an IO error during this operation.
*/
@Override
public void close() throws IOException {
if (closed) {
return;
}
closed = true;
blockData = null;
reader.close();
reader = null;
remoteObject = null;
fpos.invalidate();
try {
client.close();
} finally {
streamStatistics.close();
}
client = null;
}
@Override
public boolean markSupported() {
return false;
}
protected void throwIfClosed() throws IOException {
if (closed) {
throw new IOException(
name + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
}
}
protected void throwIfInvalidSeek(long pos) throws EOFException {
if (pos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + pos);
}
}
// Unsupported functions.
@Override
public void mark(int readlimit) {
throw new UnsupportedOperationException("mark not supported");
}
@Override
public void reset() {
throw new UnsupportedOperationException("reset not supported");
}
@Override
public long skip(long n) {
throw new UnsupportedOperationException("skip not supported");
}
}

View File

@ -0,0 +1,328 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import java.io.InputStream;
import java.util.IdentityHashMap;
import java.util.Map;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.S3Object;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.statistics.DurationTracker;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Encapsulates low level interactions with S3 object on AWS.
*/
public class S3ARemoteObject {
private static final Logger LOG =
LoggerFactory.getLogger(S3ARemoteObject.class);
/**
* Read-specific operation context.
*/
private final S3AReadOpContext context;
/**
* S3 object attributes.
*/
private final S3ObjectAttributes s3Attributes;
/**
* Callbacks used for interacting with the underlying S3 client.
*/
private final S3AInputStream.InputStreamCallbacks client;
/**
* Used for reporting input stream access statistics.
*/
private final S3AInputStreamStatistics streamStatistics;
/**
* Enforces change tracking related policies.
*/
private final ChangeTracker changeTracker;
/**
* Maps a stream returned by openForRead() to the associated S3 object.
* That allows us to close the object when closing the stream.
*/
private Map<InputStream, S3Object> s3Objects;
/**
* uri of the object being read.
*/
private final String uri;
/**
* size of a buffer to create when draining the stream.
*/
private static final int DRAIN_BUFFER_SIZE = 16384;
/**
* Initializes a new instance of the {@code S3ARemoteObject} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
* @param client callbacks used for interacting with the underlying S3 client.
* @param streamStatistics statistics about stream access.
* @param changeTracker helps enforce change tracking policy.
*
* @throws IllegalArgumentException if context is null.
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
* @throws IllegalArgumentException if streamStatistics is null.
* @throws IllegalArgumentException if changeTracker is null.
*/
public S3ARemoteObject(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics,
ChangeTracker changeTracker) {
Validate.checkNotNull(context, "context");
Validate.checkNotNull(s3Attributes, "s3Attributes");
Validate.checkNotNull(client, "client");
Validate.checkNotNull(streamStatistics, "streamStatistics");
Validate.checkNotNull(changeTracker, "changeTracker");
this.context = context;
this.s3Attributes = s3Attributes;
this.client = client;
this.streamStatistics = streamStatistics;
this.changeTracker = changeTracker;
this.s3Objects = new IdentityHashMap<>();
this.uri = this.getPath();
}
/**
* Gets an instance of {@code Invoker} for interacting with S3 API.
*
* @return an instance of {@code Invoker} for interacting with S3 API.
*/
public Invoker getReadInvoker() {
return context.getReadInvoker();
}
/**
* Gets an instance of {@code S3AInputStreamStatistics} used for reporting access metrics.
*
* @return an instance of {@code S3AInputStreamStatistics} used for reporting access metrics.
*/
public S3AInputStreamStatistics getStatistics() {
return streamStatistics;
}
/**
* Gets the path of this file.
*
* @return the path of this file.
*/
public String getPath() {
return getPath(s3Attributes);
}
/**
* Gets the path corresponding to the given s3Attributes.
*
* @param s3Attributes attributes of an S3 object.
* @return the path corresponding to the given s3Attributes.
*/
public static String getPath(S3ObjectAttributes s3Attributes) {
return String.format("s3a://%s/%s", s3Attributes.getBucket(),
s3Attributes.getKey());
}
/**
* Gets the size of this file.
* Its value is cached once obtained from AWS.
*
* @return the size of this file.
*/
public long size() {
return s3Attributes.getLen();
}
/**
* Opens a section of the file for reading.
*
* @param offset Start offset (0 based) of the section to read.
* @param size Size of the section to read.
* @return an {@code InputStream} corresponding to the given section of this file.
*
* @throws IOException if there is an error opening this file section for reading.
* @throws IllegalArgumentException if offset is negative.
* @throws IllegalArgumentException if offset is greater than or equal to file size.
* @throws IllegalArgumentException if size is greater than the remaining bytes.
*/
public InputStream openForRead(long offset, int size) throws IOException {
Validate.checkNotNegative(offset, "offset");
Validate.checkLessOrEqual(offset, "offset", size(), "size()");
Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
streamStatistics.streamOpened();
final GetObjectRequest request =
client.newGetRequest(s3Attributes.getKey())
.withRange(offset, offset + size - 1);
changeTracker.maybeApplyConstraint(request);
String operation = String.format(
"%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset);
DurationTracker tracker = streamStatistics.initiateGetRequest();
S3Object object = null;
try {
object = Invoker.once(operation, uri, () -> client.getObject(request));
} catch (IOException e) {
tracker.failed();
throw e;
} finally {
tracker.close();
}
changeTracker.processResponse(object, operation, offset);
InputStream stream = object.getObjectContent();
synchronized (s3Objects) {
s3Objects.put(stream, object);
}
return stream;
}
void close(InputStream inputStream, int numRemainingBytes) {
S3Object obj;
synchronized (s3Objects) {
obj = s3Objects.get(inputStream);
if (obj == null) {
throw new IllegalArgumentException("inputStream not found");
}
s3Objects.remove(inputStream);
}
if (numRemainingBytes <= context.getAsyncDrainThreshold()) {
// don't bother with async io.
drain(false, "close() operation", numRemainingBytes, obj, inputStream);
} else {
LOG.debug("initiating asynchronous drain of {} bytes", numRemainingBytes);
// schedule an async drain/abort with references to the fields so they
// can be reused
client.submit(
() -> drain(false, "close() operation", numRemainingBytes, obj,
inputStream));
}
}
/**
* drain the stream. This method is intended to be
* used directly or asynchronously, and measures the
* duration of the operation in the stream statistics.
*
* @param shouldAbort force an abort; used if explicitly requested.
* @param reason reason for stream being closed; used in messages
* @param remaining remaining bytes
* @param requestObject http request object;
* @param inputStream stream to close.
* @return was the stream aborted?
*/
private boolean drain(
final boolean shouldAbort,
final String reason,
final long remaining,
final S3Object requestObject,
final InputStream inputStream) {
try {
return invokeTrackingDuration(
streamStatistics.initiateInnerStreamClose(shouldAbort),
() -> drainOrAbortHttpStream(shouldAbort, reason, remaining,
requestObject, inputStream));
} catch (IOException e) {
// this is only here because invokeTrackingDuration() has it in its
// signature
return shouldAbort;
}
}
/**
* Drain or abort the inner stream.
* Exceptions are swallowed.
* If a close() is attempted and fails, the operation escalates to
* an abort.
*
* @param shouldAbort force an abort; used if explicitly requested.
* @param reason reason for stream being closed; used in messages
* @param remaining remaining bytes
* @param requestObject http request object
* @param inputStream stream to close.
* @return was the stream aborted?
*/
private boolean drainOrAbortHttpStream(
boolean shouldAbort,
final String reason,
final long remaining,
final S3Object requestObject,
final InputStream inputStream) {
if (!shouldAbort && remaining > 0) {
try {
long drained = 0;
byte[] buffer = new byte[DRAIN_BUFFER_SIZE];
while (true) {
final int count = inputStream.read(buffer);
if (count < 0) {
// no more data is left
break;
}
drained += count;
}
LOG.debug("Drained stream of {} bytes", drained);
} catch (Exception e) {
// exception escalates to an abort
LOG.debug("When closing {} stream for {}, will abort the stream", uri,
reason, e);
shouldAbort = true;
}
}
cleanupWithLogger(LOG, inputStream);
cleanupWithLogger(LOG, requestObject);
streamStatistics.streamClose(shouldAbort, remaining);
LOG.debug("Stream {} {}: {}; remaining={}", uri,
(shouldAbort ? "aborted" : "closed"), reason,
remaining);
return shouldAbort;
}
}

View File

@ -0,0 +1,175 @@
/*
* 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.s3a.prefetch;
import java.io.Closeable;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.net.SocketTimeoutException;
import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_REMOTE_BLOCK_READ;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
/**
* Provides functionality to read S3 file one block at a time.
*/
public class S3ARemoteObjectReader implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(
S3ARemoteObjectReader.class);
/** We read from the underlying input stream in blocks of this size. */
private static final int READ_BUFFER_SIZE = 64 * 1024;
/** The S3 file to read. */
private final S3ARemoteObject remoteObject;
/** Set to true by close(). */
private volatile boolean closed;
private final S3AInputStreamStatistics streamStatistics;
/**
* Constructs an instance of {@link S3ARemoteObjectReader}.
*
* @param remoteObject The S3 file to read.
*
* @throws IllegalArgumentException if remoteObject is null.
*/
public S3ARemoteObjectReader(S3ARemoteObject remoteObject) {
Validate.checkNotNull(remoteObject, "remoteObject");
this.remoteObject = remoteObject;
this.streamStatistics = this.remoteObject.getStatistics();
}
/**
* Stars reading at {@code offset} and reads upto {@code size} bytes into {@code buffer}.
*
* @param buffer the buffer into which data is returned
* @param offset the absolute offset into the underlying file where reading starts.
* @param size the number of bytes to be read.
*
* @return number of bytes actually read.
* @throws IOException if there is an error reading from the file.
*
* @throws IllegalArgumentException if buffer is null.
* @throws IllegalArgumentException if offset is outside of the range [0, file size].
* @throws IllegalArgumentException if size is zero or negative.
*/
public int read(ByteBuffer buffer, long offset, int size) throws IOException {
Validate.checkNotNull(buffer, "buffer");
Validate.checkWithinRange(offset, "offset", 0, this.remoteObject.size());
Validate.checkPositiveInteger(size, "size");
if (this.closed) {
return -1;
}
int reqSize = (int) Math.min(size, this.remoteObject.size() - offset);
return readOneBlockWithRetries(buffer, offset, reqSize);
}
@Override
public void close() {
this.closed = true;
}
private int readOneBlockWithRetries(ByteBuffer buffer, long offset, int size)
throws IOException {
this.streamStatistics.readOperationStarted(offset, size);
Invoker invoker = this.remoteObject.getReadInvoker();
int invokerResponse =
invoker.retry("read", this.remoteObject.getPath(), true,
trackDurationOfOperation(streamStatistics,
STREAM_READ_REMOTE_BLOCK_READ, () -> {
try {
this.readOneBlock(buffer, offset, size);
} catch (EOFException e) {
// the base implementation swallows EOFs.
return -1;
} catch (SocketTimeoutException e) {
throw e;
} catch (IOException e) {
this.remoteObject.getStatistics().readException();
throw e;
}
return 0;
}));
int numBytesRead = buffer.position();
buffer.limit(numBytesRead);
this.remoteObject.getStatistics()
.readOperationCompleted(size, numBytesRead);
if (invokerResponse < 0) {
return invokerResponse;
} else {
return numBytesRead;
}
}
private void readOneBlock(ByteBuffer buffer, long offset, int size)
throws IOException {
int readSize = Math.min(size, buffer.remaining());
if (readSize == 0) {
return;
}
InputStream inputStream = remoteObject.openForRead(offset, readSize);
int numRemainingBytes = readSize;
byte[] bytes = new byte[READ_BUFFER_SIZE];
int numBytesToRead;
int numBytes;
try {
do {
numBytesToRead = Math.min(READ_BUFFER_SIZE, numRemainingBytes);
numBytes = inputStream.read(bytes, 0, numBytesToRead);
if (numBytes < 0) {
String message = String.format(
"Unexpected end of stream: buffer[%d], readSize = %d, numRemainingBytes = %d",
buffer.capacity(), readSize, numRemainingBytes);
throw new EOFException(message);
}
if (numBytes > 0) {
buffer.put(bytes, 0, numBytes);
numRemainingBytes -= numBytes;
}
}
while (!this.closed && (numRemainingBytes > 0));
} finally {
remoteObject.close(inputStream, numRemainingBytes);
}
}
}

View File

@ -0,0 +1,29 @@
/*
* 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.
*/
/**
* High performance s3 input stream which reads in
* blocks and can cache blocks in the local filesystem.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.prefetch;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.s3a.statistics;
import org.apache.hadoop.fs.impl.prefetch.PrefetchingStatistics;
import org.apache.hadoop.fs.statistics.DurationTracker;
/**
@ -26,7 +27,7 @@ import org.apache.hadoop.fs.statistics.DurationTracker;
* It also contains getters for tests.
*/
public interface S3AInputStreamStatistics extends AutoCloseable,
S3AStatisticInterface {
S3AStatisticInterface, PrefetchingStatistics {
/**
* Seek backwards, incrementing the seek and backward seek counters.

View File

@ -221,6 +221,41 @@ public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
}
@Override
public DurationTracker prefetchOperationStarted() {
return stubDurationTracker();
}
@Override
public void prefetchOperationCompleted() {
}
@Override
public void blockAddedToFileCache() {
}
@Override
public void blockRemovedFromFileCache() {
}
@Override
public void executorAcquired(Duration timeInQueue) {
}
@Override
public void memoryAllocated(int size) {
}
@Override
public void memoryFreed(int size) {
}
/**
* Return an IO statistics instance.
* @return an empty IO statistics instance.
@ -354,6 +389,7 @@ public final class EmptyS3AStatisticsContext implements S3AStatisticsContext {
public DurationTracker initiateInnerStreamClose(final boolean abort) {
return stubDurationTracker();
}
}
/**

View File

@ -47,6 +47,7 @@ full details.
* [Auditing](./auditing.html).
* [Auditing Architecture](./auditing_architecture.html).
* [Testing](./testing.html)
* [Prefetching](./prefetching.html)
## <a name="overview"></a> Overview
@ -1090,6 +1091,29 @@ options are covered in [Testing](./testing.md).
</description>
</property>
<property>
<name>fs.s3a.prefetch.enabled</name>
<value>false</value>
<description>
Enables prefetching and caching when reading from input stream.
</description>
</property>
<property>
<name>fs.s3a.prefetch.block.size</name>
<value>8MB</value>
<description>
The size of a single prefetched block of data.
</description>
</property>
<property>
<name>fs.s3a.prefetch.block.count</name>
<value>8</value>
<description>
Maximum number of blocks prefetched concurrently at any given time.
</description>
</property>
```
## <a name="retry_and_recovery"></a>Retry and Recovery

View File

@ -0,0 +1,192 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
# S3A Prefetching
This document explains the `S3PrefetchingInputStream` and the various components it uses.
This input stream implements prefetching and caching to improve read performance of the input
stream.
A high level overview of this feature was published in
[Pinterest Engineering's blog post titled "Improving efficiency and reducing runtime using S3 read optimization"](https://medium.com/pinterest-engineering/improving-efficiency-and-reducing-runtime-using-s3-read-optimization-b31da4b60fa0).
With prefetching, the input stream divides the remote file into blocks of a fixed size, associates
buffers to these blocks and then reads data into these buffers asynchronously.
It also potentially caches these blocks.
### Basic Concepts
* **Remote File**: A binary blob of data stored on some storage device.
* **Block File**: Local file containing a block of the remote file.
* **Block**: A file is divided into a number of blocks.
The size of the first n-1 blocks is same, and the size of the last block may be same or smaller.
* **Block based reading**: The granularity of read is one block.
That is, either an entire block is read and returned or none at all.
Multiple blocks may be read in parallel.
### Configuring the stream
|Property |Meaning |Default |
|---|---|---|
|`fs.s3a.prefetch.enabled` |Enable the prefetch input stream |`false` |
|`fs.s3a.prefetch.block.size` |Size of a block |`8M` |
|`fs.s3a.prefetch.block.count` |Number of blocks to prefetch |`8` |
### Key Components
`S3PrefetchingInputStream` - When prefetching is enabled, S3AFileSystem will return an instance of
this class as the input stream.
Depending on the remote file size, it will either use
the `S3InMemoryInputStream` or the `S3CachingInputStream` as the underlying input stream.
`S3InMemoryInputStream` - Underlying input stream used when the remote file size < configured block
size.
Will read the entire remote file into memory.
`S3CachingInputStream` - Underlying input stream used when remote file size > configured block size.
Uses asynchronous prefetching of blocks and caching to improve performance.
`BlockData` - Holds information about the blocks in a remote file, such as:
* Number of blocks in the remote file
* Block size
* State of each block (initially all blocks have state *NOT_READY*).
Other states are: Queued, Ready, Cached.
`BufferData` - Holds the buffer and additional information about it such as:
* The block number this buffer is for
* State of the buffer (Unknown, Blank, Prefetching, Caching, Ready, Done).
Initial state of a buffer is blank.
`CachingBlockManager` - Implements reading data into the buffer, prefetching and caching.
`BufferPool` - Manages a fixed sized pool of buffers.
It's used by `CachingBlockManager` to acquire buffers.
`S3File` - Implements operations to interact with S3 such as opening and closing the input stream to
the remote file in S3.
`S3Reader` - Implements reading from the stream opened by `S3File`.
Reads from this input stream in blocks of 64KB.
`FilePosition` - Provides functionality related to tracking the position in the file.
Also gives access to the current buffer in use.
`SingleFilePerBlockCache` - Responsible for caching blocks to the local file system.
Each cache block is stored on the local disk as a separate block file.
### Operation
#### S3InMemoryInputStream
For a remote file with size 5MB, and block size = 8MB, since file size is less than the block size,
the `S3InMemoryInputStream` will be used.
If the caller makes the following read calls:
```
in.read(buffer, 0, 3MB);
in.read(buffer, 0, 2MB);
```
When the first read is issued, there is no buffer in use yet.
The `S3InMemoryInputStream` gets the data in this remote file by calling the `ensureCurrentBuffer()`
method, which ensures that a buffer with data is available to be read from.
The `ensureCurrentBuffer()` then:
* Reads data into a buffer by calling `S3Reader.read(ByteBuffer buffer, long offset, int size)`.
* `S3Reader` uses `S3File` to open an input stream to the remote file in S3 by making
a `getObject()` request with range as `(0, filesize)`.
* The `S3Reader` reads the entire remote file into the provided buffer, and once reading is complete
closes the S3 stream and frees all underlying resources.
* Now the entire remote file is in a buffer, set this data in `FilePosition` so it can be accessed
by the input stream.
The read operation now just gets the required bytes from the buffer in `FilePosition`.
When the second read is issued, there is already a valid buffer which can be used.
Don't do anything else, just read the required bytes from this buffer.
#### S3CachingInputStream
If there is a remote file with size 40MB and block size = 8MB, the `S3CachingInputStream` will be
used.
##### Sequential Reads
If the caller makes the following calls:
```
in.read(buffer, 0, 5MB)
in.read(buffer, 0, 8MB)
```
For the first read call, there is no valid buffer yet.
`ensureCurrentBuffer()` is called, and for the first `read()`, prefetch count is set as 1.
The current block (block 0) is read synchronously, while the blocks to be prefetched (block 1) is
read asynchronously.
The `CachingBlockManager` is responsible for getting buffers from the buffer pool and reading data
into them. This process of acquiring the buffer pool works as follows:
* The buffer pool keeps a map of allocated buffers and a pool of available buffers.
The size of this pool is = prefetch block count + 1.
If the prefetch block count is 8, the buffer pool has a size of 9.
* If the pool is not yet at capacity, create a new buffer and add it to the pool.
* If it's at capacity, check if any buffers with state = done can be released.
Releasing a buffer means removing it from allocated and returning it back to the pool of available
buffers.
* If there are no buffers with state = done currently then nothing will be released, so retry the
above step at a fixed interval a few times till a buffer becomes available.
* If after multiple retries there are still no available buffers, release a buffer in the ready state.
The buffer for the block furthest from the current block is released.
Once a buffer has been acquired by `CachingBlockManager`, if the buffer is in a *READY* state, it is
returned.
This means that data was already read into this buffer asynchronously by a prefetch.
If it's state is *BLANK* then data is read into it using
`S3Reader.read(ByteBuffer buffer, long offset, int size).`
For the second read call, `in.read(buffer, 0, 8MB)`, since the block sizes are of 8MB and only 5MB
of block 0 has been read so far, 3MB of the required data will be read from the current block 0.
Once all data has been read from this block, `S3CachingInputStream` requests the next block (
block 1), which will already have been prefetched and so it can just start reading from it.
Also, while reading from block 1 it will also issue prefetch requests for the next blocks.
The number of blocks to be prefetched is determined by `fs.s3a.prefetch.block.count`.
##### Random Reads
If the caller makes the following calls:
```
in.read(buffer, 0, 5MB)
in.seek(10MB)
in.read(buffer, 0, 4MB)
in.seek(2MB)
in.read(buffer, 0, 4MB)
```
The `CachingInputStream` also caches prefetched blocks.
This happens when a `seek()` is issued for outside the current block and the current block still has
not been fully read.
For the above read sequence, when the `seek(10MB)` call is issued, block 0 has not been read
completely so cache it as the caller will probably want to read from it again.
When `seek(2MB)` is called, the position is back inside block 0.
The next read can now be satisfied from the locally cached block file, which is typically orders of
magnitude faster than a network based read.

View File

@ -0,0 +1,189 @@
/*
* 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.s3a;
import java.net.URI;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.StoreStatisticNames;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Test the prefetching input stream, validates that the underlying S3ACachingInputStream and
* S3AInMemoryInputStream are working as expected.
*/
public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
public ITestS3APrefetchingInputStream() {
super(true);
}
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3APrefetchingInputStream.class);
private static final int S_1K = 1024;
private static final int S_1M = S_1K * S_1K;
// Path for file which should have length > block size so S3ACachingInputStream is used
private Path largeFile;
private FileSystem largeFileFS;
private int numBlocks;
private int blockSize;
private long largeFileSize;
// Size should be < block size so S3AInMemoryInputStream is used
private static final int SMALL_FILE_SIZE = S_1K * 16;
@Override
public Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
conf.setBoolean(PREFETCH_ENABLED_KEY, true);
return conf;
}
@Override
public void teardown() throws Exception {
super.teardown();
cleanupWithLogger(LOG, largeFileFS);
largeFileFS = null;
}
private void openFS() throws Exception {
Configuration conf = getConfiguration();
largeFile = new Path(DEFAULT_CSVTEST_FILE);
blockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE);
largeFileFS = new S3AFileSystem();
largeFileFS.initialize(new URI(DEFAULT_CSVTEST_FILE), getConfiguration());
FileStatus fileStatus = largeFileFS.getFileStatus(largeFile);
largeFileSize = fileStatus.getLen();
numBlocks = calculateNumBlocks(largeFileSize, blockSize);
}
private static int calculateNumBlocks(long largeFileSize, int blockSize) {
if (largeFileSize == 0) {
return 0;
} else {
return ((int) (largeFileSize / blockSize)) + (largeFileSize % blockSize > 0 ? 1 : 0);
}
}
@Test
public void testReadLargeFileFully() throws Throwable {
describe("read a large file fully, uses S3ACachingInputStream");
IOStatistics ioStats;
openFS();
try (FSDataInputStream in = largeFileFS.open(largeFile)) {
ioStats = in.getIOStatistics();
byte[] buffer = new byte[S_1M * 10];
long bytesRead = 0;
while (bytesRead < largeFileSize) {
in.readFully(buffer, 0, (int) Math.min(buffer.length, largeFileSize - bytesRead));
bytesRead += buffer.length;
// Blocks are fully read, no blocks should be cached
verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE,
0);
}
// Assert that first block is read synchronously, following blocks are prefetched
verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS,
numBlocks - 1);
verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, numBlocks);
verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, numBlocks);
}
// Verify that once stream is closed, all memory is freed
verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0);
}
@Test
public void testRandomReadLargeFile() throws Throwable {
describe("random read on a large file, uses S3ACachingInputStream");
IOStatistics ioStats;
openFS();
try (FSDataInputStream in = largeFileFS.open(largeFile)) {
ioStats = in.getIOStatistics();
byte[] buffer = new byte[blockSize];
// Don't read the block completely so it gets cached on seek
in.read(buffer, 0, blockSize - S_1K * 10);
in.seek(blockSize + S_1K * 10);
// Backwards seek, will use cached block
in.seek(S_1K * 5);
in.read();
verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, 2);
verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, 2);
verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, 1);
// block 0 is cached when we seek to block 1, block 1 is cached as it is being prefetched
// when we seek out of block 0, see cancelPrefetches()
verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, 2);
}
verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_BLOCKS_IN_FILE_CACHE, 0);
verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0);
}
@Test
public void testRandomReadSmallFile() throws Throwable {
describe("random read on a small file, uses S3AInMemoryInputStream");
byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26);
Path smallFile = path("randomReadSmallFile");
ContractTestUtils.writeDataset(getFileSystem(), smallFile, data, data.length, 16, true);
try (FSDataInputStream in = getFileSystem().open(smallFile)) {
IOStatistics ioStats = in.getIOStatistics();
byte[] buffer = new byte[SMALL_FILE_SIZE];
in.read(buffer, 0, S_1K * 4);
in.seek(S_1K * 12);
in.read(buffer, 0, S_1K * 4);
verifyStatisticCounterValue(ioStats, StoreStatisticNames.ACTION_HTTP_GET_REQUEST, 1);
verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_OPENED, 1);
verifyStatisticCounterValue(ioStats, StreamStatisticNames.STREAM_READ_PREFETCH_OPERATIONS, 0);
// The buffer pool is not used
verifyStatisticGaugeValue(ioStats, StreamStatisticNames.STREAM_READ_ACTIVE_MEMORY_IN_USE, 0);
}
}
}

View File

@ -31,6 +31,8 @@ import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;
import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@ -78,11 +80,16 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
inputStream.seek(0);
inputStream.readByte();
// Verify > 1 call was made, so we're sure it is correctly configured for each request
IOStatisticAssertions
.assertThatStatisticCounter(inputStream.getIOStatistics(),
StreamStatisticNames.STREAM_READ_OPENED)
.isGreaterThan(1);
if (conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT)) {
// For S3APrefetchingInputStream, verify a call was made
IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1);
} else {
// For S3AInputStream, verify > 1 call was made,
// so we're sure it is correctly configured for each request
IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
StreamStatisticNames.STREAM_READ_OPENED).isGreaterThan(1);
}
// Check list calls work without error
fs.listFiles(requesterPaysPath.getParent(), false);

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
@ -33,6 +34,7 @@ import org.junit.Test;
import java.io.IOException;
import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES;
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_BYTES_READ_CLOSE;
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_TOTAL_BYTES;
@ -72,6 +74,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
IOStatisticsSnapshot iostats = new IOStatisticsSnapshot();
// Open file, read half the data, and then call unbuffer
try (FSDataInputStream inputStream = getFileSystem().open(dest)) {
skipIfCannotUnbuffer(inputStream);
assertTrue(inputStream.getWrappedStream() instanceof S3AInputStream);
int bytesToRead = 8;
readAndAssertBytesRead(inputStream, bytesToRead);
@ -138,6 +141,7 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
Object streamStatsStr;
try {
inputStream = fs.open(dest);
skipIfCannotUnbuffer(inputStream);
streamStatsStr = demandStringifyIOStatisticsSource(inputStream);
LOG.info("initial stream statistics {}", streamStatsStr);
@ -192,6 +196,12 @@ public class ITestS3AUnbuffer extends AbstractS3ATestBase {
return ((S3AInputStream) inputStream.getWrappedStream()).isObjectStreamOpen();
}
private void skipIfCannotUnbuffer(FSDataInputStream inputStream) {
if (!inputStream.hasCapability(StreamCapabilities.UNBUFFER)) {
skip("input stream does not support unbuffer");
}
}
/**
* Read the specified number of bytes from the given
* {@link FSDataInputStream} and assert that

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
@ -1460,16 +1461,24 @@ public final class S3ATestUtils {
}
}
/**
* Get the input stream statistics of an input stream.
* Raises an exception if the inner stream is not an S3A input stream
* or prefetching input stream
* @param in wrapper
* @return the statistics for the inner stream
*/
public static S3AInputStreamStatistics getInputStreamStatistics(
FSDataInputStream in) {
return getS3AInputStream(in).getS3AStreamStatistics();
FSDataInputStream in) {
InputStream inner = in.getWrappedStream();
if (inner instanceof S3AInputStream) {
return ((S3AInputStream) inner).getS3AStreamStatistics();
} else if (inner instanceof S3APrefetchingInputStream) {
return ((S3APrefetchingInputStream) inner).getS3AStreamStatistics();
} else {
throw new AssertionError("Not an S3AInputStream or S3APrefetchingInputStream: " + inner);
}
}
/**
@ -1488,4 +1497,9 @@ public final class S3ATestUtils {
throw new AssertionError("Not an S3AInputStream: " + inner);
}
}
}

View File

@ -34,8 +34,8 @@ import static org.junit.Assert.assertEquals;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.Mockito.atLeast;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ -74,6 +74,6 @@ public class TestS3AUnbuffer extends AbstractS3AMockTest {
stream.unbuffer();
// Verify that unbuffer closed the object stream
verify(objectStream, times(1)).close();
verify(objectStream, atLeast(1)).close();
}
}

View File

@ -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.hadoop.fs.s3a.prefetch;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.concurrent.CompletableFuture;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.S3Object;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.util.functional.CallableRaisingIOE;
/**
* A mock s3 file with some fault injection.
*/
class MockS3ARemoteObject extends S3ARemoteObject {
private byte[] contents;
// If true, throws IOException on open request just once.
// That allows test code to validate behavior related to retries.
private boolean throwExceptionOnOpen;
private static final String BUCKET = "bucket";
private static final String KEY = "key";
MockS3ARemoteObject(int size) {
this(size, false);
}
MockS3ARemoteObject(int size, boolean throwExceptionOnOpen) {
super(
S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1),
S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size),
S3APrefetchFakes.createInputStreamCallbacks(BUCKET, KEY),
EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS,
S3APrefetchFakes.createChangeTracker(BUCKET, KEY, size)
);
this.throwExceptionOnOpen = throwExceptionOnOpen;
this.contents = new byte[size];
for (int b = 0; b < size; b++) {
this.contents[b] = byteAtOffset(b);
}
}
@Override
public InputStream openForRead(long offset, int size) throws IOException {
Validate.checkLessOrEqual(offset, "offset", size(), "size()");
Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
if (throwExceptionOnOpen) {
throwExceptionOnOpen = false;
throw new IOException("Throwing because throwExceptionOnOpen is true ");
}
int bufSize = (int) Math.min(size, size() - offset);
return new ByteArrayInputStream(contents, (int) offset, bufSize);
}
@Override
public void close(InputStream inputStream, int numRemainingBytes) {
// do nothing since we do not use a real S3 stream.
}
public static byte byteAtOffset(int offset) {
return (byte) (offset % 128);
}
public static S3AInputStream.InputStreamCallbacks createClient(String bucketName) {
return new S3AInputStream.InputStreamCallbacks() {
@Override
public S3Object getObject(GetObjectRequest request) {
return null;
}
@Override
public <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation) {
return null;
}
@Override
public GetObjectRequest newGetRequest(String key) {
return new GetObjectRequest(bucketName, key);
}
@Override
public void close() {
}
};
}
}

View File

@ -0,0 +1,413 @@
/*
* 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.s3a.prefetch;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.net.URI;
import java.nio.ByteBuffer;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.impl.prefetch.BlockCache;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.SingleFilePerBlockCache;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.VectoredIOContext;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.s3a.statistics.impl.CountingChangeTracker;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.util.functional.CallableRaisingIOE;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatisticsStore;
/**
* Provides 'fake' implementations of S3ARemoteInputStream variants.
*
* These implementations avoid accessing the following real resources:
* -- S3 store
* -- local filesystem
*
* This arrangement allows thorough multi-threaded testing of those
* implementations without accessing external resources. It also helps
* avoid test flakiness introduced by external factors.
*/
public final class S3APrefetchFakes {
private S3APrefetchFakes() {
}
public static final String E_TAG = "eTag";
public static final String OWNER = "owner";
public static final String VERSION_ID = "v1";
public static final long MODIFICATION_TIME = 0L;
public static final ChangeDetectionPolicy CHANGE_POLICY =
ChangeDetectionPolicy.createPolicy(
ChangeDetectionPolicy.Mode.None,
ChangeDetectionPolicy.Source.None,
false);
public static S3AFileStatus createFileStatus(String key, long fileSize) {
org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
long blockSize = fileSize;
return new S3AFileStatus(
fileSize, MODIFICATION_TIME, path, blockSize, OWNER, E_TAG, VERSION_ID);
}
public static S3ObjectAttributes createObjectAttributes(
String bucket,
String key,
long fileSize) {
org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
String encryptionKey = "";
return new S3ObjectAttributes(
bucket,
path,
key,
S3AEncryptionMethods.NONE,
encryptionKey,
E_TAG,
VERSION_ID,
fileSize);
}
public static S3AReadOpContext createReadContext(
ExecutorServiceFuturePool futurePool,
String key,
int fileSize,
int prefetchBlockSize,
int prefetchBlockCount) {
S3AFileStatus fileStatus = createFileStatus(key, fileSize);
org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
FileSystem.Statistics statistics = new FileSystem.Statistics("s3a");
S3AStatisticsContext statisticsContext = new EmptyS3AStatisticsContext();
RetryPolicy retryPolicy =
RetryPolicies.retryUpToMaximumCountWithFixedSleep(3, 10,
TimeUnit.MILLISECONDS);
return new S3AReadOpContext(
path,
new Invoker(retryPolicy, Invoker.LOG_EVENT),
statistics,
statisticsContext,
fileStatus,
new VectoredIOContext()
.setMinSeekForVectoredReads(1)
.setMaxReadSizeForVectoredReads(1)
.build(),
emptyStatisticsStore(),
futurePool,
prefetchBlockSize,
prefetchBlockCount)
.withChangeDetectionPolicy(
ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None,
ChangeDetectionPolicy.Source.ETag, false))
.withInputPolicy(S3AInputPolicy.Normal);
}
public static URI createUri(String bucket, String key) {
return URI.create(String.format("s3a://%s/%s", bucket, key));
}
public static ChangeTracker createChangeTracker(
String bucket,
String key,
long fileSize) {
return new ChangeTracker(
createUri(bucket, key).toString(),
CHANGE_POLICY,
new CountingChangeTracker(),
createObjectAttributes(bucket, key, fileSize));
}
public static S3ObjectInputStream createS3ObjectInputStream(byte[] buffer) {
return new S3ObjectInputStream(new ByteArrayInputStream(buffer), null);
}
public static S3AInputStream.InputStreamCallbacks createInputStreamCallbacks(
String bucket,
String key) {
S3Object object = new S3Object() {
@Override
public S3ObjectInputStream getObjectContent() {
return createS3ObjectInputStream(new byte[8]);
}
@Override
public ObjectMetadata getObjectMetadata() {
ObjectMetadata metadata = new ObjectMetadata();
metadata.setHeader("ETag", E_TAG);
return metadata;
}
};
return new S3AInputStream.InputStreamCallbacks() {
@Override
public S3Object getObject(GetObjectRequest request) {
return object;
}
@Override
public <T> CompletableFuture<T> submit(CallableRaisingIOE<T> operation) {
return null;
}
@Override
public GetObjectRequest newGetRequest(String key) {
return new GetObjectRequest(bucket, key);
}
@Override
public void close() {
}
};
}
public static S3ARemoteInputStream createInputStream(
Class<? extends S3ARemoteInputStream> clazz,
ExecutorServiceFuturePool futurePool,
String bucket,
String key,
int fileSize,
int prefetchBlockSize,
int prefetchBlockCount) {
org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
S3AFileStatus fileStatus = createFileStatus(key, fileSize);
S3ObjectAttributes s3ObjectAttributes =
createObjectAttributes(bucket, key, fileSize);
S3AReadOpContext s3AReadOpContext = createReadContext(
futurePool,
key,
fileSize,
prefetchBlockSize,
prefetchBlockCount);
S3AInputStream.InputStreamCallbacks callbacks =
createInputStreamCallbacks(bucket, key);
S3AInputStreamStatistics stats =
s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics();
if (clazz == FakeS3AInMemoryInputStream.class) {
return new FakeS3AInMemoryInputStream(s3AReadOpContext,
s3ObjectAttributes, callbacks, stats);
} else if (clazz == FakeS3ACachingInputStream.class) {
return new FakeS3ACachingInputStream(s3AReadOpContext, s3ObjectAttributes,
callbacks, stats);
}
throw new RuntimeException("Unsupported class: " + clazz);
}
public static FakeS3AInMemoryInputStream createS3InMemoryInputStream(
ExecutorServiceFuturePool futurePool,
String bucket,
String key,
int fileSize) {
return (FakeS3AInMemoryInputStream) createInputStream(
FakeS3AInMemoryInputStream.class, futurePool, bucket, key, fileSize, 1,
1);
}
public static FakeS3ACachingInputStream createS3CachingInputStream(
ExecutorServiceFuturePool futurePool,
String bucket,
String key,
int fileSize,
int prefetchBlockSize,
int prefetchBlockCount) {
return (FakeS3ACachingInputStream) createInputStream(
FakeS3ACachingInputStream.class,
futurePool,
bucket,
key,
fileSize,
prefetchBlockSize,
prefetchBlockCount);
}
public static class FakeS3AInMemoryInputStream
extends S3AInMemoryInputStream {
public FakeS3AInMemoryInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics) {
super(context, s3Attributes, client, streamStatistics);
}
@Override
protected S3ARemoteObject getS3File() {
randomDelay(200);
return new MockS3ARemoteObject(
(int) this.getS3ObjectAttributes().getLen(), false);
}
}
public static class FakeS3FilePerBlockCache extends SingleFilePerBlockCache {
private final Map<Path, byte[]> files;
private final int readDelay;
private final int writeDelay;
public FakeS3FilePerBlockCache(int readDelay, int writeDelay) {
super(new EmptyS3AStatisticsContext().newInputStreamStatistics());
this.files = new ConcurrentHashMap<>();
this.readDelay = readDelay;
this.writeDelay = writeDelay;
}
@Override
protected int readFile(Path path, ByteBuffer buffer) {
byte[] source = this.files.get(path);
randomDelay(this.readDelay);
buffer.put(source);
return source.length;
}
@Override
protected void writeFile(Path path, ByteBuffer buffer) throws IOException {
Validate.checkPositiveInteger(buffer.limit(), "buffer.limit()");
byte[] dest = new byte[buffer.limit()];
randomDelay(this.writeDelay);
buffer.rewind();
buffer.get(dest);
this.files.put(path, dest);
}
private long fileCount = 0;
@Override
protected Path getCacheFilePath() throws IOException {
fileCount++;
return Paths.get(Long.toString(fileCount));
}
@Override
public void close() throws IOException {
this.files.clear();
}
}
private static final Random RANDOM = new Random();
private static void randomDelay(int delay) {
try {
Thread.sleep(RANDOM.nextInt(delay));
} catch (InterruptedException e) {
}
}
public static class FakeS3ACachingBlockManager
extends S3ACachingBlockManager {
public FakeS3ACachingBlockManager(
ExecutorServiceFuturePool futurePool,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize) {
super(futurePool, reader, blockData, bufferPoolSize,
new EmptyS3AStatisticsContext().newInputStreamStatistics());
}
@Override
public int read(ByteBuffer buffer, long offset, int size)
throws IOException {
randomDelay(100);
return this.getReader().read(buffer, offset, size);
}
@Override
protected BlockCache createCache() {
final int readDelayMs = 50;
final int writeDelayMs = 200;
return new FakeS3FilePerBlockCache(readDelayMs, writeDelayMs);
}
}
public static class FakeS3ACachingInputStream extends S3ACachingInputStream {
public FakeS3ACachingInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
S3AInputStreamStatistics streamStatistics) {
super(context, s3Attributes, client, streamStatistics);
}
@Override
protected S3ARemoteObject getS3File() {
randomDelay(200);
return new MockS3ARemoteObject(
(int) this.getS3ObjectAttributes().getLen(), false);
}
@Override
protected S3ACachingBlockManager createBlockManager(
ExecutorServiceFuturePool futurePool,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize) {
return new FakeS3ACachingBlockManager(futurePool, reader, blockData,
bufferPoolSize);
}
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
public class TestS3ABlockManager extends AbstractHadoopTestBase {
static final int FILE_SIZE = 12;
static final int BLOCK_SIZE = 3;
@Test
public void testArgChecks() throws Exception {
BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
// Should not throw.
new S3ABlockManager(reader, blockData);
// Verify it throws correctly.
intercept(
IllegalArgumentException.class,
"'reader' must not be null",
() -> new S3ABlockManager(null, blockData));
intercept(
IllegalArgumentException.class,
"'blockData' must not be null",
() -> new S3ABlockManager(reader, null));
intercept(
IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> new S3ABlockManager(reader, blockData).get(-1));
intercept(
IllegalArgumentException.class,
"'data' must not be null",
() -> new S3ABlockManager(reader, blockData).release(null));
}
@Test
public void testGet() throws IOException {
BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
S3ABlockManager blockManager = new S3ABlockManager(reader, blockData);
for (int b = 0; b < blockData.getNumBlocks(); b++) {
BufferData data = blockManager.get(b);
ByteBuffer buffer = data.getBuffer();
long startOffset = blockData.getStartOffset(b);
for (int i = 0; i < BLOCK_SIZE; i++) {
assertEquals(startOffset + i, buffer.get());
}
}
}
}

View File

@ -0,0 +1,361 @@
/*
* 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.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.junit.Test;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
static final int FILE_SIZE = 15;
static final int BLOCK_SIZE = 2;
static final int POOL_SIZE = 3;
private final ExecutorService threadPool = Executors.newFixedThreadPool(4);
private final ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(threadPool);
private final S3AInputStreamStatistics streamStatistics =
new EmptyS3AStatisticsContext().newInputStreamStatistics();
private final BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
@Test
public void testArgChecks() throws Exception {
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
// Should not throw.
S3ACachingBlockManager blockManager =
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
// Verify it throws correctly.
intercept(
NullPointerException.class,
() -> new S3ACachingBlockManager(null, reader, blockData, POOL_SIZE,
streamStatistics));
intercept(
IllegalArgumentException.class,
"'reader' must not be null",
() -> new S3ACachingBlockManager(futurePool, null, blockData, POOL_SIZE,
streamStatistics));
intercept(
IllegalArgumentException.class,
"'blockData' must not be null",
() -> new S3ACachingBlockManager(futurePool, reader, null, POOL_SIZE,
streamStatistics));
intercept(
IllegalArgumentException.class,
"'bufferPoolSize' must be a positive integer",
() -> new S3ACachingBlockManager(futurePool, reader, blockData, 0,
streamStatistics));
intercept(
IllegalArgumentException.class,
"'bufferPoolSize' must be a positive integer",
() -> new S3ACachingBlockManager(futurePool, reader, blockData, -1,
streamStatistics));
intercept(NullPointerException.class,
() -> new S3ACachingBlockManager(futurePool, reader, blockData,
POOL_SIZE, null));
intercept(
IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> blockManager.get(-1));
intercept(
IllegalArgumentException.class,
"'data' must not be null",
() -> blockManager.release(null));
intercept(
IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> blockManager.requestPrefetch(-1));
intercept(
IllegalArgumentException.class,
"'data' must not be null",
() -> blockManager.requestCaching(null));
}
/**
* Extends S3ACachingBlockManager so that we can inject asynchronous failures.
*/
private static final class BlockManagerForTesting
extends S3ACachingBlockManager {
BlockManagerForTesting(
ExecutorServiceFuturePool futurePool,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize,
S3AInputStreamStatistics streamStatistics) {
super(futurePool, reader, blockData, bufferPoolSize, streamStatistics);
}
// If true, forces the next read operation to fail.
// Resets itself to false after one failure.
private boolean forceNextReadToFail;
@Override
public int read(ByteBuffer buffer, long offset, int size)
throws IOException {
if (forceNextReadToFail) {
forceNextReadToFail = false;
throw new RuntimeException("foo");
} else {
return super.read(buffer, offset, size);
}
}
// If true, forces the next cache-put operation to fail.
// Resets itself to false after one failure.
private boolean forceNextCachePutToFail;
@Override
protected void cachePut(int blockNumber, ByteBuffer buffer)
throws IOException {
if (forceNextCachePutToFail) {
forceNextCachePutToFail = false;
throw new RuntimeException("bar");
} else {
super.cachePut(blockNumber, buffer);
}
}
}
// @Ignore
@Test
public void testGet() throws Exception {
testGetHelper(false);
}
// @Ignore
@Test
public void testGetFailure() throws Exception {
testGetHelper(true);
}
private void testGetHelper(boolean forceReadFailure) throws Exception {
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, true);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
BlockManagerForTesting blockManager =
new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
for (int b = 0; b < blockData.getNumBlocks(); b++) {
// We simulate caching failure for all even numbered blocks.
boolean forceFailure = forceReadFailure && (b % 2 == 0);
BufferData data = null;
if (forceFailure) {
blockManager.forceNextReadToFail = true;
intercept(
RuntimeException.class,
"foo",
() -> blockManager.get(3));
} else {
data = blockManager.get(b);
long startOffset = blockData.getStartOffset(b);
for (int i = 0; i < blockData.getSize(b); i++) {
assertEquals(startOffset + i, data.getBuffer().get());
}
blockManager.release(data);
}
assertEquals(POOL_SIZE, blockManager.numAvailable());
}
}
// @Ignore
@Test
public void testPrefetch() throws IOException, InterruptedException {
testPrefetchHelper(false);
}
// @Ignore
@Test
public void testPrefetchFailure() throws IOException, InterruptedException {
testPrefetchHelper(true);
}
private void testPrefetchHelper(boolean forcePrefetchFailure)
throws IOException, InterruptedException {
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
BlockManagerForTesting blockManager =
new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
assertInitialState(blockManager);
int expectedNumErrors = 0;
int expectedNumSuccesses = 0;
for (int b = 0; b < POOL_SIZE; b++) {
// We simulate caching failure for all odd numbered blocks.
boolean forceFailure = forcePrefetchFailure && (b % 2 == 1);
if (forceFailure) {
expectedNumErrors++;
blockManager.forceNextReadToFail = true;
} else {
expectedNumSuccesses++;
}
blockManager.requestPrefetch(b);
}
assertEquals(0, blockManager.numCached());
blockManager.cancelPrefetches();
waitForCaching(blockManager, expectedNumSuccesses);
assertEquals(expectedNumErrors, this.totalErrors(blockManager));
assertEquals(expectedNumSuccesses, blockManager.numCached());
}
// @Ignore
@Test
public void testCachingOfPrefetched()
throws IOException, InterruptedException {
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
S3ACachingBlockManager blockManager =
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
assertInitialState(blockManager);
for (int b = 0; b < blockData.getNumBlocks(); b++) {
blockManager.requestPrefetch(b);
BufferData data = blockManager.get(b);
blockManager.requestCaching(data);
}
waitForCaching(blockManager, blockData.getNumBlocks());
assertEquals(blockData.getNumBlocks(), blockManager.numCached());
assertEquals(0, this.totalErrors(blockManager));
}
// @Ignore
@Test
public void testCachingOfGet() throws IOException, InterruptedException {
testCachingOfGetHelper(false);
}
// @Ignore
@Test
public void testCachingFailureOfGet()
throws IOException, InterruptedException {
testCachingOfGetHelper(true);
}
public void testCachingOfGetHelper(boolean forceCachingFailure)
throws IOException, InterruptedException {
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
BlockManagerForTesting blockManager =
new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
assertInitialState(blockManager);
int expectedNumErrors = 0;
int expectedNumSuccesses = 0;
for (int b = 0; b < blockData.getNumBlocks(); b++) {
// We simulate caching failure for all odd numbered blocks.
boolean forceFailure = forceCachingFailure && (b % 2 == 1);
if (forceFailure) {
expectedNumErrors++;
} else {
expectedNumSuccesses++;
}
BufferData data = blockManager.get(b);
if (forceFailure) {
blockManager.forceNextCachePutToFail = true;
}
blockManager.requestCaching(data);
waitForCaching(blockManager, expectedNumSuccesses);
assertEquals(expectedNumSuccesses, blockManager.numCached());
if (forceCachingFailure) {
assertEquals(expectedNumErrors, this.totalErrors(blockManager));
} else {
assertEquals(0, this.totalErrors(blockManager));
}
}
}
private void waitForCaching(
S3ACachingBlockManager blockManager,
int expectedCount)
throws InterruptedException {
// Wait for async cache operation to be over.
int numTrys = 0;
int count;
do {
Thread.sleep(100);
count = blockManager.numCached();
numTrys++;
if (numTrys > 600) {
String message = String.format(
"waitForCaching: expected: %d, actual: %d, read errors: %d, caching errors: %d",
expectedCount, count, blockManager.numReadErrors(),
blockManager.numCachingErrors());
throw new IllegalStateException(message);
}
}
while (count < expectedCount);
}
private int totalErrors(S3ACachingBlockManager blockManager) {
return blockManager.numCachingErrors() + blockManager.numReadErrors();
}
private void assertInitialState(S3ACachingBlockManager blockManager) {
assertEquals(0, blockManager.numCached());
}
}

View File

@ -0,0 +1,279 @@
/*
* 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.s3a.prefetch;
import java.io.EOFException;
import java.io.IOException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.junit.Test;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.junit.Assert.assertEquals;
/**
* Applies the same set of tests to both S3ACachingInputStream and S3AInMemoryInputStream.
*/
public class TestS3ARemoteInputStream extends AbstractHadoopTestBase {
private static final int FILE_SIZE = 10;
private final ExecutorService threadPool = Executors.newFixedThreadPool(4);
private final ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(threadPool);
private final S3AInputStream.InputStreamCallbacks client =
MockS3ARemoteObject.createClient("bucket");
@Test
public void testArgChecks() throws Exception {
S3AReadOpContext readContext =
S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
S3ObjectAttributes attrs =
S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
S3AInputStreamStatistics stats =
readContext.getS3AStatisticsContext().newInputStreamStatistics();
// Should not throw.
new S3ACachingInputStream(readContext, attrs, client, stats);
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3ACachingInputStream(null, attrs, client, stats));
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3ACachingInputStream(readContext, null, client, stats));
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3ACachingInputStream(readContext, attrs, null, stats));
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3ACachingInputStream(readContext, attrs, client, null));
}
@Test
public void testRead0SizedFile() throws Exception {
S3ARemoteInputStream inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 0);
testRead0SizedFileHelper(inputStream, 9);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
0, 5, 2);
testRead0SizedFileHelper(inputStream, 5);
}
private void testRead0SizedFileHelper(S3ARemoteInputStream inputStream,
int bufferSize)
throws Exception {
assertEquals(0, inputStream.available());
assertEquals(-1, inputStream.read());
assertEquals(-1, inputStream.read());
byte[] buffer = new byte[2];
assertEquals(-1, inputStream.read(buffer));
assertEquals(-1, inputStream.read());
}
@Test
public void testRead() throws Exception {
S3ARemoteInputStream inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", FILE_SIZE);
testReadHelper(inputStream, FILE_SIZE);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
FILE_SIZE, 5, 2);
testReadHelper(inputStream, 5);
}
private void testReadHelper(S3ARemoteInputStream inputStream, int bufferSize)
throws Exception {
assertEquals(bufferSize, inputStream.available());
assertEquals(0, inputStream.read());
assertEquals(1, inputStream.read());
byte[] buffer = new byte[2];
assertEquals(2, inputStream.read(buffer));
assertEquals(2, buffer[0]);
assertEquals(3, buffer[1]);
assertEquals(4, inputStream.read());
buffer = new byte[10];
int curPos = (int) inputStream.getPos();
int expectedRemainingBytes = (int) (FILE_SIZE - curPos);
int readStartOffset = 2;
assertEquals(
expectedRemainingBytes,
inputStream.read(buffer, readStartOffset, expectedRemainingBytes));
for (int i = 0; i < expectedRemainingBytes; i++) {
assertEquals(curPos + i, buffer[readStartOffset + i]);
}
assertEquals(-1, inputStream.read());
Thread.sleep(100);
assertEquals(-1, inputStream.read());
assertEquals(-1, inputStream.read());
assertEquals(-1, inputStream.read(buffer));
assertEquals(-1, inputStream.read(buffer, 1, 3));
}
@Test
public void testSeek() throws Exception {
S3ARemoteInputStream inputStream;
inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 9);
testSeekHelper(inputStream, 9, 9);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
9, 5, 1);
testSeekHelper(inputStream, 5, 9);
}
private void testSeekHelper(S3ARemoteInputStream inputStream,
int bufferSize,
int fileSize)
throws Exception {
assertEquals(0, inputStream.getPos());
inputStream.seek(7);
assertEquals(7, inputStream.getPos());
inputStream.seek(0);
assertEquals(bufferSize, inputStream.available());
for (int i = 0; i < fileSize; i++) {
assertEquals(i, inputStream.read());
}
for (int i = 0; i < fileSize; i++) {
inputStream.seek(i);
for (int j = i; j < fileSize; j++) {
assertEquals(j, inputStream.read());
}
}
// Test invalid seeks.
ExceptionAsserts.assertThrows(
EOFException.class,
FSExceptionMessages.NEGATIVE_SEEK,
() -> inputStream.seek(-1));
}
@Test
public void testRandomSeek() throws Exception {
S3ARemoteInputStream inputStream;
inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 9);
testRandomSeekHelper(inputStream, 9, 9);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
9, 5, 1);
testRandomSeekHelper(inputStream, 5, 9);
}
private void testRandomSeekHelper(S3ARemoteInputStream inputStream,
int bufferSize,
int fileSize)
throws Exception {
assertEquals(0, inputStream.getPos());
inputStream.seek(7);
assertEquals(7, inputStream.getPos());
inputStream.seek(0);
assertEquals(bufferSize, inputStream.available());
for (int i = 0; i < fileSize; i++) {
assertEquals(i, inputStream.read());
}
for (int i = 0; i < fileSize; i++) {
inputStream.seek(i);
for (int j = i; j < fileSize; j++) {
assertEquals(j, inputStream.read());
}
int seekFromEndPos = fileSize - i - 1;
inputStream.seek(seekFromEndPos);
for (int j = seekFromEndPos; j < fileSize; j++) {
assertEquals(j, inputStream.read());
}
}
}
@Test
public void testClose() throws Exception {
S3ARemoteInputStream inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 9);
testCloseHelper(inputStream, 9);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
9, 5, 3);
testCloseHelper(inputStream, 5);
}
private void testCloseHelper(S3ARemoteInputStream inputStream, int bufferSize)
throws Exception {
assertEquals(bufferSize, inputStream.available());
assertEquals(0, inputStream.read());
assertEquals(1, inputStream.read());
inputStream.close();
ExceptionAsserts.assertThrows(
IOException.class,
FSExceptionMessages.STREAM_IS_CLOSED,
() -> inputStream.available());
ExceptionAsserts.assertThrows(
IOException.class,
FSExceptionMessages.STREAM_IS_CLOSED,
() -> inputStream.read());
byte[] buffer = new byte[10];
ExceptionAsserts.assertThrows(
IOException.class,
FSExceptionMessages.STREAM_IS_CLOSED,
() -> inputStream.read(buffer));
// Verify a second close() does not throw.
inputStream.close();
}
}

View File

@ -0,0 +1,88 @@
/*
* 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.s3a.prefetch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.junit.Test;
import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.test.AbstractHadoopTestBase;
public class TestS3ARemoteObject extends AbstractHadoopTestBase {
private final ExecutorService threadPool = Executors.newFixedThreadPool(1);
private final ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(threadPool);
private final S3AInputStream.InputStreamCallbacks client =
MockS3ARemoteObject.createClient("bucket");
@Test
public void testArgChecks() throws Exception {
S3AReadOpContext readContext =
S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
S3ObjectAttributes attrs =
S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
S3AInputStreamStatistics stats =
readContext.getS3AStatisticsContext().newInputStreamStatistics();
ChangeTracker changeTracker =
S3APrefetchFakes.createChangeTracker("bucket", "key", 10);
// Should not throw.
new S3ARemoteObject(readContext, attrs, client, stats, changeTracker);
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'context' must not be null",
() -> new S3ARemoteObject(null, attrs, client, stats, changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'s3Attributes' must not be null",
() -> new S3ARemoteObject(readContext, null, client, stats,
changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'client' must not be null",
() -> new S3ARemoteObject(readContext, attrs, null, stats,
changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'streamStatistics' must not be null",
() -> new S3ARemoteObject(readContext, attrs, client, null,
changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'changeTracker' must not be null",
() -> new S3ARemoteObject(readContext, attrs, client, stats, null));
}
}

View File

@ -0,0 +1,112 @@
/*
* 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.s3a.prefetch;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
public class TestS3ARemoteObjectReader extends AbstractHadoopTestBase {
private static final int FILE_SIZE = 9;
private static final int BUFFER_SIZE = 2;
private final S3ARemoteObject remoteObject =
new MockS3ARemoteObject(FILE_SIZE, false);
@Test
public void testArgChecks() throws Exception {
// Should not throw.
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(remoteObject);
// Verify it throws correctly.
intercept(
IllegalArgumentException.class,
"'remoteObject' must not be null",
() -> new S3ARemoteObjectReader(null));
intercept(
IllegalArgumentException.class,
"'buffer' must not be null",
() -> reader.read(null, 10, 2));
ByteBuffer buffer = ByteBuffer.allocate(BUFFER_SIZE);
intercept(
IllegalArgumentException.class,
"'offset' (-1) must be within the range [0, 9]",
() -> reader.read(buffer, -1, 2));
intercept(
IllegalArgumentException.class,
"'offset' (11) must be within the range [0, 9]",
() -> reader.read(buffer, 11, 2));
intercept(
IllegalArgumentException.class,
"'size' must be a positive integer",
() -> reader.read(buffer, 1, 0));
intercept(
IllegalArgumentException.class,
"'size' must be a positive integer",
() -> reader.read(buffer, 1, -1));
}
@Test
public void testGetWithOffset() throws Exception {
for (int i = 0; i < FILE_SIZE; i++) {
testGetHelper(false, i); // no retry
testGetHelper(true, i); // with retry
}
}
private void testGetHelper(boolean testWithRetry, long startOffset)
throws Exception {
int numBlocks = 0;
ByteBuffer buffer;
S3ARemoteObjectReader reader =
new S3ARemoteObjectReader(
new MockS3ARemoteObject(FILE_SIZE, testWithRetry));
int remainingSize = FILE_SIZE - (int) startOffset;
for (int bufferSize = 0; bufferSize <= FILE_SIZE + 1; bufferSize++) {
buffer = ByteBuffer.allocate(bufferSize);
for (int readSize = 1; readSize <= FILE_SIZE; readSize++) {
buffer.clear();
int numBytesRead = reader.read(buffer, startOffset, readSize);
int expectedNumBytesRead = Math.min(readSize, remainingSize);
expectedNumBytesRead = Math.min(bufferSize, expectedNumBytesRead);
assertEquals(expectedNumBytesRead, numBytesRead);
byte[] bytes = buffer.array();
for (int i = 0; i < expectedNumBytesRead; i++) {
assertEquals(startOffset + i, bytes[i]);
}
}
}
}
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
@ -94,6 +95,14 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
private boolean testDataAvailable = true;
private String assumptionMessage = "test file";
@Override
protected Configuration createScaleConfiguration() {
Configuration conf = super.createScaleConfiguration();
S3ATestUtils.removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
conf.setBoolean(PREFETCH_ENABLED_KEY, false);
return conf;
}
/**
* Open the FS and the test data. The input stream is always set up here.
* @throws IOException IO Problems.