Browse Source

HADOOP-18757. S3A Committer only finalizes the commits in a single thread (#5706)

Contributed by Moditha Hewasinghage
Moditha Hewasinghage 1 year ago
parent
commit
b6b259066f

+ 1 - 1
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<>(),

+ 2 - 0
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;
   }