[
https://issues.apache.org/jira/browse/FLINK-33693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Rui Fan updated FLINK-33693:
----------------------------
Description:
FLINK-21945 will convert the unaligned checkpoint barrier and timeoutable
aligned checkpoint barrier to force aligned barrier. However,
Bug at org.apache.flink.streaming.runtime.io.RecordWriterOutput#broadcastEvent,
we will call withUnalignedUnsupported to transmit the unaligned checkpoint
barrier to {color:#9876aa}FORCED_ALIGNED {color}barrier when the shuffle type
cannot use unaligned checkpoint.
However, the if has one condition: isPriorityEvent. When aligned checkpoint
timeout is enabled, flink will emit one timeoutable barrier. It isn't
PriorityEvent.
But timeoutable barrier should call withUnalignedUnsupported here.
!image-2023-11-29-20-41-17-038.png!
was:
The disable unaligned checkpoint isn't respected when the aligned checkpoint
timeout is enabled.
Bug at org.apache.flink.streaming.runtime.io.RecordWriterOutput#broadcastEvent,
we will call withUnalignedUnsupported to transmit the unaligned checkpoint
barrier to {color:#9876aa}FORCED_ALIGNED {color}barrier when the shuffle type
cannot use unaligned checkpoint.
However, the if has one condition: isPriorityEvent. When aligned checkpoint
timeout is enabled, flink will emit one timeoutable barrier. It isn't
PriorityEvent.
But timeoutable barrier should call withUnalignedUnsupported here.
!image-2023-11-29-20-41-17-038.png!
> Force aligned barrier logic doesn't work when the aligned checkpoint timeout
> is enabled
> ----------------------------------------------------------------------------------------
>
> Key: FLINK-33693
> URL: https://issues.apache.org/jira/browse/FLINK-33693
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing
> Affects Versions: 1.18.0, 1.17.2
> Reporter: Rui Fan
> Assignee: Rui Fan
> Priority: Major
> Attachments: image-2023-11-29-20-41-17-038.png
>
>
> FLINK-21945 will convert the unaligned checkpoint barrier and timeoutable
> aligned checkpoint barrier to force aligned barrier. However,
>
> Bug at
> org.apache.flink.streaming.runtime.io.RecordWriterOutput#broadcastEvent, we
> will call withUnalignedUnsupported to transmit the unaligned checkpoint
> barrier to {color:#9876aa}FORCED_ALIGNED {color}barrier when the shuffle type
> cannot use unaligned checkpoint.
> However, the if has one condition: isPriorityEvent. When aligned checkpoint
> timeout is enabled, flink will emit one timeoutable barrier. It isn't
> PriorityEvent.
>
> But timeoutable barrier should call withUnalignedUnsupported here.
> !image-2023-11-29-20-41-17-038.png!
--
This message was sent by Atlassian Jira
(v8.20.10#820010)