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

Flink: Fix IcebergSource tableloader lifecycle management in batch mode #9173

Merged
merged 1 commit into from
Dec 9, 2023

Conversation

mas-chen
Copy link
Contributor

This is to fix a connection pool issue that prevents IcebergSource to startup in batch mode. The root cause is that the FileIO is closed when the tableLoader is closed via the try-with-resources statements. We need multiple copies of the user tableLoader parameter to retrieve the table name and do split planning and to close the loader after each use. The fix is to utilize clone the table loader to that it can be opened/closed by the batch split planning mechanism. This patch has been manually verified.

@github-actions github-actions bot added the flink label Nov 28, 2023
@mas-chen
Copy link
Contributor Author

cc: @stevenzwu

@stevenzwu stevenzwu self-requested a review November 29, 2023 05:47
ExecutorService workerPool =
ThreadPools.newWorkerPool(threadName, scanContext.planParallelism());
try {
plannerTableLoader.open();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I forgot to rename this variable earlier

Comment on lines 210 to 212
// Create a copy of the table loader to avoid lifecycle management conflicts with the user
// provided table loader. This copy is only required for split planning, which uses the
// underlying io, and should be closed after split planning is complete
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not clone/open/close it here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Just to keep it the same as ContinuousSplitPlannerImpl and tableName(), which perform the open/close themselves. and for the ContinuousSplitPlannerImpl we can't close it until the source is closed.

Would it be more readable if the logic that requires a clone--abstract the clone, open, and then close? I think that's also fine but the original table loader needs to be closed by the source at the end of initialization

Copy link
Contributor

Choose a reason for hiding this comment

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

I usually prefer if the objects own the whole lifecycle of the child objects.

So ideally:

  • new ContinuousSplitPlannerImpl should clone the loader itself, and keep it as long as it needs, and closes it at the end of the reading
  • planSplitsForBatch should clone the loader itself, and keep it as long as it needs, and closes it at the end of the reading
  • For tableName, I am a bit confused. In planSplitsForBatch we need to clone the loader to do the planning, but use the old loader to get the tableName for logging? Why not set the tableName value in the constructor, and forget the whole lazyTable thing?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I agree with the pattern.

wrt to the lazytable thing, this is because the Table needed to be a transient value. But it seems we do not make a reference to it outside of the lazytable, so I will remove it.

assigner = assignerFactory.createAssigner(enumState.pendingSplits());
}

TableLoader tableLoaderCopy = tableLoader.clone();
Copy link
Contributor

Choose a reason for hiding this comment

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

Not needed

this.emitter = emitter;
this.tableName = table.name();
Copy link
Contributor

Choose a reason for hiding this comment

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

this is incorrect. table can be null and loaded lazily.

Copy link
Contributor Author

@mas-chen mas-chen Dec 5, 2023

Choose a reason for hiding this comment

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

The builder method already handles the nullity case. I added a preconditions to verify that table is non-null before invoking the source constructor.

We can move checkRequired into the constructor too (I think that makes the code more readable)

Copy link
Contributor

Choose a reason for hiding this comment

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

Previously, users don't have to provide a valid table. this is a breaking change that requires users to supply a valid table object

Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't this part of the code takes care of creating the table?

public IcebergSource<T> build() {
if (table == null) {
try (TableLoader loader = tableLoader) {
loader.open();
this.table = tableLoader.loadTable();
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for pointing out the code from the build() method. yeah, this is fine. I missed it earlier

Copy link
Contributor

Choose a reason for hiding this comment

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

Now I understand Mason's comment. totally agree that it is more readable to move the checkRequired() inside the constructor. now it is detached.

Copy link
Contributor

@stevenzwu stevenzwu left a comment

Choose a reason for hiding this comment

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

can't assume table is not null

Copy link
Contributor

@pvary pvary left a comment

Choose a reason for hiding this comment

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

LGTM, Thanks!

@pvary pvary merged commit 4d0b69b into apache:main Dec 9, 2023
13 checks passed
@pvary
Copy link
Contributor

pvary commented Dec 9, 2023

Merged to main.
Thanks for the PR @mas-chen and @stevenzwu for the review!

@pvary
Copy link
Contributor

pvary commented Dec 9, 2023

@mas-chen: Please do not forget to port the changes to the other Flink versions.
Thanks, Peter

mas-chen added a commit to mas-chen/iceberg that referenced this pull request Dec 18, 2023
mas-chen added a commit to mas-chen/iceberg that referenced this pull request Dec 18, 2023
mas-chen added a commit to mas-chen/iceberg that referenced this pull request Dec 18, 2023
mas-chen added a commit to mas-chen/iceberg that referenced this pull request Dec 18, 2023
pvary pushed a commit that referenced this pull request Dec 19, 2023
lisirrx pushed a commit to lisirrx/iceberg that referenced this pull request Jan 4, 2024
lisirrx pushed a commit to lisirrx/iceberg that referenced this pull request Jan 4, 2024
@javrasya
Copy link
Contributor

javrasya commented Jan 8, 2024

Why not supporting this for 1.15? Unfortunately that is still the latest version one can use on managed AWS Flink service.

@pvary
Copy link
Contributor

pvary commented Jan 9, 2024

@javrasya: The general policy is that we support the last 3 Flink release in the Flink connectors. We follow the same pattern with the Iceberg connector, and support Flink 1.18/1.17/1.16 in the upcoming Iceberg releases.

If you think that you would need this in older Iceberg releases, like 1.14.4 (if there will be any), then it should be backported to 1.4.x branch.

geruh pushed a commit to geruh/iceberg that referenced this pull request Jan 26, 2024
devangjhabakh pushed a commit to cdouglas/iceberg that referenced this pull request Apr 22, 2024
devangjhabakh pushed a commit to cdouglas/iceberg that referenced this pull request Apr 22, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants