forked from apache/paimon
-
Notifications
You must be signed in to change notification settings - Fork 3
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[flink] Cdc ingestion support warkmark strategy (apache#2640)
- Loading branch information
1 parent
f3b3735
commit 7992ee9
Showing
13 changed files
with
518 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
121 changes: 121 additions & 0 deletions
121
.../main/java/org/apache/paimon/flink/action/cdc/watermark/CdcTimestampExtractorFactory.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,121 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.paimon.flink.action.cdc.watermark; | ||
|
||
import org.apache.paimon.utils.JsonSerdeUtil; | ||
|
||
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; | ||
|
||
import com.ververica.cdc.connectors.mongodb.source.MongoDBSource; | ||
import com.ververica.cdc.connectors.mysql.source.MySqlSource; | ||
import org.apache.flink.connector.kafka.source.KafkaSource; | ||
import org.apache.flink.connector.pulsar.source.PulsarSource; | ||
|
||
import java.io.Serializable; | ||
import java.time.LocalDateTime; | ||
import java.time.ZoneId; | ||
import java.time.format.DateTimeFormatter; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.function.Supplier; | ||
|
||
/** Factory for creating CDC timestamp extractors based on different source types. */ | ||
public class CdcTimestampExtractorFactory implements Serializable { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private static final Map<Class<?>, Supplier<CdcTimestampExtractor>> extractorMap = | ||
new HashMap<>(); | ||
|
||
static { | ||
extractorMap.put(MongoDBSource.class, MongoDBCdcTimestampExtractor::new); | ||
extractorMap.put(MySqlSource.class, MysqlCdcTimestampExtractor::new); | ||
extractorMap.put(PulsarSource.class, MessageQueueCdcTimestampExtractor::new); | ||
extractorMap.put(KafkaSource.class, MessageQueueCdcTimestampExtractor::new); | ||
} | ||
|
||
public static CdcTimestampExtractor createExtractor(Object source) { | ||
Supplier<CdcTimestampExtractor> extractorSupplier = extractorMap.get(source.getClass()); | ||
if (extractorSupplier != null) { | ||
return extractorSupplier.get(); | ||
} | ||
throw new IllegalArgumentException( | ||
"Unsupported source type: " + source.getClass().getName()); | ||
} | ||
|
||
/** Timestamp extractor for MongoDB sources in CDC applications. */ | ||
public static class MongoDBCdcTimestampExtractor implements CdcTimestampExtractor { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
@Override | ||
public long extractTimestamp(String record) throws JsonProcessingException { | ||
return JsonSerdeUtil.extractValue(record, Long.class, "payload", "ts_ms"); | ||
} | ||
} | ||
|
||
/** Timestamp extractor for Kafka/Pulsar sources in CDC applications. */ | ||
public static class MessageQueueCdcTimestampExtractor implements CdcTimestampExtractor { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
@Override | ||
public long extractTimestamp(String record) throws JsonProcessingException { | ||
if (JsonSerdeUtil.isNodeExists(record, "mysqlType")) { | ||
// Canal json | ||
return JsonSerdeUtil.extractValue(record, Long.class, "ts"); | ||
} else if (JsonSerdeUtil.isNodeExists(record, "pos")) { | ||
// Ogg json | ||
String dateTimeString = JsonSerdeUtil.extractValue(record, String.class, "op_ts"); | ||
DateTimeFormatter formatter = | ||
DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS"); | ||
LocalDateTime dateTime = LocalDateTime.parse(dateTimeString, formatter); | ||
return dateTime.atZone(ZoneId.systemDefault()).toInstant().toEpochMilli(); | ||
} else if (JsonSerdeUtil.isNodeExists(record, "xid")) { | ||
// Maxwell json | ||
return JsonSerdeUtil.extractValue(record, Long.class, "ts") * 1000; | ||
} else if (JsonSerdeUtil.isNodeExists(record, "payload", "source", "connector")) { | ||
// Dbz json | ||
return JsonSerdeUtil.extractValue(record, Long.class, "payload", "ts_ms"); | ||
} else if (JsonSerdeUtil.isNodeExists(record, "source", "connector")) { | ||
// Dbz json | ||
return JsonSerdeUtil.extractValue(record, Long.class, "ts_ms"); | ||
} | ||
throw new RuntimeException( | ||
String.format( | ||
"Failed to extract timestamp: The JSON format of the message queue is unsupported. Record details: %s", | ||
record)); | ||
} | ||
} | ||
|
||
/** Timestamp extractor for MySQL sources in CDC applications. */ | ||
public static class MysqlCdcTimestampExtractor implements CdcTimestampExtractor { | ||
|
||
@Override | ||
public long extractTimestamp(String record) throws JsonProcessingException { | ||
return JsonSerdeUtil.extractValue(record, Long.class, "source", "ts_ms"); | ||
} | ||
} | ||
|
||
/** Interface defining the contract for CDC timestamp extraction. */ | ||
public interface CdcTimestampExtractor extends Serializable { | ||
|
||
long extractTimestamp(String record) throws JsonProcessingException; | ||
} | ||
} |
70 changes: 70 additions & 0 deletions
70
...-cdc/src/main/java/org/apache/paimon/flink/action/cdc/watermark/CdcWatermarkStrategy.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.paimon.flink.action.cdc.watermark; | ||
|
||
import org.apache.paimon.flink.action.cdc.watermark.CdcTimestampExtractorFactory.CdcTimestampExtractor; | ||
|
||
import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; | ||
|
||
import org.apache.flink.api.common.eventtime.Watermark; | ||
import org.apache.flink.api.common.eventtime.WatermarkGenerator; | ||
import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; | ||
import org.apache.flink.api.common.eventtime.WatermarkOutput; | ||
import org.apache.flink.api.common.eventtime.WatermarkStrategy; | ||
|
||
/** | ||
* Watermark strategy for CDC sources, generating watermarks based on timestamps extracted from | ||
* records. | ||
*/ | ||
public class CdcWatermarkStrategy implements WatermarkStrategy<String> { | ||
|
||
private final CdcTimestampExtractor timestampExtractor; | ||
private static final long serialVersionUID = 1L; | ||
private long currentMaxTimestamp; | ||
|
||
public CdcWatermarkStrategy(CdcTimestampExtractor extractor) { | ||
this.timestampExtractor = extractor; | ||
} | ||
|
||
@Override | ||
public WatermarkGenerator<String> createWatermarkGenerator( | ||
WatermarkGeneratorSupplier.Context context) { | ||
return new WatermarkGenerator<String>() { | ||
|
||
@Override | ||
public void onEvent(String record, long timestamp, WatermarkOutput output) { | ||
long tMs; | ||
try { | ||
tMs = timestampExtractor.extractTimestamp(record); | ||
} catch (JsonProcessingException e) { | ||
throw new RuntimeException(e); | ||
} | ||
currentMaxTimestamp = Math.max(currentMaxTimestamp, tMs); | ||
output.emitWatermark(new Watermark(currentMaxTimestamp - 1)); | ||
} | ||
|
||
@Override | ||
public void onPeriodicEmit(WatermarkOutput output) { | ||
long timeMillis = System.currentTimeMillis(); | ||
currentMaxTimestamp = Math.max(timeMillis, currentMaxTimestamp); | ||
output.emitWatermark(new Watermark(currentMaxTimestamp - 1)); | ||
} | ||
}; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.