Skip to content

HADOOP-17250 Lot of short reads can be merged with readahead. #3110

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 4 commits into from
Jul 5, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,12 @@ public class AbfsConfiguration{
DefaultValue = DEFAULT_READ_BUFFER_SIZE)
private int readBufferSize;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_AHEAD_RANGE,
MinValue = MIN_BUFFER_SIZE,
MaxValue = MAX_BUFFER_SIZE,
DefaultValue = DEFAULT_READ_AHEAD_RANGE)
private int readAheadRange;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MIN_BACKOFF_INTERVAL,
DefaultValue = DEFAULT_MIN_BACKOFF_INTERVAL)
private int minBackoffInterval;
Expand Down Expand Up @@ -900,6 +906,10 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio
}
}

public int getReadAheadRange() {
return this.readAheadRange;
}

int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
String value = get(validator.ConfigurationKey());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -721,6 +721,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext(
.withTolerateOobAppends(abfsConfiguration.getTolerateOobAppends())
.withReadSmallFilesCompletely(abfsConfiguration.readSmallFilesCompletely())
.withOptimizeFooterRead(abfsConfiguration.optimizeFooterRead())
.withReadAheadRange(abfsConfiguration.getReadAheadRange())
.withStreamStatistics(new AbfsInputStreamStatisticsImpl())
.withShouldReadBufferSizeAlways(
abfsConfiguration.shouldReadBufferSizeAlways())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,14 @@ public final class ConfigurationKeys {
public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size";
public static final String AZURE_READ_SMALL_FILES_COMPLETELY = "fs.azure.read.smallfilescompletely";
public static final String AZURE_READ_OPTIMIZE_FOOTER_READ = "fs.azure.read.optimizefooterread";

/**
* Read ahead range parameter which can be set by user.
* Default value is {@link FileSystemConfigurations#DEFAULT_READ_AHEAD_RANGE}.
* This might reduce number of calls to remote as next requested
* data could already be present in buffer {@value}.
*/
public static final String AZURE_READ_AHEAD_RANGE = "fs.azure.readahead.range";
public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
public static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost";
public static final String AZURE_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_OPTIMIZE_FOOTER_READ = false;
public static final boolean DEFAULT_ALWAYS_READ_BUFFER_SIZE = false;
public static final int DEFAULT_READ_AHEAD_BLOCK_SIZE = 4 * ONE_MB;
public static final int DEFAULT_READ_AHEAD_RANGE = 64 * ONE_KB; // 64 KB
public static final int MIN_BUFFER_SIZE = 16 * ONE_KB; // 16 KB
public static final int MAX_BUFFER_SIZE = 100 * ONE_MB; // 100 MB
public static final long MAX_AZURE_BLOCK_SIZE = 256 * 1024 * 1024L; // changing default abfs blocksize to 256MB
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,8 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,
* @see #read(long, byte[], int, int)
*/
private final boolean bufferedPreadDisabled;
// User configured size of read ahead.
private final int readAheadRange;

private boolean firstRead = true;
// SAS tokens can be re-used until they expire
Expand Down Expand Up @@ -112,6 +114,11 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer,

private final AbfsInputStreamContext context;
private IOStatistics ioStatistics;
/**
* This is the actual position within the object, used by
* lazy seek to decide whether to seek on the next read or not.
*/
private long nextReadPos;

public AbfsInputStream(
final AbfsClient client,
Expand All @@ -129,6 +136,7 @@ public AbfsInputStream(
this.readAheadQueueDepth = abfsInputStreamContext.getReadAheadQueueDepth();
this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
this.eTag = eTag;
this.readAheadRange = abfsInputStreamContext.getReadAheadRange();
this.readAheadEnabled = true;
this.alwaysReadBufferSize
= abfsInputStreamContext.shouldReadBufferSizeAlways();
Expand Down Expand Up @@ -225,6 +233,28 @@ public synchronized int read(final byte[] b, final int off, final int len) throw
}
incrementReadOps();
do {

// limit is the maximum amount of data present in buffer.
// fCursor is the current file pointer. Thus maximum we can
// go back and read from buffer is fCursor - limit.
// There maybe case that we read less than requested data.
long filePosAtStartOfBuffer = fCursor - limit;
if (nextReadPos >= filePosAtStartOfBuffer && nextReadPos <= fCursor) {
// Determining position in buffer from where data is to be read.
bCursor = (int) (nextReadPos - filePosAtStartOfBuffer);

// When bCursor == limit, buffer will be filled again.
// So in this case we are not actually reading from buffer.
if (bCursor != limit && streamStatistics != null) {
streamStatistics.seekInBuffer();
}
} else {
// Clearing the buffer and setting the file pointer
// based on previous seek() call.
fCursor = nextReadPos;
limit = 0;
bCursor = 0;
}
if (shouldReadFully()) {
lastReadBytes = readFileCompletely(b, currentOff, currentLen);
} else if (shouldReadLastBlock()) {
Expand Down Expand Up @@ -283,9 +313,13 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO
} else {
// Enable readAhead when reading sequentially
if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
LOG.debug("Sequential read with read ahead size of {}", bufferSize);
bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);
} else {
bytesRead = readInternal(fCursor, buffer, 0, b.length, true);
// Enabling read ahead for random reads as well to reduce number of remote calls.
int lengthWithReadAhead = Math.min(b.length + readAheadRange, bufferSize);
LOG.debug("Random read with read ahead size of {}", lengthWithReadAhead);
bytesRead = readInternal(fCursor, buffer, 0, lengthWithReadAhead, true);
}
}
if (firstRead) {
Expand Down Expand Up @@ -419,6 +453,7 @@ private int copyToUserBuffer(byte[] b, int off, int len){
int bytesToRead = min(len, bytesRemaining);
System.arraycopy(buffer, bCursor, b, off, bytesToRead);
bCursor += bytesToRead;
nextReadPos += bytesToRead;
if (statistics != null) {
statistics.incrementBytesRead(bytesToRead);
}
Expand Down Expand Up @@ -502,13 +537,13 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t
final AbfsRestOperation op;
AbfsPerfTracker tracker = client.getAbfsPerfTracker();
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) {
if (streamStatistics != null) {
streamStatistics.remoteReadOperation();
}
LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length);
op = client.read(path, position, b, offset, length,
tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), tracingContext);
cachedSasToken.update(op.getSasToken());
if (streamStatistics != null) {
streamStatistics.remoteReadOperation();
}
LOG.debug("issuing HTTP GET request params position = {} b.length = {} "
+ "offset = {} length = {}", position, b.length, offset, length);
perfInfo.registerResult(op.getResult()).registerSuccess(true);
Expand Down Expand Up @@ -566,21 +601,9 @@ public synchronized void seek(long n) throws IOException {
streamStatistics.seek(n, fCursor);
}

if (n>=fCursor-limit && n<=fCursor) { // within buffer
bCursor = (int) (n-(fCursor-limit));
if (streamStatistics != null) {
streamStatistics.seekInBuffer();
}
return;
}

// next read will read from here
fCursor = n;
LOG.debug("set fCursor to {}", fCursor);

//invalidate buffer
limit = 0;
bCursor = 0;
nextReadPos = n;
LOG.debug("set nextReadPos to {}", nextReadPos);
}

@Override
Expand Down Expand Up @@ -651,7 +674,7 @@ public synchronized long getPos() throws IOException {
if (closed) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
return fCursor - limit + bCursor;
return nextReadPos < 0 ? 0 : nextReadPos;
}

public TracingContext getTracingContext() {
Expand Down Expand Up @@ -721,6 +744,11 @@ byte[] getBuffer() {
return buffer;
}

@VisibleForTesting
public int getReadAheadRange() {
return readAheadRange;
}

@VisibleForTesting
protected void setCachedSasToken(final CachedSASToken cachedSasToken) {
this.cachedSasToken = cachedSasToken;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;

/**
* Class to hold extra input stream configs.
Expand All @@ -38,6 +39,8 @@ public class AbfsInputStreamContext extends AbfsStreamContext {

private int readAheadBlockSize;

private int readAheadRange;

private AbfsInputStreamStatistics streamStatistics;

private boolean readSmallFilesCompletely;
Expand Down Expand Up @@ -69,6 +72,12 @@ public AbfsInputStreamContext withTolerateOobAppends(
return this;
}

public AbfsInputStreamContext withReadAheadRange(
final int readAheadRange) {
this.readAheadRange = readAheadRange;
return this;
}

public AbfsInputStreamContext withStreamStatistics(
final AbfsInputStreamStatistics streamStatistics) {
this.streamStatistics = streamStatistics;
Expand Down Expand Up @@ -115,6 +124,8 @@ public AbfsInputStreamContext build() {
readAheadBlockSize = readBufferSize;
}
// Validation of parameters to be done here.
Preconditions.checkArgument(readAheadRange > 0,
"Read ahead range should be greater than 0");
return this;
}

Expand All @@ -130,6 +141,10 @@ public boolean isTolerateOobAppends() {
return tolerateOobAppends;
}

public int getReadAheadRange() {
return readAheadRange;
}

public AbfsInputStreamStatistics getStreamStatistics() {
return streamStatistics;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,7 @@ public void testSeekStatistics() throws IOException {
*/
for (int i = 0; i < OPERATIONS; i++) {
in.seek(0);
in.read();
in.seek(ONE_MB);
}

Expand Down Expand Up @@ -157,7 +158,7 @@ public void testSeekStatistics() throws IOException {
* are in buffer.
*
* seekInBuffer - Since all seeks were in buffer, the seekInBuffer
* would be equal to 2 * OPERATIONS.
* would be equal to OPERATIONS.
*
*/
assertEquals("Mismatch in seekOps value", 2 * OPERATIONS,
Expand All @@ -170,7 +171,7 @@ public void testSeekStatistics() throws IOException {
OPERATIONS * ONE_MB, stats.getBytesBackwardsOnSeek());
assertEquals("Mismatch in bytesSkippedOnSeek value",
0, stats.getBytesSkippedOnSeek());
assertEquals("Mismatch in seekInBuffer value", 2 * OPERATIONS,
assertEquals("Mismatch in seekInBuffer value", OPERATIONS,
stats.getSeekInBuffer());

in.close();
Expand Down Expand Up @@ -263,6 +264,7 @@ public void testWithNullStreamStatistics() throws IOException {
.withReadBufferSize(getConfiguration().getReadBufferSize())
.withReadAheadQueueDepth(getConfiguration().getReadAheadQueueDepth())
.withStreamStatistics(null)
.withReadAheadRange(getConfiguration().getReadAheadRange())
.build();

AbfsOutputStream out = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.hadoop.fs.azurebfs.utils.Base64;

import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_AHEAD_RANGE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_WRITE_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_MAX_RETRY_ATTEMPTS;
Expand Down Expand Up @@ -141,6 +142,7 @@ public void testConfigServiceImplAnnotatedFieldsInitialized() throws Exception {
assertEquals(DEFAULT_MAX_RETRY_ATTEMPTS, abfsConfiguration.getMaxIoRetries());
assertEquals(MAX_AZURE_BLOCK_SIZE, abfsConfiguration.getAzureBlockSize());
assertEquals(AZURE_BLOCK_LOCATION_HOST_DEFAULT, abfsConfiguration.getAzureBlockLocationHost());
assertEquals(DEFAULT_READ_AHEAD_RANGE, abfsConfiguration.getReadAheadRange());
}

@Test
Expand Down
Loading