Skip to content

[FLINK-37608][fs-connector] File sink supports to close buckets in parallel when snapshot #26498

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

Open
wants to merge 3 commits into
base: master
Choose a base branch
from

Conversation

lianghongjia6
Copy link

What is the purpose of the change

  • File sink supports to close buckets in parallel when snapshot

Brief change log

  • File sink supports to close buckets in parallel when snapshot

Verifying this change

This change is already covered by existing tests, such as BucketsTest#testSnapshotAndRestore.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (yes / no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yes / no)
  • The serializers: (yes / no / don't know)
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
  • The S3 file system connector: (yes / no / don't know)

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

@flinkbot
Copy link
Collaborator

flinkbot commented Apr 22, 2025

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@@ -85,6 +92,8 @@ public class Buckets<IN, BucketID> {

private final BucketStateSerializer<BucketID> bucketStateSerializer;

private final ExecutorService snapshotActiveBucketsThreadPool;
Copy link
Contributor

@davidradl davidradl Apr 22, 2025

Choose a reason for hiding this comment

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

some comments:

  • I am not convinced that this is covered by existing tests as there is a new thread pool that has been introduced.
  • Also I would like to see a test showing the expected behaviour when the CompletionException is thrown. I am unsure what we are looking to do when there is a CompletionException is thrown. Does everything get cleaned up when this exception is thrown?
  • the title talks of close buckets but the code is in snapshotActiveBuckets. It would be easier to understand if I could see an obvious connection between close buckets and snapshotActiveBuckets.
  • I see the Jira talks about introducing an option for this - but the code has not done this - it has changed it from synchronous to asynchronous. what is the thinking here?
  • the CI has failed - I am not sure if it relates to this change.

@lianghongjia6
Copy link
Author

@flinkbot run azure

@lianghongjia6
Copy link
Author

@flinkbot run azure

Copy link
Contributor

@masteryhx masteryhx left a comment

Choose a reason for hiding this comment

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

Thanks for the PR!
PTAL my comments.

}
return bucketState;
} catch (IOException e) {
throw new CompletionException(e);
Copy link
Contributor

Choose a reason for hiding this comment

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

Add some context for any exception like subtask, bucket?

import java.util.concurrent.CompletionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.stream.Collectors;

/**
* The manager of the different active buckets in the {@link StreamingFileSink}.
Copy link
Contributor

Choose a reason for hiding this comment

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

How about also supporting FileSink, the refactored one ?

final BucketState<BucketID> bucketState = bucket.onReceptionOfCheckpoint(checkpointId);

long start = System.currentTimeMillis();
List<CompletableFuture<BucketState<BucketID>>> futures =
Copy link
Contributor

Choose a reason for hiding this comment

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

How about adding a test about partial active buckets failure ?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants