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
Open
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 @@ -26,6 +26,7 @@
import org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink;
import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.concurrent.ExecutorThreadFactory;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -36,7 +37,13 @@
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
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 ?

Expand Down Expand Up @@ -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.


/**
* A constructor creating a new empty bucket manager.
*
Expand Down Expand Up @@ -121,6 +130,8 @@ public Buckets(
bucketWriter.getProperties().getPendingFileRecoverableSerializer(),
bucketAssigner.getSerializer());
this.maxPartCounter = 0L;
this.snapshotActiveBucketsThreadPool =
Executors.newCachedThreadPool(new ExecutorThreadFactory("snapshot-active-buckets"));
}

public void setBucketLifeCycleListener(
Expand Down Expand Up @@ -267,18 +278,49 @@ private void snapshotActiveBuckets(
final long checkpointId, final ListState<byte[]> bucketStatesContainer)
throws Exception {

for (Bucket<IN, BucketID> bucket : activeBuckets.values()) {
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 ?

activeBuckets.values().stream()
.map(
bucket ->
CompletableFuture.supplyAsync(
() -> {
try {
BucketState<BucketID> bucketState =
bucket.onReceptionOfCheckpoint(
checkpointId);
if (LOG.isDebugEnabled()) {
LOG.debug(
"Subtask {} checkpointing: {}",
subtaskIndex,
bucketState);
}
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?

}
},
snapshotActiveBucketsThreadPool))
.collect(Collectors.toList());

CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join();

for (CompletableFuture<BucketState<BucketID>> future : futures) {
BucketState<BucketID> bucketState = future.get();
final byte[] serializedBucketState =
SimpleVersionedSerialization.writeVersionAndSerialize(
bucketStateSerializer, bucketState);

bucketStatesContainer.add(serializedBucketState);
}

if (LOG.isDebugEnabled()) {
LOG.debug("Subtask {} checkpointing: {}", subtaskIndex, bucketState);
}
long duration = System.currentTimeMillis() - start;
if (LOG.isDebugEnabled()) {
LOG.debug(
"Subtask {} has completely snapshot the active buckets for the checkpoint with id={} , active buckets size: {}, cost: {}ms",
subtaskIndex,
checkpointId,
activeBuckets.size(),
duration);
}
}

Expand Down Expand Up @@ -351,6 +393,9 @@ public void close() {
if (activeBuckets != null) {
activeBuckets.values().forEach(Bucket::disposePartFile);
}
if (snapshotActiveBucketsThreadPool != null) {
snapshotActiveBucketsThreadPool.shutdown();
}
}

private Path assembleBucketPath(BucketID bucketId) {
Expand Down