Skip to content

Conversation

@yittg
Copy link
Contributor

@yittg yittg commented Feb 21, 2022

Fixes #3776

@yittg yittg changed the title Use particuar worker pool for flink jobs Use particular worker pool for flink jobs Feb 21, 2022
@yittg yittg force-pushed the flink-class-loader-closed branch from 93762c7 to 82ea9c1 Compare February 22, 2022 01:53
@yittg yittg marked this pull request as ready for review February 22, 2022 03:43
@yittg
Copy link
Contributor Author

yittg commented Feb 22, 2022

@rdblue @openinx Please help review this change.

}

@Override
public void open(Configuration parameters) throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

Should the pool size be configured by parameters?

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, is there a way to share pools if there are multiple Iceberg operators in the same Flink job?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Should the pool size be configured by parameters?

Configured from scan context;

Also, is there a way to share pools if there are multiple Iceberg operators in the same Flink job?

I think it's hard to share, and it will easily get meaningless across distributed nodes.
What do you think, @rdblue?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@rdblue do you think that sharing a pool in one job is a blocking issue? If that, we can provide a pool keyed by job. it's somehow reasonable to replace the original pool equivalently :)

Copy link
Contributor

Choose a reason for hiding this comment

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

Hey, sorry about this. I think my comment here is probably what caused the confusion about sharing pools by job ID. I think there are use cases around this (Steven has one at least) but let's focus on fixing the problem here and sharing resources later.

Thanks for your patience, @yittg!

@yittg yittg force-pushed the flink-class-loader-closed branch from 8279039 to 6b7666a Compare February 23, 2022 05:44
@yittg yittg force-pushed the flink-class-loader-closed branch 2 times, most recently from 1289e7c to 6af99ed Compare February 23, 2022 07:39
@yittg yittg requested a review from rdblue February 24, 2022 01:57
@yittg yittg force-pushed the flink-class-loader-closed branch 2 times, most recently from 45f3de2 to 8f3733a Compare March 1, 2022 04:17
return WORKER_POOL;
}

public static ExecutorService newWorkerPool(String namePrefix, Integer parallelism) {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: is poolSize more intuitive than parallelism?

public static ExecutorService newWorkerPool(String namePrefix, Integer parallelism) {
return MoreExecutors.getExitingExecutorService(
(ThreadPoolExecutor) Executors.newFixedThreadPool(
Optional.ofNullable(parallelism).orElse(WORKER_THREAD_POOL_SIZE),
Copy link
Contributor

Choose a reason for hiding this comment

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

should we make the param a primitive type and provide an overload method without the poolSize param?

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm okay either way.

super.open(parameters);

final String jobId = getRuntimeContext().getJobId().toString();
this.workerPool = ThreadPools.newKeyedWorkerPool(jobId, "flink-worker-pool", scanContext.planParallelism());
Copy link
Contributor

Choose a reason for hiding this comment

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

I saw this shares the same key as IcebergFilesCommitter, but not FlinkInputFormat. Trying to understand the reasons.

Copy link
Contributor

Choose a reason for hiding this comment

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

I agree here. Since this is creating a different thread pool per job ID, the thread name prefix should also include the job ID to get unique names.

final String jobId = getRuntimeContext().getJobId().toString();
this.workerPool = ThreadPools.newKeyedWorkerPool(jobId, "flink-worker-pool", scanContext.planParallelism());
getRuntimeContext().registerUserCodeClassLoaderReleaseHookIfAbsent(
"release-flink-worker-pool", () -> ThreadPools.shutdownKeyedWorkerPool(jobId));
Copy link
Contributor

Choose a reason for hiding this comment

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

Is the key here also going to be a problem? Or is this a description?

.build()));
}

public static ExecutorService newKeyedWorkerPool(String key, String namePrefix, Integer parallelism) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think that we need to keep worker pools here in a static map.

The two places where this is called immediately set up a callback that calls shutdown, but could easily keep a reference to the worker pool locally instead of storing it here by name.

I think it would be better to avoid keeping track of pools here.

Copy link
Contributor

Choose a reason for hiding this comment

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

if the intention is to reuse the job specific thread pool in Flink, then we do need the static cache as the same keyed pool may be requested from multiple code path.

Is this a Flink only problem regarding classloader issue on thread pool? if so, maybe we can move the keyed cache into Flink module.

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh, so the job can share between the monitor and the sink? I don't really mind having two pools for that.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@rdblue ,sorry, i don't get your point exactly. let me guess, what you really mean is sharing pools for all sources or all sinks, not for all sources and sinks?
To be clear, for example, if a job consists of:
Source: Iceberg A(parallelism: 3), Source:Iceberg B, Sink:Iceberg C, Sink: Iceberg D.
What's your favor?

  1. share btw all parallelism of one operator, like 3 subtask for Iceberg A (it can be run in different slots in one TaskManager or different TaskManagers) ;
  2. share btw all sources or all sinks, like sharing one for btw A and B, and another one for C and D;
  3. share btw all operators, like sharing btw A, B, C, and D; all subtasks in same TaskManager can share.

Copy link
Contributor

Choose a reason for hiding this comment

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

What I was thinking was a pool per operator in a job, rather than a pool per job. That avoids the need to track thread pools by some key in static state. I think it is probably fine to have more pools since these are primarily for IO. Does that sound reasonable?

Copy link
Contributor

@stevenzwu stevenzwu Mar 11, 2022

Choose a reason for hiding this comment

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

after reviewing the usage of the thread pools, I am also in favor of no sharing of thread pools so that we can avoid the static cache. None of the usage is on parallel tasks.

  • source: split planning (running on jobmanager or the single-parallelism StreamingMonitorFunction)
  • sink: single-parallelism committer

But we do need to add some user doc to clarify the behavior change regarding I/O thread pool. Previously, there is a global shared thread pool per JVM. Now it is per source/sink. E.g., Internally we had a rather unique setup where a single Flink job (running on many taskmanagers) can ingest data to dozens or hundreds of Iceberg tables. For those setups, users would need to tune down the pool size to probably 1 to avoid excessive number of threads created in JVM.

Copy link
Contributor

Choose a reason for hiding this comment

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

@stevenzwu, for that use case, maybe we should follow up to this PR with one that allows you to configure a named threadpool? I think that's probably the use case that @yittg had in mind when he set up sharing.

Copy link
Contributor

Choose a reason for hiding this comment

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

In addition to documentation change, we should also make sure this behavior change is captured in the release note of the next minor version release of 0.14.0. @rdblue where do we track future release note?

Copy link
Contributor

Choose a reason for hiding this comment

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

@stevenzwu, I added the "release notes" tag to this PR and added it to the 0.14.0 release milestone so we add this to release notes. If you want, you can add a comment with the suggested release notes at the end.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, although I didn't think about sharing at the beginning. but some kind of sharing or global limit sounds good to me after some consideration. We can provide a reasonable solution next I think.
Thanks, @rdblue and @stevenzwu .

Copy link
Contributor

@rdblue rdblue left a comment

Choose a reason for hiding this comment

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

Thanks, @yittg! I really appreciate how patient you've been with me getting back to this review.

There are two main things to fix now. First, I don't think we need to keep track of open pools in ThreadPools. Second, I agree with @stevenzwu's comment about passing the same name prefix for all of the pools created by the monitor and the sink. We should make sure the prefix is also unique by job ID.

Thanks!

@yittg yittg force-pushed the flink-class-loader-closed branch from 8f3733a to 62b04d9 Compare March 11, 2022 08:33
Comment on lines +81 to +86
final ExecutorService workerPool = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism());
try (TableLoader loader = tableLoader) {
Table table = loader.loadTable();
return FlinkSplitPlanner.planInputSplits(table, context);
return FlinkSplitPlanner.planInputSplits(table, context, workerPool);
} finally {
workerPool.shutdown();
Copy link
Contributor Author

@yittg yittg Mar 11, 2022

Choose a reason for hiding this comment

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

This function is called in client and job manager. So there is no context here. Given it's a adhoc pool and will be shut down after planning, i think it's ok to name it in this way.

@yittg yittg requested a review from rdblue March 11, 2022 08:39
Copy link
Contributor Author

@yittg yittg left a comment

Choose a reason for hiding this comment

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

Given the StreamingMonitorFunction and IcebergFilesCommitter are both 1-parallelism. We can new worker pool in subtask open along with guaranteeing one worker pool per operator.

@yittg yittg force-pushed the flink-class-loader-closed branch from 62b04d9 to efcd332 Compare March 11, 2022 09:18
@rdblue rdblue merged commit 1b774c2 into apache:master Mar 11, 2022
@rdblue rdblue added this to the Iceberg 0.14.0 Release milestone Mar 11, 2022
@rdblue rdblue added the release notes PR should be included in the release notes label Mar 11, 2022
@yittg yittg deleted the flink-class-loader-closed branch March 12, 2022 03:11
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

API core flink release notes PR should be included in the release notes

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Flink connector thrown Trying to access closed classloader

3 participants