leletan commented on code in PR #20852:
URL: https://github.com/apache/flink/pull/20852#discussion_r988631290


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNG.java:
##########
@@ -125,7 +127,8 @@ void notifyKvStateUnregistered(
     CompletableFuture<String> triggerSavepoint(
             @Nullable String targetDirectory, boolean cancelJob, 
SavepointFormatType formatType);
 
-    CompletableFuture<String> triggerCheckpoint();
+    CompletableFuture<CompletedCheckpoint> triggerCheckpoint(
+            @Nullable CheckpointType checkpointType);

Review Comment:
   Good question. Correct, this is nullable here but not nullable in the 
request body. The reason is a little bit complicated to explain:
   At essence, the `nonNull` is expected for normal restful API call all the 
way to CheckpointCoordinator, and the `nullable` is only to take care of 
[this](https://github.com/apache/flink/pull/17278) legacy case.
   
   Here are some more context:
   
   The call sequence of the above mentioned legacy case 
`CompletableFuture<String> triggerCheckpoint(@RpcTimeout final Time timeout)` 
is 
   **_Minicluster -> DispatcherGateway -> Dispatcher -> JobMasterGateway -> 
..._** 
   and it expect a return type of `CompletableFuture<String>` (String for 
checkpointLocation). This one does not have any CheckpointType in the function 
parameter.
   
   The call sequence of the new API is 
   **_CheckpointTriggerHandler -> RestfulGateway -> Dispatcher -> 
JobMasterGateway -> ..._** 
   and it expect a return type of `CompletableFuture<Long>` (Long for 
checkpointId). In this call sequence we expect `checkpointType` to be `nonNull`
   
   To avoid keeping 2 very similar functions down the same call sequence 
(**_...-> JobMasterGateway -> JobMaster -> SchedulerNG -> Scheduler -> ..._**) 
and duplicated a lot of code, we combine them through
   ```
       default CompletableFuture<String> triggerCheckpoint(@RpcTimeout final 
Time timeout) {
           return triggerCheckpoint(null, 
timeout).thenApply(CompletedCheckpoint::getExternalPointer);
       }
   ``` 
   in JobMasterGateway.
   
   We may also combine them in `Dispatcher`  - the first class where 2 call 
sequence meet, however the legacy call sequence already had a non HA safe 
implementation (more context 
[here](https://issues.apache.org/jira/browse/FLINK-18312)) which is good enough 
for Minicluster but not good enough for RestfulAPI. 
   
   I know this is NOT a perfect solution, but it is the best I can think of 
after considering trade-offs among all the solutions I can find. Please let me 
know if you have a better way.
   



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