You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/09/21 10:43:28 UTC

[inlong] branch release-1.3.0 updated: [INLONG-5950][Sort] Support metric state recovery for mongo-cdc (#5951)

This is an automated email from the ASF dual-hosted git repository.

dockerzhang pushed a commit to branch release-1.3.0
in repository https://gitbox.apache.org/repos/asf/inlong.git


The following commit(s) were added to refs/heads/release-1.3.0 by this push:
     new 58cfcd29c [INLONG-5950][Sort] Support metric state recovery for mongo-cdc (#5951)
58cfcd29c is described below

commit 58cfcd29ced2e2d2de0b5e83e877d7dfee99eefb
Author: Schnapps <zp...@connect.ust.hk>
AuthorDate: Tue Sep 20 17:48:37 2022 +0800

    [INLONG-5950][Sort] Support metric state recovery for mongo-cdc (#5951)
---
 .../sort/cdc/mongodb/DebeziumSourceFunction.java   | 38 ++++++++++++++++++----
 1 file changed, 32 insertions(+), 6 deletions(-)

diff --git a/inlong-sort/sort-connectors/mongodb-cdc/src/main/java/org/apache/inlong/sort/cdc/mongodb/DebeziumSourceFunction.java b/inlong-sort/sort-connectors/mongodb-cdc/src/main/java/org/apache/inlong/sort/cdc/mongodb/DebeziumSourceFunction.java
index ba899e980..6c80ca538 100644
--- a/inlong-sort/sort-connectors/mongodb-cdc/src/main/java/org/apache/inlong/sort/cdc/mongodb/DebeziumSourceFunction.java
+++ b/inlong-sort/sort-connectors/mongodb-cdc/src/main/java/org/apache/inlong/sort/cdc/mongodb/DebeziumSourceFunction.java
@@ -44,6 +44,7 @@ import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.OperatorStateStore;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.configuration.Configuration;
@@ -59,7 +60,9 @@ import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.inlong.sort.base.metric.MetricOption;
 import org.apache.inlong.sort.base.metric.MetricOption.RegisteredMetric;
+import org.apache.inlong.sort.base.metric.MetricState;
 import org.apache.inlong.sort.base.metric.SourceMetricData;
+import org.apache.inlong.sort.base.util.MetricStateUtils;
 import org.apache.kafka.connect.source.SourceRecord;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -79,6 +82,9 @@ import java.util.concurrent.TimeUnit;
 
 import static com.ververica.cdc.debezium.utils.DatabaseHistoryUtil.registerHistory;
 import static com.ververica.cdc.debezium.utils.DatabaseHistoryUtil.retrieveHistory;
+import static org.apache.inlong.sort.base.Constants.INLONG_METRIC_STATE_NAME;
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
 
 /**
  * The {@link DebeziumSourceFunction} is a streaming data source that pulls captured change data
@@ -225,7 +231,11 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
 
     private String inlongAudit;
 
-    private SourceMetricData metricData;
+    private SourceMetricData sourceMetricData;
+
+    private transient ListState<MetricState> metricStateListState;
+
+    private MetricState metricState;
 
     // ---------------------------------------------------------------------------------------
 
@@ -270,9 +280,19 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
                         new ListStateDescriptor<>(
                                 HISTORY_RECORDS_STATE_NAME, BasicTypeInfo.STRING_TYPE_INFO));
 
+        if (this.inlongMetric != null) {
+            this.metricStateListState =
+                stateStore.getUnionListState(
+                    new ListStateDescriptor<>(
+                        INLONG_METRIC_STATE_NAME, TypeInformation.of(new TypeHint<MetricState>() {
+                    })));
+        }
+
         if (context.isRestored()) {
             restoreOffsetState();
             restoreHistoryRecordsState();
+            metricState = MetricStateUtils.restoreMetricState(metricStateListState,
+                getRuntimeContext().getIndexOfThisSubtask(), getRuntimeContext().getNumberOfParallelSubtasks());
         } else {
             if (specificOffset != null) {
                 byte[] serializedOffset =
@@ -342,6 +362,10 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
         } else {
             snapshotOffsetState(functionSnapshotContext.getCheckpointId());
             snapshotHistoryRecordsState();
+            if (sourceMetricData != null && metricStateListState != null) {
+                MetricStateUtils.snapshotMetricStateForSourceMetricData(metricStateListState, sourceMetricData,
+                    getRuntimeContext().getIndexOfThisSubtask());
+            }
         }
     }
 
@@ -416,10 +440,12 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
         MetricOption metricOption = MetricOption.builder()
                 .withInlongLabels(inlongMetric)
                 .withInlongAudit(inlongAudit)
+                .withInitRecords(metricState != null ? metricState.getMetricValue(NUM_RECORDS_IN) : 0L)
+                .withInitBytes(metricState != null ? metricState.getMetricValue(NUM_BYTES_IN) : 0L)
                 .withRegisterMetric(RegisteredMetric.ALL)
                 .build();
         if (metricOption != null) {
-            metricData = new SourceMetricData(metricOption, metricGroup);
+            sourceMetricData = new SourceMetricData(metricOption, metricGroup);
         }
         properties.setProperty("name", "engine");
         properties.setProperty("offset.storage", FlinkOffsetBackingStore.class.getCanonicalName());
@@ -458,8 +484,8 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
                         new DebeziumDeserializationSchema<T>() {
                             @Override
                             public void deserialize(SourceRecord record, Collector<T> out) throws Exception {
-                                if (metricData != null) {
-                                    metricData.outputMetricsWithEstimate(record.value());
+                                if (sourceMetricData != null) {
+                                    sourceMetricData.outputMetricsWithEstimate(record.value());
                                 }
                                 deserializer.deserialize(record, out);
                             }
@@ -631,7 +657,7 @@ public class DebeziumSourceFunction<T> extends RichSourceFunction<T>
         return engineInstanceName;
     }
 
-    public SourceMetricData getMetricData() {
-        return metricData;
+    public SourceMetricData getSourceMetricData() {
+        return sourceMetricData;
     }
 }