github-actions[bot] commented on code in PR #63427:
URL: https://github.com/apache/doris/pull/63427#discussion_r3273092766


##########
fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java:
##########
@@ -1437,6 +1437,12 @@ public void commitOffset(CommitOffsetRequest 
offsetRequest) throws JobException
         try {
             if (this.runningStreamTask != null
                     && this.runningStreamTask instanceof 
StreamingMultiTblTask) {
+                if (this.runningStreamTask.getIsCanceled().get()) {

Review Comment:
   This guard skips the entire offset commit for any canceled 
`StreamingMultiTblTask`, including tasks canceled by a manual pause/stop or by 
timeout after the BE request has already completed its stream loads but before 
the callback reaches FE. In that timeline the target rows are already visible, 
but returning here leaves `offsetProvider` and `offsetProviderPersist` 
unchanged; after resume the next task starts from the old source offset and can 
load the same changes again. Before this change, `commitOffset` still 
advanced/persisted the offset and only `successCallback()` was suppressed by 
`isCanceled`, which avoided flipping the task back to success while keeping 
data and offsets aligned. Please narrow this to the truly stale/failed callback 
case or split the handling so late callbacks cannot mark the task successful 
but still persist the offset when the BE has already loaded data.



-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to