Hisoka-X commented on code in PR #7338:
URL: https://github.com/apache/seatunnel/pull/7338#discussion_r1713129674


##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.seatunnel.engine.server.metrics;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.Meter;
+import org.apache.seatunnel.api.common.metrics.MetricsContext;
+import org.apache.seatunnel.api.table.catalog.TablePath;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS;
+
+public class TaskMetricsCalcContext {
+
+    private final MetricsContext metricsContext;
+
+    private final String type;
+
+    private Counter count;
+
+    private Map<String, Counter> countPerTable = new ConcurrentHashMap<>();
+
+    private Meter QPS;
+
+    private Map<String, Meter> QPSPerTable = new ConcurrentHashMap<>();
+
+    private Counter bytes;
+
+    private Map<String, Counter> bytesPerTable = new ConcurrentHashMap<>();
+
+    private Meter bytesPerSeconds;
+
+    private Map<String, Meter> bytesPerSecondsPerTable = new 
ConcurrentHashMap<>();
+
+    public TaskMetricsCalcContext(
+            MetricsContext metricsContext, String type, boolean isMulti, 
List<TablePath> tables) {

Review Comment:
   ```suggestion
               MetricsContext metricsContext, String type, boolean isMulti, 
List<TablePath> tables) {
   ```
   ```suggestion
               MetricsContext metricsContext, PluginType type, boolean isMulti, 
List<TablePath> tables) {
   ```



##########
seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java:
##########
@@ -82,33 +83,101 @@ public void multiTableMetrics() {
         Collections.singletonList(node1)
                 .forEach(
                         instance -> {
-                            given().get(
-                                            HOST
-                                                    + instance.getCluster()
-                                                            .getLocalMember()
-                                                            .getAddress()
-                                                            .getPort()
-                                                    + RestConstant.JOB_INFO_URL
-                                                    + "/"
-                                                    + batchJobProxy.getJobId())
-                                    .then()
+                            Response response =
+                                    given().get(
+                                                    HOST
+                                                            + 
instance.getCluster()
+                                                                    
.getLocalMember()
+                                                                    
.getAddress()
+                                                                    .getPort()
+                                                            + 
RestConstant.JOB_INFO_URL
+                                                            + "/"
+                                                            + 
batchJobProxy.getJobId());
+                            // 测试示例中单条 [3, "C", 100] 的数据待大小是13

Review Comment:
   Please change comment to english.



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.seatunnel.engine.server.metrics;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.Meter;
+import org.apache.seatunnel.api.common.metrics.MetricsContext;
+import org.apache.seatunnel.api.table.catalog.TablePath;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS;
+
+public class TaskMetricsCalcContext {
+
+    private final MetricsContext metricsContext;
+
+    private final String type;
+
+    private Counter count;
+
+    private Map<String, Counter> countPerTable = new ConcurrentHashMap<>();
+
+    private Meter QPS;
+
+    private Map<String, Meter> QPSPerTable = new ConcurrentHashMap<>();
+
+    private Counter bytes;
+
+    private Map<String, Counter> bytesPerTable = new ConcurrentHashMap<>();
+
+    private Meter bytesPerSeconds;
+
+    private Map<String, Meter> bytesPerSecondsPerTable = new 
ConcurrentHashMap<>();
+
+    public TaskMetricsCalcContext(
+            MetricsContext metricsContext, String type, boolean isMulti, 
List<TablePath> tables) {
+        this.metricsContext = metricsContext;
+        this.type = type;
+        initializeMetrics(isMulti, tables);
+    }
+
+    private void initializeMetrics(boolean isMulti, List<TablePath> tables) {
+        if ("SINK".equalsIgnoreCase(type)) {
+            this.initializeMetrics(
+                    isMulti,
+                    tables,
+                    SINK_WRITE_COUNT,
+                    SINK_WRITE_QPS,
+                    SINK_WRITE_BYTES,
+                    SINK_WRITE_BYTES_PER_SECONDS);
+        } else if ("SOURCE".equalsIgnoreCase(type)) {
+            this.initializeMetrics(
+                    isMulti,
+                    tables,
+                    SOURCE_RECEIVED_COUNT,
+                    SOURCE_RECEIVED_QPS,
+                    SOURCE_RECEIVED_BYTES,
+                    SOURCE_RECEIVED_BYTES_PER_SECONDS);
+        }
+    }
+
+    private void initializeMetrics(
+            boolean isMulti,
+            List<TablePath> tables,
+            String sinkWriteCount,
+            String sinkWriteQps,
+            String sinkWriteBytes,
+            String sinkWriteBytesPerSeconds) {

Review Comment:
   ```suggestion
       private void initializeMetrics(
               boolean isMulti,
               List<TablePath> tables,
               String writeCountName,
               String writeQpsName,
               String writeBytesName,
               String writeBytesPerSecondsName) {
   ```



##########
seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/metrics/TaskMetricsCalcContext.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.seatunnel.engine.server.metrics;
+
+import org.apache.seatunnel.api.common.metrics.Counter;
+import org.apache.seatunnel.api.common.metrics.Meter;
+import org.apache.seatunnel.api.common.metrics.MetricsContext;
+import org.apache.seatunnel.api.table.catalog.TablePath;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT;
+import static 
org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS;
+
+public class TaskMetricsCalcContext {
+
+    private final MetricsContext metricsContext;
+
+    private final String type;
+
+    private Counter count;
+
+    private Map<String, Counter> countPerTable = new ConcurrentHashMap<>();
+
+    private Meter QPS;
+
+    private Map<String, Meter> QPSPerTable = new ConcurrentHashMap<>();
+
+    private Counter bytes;
+
+    private Map<String, Counter> bytesPerTable = new ConcurrentHashMap<>();
+
+    private Meter bytesPerSeconds;
+
+    private Map<String, Meter> bytesPerSecondsPerTable = new 
ConcurrentHashMap<>();
+
+    public TaskMetricsCalcContext(
+            MetricsContext metricsContext, String type, boolean isMulti, 
List<TablePath> tables) {
+        this.metricsContext = metricsContext;
+        this.type = type;
+        initializeMetrics(isMulti, tables);
+    }
+
+    private void initializeMetrics(boolean isMulti, List<TablePath> tables) {
+        if ("SINK".equalsIgnoreCase(type)) {
+            this.initializeMetrics(
+                    isMulti,
+                    tables,
+                    SINK_WRITE_COUNT,
+                    SINK_WRITE_QPS,
+                    SINK_WRITE_BYTES,
+                    SINK_WRITE_BYTES_PER_SECONDS);
+        } else if ("SOURCE".equalsIgnoreCase(type)) {
+            this.initializeMetrics(
+                    isMulti,
+                    tables,
+                    SOURCE_RECEIVED_COUNT,
+                    SOURCE_RECEIVED_QPS,
+                    SOURCE_RECEIVED_BYTES,
+                    SOURCE_RECEIVED_BYTES_PER_SECONDS);
+        }
+    }
+
+    private void initializeMetrics(
+            boolean isMulti,
+            List<TablePath> tables,
+            String sinkWriteCount,
+            String sinkWriteQps,
+            String sinkWriteBytes,
+            String sinkWriteBytesPerSeconds) {
+        count = metricsContext.counter(sinkWriteCount);
+        QPS = metricsContext.meter(sinkWriteQps);
+        bytes = metricsContext.counter(sinkWriteBytes);
+        bytesPerSeconds = metricsContext.meter(sinkWriteBytesPerSeconds);
+        if (isMulti) {
+            tables.forEach(
+                    tablePath -> {
+                        countPerTable.put(
+                                tablePath.getFullName(),
+                                metricsContext.counter(
+                                        sinkWriteCount + "#" + 
tablePath.getFullName()));
+                        QPSPerTable.put(
+                                tablePath.getFullName(),
+                                metricsContext.meter(sinkWriteQps + "#" + 
tablePath.getFullName()));
+                        bytesPerTable.put(
+                                tablePath.getFullName(),
+                                metricsContext.counter(
+                                        sinkWriteBytes + "#" + 
tablePath.getFullName()));
+                        bytesPerSecondsPerTable.put(
+                                tablePath.getFullName(),
+                                metricsContext.meter(
+                                        sinkWriteBytesPerSeconds + "#" + 
tablePath.getFullName()));
+                    });
+        }
+    }
+
+    public void updateMetrics(Object data) {
+        count.inc();
+        QPS.markEvent();
+        if (data instanceof SeaTunnelRow) {
+            SeaTunnelRow row = (SeaTunnelRow) data;
+            bytes.inc(row.getBytesSize());
+            bytesPerSeconds.markEvent(row.getBytesSize());
+            String tableId = row.getTableId();
+
+            if (StringUtils.isNotBlank(tableId)) {
+                String tableName = TablePath.of(tableId).getFullName();
+
+                // Processing count
+                processMetrics(
+                        countPerTable,
+                        Counter.class,
+                        tableName,
+                        SINK_WRITE_COUNT,
+                        SOURCE_RECEIVED_COUNT,
+                        Counter::inc);
+
+                // Processing bytes
+                processMetrics(
+                        bytesPerTable,
+                        Counter.class,
+                        tableName,
+                        SINK_WRITE_BYTES,
+                        SOURCE_RECEIVED_BYTES,
+                        counter -> counter.inc(row.getBytesSize()));
+
+                // Processing QPS
+                processMetrics(
+                        QPSPerTable,
+                        Meter.class,
+                        tableName,
+                        SINK_WRITE_QPS,
+                        SOURCE_RECEIVED_QPS,
+                        Meter::markEvent);
+
+                // Processing bytes rate
+                processMetrics(
+                        bytesPerSecondsPerTable,
+                        Meter.class,
+                        tableName,
+                        SINK_WRITE_BYTES_PER_SECONDS,
+                        SOURCE_RECEIVED_BYTES_PER_SECONDS,
+                        meter -> meter.markEvent(row.getBytesSize()));
+            }
+        }
+    }
+
+    private <T> void processMetrics(
+            Map<String, T> metricMap,
+            Class<T> cls,
+            String tableName,
+            String sinkMetric,
+            String sourceMetric,
+            MetricProcessor<T> processor) {
+        T metric = metricMap.get(tableName);
+        if (Objects.nonNull(metric)) {
+            processor.process(metric);
+        } else {
+            String metricName =
+                    "sink".equalsIgnoreCase(type)
+                            ? sinkMetric + "#" + tableName
+                            : sourceMetric + "#" + tableName;
+            T newMetric = createMetric(metricsContext, metricName, cls);
+            processor.process(newMetric);
+            metricMap.put(tableName, newMetric);
+        }
+    }
+
+    private <T> T createMetric(
+            MetricsContext metricsContext, String metricName, Class<T> 
metricClass) {
+        if (metricClass == Counter.class) {
+            return metricClass.cast(metricsContext.counter(metricName));
+        } else if (metricClass == Meter.class) {
+            return metricClass.cast(metricsContext.meter(metricName));
+        }
+        throw new IllegalArgumentException("Unsupported metric class: " + 
metricClass.getName());
+    }
+
+    @FunctionalInterface
+    interface MetricProcessor<T> {
+        void process(T t);
+    }
+
+    private String getFullName(TablePath tablePath) {
+        if (StringUtils.isBlank(tablePath.getTableName())) {
+            tablePath =
+                    TablePath.of(tablePath.getDatabaseName(), 
tablePath.getSchemaName(), "default");
+        }
+        return tablePath.getFullName();
+    }

Review Comment:
   Already removed in 
https://github.com/apache/seatunnel/pull/7252/files#diff-88ba3dc2d502d1b50f1b160cb32c6fe40e0e8a74570049cf1a2a05094f4d0ab3



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to