1717
1818package org .apache .flink .cdc .connectors .hudi .sink .coordinator ;
1919
20- import org .apache .flink .cdc .common .event .CreateTableEvent ;
2120import org .apache .flink .cdc .common .event .TableId ;
2221import org .apache .flink .cdc .common .schema .Schema ;
2322import org .apache .flink .cdc .connectors .hudi .sink .event .EnhancedWriteMetadataEvent ;
24- import org .apache .flink .cdc .connectors .hudi .sink .event .SchemaChangeOperatorEvent ;
2523import org .apache .flink .cdc .connectors .hudi .sink .event .TableAwareCorrespondent .CreateTableRequest ;
26- import org .apache .flink .cdc .connectors .hudi .sink .event .TableAwareCorrespondent .CreateTableResponse ;
2724import org .apache .flink .cdc .connectors .hudi .sink .event .TableAwareCorrespondent .MultiTableInstantTimeRequest ;
25+ import org .apache .flink .cdc .connectors .hudi .sink .event .TableAwareCorrespondent .SchemaChangeRequest ;
26+ import org .apache .flink .cdc .connectors .hudi .sink .event .TableAwareCorrespondent .SchemaChangeResponse ;
2827import org .apache .flink .cdc .connectors .hudi .sink .util .RowDataUtils ;
2928import org .apache .flink .configuration .Configuration ;
3029import org .apache .flink .runtime .jobgraph .OperatorID ;
@@ -392,13 +391,30 @@ public CompletableFuture<CoordinationResponse> handleCoordinationRequest(
392391 CompletableFuture <CoordinationResponse > future = new CompletableFuture <>();
393392 executor .execute (
394393 () -> {
395- boolean isSuccess = handleCreateTableEvent (((CreateTableRequest ) request ));
394+ CreateTableRequest createTableRequest = (CreateTableRequest ) request ;
395+ boolean isSuccess = handleCreateTableRequest (createTableRequest );
396396 future .complete (
397- CoordinationResponseSerDe .wrap (new CreateTableResponse (isSuccess )));
397+ CoordinationResponseSerDe .wrap (
398+ SchemaChangeResponse .of (
399+ createTableRequest .getTableId (), isSuccess )));
398400 },
399401 "Handling create table request: " ,
400402 request );
401403 return future ;
404+ } else if (request instanceof SchemaChangeRequest ) {
405+ CompletableFuture <CoordinationResponse > future = new CompletableFuture <>();
406+ executor .execute (
407+ () -> {
408+ SchemaChangeRequest createTableRequest = (SchemaChangeRequest ) request ;
409+ boolean isSuccess = handleSchemaChangeRequest (createTableRequest );
410+ future .complete (
411+ CoordinationResponseSerDe .wrap (
412+ SchemaChangeResponse .of (
413+ createTableRequest .getTableId (), isSuccess )));
414+ },
415+ "Handling create schema change request: " ,
416+ request );
417+ return future ;
402418 } else {
403419 LOG .warn ("Received an unknown coordination request: {}" , request .getClass ().getName ());
404420 return super .handleCoordinationRequest (request );
@@ -423,9 +439,7 @@ public void handleEventFromOperator(
423439 int subtask , int attemptNumber , OperatorEvent operatorEvent ) {
424440 executor .execute (
425441 () -> {
426- if (operatorEvent instanceof SchemaChangeOperatorEvent ) {
427- handleSchemaChangeEvent ((SchemaChangeOperatorEvent ) operatorEvent );
428- } else if (operatorEvent instanceof EnhancedWriteMetadataEvent ) {
442+ if (operatorEvent instanceof EnhancedWriteMetadataEvent ) {
429443 handleEnhancedWriteMetadataEvent (
430444 (EnhancedWriteMetadataEvent ) operatorEvent );
431445 } else {
@@ -438,16 +452,14 @@ public void handleEventFromOperator(
438452 operatorEvent );
439453 }
440454
441- private boolean handleCreateTableEvent (CreateTableRequest createTableRequest ) {
442- CreateTableEvent event = createTableRequest .getCreateTableEvent ();
443- TableId tableId = event .tableId ();
444-
455+ private boolean handleCreateTableRequest (CreateTableRequest createTableRequest ) {
456+ TableId tableId = createTableRequest .getTableId ();
445457 // Store the schema for this table
446- tableSchemas .put (tableId , event .getSchema ());
458+ tableSchemas .put (tableId , createTableRequest .getSchema ());
447459 LOG .info (
448460 "Cached schema for table {}: {} columns" ,
449461 tableId ,
450- event .getSchema ().getColumnCount ());
462+ createTableRequest .getSchema ().getColumnCount ());
451463
452464 TableContext tableContext =
453465 tableContexts .computeIfAbsent (
@@ -493,28 +505,36 @@ private boolean handleCreateTableEvent(CreateTableRequest createTableRequest) {
493505 * Handles schema change events from the sink functions. Updates the cached schema and recreates
494506 * the write client to ensure it uses the new schema.
495507 *
496- * @param event The schema change event containing the table ID and new schema
508+ * @param request The schema change request containing the table ID and new schema
497509 */
498- private void handleSchemaChangeEvent ( SchemaChangeOperatorEvent event ) {
499- TableId tableId = event .getTableId ();
500- Schema newSchema = event . getNewSchema ();
510+ private boolean handleSchemaChangeRequest ( SchemaChangeRequest request ) {
511+ TableId tableId = request .getTableId ();
512+ Schema newSchema = request . getSchema ();
501513
502514 LOG .info (
503515 "Received schema change event for table {}: {} columns" ,
504516 tableId ,
505517 newSchema .getColumnCount ());
506518
519+ Schema oldSchema = tableSchemas .get (tableId );
520+ if (Objects .equals (oldSchema , newSchema )) {
521+ LOG .warn ("Schema change already applied, tableId: {}, schema: {}." , tableId , newSchema );
522+ return true ;
523+ }
507524 // Update the cached schema
508525 tableSchemas .put (tableId , newSchema );
509- LOG .info ("Updated coordinator's schema cache for table: {}" , tableId );
526+ LOG .info (
527+ "Updated coordinator's schema cache for table: {}, new schema: {}" ,
528+ tableId ,
529+ newSchema );
510530
511531 // Get the existing table context
512532 TableContext oldContext = tableContexts .get (tableId );
513533 if (oldContext == null ) {
514534 LOG .warn (
515535 "Received schema change for unknown table: {}. Skipping write client update." ,
516536 tableId );
517- return ;
537+ return true ;
518538 }
519539
520540 try {
@@ -543,6 +563,7 @@ private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) {
543563 tableContexts .put (tableId , newContext );
544564
545565 LOG .info ("Successfully updated write client for table {} after schema change" , tableId );
566+ return true ;
546567 } catch (Exception e ) {
547568 LOG .error ("Failed to update write client for table {} after schema change" , tableId , e );
548569 context .failJob (
@@ -551,6 +572,7 @@ private void handleSchemaChangeEvent(SchemaChangeOperatorEvent event) {
551572 + tableId
552573 + " after schema change" ,
553574 e ));
575+ return false ;
554576 }
555577 }
556578
@@ -736,7 +758,21 @@ private void commitInstantForTable(
736758 long checkpointId ,
737759 String instant ,
738760 WriteMetadataEvent [] eventBuffer ) {
739-
761+ final HoodieTimeline completedTimeline =
762+ tableContext
763+ .writeClient
764+ .getHoodieTable ()
765+ .getMetaClient ()
766+ .getActiveTimeline ()
767+ .filterCompletedInstants ();
768+ if (completedTimeline .containsInstant (instant )) {
769+ LOG .info (
770+ "Instant {} already committed, table {}, checkpoint id: {}." ,
771+ instant ,
772+ tableId ,
773+ checkpointId );
774+ return ;
775+ }
740776 if (Arrays .stream (eventBuffer ).allMatch (Objects ::isNull )) {
741777 LOG .info ("No events for instant {}, table {}. Resetting buffer." , instant , tableId );
742778 tableContext .eventBuffers .reset (checkpointId );
0 commit comments