This is an automated email from the ASF dual-hosted git repository. stevel pushed a commit to branch branch-3.3 in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/branch-3.3 by this push: new 236b9aa124e HADOOP-18757. S3A Committer only finalizes the commits in a single thread (#5706) 236b9aa124e is described below commit 236b9aa124e385308bed0bec790d5a2cf3a2425b Author: Moditha Hewasinghage <33624668+modit...@users.noreply.github.com> AuthorDate: Wed Jul 19 11:03:41 2023 +0200 HADOOP-18757. S3A Committer only finalizes the commits in a single thread (#5706) Contributed by Moditha Hewasinghage --- .../main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java | 2 +- .../commit/staging/integration/ITestStagingCommitProtocolFailure.java | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java index c93d2d8f739..2a529fc8648 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java @@ -236,7 +236,7 @@ public final class CommitContext implements Closeable { .setDaemon(true) .setNameFormat(THREAD_PREFIX + jobId + "-%d") .build(); - return new HadoopThreadPoolExecutor(0, numThreads, + return new HadoopThreadPoolExecutor(numThreads, numThreads, THREAD_KEEP_ALIVE_TIME, TimeUnit.SECONDS, new LinkedBlockingQueue<>(), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java index a6d2c57d1d2..08b6c21a863 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocolFailure.java @@ -32,6 +32,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_UPLOADS_ENABLED; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_NAME; @@ -54,6 +55,7 @@ public class ITestStagingCommitProtocolFailure extends AbstractS3ATestBase { conf.setBoolean(MULTIPART_UPLOADS_ENABLED, false); conf.set(S3A_COMMITTER_FACTORY_KEY, CommitConstants.S3A_COMMITTER_FACTORY); conf.set(FS_S3A_COMMITTER_NAME, InternalCommitterConstants.COMMITTER_NAME_STAGING); + disableFilesystemCaching(conf); return conf; } --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org