Skip to content
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

HADOOP-17166. ABFS: making max concurrent requests and max requests that can be que… #2179

Merged
merged 7 commits into from
Sep 9, 2020
Merged
Show file tree
Hide file tree
Changes from 5 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 @@ -86,6 +86,14 @@ public class AbfsConfiguration{
DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED)
private String isNamespaceEnabledAccount;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_MAX_CONCURRENT_REQUESTS,
DefaultValue = -1)
private int writeMaxConcurrentRequestCount;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_MAX_REQUESTS_TO_QUEUE,
DefaultValue = -1)
private int maxWriteRequestsToQueue;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE,
MinValue = MIN_BUFFER_SIZE,
MaxValue = MAX_BUFFER_SIZE,
Expand Down Expand Up @@ -822,6 +830,20 @@ public ExponentialRetryPolicy getOauthTokenFetchRetryPolicy() {
oauthTokenFetchRetryDeltaBackoff);
}

public int getWriteMaxConcurrentRequestCount() {
if (this.writeMaxConcurrentRequestCount < 1) {
return 4 * Runtime.getRuntime().availableProcessors();
}
return this.writeMaxConcurrentRequestCount;
}

public int getMaxWriteRequestsToQueue() {
if (this.maxWriteRequestsToQueue < 1) {
return 2 * getWriteMaxConcurrentRequestCount();
}
return this.maxWriteRequestsToQueue;
}

@VisibleForTesting
void setReadBufferSize(int bufferSize) {
this.readBufferSize = bufferSize;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -486,6 +486,8 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend
.disableOutputStreamFlush(abfsConfiguration.isOutputStreamFlushDisabled())
.withStreamStatistics(new AbfsOutputStreamStatisticsImpl())
.withAppendBlob(isAppendBlob)
.withWriteMaxConcurrentRequestCount(abfsConfiguration.getWriteMaxConcurrentRequestCount())
.withMaxWriteRequestsToQueue(abfsConfiguration.getMaxWriteRequestsToQueue())
.build();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,8 @@ public final class ConfigurationKeys {
public static final String AZURE_OAUTH_TOKEN_FETCH_RETRY_DELTA_BACKOFF = "fs.azure.oauth.token.fetch.retry.delta.backoff";

// Read and write buffer sizes defined by the user
public static final String AZURE_WRITE_MAX_CONCURRENT_REQUESTS = "fs.azure.write.max.concurrent.requests";
Copy link
Contributor

Choose a reason for hiding this comment

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

use experimental in name to show they are exactly that

Copy link
Contributor Author

Choose a reason for hiding this comment

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

These configs are tested on prod environments. The same can remain as a means to controle the resource usage. With the internal discussions we had we would like to keep the same this way.

public static final String AZURE_WRITE_MAX_REQUESTS_TO_QUEUE = "fs.azure.write.max.requests.to.queue";
public static final String AZURE_WRITE_BUFFER_SIZE = "fs.azure.write.request.size";
public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size";
public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa
private byte[] buffer;
private int bufferIndex;
private final int maxConcurrentRequestCount;
private final int maxRequestsThatCanBeQueued;

private ConcurrentLinkedDeque<WriteOperation> writeOperations;
private final ThreadPoolExecutor threadExecutor;
Expand Down Expand Up @@ -119,8 +120,11 @@ public AbfsOutputStream(
if (this.isAppendBlob) {
this.maxConcurrentRequestCount = 1;
} else {
this.maxConcurrentRequestCount = 4 * Runtime.getRuntime().availableProcessors();
this.maxConcurrentRequestCount = abfsOutputStreamContext
.getWriteMaxConcurrentRequestCount();
}
this.maxRequestsThatCanBeQueued = abfsOutputStreamContext
.getMaxWriteRequestsToQueue();
this.threadExecutor
= new ThreadPoolExecutor(maxConcurrentRequestCount,
maxConcurrentRequestCount,
Expand Down Expand Up @@ -371,7 +375,7 @@ private synchronized void writeCurrentBufferToService() throws IOException {
final long offset = position;
position += bytesLength;

if (threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
if (threadExecutor.getQueue().size() >= maxRequestsThatCanBeQueued) {
long start = System.currentTimeMillis();
waitForTaskToComplete();
outputStreamStatistics.timeSpentTaskWait(start, System.currentTimeMillis());
Expand Down Expand Up @@ -543,6 +547,16 @@ public int getWriteOperationsSize() {
return writeOperations.size();
}

@VisibleForTesting
int getMaxConcurrentRequestCount() {
return this.maxConcurrentRequestCount;
}

@VisibleForTesting
int getMaxRequestsThatCanBeQueued() {
return maxRequestsThatCanBeQueued;
}

/**
* Appending AbfsOutputStream statistics to base toString().
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,10 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {

private boolean isAppendBlob;

private int writeMaxConcurrentRequestCount;

private int maxWriteRequestsToQueue;

public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) {
super(sasTokenRenewPeriodForStreamsInSeconds);
}
Expand Down Expand Up @@ -71,6 +75,18 @@ public AbfsOutputStreamContext build() {
return this;
}

public AbfsOutputStreamContext withWriteMaxConcurrentRequestCount(
final int writeMaxConcurrentRequestCount) {
this.writeMaxConcurrentRequestCount = writeMaxConcurrentRequestCount;
return this;
}

public AbfsOutputStreamContext withMaxWriteRequestsToQueue(
final int maxWriteRequestsToQueue) {
this.maxWriteRequestsToQueue = maxWriteRequestsToQueue;
return this;
}

public int getWriteBufferSize() {
return writeBufferSize;
}
Expand All @@ -90,4 +106,12 @@ public AbfsOutputStreamStatistics getStreamStatistics() {
public boolean isAppendBlob() {
return isAppendBlob;
}

public int getWriteMaxConcurrentRequestCount() {
return this.writeMaxConcurrentRequestCount;
}

public int getMaxWriteRequestsToQueue() {
return this.maxWriteRequestsToQueue;
}
}
10 changes: 10 additions & 0 deletions hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
Original file line number Diff line number Diff line change
Expand Up @@ -785,6 +785,16 @@ greater than or equal to 0.
in bytes. The value should be between 16384 to 104857600 both inclusive (16 KB
to 100 MB). The default value will be 8388608 (8 MB).

`fs.azure.write.max.concurrent.requests`: To set the maximum concurrent
write requests from an AbfsOutputStream instance to server at any point of
time. Effectively this will be the threadpool size within the
AbfsOutputStream instance. Set the value in between 1 to 8 both inclusive.

`fs.azure.write.max.requests.to.queue`: To set the maximum write requests
that can be queued. Memory consumption of AbfsOutputStream instance can be
tuned with this config considering each queued request holds a buffer. Set
the value 3 or 4 times the value set for s.azure.write.max.concurrent.requests.

`fs.azure.read.request.size`: To set the read buffer size.Specify the value in
bytes. The value should be between 16384 to 104857600 both inclusive (16 KB to
100 MB). The default value will be 4194304 (4 MB).
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.azurebfs.services;

import org.assertj.core.api.Assertions;
import org.junit.Test;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;

/**
* Test create operation.
*/
public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
private static final Path TEST_FILE_PATH = new Path("testfile");

public ITestAbfsOutputStream() throws Exception {
super();
}

@Test
public void testMaxRequestsAndQueueCapacityDefaults() throws Exception {
Configuration conf = getRawConfiguration();
final AzureBlobFileSystem fs = getFileSystem(conf);
FSDataOutputStream out = fs.create(TEST_FILE_PATH);
steveloughran marked this conversation as resolved.
Show resolved Hide resolved
AbfsOutputStream stream = (AbfsOutputStream) out.getWrappedStream();
Assertions.assertThat(stream.getMaxConcurrentRequestCount()).describedAs(
"maxConcurrentRequests should be " + getConfiguration()
.getWriteMaxConcurrentRequestCount())
.isEqualTo(getConfiguration().getWriteMaxConcurrentRequestCount());
Assertions.assertThat(stream.getMaxRequestsThatCanBeQueued()).describedAs(
"maxRequestsToQueue should be " + getConfiguration()
.getMaxWriteRequestsToQueue())
.isEqualTo(getConfiguration().getMaxWriteRequestsToQueue());
}

@Test
public void testMaxRequestsAndQueueCapacity() throws Exception {
Configuration conf = getRawConfiguration();
int maxConcurrentRequests = 6;
int maxRequestsToQueue = 10;
conf.set(ConfigurationKeys.AZURE_WRITE_MAX_CONCURRENT_REQUESTS,
"" + maxConcurrentRequests);
conf.set(ConfigurationKeys.AZURE_WRITE_MAX_REQUESTS_TO_QUEUE,
"" + maxRequestsToQueue);
final AzureBlobFileSystem fs = getFileSystem(conf);
FSDataOutputStream out = fs.create(TEST_FILE_PATH);
AbfsOutputStream stream = (AbfsOutputStream) out.getWrappedStream();
Assertions.assertThat(stream.getMaxConcurrentRequestCount())
.describedAs("maxConcurrentRequests should be " + maxConcurrentRequests)
.isEqualTo(maxConcurrentRequests);
Assertions.assertThat(stream.getMaxRequestsThatCanBeQueued())
.describedAs("maxRequestsToQueue should be " + maxRequestsToQueue)
.isEqualTo(maxRequestsToQueue);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hadoop.fs.azurebfs.services;

import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Random;
Expand Down Expand Up @@ -54,13 +55,17 @@ public final class TestAbfsOutputStream {
private AbfsOutputStreamContext populateAbfsOutputStreamContext(int writeBufferSize,
boolean isFlushEnabled,
boolean disableOutputStreamFlush,
boolean isAppendBlob) {
boolean isAppendBlob) throws IOException, IllegalAccessException {
AbfsConfiguration abfsConf = new AbfsConfiguration(new Configuration(),
accountName1);
return new AbfsOutputStreamContext(2)
.withWriteBufferSize(writeBufferSize)
.enableFlush(isFlushEnabled)
.disableOutputStreamFlush(disableOutputStreamFlush)
.withStreamStatistics(new AbfsOutputStreamStatisticsImpl())
.withAppendBlob(isAppendBlob)
.withWriteMaxConcurrentRequestCount(abfsConf.getWriteMaxConcurrentRequestCount())
.withMaxWriteRequestsToQueue(abfsConf.getMaxWriteRequestsToQueue())
.build();
}

Expand Down