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-18757: Bump corePoolSize of HadoopThreadPoolExecutor in s3a committer #5706

Merged
merged 5 commits into from
Jul 19, 2023
Merged
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
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,
Copy link
Member

Choose a reason for hiding this comment

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

Hi, @steveloughran . In this case, it seems that there is no workaround in Apache Hadoop 3.3.5 and 3.3.6. Do you have any recommendation for this issue? Currently, Apache Spark 3.5.0 RC1 tag is using Apache Hadoop 3.3.6.

[SPARK-44197][BUILD] Upgrade Hadoop to 3.3.6
[SPARK-42913][BUILD] Upgrade Hadoop to 3.3.5

Copy link
Contributor

Choose a reason for hiding this comment

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

you are going to have to go with it for now; i think we should be thinking about a 3.3.7 before long as we have some other abfs and s3a issues causing pain...

Copy link
Member

Choose a reason for hiding this comment

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

Thank you, @steveloughran .

Copy link
Contributor

Choose a reason for hiding this comment

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

  • a cve rollup on protobuf, guava etc... not jackson, sadly

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