1996fanrui opened a new pull request, #26619:
URL: https://github.com/apache/flink/pull/26619

   ## What is the purpose of the change
   
   [FLINK-37870][checkpoint] Fix the bug that unaligned checkpoint is disabled 
for all connections unexpectedly
   
   ### Bug reason:
   
   `Set<Integer> seen = new HashSet<>(writer.getId());` creates a HashSet to 
record all seen transformations. It expects to create a HashSet and put the 
first writer transformation into the set to end the recursive loop (to prevent 
traversing the previous transformation of the writer).
   
   But the constructor parameter of new HashSet is initialCapacity, so it only 
created a HashSet, and didn't put the first writer transformation into the set.
   
   ## Brief change log
   
   ```
   -   Set<Integer> seen = new HashSet<>(writer.getId());
   +   Set<Integer> seen = new HashSet<>(sinkTransformations.size() * 2);
   +   seen.add(writer.getId());
   ```
   
   ## Verifying this change
   
   Improve `SinkV2ITCase` and `SinkV2TransformationTranslatorITCase`.
   
   -  Not only check all the out edges between sink nodes should not support 
unaligned checkpoints, but also check all the out edges of nodes upstream of 
the sink should support unaligned checkpoints
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to