fuweng11 commented on code in PR #10916:
URL: https://github.com/apache/inlong/pull/10916#discussion_r1737738348


##########
inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkParallelismOptimizer.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.manager.plugin.flink;
+
+import org.apache.inlong.audit.AuditIdEnum;
+import org.apache.inlong.audit.entity.FlowType;
+import org.apache.inlong.manager.pojo.audit.AuditInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.stereotype.Component;
+
+import java.io.IOException;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.StringJoiner;
+
+import static org.apache.inlong.audit.consts.OpenApiConstants.*;
+import static org.apache.inlong.manager.common.consts.InlongConstants.*;
+
+/**
+ *     This class is used to calculate the recommended parallelism based on 
the maximum message per second per core.
+ *     The data volume is calculated based on the average data count per hour.
+ *     The data count is retrieved from the inlong audit API.
+ */
+@Slf4j
+@Component
+public class FlinkParallelismOptimizer {
+
+    @Value("${audit.query.url:http://127.0.0.1:10080}";)
+    public String auditQueryUrl;
+
+    private static final int MAX_PARALLELISM = 2048;
+    private long maximumMessagePerSecondPerCore = 1000L;
+    private static final long DEFAULT_ERROR_DATA_VOLUME = 0L;
+    private static final FlowType DEFAULT_FLOWTYPE = FlowType.OUTPUT;
+    private static final String DEFAULT_AUDIT_TYPE = "DataProxy";
+    private static final String AUDIT_CYCLE_REALTIME = "1";
+    // maxmimum data scale counting range in hours
+    private static final int DATA_SCALE_COUNTING_RANGE_IN_HOURS = 1;
+    // sample time format: 2024-08-23T22:47:38.866
+    private static final String AUDIT_QUERY_DATE_TIME_FORMAT = 
"yyyy-MM-dd'T'HH:mm:ss.SSS";
+
+    private static final String LOGTS_DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+    private static final String TIMEZONE_REGEX = "([+-])(\\d):";
+
+    /**
+     * Calculate recommended parallelism based on maximum message per second 
per core
+     *
+     * @return Recommended parallelism
+     */
+    public int calculateRecommendedParallelism(List<InlongStreamInfo> 
streamInfos) {
+        long averageDataVolume;
+        InlongStreamInfo streamInfo = streamInfos.get(0);
+        try {
+            averageDataVolume = getAverageDataVolume(streamInfo);
+            log.info("Retrieved data volume: {}", averageDataVolume);
+        } catch (Exception e) {
+            log.error("Error retrieving data volume: {}", e.getMessage(), e);
+            log.warn("Using default data volume: {}", 
DEFAULT_ERROR_DATA_VOLUME);
+            averageDataVolume = DEFAULT_ERROR_DATA_VOLUME;
+        }
+        int newParallelism = (int) (averageDataVolume / 
maximumMessagePerSecondPerCore);
+        // Ensure parallelism is at most MAX_PARALLELISM
+        newParallelism = Math.min(newParallelism, MAX_PARALLELISM);
+        log.info("Calculated parallelism: {} for data volume: {}", 
newParallelism, averageDataVolume);
+        return newParallelism;
+    }
+
+    /**
+     * Initialize maximum message per second per core based on configuration
+     *
+     * @param maximumMessagePerSecondPerCore The maximum messages per second 
per core
+     */
+    public void setMaximumMessagePerSecondPerCore(Integer 
maximumMessagePerSecondPerCore) {
+        if (maximumMessagePerSecondPerCore == null || 
maximumMessagePerSecondPerCore <= 0) {
+            log.error("Illegal flink.maxpercore property, must be nonnull and 
positive, using default value: {}",
+                    maximumMessagePerSecondPerCore);
+        } else {
+            this.maximumMessagePerSecondPerCore = 
maximumMessagePerSecondPerCore;
+        }
+    }
+
+    /**
+     * Get average data volume on the scale specified by 
DATA_SCALE_COUNTING_RANGE_IN_HOURS
+     *
+     * @param streamInfo inlong stream info
+     * @return The average data count per hour
+     */
+    private long getAverageDataVolume(InlongStreamInfo streamInfo) {
+        // Since the audit module uses local time, we need to use 
ZonedDateTime to get the current time
+        String dataTimeZone = 
streamInfo.getSourceList().get(0).getDataTimeZone();
+
+        // This regex pattern matches a time zone offset in the format of 
"GMT+/-X:00"
+        // where X is a single digit (e.g., "GMT+8:00"). The pattern captures 
the "+" or "-" sign
+        // and the single digit, then it replaces the single digit with two 
digits by adding a "0" in front of it.
+        // For example, "GMT+8:00" becomes "GMT+08:00" in order to match 
standard offset-based ZoneId.
+        dataTimeZone = dataTimeZone.replaceAll(TIMEZONE_REGEX, "$10$2:");
+        ZoneId dataZone = ZoneId.of(dataTimeZone);
+
+        ZonedDateTime endTime = ZonedDateTime.now(dataZone);
+        ZonedDateTime startTime = 
endTime.minusHours(DATA_SCALE_COUNTING_RANGE_IN_HOURS);
+        DateTimeFormatter formatter = 
DateTimeFormatter.ofPattern(AUDIT_QUERY_DATE_TIME_FORMAT);
+
+        // counting data volume on with DATA_PROXY_OUTPUT auditId
+        int auditId = AuditIdEnum.getAuditId(DEFAULT_AUDIT_TYPE, 
DEFAULT_FLOWTYPE).getValue();
+        StringJoiner urlParameters = new StringJoiner(AMPERSAND)
+                .add(PARAMS_START_TIME + EQUAL + startTime.format(formatter))
+                .add(PARAMS_END_TIME + EQUAL + endTime.format(formatter))
+                .add(PARAMS_INLONG_GROUP_Id + EQUAL + 
streamInfo.getInlongGroupId())
+                .add(PARAMS_INLONG_STREAM_Id + EQUAL + 
streamInfo.getInlongStreamId())

Review Comment:
   Ditto.



##########
inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkParallelismOptimizer.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.manager.plugin.flink;
+
+import org.apache.inlong.audit.AuditIdEnum;
+import org.apache.inlong.audit.entity.FlowType;
+import org.apache.inlong.manager.pojo.audit.AuditInfo;
+import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.stereotype.Component;
+
+import java.io.IOException;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.StringJoiner;
+
+import static org.apache.inlong.audit.consts.OpenApiConstants.*;
+import static org.apache.inlong.manager.common.consts.InlongConstants.*;
+
+/**
+ *     This class is used to calculate the recommended parallelism based on 
the maximum message per second per core.
+ *     The data volume is calculated based on the average data count per hour.
+ *     The data count is retrieved from the inlong audit API.
+ */
+@Slf4j
+@Component
+public class FlinkParallelismOptimizer {
+
+    @Value("${audit.query.url:http://127.0.0.1:10080}";)
+    public String auditQueryUrl;
+
+    private static final int MAX_PARALLELISM = 2048;
+    private long maximumMessagePerSecondPerCore = 1000L;
+    private static final long DEFAULT_ERROR_DATA_VOLUME = 0L;
+    private static final FlowType DEFAULT_FLOWTYPE = FlowType.OUTPUT;
+    private static final String DEFAULT_AUDIT_TYPE = "DataProxy";
+    private static final String AUDIT_CYCLE_REALTIME = "1";
+    // maxmimum data scale counting range in hours
+    private static final int DATA_SCALE_COUNTING_RANGE_IN_HOURS = 1;
+    // sample time format: 2024-08-23T22:47:38.866
+    private static final String AUDIT_QUERY_DATE_TIME_FORMAT = 
"yyyy-MM-dd'T'HH:mm:ss.SSS";
+
+    private static final String LOGTS_DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+    private static final String TIMEZONE_REGEX = "([+-])(\\d):";
+
+    /**
+     * Calculate recommended parallelism based on maximum message per second 
per core
+     *
+     * @return Recommended parallelism
+     */
+    public int calculateRecommendedParallelism(List<InlongStreamInfo> 
streamInfos) {
+        long averageDataVolume;
+        InlongStreamInfo streamInfo = streamInfos.get(0);
+        try {
+            averageDataVolume = getAverageDataVolume(streamInfo);
+            log.info("Retrieved data volume: {}", averageDataVolume);
+        } catch (Exception e) {
+            log.error("Error retrieving data volume: {}", e.getMessage(), e);
+            log.warn("Using default data volume: {}", 
DEFAULT_ERROR_DATA_VOLUME);
+            averageDataVolume = DEFAULT_ERROR_DATA_VOLUME;
+        }
+        int newParallelism = (int) (averageDataVolume / 
maximumMessagePerSecondPerCore);
+        // Ensure parallelism is at most MAX_PARALLELISM
+        newParallelism = Math.min(newParallelism, MAX_PARALLELISM);
+        log.info("Calculated parallelism: {} for data volume: {}", 
newParallelism, averageDataVolume);
+        return newParallelism;
+    }
+
+    /**
+     * Initialize maximum message per second per core based on configuration
+     *
+     * @param maximumMessagePerSecondPerCore The maximum messages per second 
per core
+     */
+    public void setMaximumMessagePerSecondPerCore(Integer 
maximumMessagePerSecondPerCore) {
+        if (maximumMessagePerSecondPerCore == null || 
maximumMessagePerSecondPerCore <= 0) {
+            log.error("Illegal flink.maxpercore property, must be nonnull and 
positive, using default value: {}",
+                    maximumMessagePerSecondPerCore);
+        } else {
+            this.maximumMessagePerSecondPerCore = 
maximumMessagePerSecondPerCore;
+        }
+    }
+
+    /**
+     * Get average data volume on the scale specified by 
DATA_SCALE_COUNTING_RANGE_IN_HOURS
+     *
+     * @param streamInfo inlong stream info
+     * @return The average data count per hour
+     */
+    private long getAverageDataVolume(InlongStreamInfo streamInfo) {
+        // Since the audit module uses local time, we need to use 
ZonedDateTime to get the current time
+        String dataTimeZone = 
streamInfo.getSourceList().get(0).getDataTimeZone();
+
+        // This regex pattern matches a time zone offset in the format of 
"GMT+/-X:00"
+        // where X is a single digit (e.g., "GMT+8:00"). The pattern captures 
the "+" or "-" sign
+        // and the single digit, then it replaces the single digit with two 
digits by adding a "0" in front of it.
+        // For example, "GMT+8:00" becomes "GMT+08:00" in order to match 
standard offset-based ZoneId.
+        dataTimeZone = dataTimeZone.replaceAll(TIMEZONE_REGEX, "$10$2:");
+        ZoneId dataZone = ZoneId.of(dataTimeZone);
+
+        ZonedDateTime endTime = ZonedDateTime.now(dataZone);
+        ZonedDateTime startTime = 
endTime.minusHours(DATA_SCALE_COUNTING_RANGE_IN_HOURS);
+        DateTimeFormatter formatter = 
DateTimeFormatter.ofPattern(AUDIT_QUERY_DATE_TIME_FORMAT);
+
+        // counting data volume on with DATA_PROXY_OUTPUT auditId
+        int auditId = AuditIdEnum.getAuditId(DEFAULT_AUDIT_TYPE, 
DEFAULT_FLOWTYPE).getValue();
+        StringJoiner urlParameters = new StringJoiner(AMPERSAND)
+                .add(PARAMS_START_TIME + EQUAL + startTime.format(formatter))
+                .add(PARAMS_END_TIME + EQUAL + endTime.format(formatter))
+                .add(PARAMS_INLONG_GROUP_Id + EQUAL + 
streamInfo.getInlongGroupId())

Review Comment:
   Please rename `PARAMS_INLONG_GROUP_Id` to `PARAMS_INLONG_GROUP_ID`.



-- 
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