Skip to content

Commit

Permalink
HADOOP-18757. S3A Committer only finalizes the commits in a single th…
Browse files Browse the repository at this point in the history
…read (#5706)


Contributed by Moditha Hewasinghage
  • Loading branch information
modithah authored Jul 19, 2023
1 parent 8df1616 commit b6b2590
Show file tree
Hide file tree
Showing 2 changed files with 3 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,7 @@ private ExecutorService buildThreadPool(
.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<>(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
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;
Expand All @@ -54,6 +55,7 @@ protected Configuration createConfiguration() {
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;
}

Expand Down

0 comments on commit b6b2590

Please sign in to comment.