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 7934156e6 [INLONG-5930][Sort] Support metric state recovery for
sqlserver-cdc (#5934)
7934156e6 is described below
commit 7934156e63b46faaa157bbfed74b374c796b2800
Author: Schnapps <[email protected]>
AuthorDate: Tue Sep 20 15:38:02 2022 +0800
[INLONG-5930][Sort] Support metric state recovery for sqlserver-cdc (#5934)
---
.../inlong/sort/base/util/MetricStateUtils.java | 12 ++++---
.../sort/cdc/mongodb/DebeziumSourceFunction.java | 2 +-
.../sort/cdc/debezium/DebeziumSourceFunction.java | 2 +-
.../sort/cdc/oracle/DebeziumSourceFunction.java | 2 +-
.../DebeziumSourceFunction.java | 2 +-
.../sqlserver/table/DebeziumSourceFunction.java | 38 ++++++++++++++++++----
6 files changed, 43 insertions(+), 15 deletions(-)
diff --git
a/inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java
b/inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java
index 416c8b719..4072adfae 100644
---
a/inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java
+++
b/inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java
@@ -18,7 +18,6 @@
package org.apache.inlong.sort.base.util;
-import lombok.extern.slf4j.Slf4j;
import org.apache.flink.api.common.state.ListState;
import org.apache.inlong.sort.base.metric.MetricState;
import org.apache.inlong.sort.base.metric.SinkMetricData;
@@ -28,6 +27,8 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
import static org.apache.inlong.sort.base.Constants.NUM_BYTES_OUT;
@@ -37,9 +38,10 @@ import static
org.apache.inlong.sort.base.Constants.NUM_RECORDS_OUT;
/**
* metric state for {@link MetricState} supporting snapshot and restore
*/
-@Slf4j
public class MetricStateUtils {
+ public static final Logger LOGGER =
LoggerFactory.getLogger(MetricStateUtils.class);
+
/**
*
* restore metric state data
@@ -54,7 +56,7 @@ public class MetricStateUtils {
if (metricStateListState == null || metricStateListState.get() ==
null) {
return null;
}
- log.info("restoreMetricState:{}, subtaskIndex:{},
currentSubtaskNum:{}", metricStateListState, subtaskIndex,
+ LOGGER.info("restoreMetricState:{}, subtaskIndex:{},
currentSubtaskNum:{}", metricStateListState, subtaskIndex,
currentSubtaskNum);
MetricState currentMetricState;
Map<Integer, MetricState> map = new HashMap<>(16);
@@ -118,7 +120,7 @@ public class MetricStateUtils {
public static void
snapshotMetricStateForSourceMetricData(ListState<MetricState>
metricStateListState,
SourceMetricData sourceMetricData, Integer subtaskIndex)
throws Exception {
- log.info("snapshotMetricStateForSourceMetricData:{},
sourceMetricData:{}, subtaskIndex:{}",
+ LOGGER.info("snapshotMetricStateForSourceMetricData:{},
sourceMetricData:{}, subtaskIndex:{}",
metricStateListState, sourceMetricData, subtaskIndex);
metricStateListState.clear();
Map<String, Long> metricDataMap = new HashMap<>();
@@ -139,7 +141,7 @@ public class MetricStateUtils {
public static void
snapshotMetricStateForSinkMetricData(ListState<MetricState>
metricStateListState,
SinkMetricData sinkMetricData, Integer subtaskIndex)
throws Exception {
- log.info("snapshotMetricStateForSinkMetricData:{}, sinkMetricData:{},
subtaskIndex:{}",
+ LOGGER.info("snapshotMetricStateForSinkMetricData:{},
sinkMetricData:{}, subtaskIndex:{}",
metricStateListState, sinkMetricData, subtaskIndex);
metricStateListState.clear();
Map<String, Long> metricDataMap = new HashMap<>();
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 932e249d5..ba899e980 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
@@ -419,7 +419,7 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
.withRegisterMetric(RegisteredMetric.ALL)
.build();
if (metricOption != null) {
- metricData = new SourceMetricData(metricOption,
getRuntimeContext().getMetricGroup());
+ metricData = new SourceMetricData(metricOption, metricGroup);
}
properties.setProperty("name", "engine");
properties.setProperty("offset.storage",
FlinkOffsetBackingStore.class.getCanonicalName());
diff --git
a/inlong-sort/sort-connectors/mysql-cdc/src/main/java/org/apache/inlong/sort/cdc/debezium/DebeziumSourceFunction.java
b/inlong-sort/sort-connectors/mysql-cdc/src/main/java/org/apache/inlong/sort/cdc/debezium/DebeziumSourceFunction.java
index 1ac48f97a..f9b4d96f6 100644
---
a/inlong-sort/sort-connectors/mysql-cdc/src/main/java/org/apache/inlong/sort/cdc/debezium/DebeziumSourceFunction.java
+++
b/inlong-sort/sort-connectors/mysql-cdc/src/main/java/org/apache/inlong/sort/cdc/debezium/DebeziumSourceFunction.java
@@ -419,7 +419,7 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
.withRegisterMetric(RegisteredMetric.ALL)
.build();
if (metricOption != null) {
- sourceMetricData = new SourceMetricData(metricOption,
getRuntimeContext().getMetricGroup());
+ sourceMetricData = new SourceMetricData(metricOption, metricGroup);
}
properties.setProperty("name", "engine");
diff --git
a/inlong-sort/sort-connectors/oracle-cdc/src/main/java/org/apache/inlong/sort/cdc/oracle/DebeziumSourceFunction.java
b/inlong-sort/sort-connectors/oracle-cdc/src/main/java/org/apache/inlong/sort/cdc/oracle/DebeziumSourceFunction.java
index 24eb737d8..ec9fa46d3 100644
---
a/inlong-sort/sort-connectors/oracle-cdc/src/main/java/org/apache/inlong/sort/cdc/oracle/DebeziumSourceFunction.java
+++
b/inlong-sort/sort-connectors/oracle-cdc/src/main/java/org/apache/inlong/sort/cdc/oracle/DebeziumSourceFunction.java
@@ -445,7 +445,7 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
.withRegisterMetric(RegisteredMetric.ALL)
.build();
if (metricOption != null) {
- sourceMetricData = new SourceMetricData(metricOption,
getRuntimeContext().getMetricGroup());
+ sourceMetricData = new SourceMetricData(metricOption, metricGroup);
}
properties.setProperty("name", "engine");
properties.setProperty("offset.storage",
FlinkOffsetBackingStore.class.getCanonicalName());
diff --git
a/inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java
b/inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java
index 2ccf92421..6951c0721 100644
---
a/inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java
+++
b/inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java
@@ -445,7 +445,7 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
.withRegisterMetric(RegisteredMetric.ALL)
.build();
if (metricOption != null) {
- sourceMetricData = new SourceMetricData(metricOption,
getRuntimeContext().getMetricGroup());
+ sourceMetricData = new SourceMetricData(metricOption, metricGroup);
}
properties.setProperty("name", "engine");
properties.setProperty("offset.storage",
FlinkOffsetBackingStore.class.getCanonicalName());
diff --git
a/inlong-sort/sort-connectors/sqlserver-cdc/src/main/java/org/apache/inlong/sort/cdc/sqlserver/table/DebeziumSourceFunction.java
b/inlong-sort/sort-connectors/sqlserver-cdc/src/main/java/org/apache/inlong/sort/cdc/sqlserver/table/DebeziumSourceFunction.java
index 16ee84362..b50d6fed0 100644
---
a/inlong-sort/sort-connectors/sqlserver-cdc/src/main/java/org/apache/inlong/sort/cdc/sqlserver/table/DebeziumSourceFunction.java
+++
b/inlong-sort/sort-connectors/sqlserver-cdc/src/main/java/org/apache/inlong/sort/cdc/sqlserver/table/DebeziumSourceFunction.java
@@ -20,6 +20,9 @@ package org.apache.inlong.sort.cdc.sqlserver.table;
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;
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
import com.ververica.cdc.debezium.Validator;
@@ -59,6 +62,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;
@@ -74,7 +78,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;
@@ -211,10 +217,14 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
private String inlongMetric;
- private SourceMetricData metricData;
+ private SourceMetricData sourceMetricData;
private String auditHostAndPorts;
+ private transient ListState<MetricState> metricStateListState;
+
+ private MetricState metricState;
+
//
---------------------------------------------------------------------------------------
public DebeziumSourceFunction(
@@ -259,9 +269,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 =
@@ -331,6 +351,10 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
} else {
snapshotOffsetState(functionSnapshotContext.getCheckpointId());
snapshotHistoryRecordsState();
+ if (sourceMetricData != null && metricStateListState != null) {
+
MetricStateUtils.snapshotMetricStateForSourceMetricData(metricStateListState,
sourceMetricData,
+ getRuntimeContext().getIndexOfThisSubtask());
+ }
}
}
@@ -405,10 +429,12 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
MetricOption metricOption = MetricOption.builder()
.withInlongLabels(inlongMetric)
.withInlongAudit(auditHostAndPorts)
+ .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());
@@ -447,8 +473,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);
}
@@ -620,7 +646,7 @@ public class DebeziumSourceFunction<T> extends
RichSourceFunction<T>
return engineInstanceName;
}
- public SourceMetricData getMetricData() {
- return metricData;
+ public SourceMetricData getSourceMetricData() {
+ return sourceMetricData;
}
}