8000 HADOOP-19472: [ABFS] Improve write workload performance for ABFS by efficient concurrency utilization by anmolanmol1234 · Pull Request #7669 · apache/hadoop · GitHub
[go: up one dir, main page]
More Web Proxy on the site http://driver.im/
Skip to content

HADOOP-19472: [ABFS] Improve write workload performance for ABFS by efficient concurrency utilization #7669

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

Draft
wants to merge 16 commits into
base: trunk
Choose a base branch
from
Draft
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
8000
Original file line number Diff line number Diff line change
Expand Up @@ -433,6 +433,14 @@ public class AbfsConfiguration{
FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES, DefaultValue = DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES)
private int maxApacheHttpClientIoExceptionsRetries;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_WRITE_THREADPOOL_KEEP_ALIVE_TIME,
DefaultValue = DEFAULT_WRITE_THREADPOOL_KEEP_ALIVE_TIME)
private int writeThreadPoolKeepAliveTime;

@IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_WRITE_THREADPOOL_CORE_POOL_SIZE,
DefaultValue = DEFAULT_WRITE_THREADPOOL_CORE_POOL_SIZE)
private int writeCorePoolSize;

/**
* Max idle TTL configuration for connection given in
* {@value org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys#FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL}
Expand Down Expand Up @@ -1460,6 +1468,14 @@ public int getWriteMaxConcurrentRequestCount() {
return this.writeMaxConcurrentRequestCount;
}

public int getWriteThreadPoolKeepAliveTime() {
return writeThreadPoolKeepAliveTime;
}

public int getWriteCorePoolSize() {
return writeCorePoolSize;
}

public int getMaxWriteRequestsToQueue() {
if (this.maxWriteRequestsToQueue < 1) {
return 2 * getWriteMaxConcurrentRequestCount();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,10 +45,10 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.WeakHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -131,10 +131,8 @@
import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture;
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
import org.apache.hadoop.util.Preconditions;
import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.apache.http.client.utils.URIBuilder;

import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.METADATA_INCOMPLETE_RENAME_FAILURES;
Expand Down Expand Up @@ -204,6 +202,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
private int blockOutputActiveBlocks;
/** Bounded ThreadPool for this instance. */
private ExecutorService boundedThreadPool;
private WriteThreadPoolSizeManager poolSizeManager;

/** ABFS instance reference to be held by the store to avoid GC close. */
private BackReference fsBackRef;
Expand Down Expand Up @@ -280,11 +279,11 @@ public AzureBlobFileSystemStore(
}
this.blockFactory = abfsStoreBuilder.blockFactory;
this.blockOutputActiveBlocks = abfsStoreBuilder.blockOutputActiveBlocks;
this.boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
abfsConfiguration.getWriteMaxConcurrentRequestCount(),
abfsConfiguration.getMaxWriteRequestsToQueue(),
10L, TimeUnit.SECONDS,
"abfs-bounded");
this.poolSizeManager = WriteThreadPoolSizeManager.getInstance(
getClient().getFileSystem() + "-" + UUID.randomUUID(),
abfsConfiguration);
poolSizeManager.startCPUMonitoring();
this.boundedThreadPool = poolSizeManager.getExecutorService();
}

/**
Expand Down Expand Up @@ -336,16 +335,13 @@ public void close() throws IOException {
try {
Futures.allAsList(futures).get();
// shutdown the threadPool and set it to null.
HadoopExecutors.shutdown(boundedThreadPool, LOG,
30, TimeUnit.SECONDS);
boundedThreadPool = null;
} catch (InterruptedException e) {
LOG.error("Interrupted freeing leases", e);
Thread.currentThread().interrupt();
} catch (ExecutionException e) {
LOG.error("Error freeing leases", e);
} finally {
IOUtils.cleanupWithLogger(LOG, getClient());
IOUtils.cleanupWithLogger(LOG, poolSizeManager, getClient());
}
}

Expand Down
Loading
0