You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by GitBox <gi...@apache.org> on 2022/09/05 15:04:19 UTC

[GitHub] [inlong] gong opened a new pull request, #5785: [INLONG-5784][Sort] Add metric state for postgreSQL

gong opened a new pull request, #5785:
URL: https://github.com/apache/inlong/pull/5785

   ### Prepare a Pull Request
   
   - [INLONG-5784][Sort] Add metric state for postgreSQL
   
   - Fixes #5784 
   
   ### Motivation
   
   [INLONG-5784][Sort] Add metric state for postgreSQL
   
   ### Modifications
   
   * Add metric state
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] thesumery commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for postgreSQL

Posted by GitBox <gi...@apache.org>.
thesumery commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r963446399


##########
inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java:
##########
@@ -273,9 +283,15 @@ public void initializeState(FunctionInitializationContext context) throws Except
                         new ListStateDescriptor<>(
                                 HISTORY_RECORDS_STATE_NAME, BasicTypeInfo.STRING_TYPE_INFO));
 
+        this.metricState =
+                stateStore.getUnionListState(

Review Comment:
   `getUnionListState` is aggregating all state,here if  before multiple subtask have been snapshotted, maybe here can be List<Map<String, Integer>>, for every subtask has a different counter value ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] gong commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for PostgreSQL

Posted by GitBox <gi...@apache.org>.
gong commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r972924837


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ *   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.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.SourceMetricData;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric state for {@link MetricState} supporting snapshot and restore
+ */
+@Slf4j
+public class MetricStateUtils {
+
+    /**
+     *
+     * restore metric state data
+     * @param metricStateListState state data list
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @return metric state
+     * @throws Exception throw exception metricStateListState.get()
+     */
+    public static MetricState restoreMetricState(ListState<MetricState> metricStateListState, Integer subtaskIndex,
+            Integer currentSubtaskNum) throws Exception {
+        if (metricStateListState == null || metricStateListState.get() == null) {
+            return null;
+        }
+        log.info("restoreMetricState:{}, subtaskIndex:{}, currentSubtaskNum:{}", metricStateListState, subtaskIndex,
+                currentSubtaskNum);
+        MetricState currentMetricState;
+        Map<Integer, MetricState> map = new HashMap<>(16);
+        for (MetricState metricState : metricStateListState.get()) {
+            map.put(metricState.getSubtaskIndex(), metricState);
+        }
+        int previousSubtaskNum = map.size();
+        if (currentSubtaskNum >= previousSubtaskNum) {
+            currentMetricState = map.get(subtaskIndex);
+        } else {
+            Map<String, Long> metrics = new HashMap<>(4);
+            currentMetricState = new MetricState(subtaskIndex, metrics);
+            List<Integer> indexList = computeIndexList(subtaskIndex, currentSubtaskNum, previousSubtaskNum);
+            for (Integer index : indexList) {
+                MetricState metricState = map.get(index);
+                for (Map.Entry<String, Long> entry : metricState.getMetrics().entrySet()) {
+                    if (metrics.containsKey(entry.getKey())) {
+                        metrics.put(entry.getKey(), metrics.get(entry.getKey()) + entry.getValue());
+                    } else {
+                        metrics.put(entry.getKey(), entry.getValue());
+                    }
+                }
+            }
+        }
+        return currentMetricState;
+    }
+
+    /**
+     *
+     * Assignment previous subtask index to current subtask when reduce parallelism
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @param previousSubtaskNum number of previous parallel subtask
+     * @return index list
+     */
+    public static List<Integer> computeIndexList(Integer subtaskIndex, Integer currentSubtaskNum,
+            Integer previousSubtaskNum) {
+        List<Integer> indexList = new ArrayList<>();
+        int taskNum = previousSubtaskNum / currentSubtaskNum;

Review Comment:
   I modify this name to `assignTaskNum`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] EMsnap commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for PostgreSQL

Posted by GitBox <gi...@apache.org>.
EMsnap commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r972660205


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ *   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.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.SourceMetricData;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric state for {@link MetricState} supporting snapshot and restore
+ */
+@Slf4j
+public class MetricStateUtils {
+
+    /**
+     *
+     * restore metric state data
+     * @param metricStateListState state data list
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @return metric state
+     * @throws Exception throw exception metricStateListState.get()
+     */
+    public static MetricState restoreMetricState(ListState<MetricState> metricStateListState, Integer subtaskIndex,
+            Integer currentSubtaskNum) throws Exception {
+        if (metricStateListState == null || metricStateListState.get() == null) {
+            return null;
+        }
+        log.info("restoreMetricState:{}, subtaskIndex:{}, currentSubtaskNum:{}", metricStateListState, subtaskIndex,
+                currentSubtaskNum);
+        MetricState currentMetricState;
+        Map<Integer, MetricState> map = new HashMap<>(16);
+        for (MetricState metricState : metricStateListState.get()) {
+            map.put(metricState.getSubtaskIndex(), metricState);
+        }
+        int previousSubtaskNum = map.size();
+        if (currentSubtaskNum >= previousSubtaskNum) {
+            currentMetricState = map.get(subtaskIndex);
+        } else {
+            Map<String, Long> metrics = new HashMap<>(4);
+            currentMetricState = new MetricState(subtaskIndex, metrics);
+            List<Integer> indexList = computeIndexList(subtaskIndex, currentSubtaskNum, previousSubtaskNum);
+            for (Integer index : indexList) {
+                MetricState metricState = map.get(index);
+                for (Map.Entry<String, Long> entry : metricState.getMetrics().entrySet()) {
+                    if (metrics.containsKey(entry.getKey())) {
+                        metrics.put(entry.getKey(), metrics.get(entry.getKey()) + entry.getValue());
+                    } else {
+                        metrics.put(entry.getKey(), entry.getValue());
+                    }
+                }
+            }
+        }
+        return currentMetricState;
+    }
+
+    /**
+     *
+     * Assignment previous subtask index to current subtask when reduce parallelism
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @param previousSubtaskNum number of previous parallel subtask
+     * @return index list
+     */
+    public static List<Integer> computeIndexList(Integer subtaskIndex, Integer currentSubtaskNum,
+            Integer previousSubtaskNum) {
+        List<Integer> indexList = new ArrayList<>();
+        int taskNum = previousSubtaskNum / currentSubtaskNum;

Review Comment:
   previousSubtaskNum / currentSubtaskNum doesn't equal to tasknum



##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ *   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.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.SourceMetricData;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric state for {@link MetricState} supporting snapshot and restore
+ */
+@Slf4j
+public class MetricStateUtils {
+
+    /**
+     *
+     * restore metric state data
+     * @param metricStateListState state data list
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @return metric state
+     * @throws Exception throw exception metricStateListState.get()
+     */
+    public static MetricState restoreMetricState(ListState<MetricState> metricStateListState, Integer subtaskIndex,
+            Integer currentSubtaskNum) throws Exception {
+        if (metricStateListState == null || metricStateListState.get() == null) {
+            return null;
+        }
+        log.info("restoreMetricState:{}, subtaskIndex:{}, currentSubtaskNum:{}", metricStateListState, subtaskIndex,
+                currentSubtaskNum);
+        MetricState currentMetricState;
+        Map<Integer, MetricState> map = new HashMap<>(16);
+        for (MetricState metricState : metricStateListState.get()) {
+            map.put(metricState.getSubtaskIndex(), metricState);
+        }
+        int previousSubtaskNum = map.size();
+        if (currentSubtaskNum >= previousSubtaskNum) {
+            currentMetricState = map.get(subtaskIndex);
+        } else {
+            Map<String, Long> metrics = new HashMap<>(4);
+            currentMetricState = new MetricState(subtaskIndex, metrics);
+            List<Integer> indexList = computeIndexList(subtaskIndex, currentSubtaskNum, previousSubtaskNum);
+            for (Integer index : indexList) {
+                MetricState metricState = map.get(index);
+                for (Map.Entry<String, Long> entry : metricState.getMetrics().entrySet()) {
+                    if (metrics.containsKey(entry.getKey())) {
+                        metrics.put(entry.getKey(), metrics.get(entry.getKey()) + entry.getValue());
+                    } else {
+                        metrics.put(entry.getKey(), entry.getValue());
+                    }
+                }
+            }
+        }
+        return currentMetricState;
+    }
+
+    /**
+     *
+     * Assignment previous subtask index to current subtask when reduce parallelism
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @param previousSubtaskNum number of previous parallel subtask
+     * @return index list
+     */
+    public static List<Integer> computeIndexList(Integer subtaskIndex, Integer currentSubtaskNum,
+            Integer previousSubtaskNum) {
+        List<Integer> indexList = new ArrayList<>();
+        int taskNum = previousSubtaskNum / currentSubtaskNum;

Review Comment:
   maybe use a more explicit name



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] EMsnap commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for PostgreSQL

Posted by GitBox <gi...@apache.org>.
EMsnap commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r972656995


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ *   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.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.SourceMetricData;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric state for {@link MetricState} supporting snapshot and restore
+ */
+@Slf4j
+public class MetricStateUtils {
+
+    /**
+     *
+     * restore metric state data
+     * @param metricStateListState state data list
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @return metric state
+     * @throws Exception throw exception metricStateListState.get()
+     */
+    public static MetricState restoreMetricState(ListState<MetricState> metricStateListState, Integer subtaskIndex,
+            Integer currentSubtaskNum) throws Exception {
+        if (metricStateListState == null || metricStateListState.get() == null) {
+            return null;
+        }
+        log.info("restoreMetricState:{}, subtaskIndex:{}, currentSubtaskNum:{}", metricStateListState, subtaskIndex,
+                currentSubtaskNum);
+        MetricState currentMetricState;
+        Map<Integer, MetricState> map = new HashMap<>(16);
+        for (MetricState metricState : metricStateListState.get()) {
+            map.put(metricState.getSubtaskIndex(), metricState);
+        }
+        int previousSubtaskNum = map.size();
+        if (currentSubtaskNum >= previousSubtaskNum) {
+            currentMetricState = map.get(subtaskIndex);
+        } else {
+            Map<String, Long> metrics = new HashMap<>(4);
+            currentMetricState = new MetricState(subtaskIndex, metrics);
+            List<Integer> indexList = computeIndexList(subtaskIndex, currentSubtaskNum, previousSubtaskNum);
+            for (Integer index : indexList) {
+                MetricState metricState = map.get(index);
+                for (Map.Entry<String, Long> entry : metricState.getMetrics().entrySet()) {
+                    if (metrics.containsKey(entry.getKey())) {
+                        metrics.put(entry.getKey(), metrics.get(entry.getKey()) + entry.getValue());
+                    } else {
+                        metrics.put(entry.getKey(), entry.getValue());
+                    }
+                }
+            }
+        }
+        return currentMetricState;
+    }
+
+    /**
+     *
+     * Assignment previous subtask index to current subtask when reduce parallelism
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @param previousSubtaskNum number of previous parallel subtask
+     * @return index list
+     */
+    public static List<Integer> computeIndexList(Integer subtaskIndex, Integer currentSubtaskNum,

Review Comment:
   the doc should explain the rule on how are the indexs are assigned 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] gong commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for postgreSQL

Posted by GitBox <gi...@apache.org>.
gong commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r963845851


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricDataSerializationUtils.java:
##########
@@ -0,0 +1,61 @@
+/*
+ *   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.inlong.sort.base.util;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.inlong.sort.base.metric.SourceMetricData;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric data state serialization and deserialization
+ */
+public class MetricDataSerializationUtils {
+
+    public static Map<String, Long> restoreMetricState(ListState<byte[]> metricState) throws Exception {
+        Map<String, Long> metricDataMap = new HashMap<>();
+        for (byte[] metricData : metricState.get()) {
+            ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(metricData);
+            ObjectInputStream objectInputStream = new ObjectInputStream(byteArrayInputStream);
+            metricDataMap = (Map<String, Long>) objectInputStream.readObject();
+        }
+        return metricDataMap;
+    }
+
+    public static void snapshotMetricState(ListState<byte[]> metricState, SourceMetricData sourceMetricData)

Review Comment:
   A subtask should have a sourceMetricData. Here, pg cdc only have a subtask, it will use single task to comsum log data.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] gong commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for PostgreSQL

Posted by GitBox <gi...@apache.org>.
gong commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r972925038


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricStateUtils.java:
##########
@@ -0,0 +1,127 @@
+/*
+ *   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.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.SourceMetricData;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric state for {@link MetricState} supporting snapshot and restore
+ */
+@Slf4j
+public class MetricStateUtils {
+
+    /**
+     *
+     * restore metric state data
+     * @param metricStateListState state data list
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @return metric state
+     * @throws Exception throw exception metricStateListState.get()
+     */
+    public static MetricState restoreMetricState(ListState<MetricState> metricStateListState, Integer subtaskIndex,
+            Integer currentSubtaskNum) throws Exception {
+        if (metricStateListState == null || metricStateListState.get() == null) {
+            return null;
+        }
+        log.info("restoreMetricState:{}, subtaskIndex:{}, currentSubtaskNum:{}", metricStateListState, subtaskIndex,
+                currentSubtaskNum);
+        MetricState currentMetricState;
+        Map<Integer, MetricState> map = new HashMap<>(16);
+        for (MetricState metricState : metricStateListState.get()) {
+            map.put(metricState.getSubtaskIndex(), metricState);
+        }
+        int previousSubtaskNum = map.size();
+        if (currentSubtaskNum >= previousSubtaskNum) {
+            currentMetricState = map.get(subtaskIndex);
+        } else {
+            Map<String, Long> metrics = new HashMap<>(4);
+            currentMetricState = new MetricState(subtaskIndex, metrics);
+            List<Integer> indexList = computeIndexList(subtaskIndex, currentSubtaskNum, previousSubtaskNum);
+            for (Integer index : indexList) {
+                MetricState metricState = map.get(index);
+                for (Map.Entry<String, Long> entry : metricState.getMetrics().entrySet()) {
+                    if (metrics.containsKey(entry.getKey())) {
+                        metrics.put(entry.getKey(), metrics.get(entry.getKey()) + entry.getValue());
+                    } else {
+                        metrics.put(entry.getKey(), entry.getValue());
+                    }
+                }
+            }
+        }
+        return currentMetricState;
+    }
+
+    /**
+     *
+     * Assignment previous subtask index to current subtask when reduce parallelism
+     * @param subtaskIndex current subtask index
+     * @param currentSubtaskNum number of current parallel subtask
+     * @param previousSubtaskNum number of previous parallel subtask
+     * @return index list
+     */
+    public static List<Integer> computeIndexList(Integer subtaskIndex, Integer currentSubtaskNum,

Review Comment:
   add description



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] dockerzhang merged pull request #5785: [INLONG-5784][Sort] Add metric state for PostgreSQL

Posted by GitBox <gi...@apache.org>.
dockerzhang merged PR #5785:
URL: https://github.com/apache/inlong/pull/5785


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] gong commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for postgreSQL

Posted by GitBox <gi...@apache.org>.
gong commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r963848158


##########
inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java:
##########
@@ -273,9 +283,15 @@ public void initializeState(FunctionInitializationContext context) throws Except
                         new ListStateDescriptor<>(
                                 HISTORY_RECORDS_STATE_NAME, BasicTypeInfo.STRING_TYPE_INFO));
 
+        this.metricState =
+                stateStore.getUnionListState(

Review Comment:
   every subtask should have a counter



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] EMsnap commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for postgreSQL

Posted by GitBox <gi...@apache.org>.
EMsnap commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r963189543


##########
inlong-sort/sort-connectors/postgres-cdc/src/main/java/org.apache.inlong.sort.cdc.postgres/DebeziumSourceFunction.java:
##########
@@ -273,9 +283,15 @@ public void initializeState(FunctionInitializationContext context) throws Except
                         new ListStateDescriptor<>(
                                 HISTORY_RECORDS_STATE_NAME, BasicTypeInfo.STRING_TYPE_INFO));
 
+        this.metricState =
+                stateStore.getUnionListState(
+                        new ListStateDescriptor<>(
+                                INLONG_METRIC_STATE_NAME, PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO));

Review Comment:
   metric state should not use the same name for different operator



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [inlong] thesumery commented on a diff in pull request #5785: [INLONG-5784][Sort] Add metric state for postgreSQL

Posted by GitBox <gi...@apache.org>.
thesumery commented on code in PR #5785:
URL: https://github.com/apache/inlong/pull/5785#discussion_r963442311


##########
inlong-sort/sort-connectors/base/src/main/java/org/apache/inlong/sort/base/util/MetricDataSerializationUtils.java:
##########
@@ -0,0 +1,61 @@
+/*
+ *   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.inlong.sort.base.util;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.inlong.sort.base.metric.SourceMetricData;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.inlong.sort.base.Constants.NUM_BYTES_IN;
+import static org.apache.inlong.sort.base.Constants.NUM_RECORDS_IN;
+
+/**
+ * metric data state serialization and deserialization
+ */
+public class MetricDataSerializationUtils {
+
+    public static Map<String, Long> restoreMetricState(ListState<byte[]> metricState) throws Exception {
+        Map<String, Long> metricDataMap = new HashMap<>();
+        for (byte[] metricData : metricState.get()) {
+            ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(metricData);
+            ObjectInputStream objectInputStream = new ObjectInputStream(byteArrayInputStream);
+            metricDataMap = (Map<String, Long>) objectInputStream.readObject();
+        }
+        return metricDataMap;
+    }
+
+    public static void snapshotMetricState(ListState<byte[]> metricState, SourceMetricData sourceMetricData)

Review Comment:
   Can this guarantee that other subtasks are not increasing the value when saving the counter?maybe other subtask is processingElement but this subtask is snapshotState, so this value is not stable



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org