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 09:25:50 UTC
[inlong] 04/05: [INLONG-5944][Sort] Add metric state for es6 and es7 (#5968)
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
commit 5417acad8749a6cb8d88a51ec6fa28df9e382f80
Author: Oneal65 <li...@foxmail.com>
AuthorDate: Wed Sep 21 16:44:49 2022 +0800
[INLONG-5944][Sort] Add metric state for es6 and es7 (#5968)
---
.../sort/elasticsearch/ElasticsearchSinkBase.java | 3 ++
.../elasticsearch/ElasticsearchSinkFunction.java | 10 ++++++
.../table/RowElasticsearchSinkFunction.java | 38 ++++++++++++++++++++++
3 files changed, 51 insertions(+)
diff --git a/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkBase.java b/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkBase.java
index cdea07fa5..dff8b9335 100644
--- a/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkBase.java
+++ b/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkBase.java
@@ -293,6 +293,7 @@ public abstract class ElasticsearchSinkBase<T, C extends AutoCloseable> extends
@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
// no initialization needed
+ elasticsearchSinkFunction.initializeState(context);
}
@Override
@@ -305,6 +306,8 @@ public abstract class ElasticsearchSinkBase<T, C extends AutoCloseable> extends
checkAsyncErrorsAndRequests();
}
}
+
+ elasticsearchSinkFunction.snapshotState(context);
}
@Override
diff --git a/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkFunction.java b/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkFunction.java
index dc4bf5af1..e1c019918 100644
--- a/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkFunction.java
+++ b/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/ElasticsearchSinkFunction.java
@@ -21,6 +21,8 @@ package org.apache.inlong.sort.elasticsearch;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
import org.elasticsearch.action.ActionRequest;
@@ -59,4 +61,12 @@ public interface ElasticsearchSinkFunction<T> extends Serializable, Function {
* @param indexer request indexer that {@code ActionRequest} should be added to
*/
void process(T element, RuntimeContext ctx, RequestIndexer indexer);
+
+ default void initializeState(FunctionInitializationContext context) throws Exception {
+ // no initialization needed
+ }
+
+ default void snapshotState(FunctionSnapshotContext context) throws Exception {
+
+ }
}
diff --git a/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/table/RowElasticsearchSinkFunction.java b/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/table/RowElasticsearchSinkFunction.java
index e4f1419c5..a5abc4690 100644
--- a/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/table/RowElasticsearchSinkFunction.java
+++ b/inlong-sort/sort-connectors/elasticsearch-base/src/main/java/org/apache/inlong/sort/elasticsearch/table/RowElasticsearchSinkFunction.java
@@ -20,8 +20,16 @@ package org.apache.inlong.sort.elasticsearch.table;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
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.util.MetricStateUtils;
import org.apache.inlong.sort.elasticsearch.ElasticsearchSinkFunction;
import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
import org.apache.flink.table.api.TableException;
@@ -40,6 +48,10 @@ import javax.annotation.Nullable;
import java.util.Objects;
import java.util.function.Function;
+import static org.apache.inlong.sort.base.Constants.INLONG_METRIC_STATE_NAME;
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_OUT;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_OUT;
+
/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */
public class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction<RowData> {
@@ -51,6 +63,8 @@ public class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction<R
private final XContentType contentType;
private final RequestFactory requestFactory;
private final Function<RowData, String> createKey;
+ private transient ListState<MetricState> metricStateListState;
+ private transient MetricState metricState;
private final String inlongMetric;
private final String auditHostAndPorts;
@@ -88,6 +102,8 @@ public class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction<R
MetricOption metricOption = MetricOption.builder()
.withInlongLabels(inlongMetric)
.withInlongAudit(auditHostAndPorts)
+ .withInitRecords(metricState != null ? metricState.getMetricValue(NUM_RECORDS_OUT) : 0L)
+ .withInitBytes(metricState != null ? metricState.getMetricValue(NUM_BYTES_OUT) : 0L)
.withRegisterMetric(RegisteredMetric.NORMAL)
.build();
if (metricOption != null) {
@@ -101,6 +117,28 @@ public class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction<R
}
}
+ @Override
+ public void initializeState(FunctionInitializationContext context) throws Exception {
+ if (this.inlongMetric != null) {
+ this.metricStateListState = context.getOperatorStateStore().getUnionListState(
+ new ListStateDescriptor<>(
+ INLONG_METRIC_STATE_NAME, TypeInformation.of(new TypeHint<MetricState>() {
+ })));
+ }
+ if (context.isRestored()) {
+ metricState = MetricStateUtils.restoreMetricState(metricStateListState,
+ runtimeContext.getIndexOfThisSubtask(), runtimeContext.getNumberOfParallelSubtasks());
+ }
+ }
+
+ @Override
+ public void snapshotState(FunctionSnapshotContext context) throws Exception {
+ if (sinkMetricData != null && metricStateListState != null) {
+ MetricStateUtils.snapshotMetricStateForSinkMetricData(metricStateListState, sinkMetricData,
+ runtimeContext.getIndexOfThisSubtask());
+ }
+ }
+
@Override
public void process(RowData element, RuntimeContext ctx, RequestIndexer indexer) {
switch (element.getRowKind()) {