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/07/31 13:46:12 UTC

[GitHub] [inlong] Greedyu opened a new pull request, #5301: [INLONG-4958][Agent] Feature unify the exposed metrics for Agent

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

   Feature unify the exposed metrics for Agent
   - Fixes #4958 
   
   ### Motivation
   1. Metric related classes are migrated to the agent-common module
   2. Change the isPrometheusEnabled() method to create the indicator processing method and change it to the class name specification
   3. Abstraction indicator processing


-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java:
##########
@@ -120,6 +120,6 @@ public void destroy() {
             queue.clear();
         }
         LOGGER.info("destroy channel, show memory channel metric:");
-        GlobalMetrics.showMemoryChannelStatics();
+        // GlobalMetrics.showMemoryChannelStatics();

Review Comment:
   rollback



-- 
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 commented on pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

Posted by GitBox <gi...@apache.org>.
dockerzhang commented on PR #5301:
URL: https://github.com/apache/inlong/pull/5301#issuecomment-1200917816

   @luchunliang @pocozh PTAL, thanks.


-- 
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] pocozh commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentPrometheusMetricHandler.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.agent.metrics;
+
+import io.prometheus.client.exporter.HTTPServer;
+import io.prometheus.client.hotspot.DefaultExports;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.metrics.global.PrometheusGlobalMetrics;
+import org.apache.inlong.agent.metrics.job.JobPrometheusMetrics;
+import org.apache.inlong.agent.metrics.task.TaskPrometheusMetrics;
+import org.apache.inlong.common.metric.MetricItemValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.AgentConstants.DEFAULT_PROMETHEUS_EXPORTER_PORT;
+import static org.apache.inlong.agent.constant.AgentConstants.PROMETHEUS_EXPORTER_PORT;
+
+/**
+ * prometheus metric handler
+ */
+public class AgentPrometheusMetricHandler extends AgentMetricBaseHandler {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentPrometheusMetricHandler.class);
+    private static HTTPServer metricsServer;
+
+    static {
+        DefaultExports.initialize();
+    }
+
+    public AgentPrometheusMetricHandler() {
+        jobMetrics = new JobPrometheusMetrics();
+        taskMetrics = new TaskPrometheusMetrics();
+        globalMetrics = new PrometheusGlobalMetrics();
+    }
+
+    @Override
+    public void snapshot(String domain, List<MetricItemValue> itemValues) {
+        // nothing
+    }
+
+    @Override
+    public void init() {
+        // starting metrics server
+        int metricsServerPort = AgentConfiguration.getAgentConf()
+                .getInt(PROMETHEUS_EXPORTER_PORT, DEFAULT_PROMETHEUS_EXPORTER_PORT);

Review Comment:
   > If there are multiple listeners later, it should be a listener object corresponding to a processing class, so even if there are multiple ports, each class initializes itself. Or that the listener will correspond to multiple ports. If this is the case, is it okay to wait until there is such a business and then modify it?
   
   Okay, let's improve it later.



-- 
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] healchow commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java:
##########
@@ -120,6 +120,6 @@ public void destroy() {
             queue.clear();
         }
         LOGGER.info("destroy channel, show memory channel metric:");
-        GlobalMetrics.showMemoryChannelStatics();
+        // GlobalMetrics.showMemoryChannelStatics();

Review Comment:
   Why not remove this line directly?



-- 
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] healchow commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseListener agentMetricBaseHandler;

Review Comment:
   Does it need `volatile` for `agentMetricBaseHandler`?



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseListener agentMetricBaseHandler;

Review Comment:
   It was added at the beginning, but the checkstyle is abnormal
   ![image](https://user-images.githubusercontent.com/20356765/182333107-82942099-3a8c-465f-b0a1-b0449d90f760.png)
   



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -112,14 +112,13 @@ agent.manager.auth.secretKey=
 
 
 ############################
-# prometheus config
+# metric config
 ############################
-# whether to enable prometheus
-agent.prometheus.enable=true
+metricDomains=Agent

Review Comment:
   has been deleted



##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseHandler agentMetricBaseHandler;
+
+    private AgentMetricSingleton() {
+    }
+
+    public static AgentMetricBaseHandler getAgentMetricHandler() {
+        if (agentMetricBaseHandler == null) {
+            synchronized (AgentJmxMetricHandler.class) {
+                if (agentMetricBaseHandler == null) {
+                    agentMetricBaseHandler = getAgentMetricByConf();
+                    agentMetricBaseHandler.init();
+                }
+            }
+        }
+        return agentMetricBaseHandler;
+    }
+
+    private static AgentMetricBaseHandler getAgentMetricByConf() {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        try {
+            Class<?> handlerClass = ClassUtils
+                    .getClass(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT));
+            Object handlerObject = handlerClass.getDeclaredConstructor().newInstance();
+            //
+            // final MetricListener listener = (MetricListener) listenerObject;

Review Comment:
   resolve



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentPrometheusMetricHandler.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.agent.metrics;
+
+import io.prometheus.client.exporter.HTTPServer;
+import io.prometheus.client.hotspot.DefaultExports;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.metrics.global.PrometheusGlobalMetrics;
+import org.apache.inlong.agent.metrics.job.JobPrometheusMetrics;
+import org.apache.inlong.agent.metrics.task.TaskPrometheusMetrics;
+import org.apache.inlong.common.metric.MetricItemValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.AgentConstants.DEFAULT_PROMETHEUS_EXPORTER_PORT;
+import static org.apache.inlong.agent.constant.AgentConstants.PROMETHEUS_EXPORTER_PORT;
+
+/**
+ * prometheus metric handler
+ */
+public class AgentPrometheusMetricHandler extends AgentMetricBaseHandler {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentPrometheusMetricHandler.class);
+    private static HTTPServer metricsServer;
+
+    static {
+        DefaultExports.initialize();
+    }
+
+    public AgentPrometheusMetricHandler() {
+        jobMetrics = new JobPrometheusMetrics();
+        taskMetrics = new TaskPrometheusMetrics();
+        globalMetrics = new PrometheusGlobalMetrics();
+    }
+
+    @Override
+    public void snapshot(String domain, List<MetricItemValue> itemValues) {
+        // nothing
+    }
+
+    @Override
+    public void init() {
+        // starting metrics server
+        int metricsServerPort = AgentConfiguration.getAgentConf()
+                .getInt(PROMETHEUS_EXPORTER_PORT, DEFAULT_PROMETHEUS_EXPORTER_PORT);

Review Comment:
   If there are multiple listeners later, it should be a listener object corresponding to a processing class, so even if there are multiple ports, each class initializes itself.
   Or that the listener will correspond to multiple ports. If this is the case, is it okay to wait until there is such a business and then modify it?



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseListener agentMetricBaseHandler;

Review Comment:
   resolve



-- 
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 commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -108,18 +108,12 @@ agent.manager.vip.http.host=127.0.0.1
 agent.manager.vip.http.port=8083
 agent.manager.auth.secretId=
 agent.manager.auth.secretKey=
-
-
-
 ############################
-# prometheus config
+# metric config
 ############################
-# whether to enable prometheus
-agent.prometheus.enable=true
-# prometheus exporter server default port
+agent.domainListeners=org.apache.inlong.agent.metrics.AgentPrometheusMetricListener

Review Comment:
   please list all available listeners in the annotation.



-- 
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 commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -112,14 +112,13 @@ agent.manager.auth.secretKey=
 
 
 ############################
-# prometheus config
+# metric config
 ############################
-# whether to enable prometheus
-agent.prometheus.enable=true
+metricDomains=Agent
+metricDomains.Agent.domainListeners=org.apache.inlong.agent.metrics.AgentPrometheusMetricHandler

Review Comment:
   `agent.prometheus.enable` be removed. It's better to remove all related codes.



-- 
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 commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -112,14 +112,13 @@ agent.manager.auth.secretKey=
 
 
 ############################
-# prometheus config
+# metric config
 ############################
-# whether to enable prometheus
-agent.prometheus.enable=true
+metricDomains=Agent

Review Comment:
    it's designed for `agent`, maybe the `metricDomains` param is needless? 



##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java:
##########
@@ -54,12 +54,12 @@ public void push(Message message) {
                 if (message instanceof ProxyMessage) {
                     groupId = ((ProxyMessage) message).getInlongGroupId();
                 }
-                GlobalMetrics.incReadNum(groupId);
+                AgentMetricSingleton.getAgentMetricHandler().globalMetrics.incReadNum(groupId);

Review Comment:
   it's better to add a global variate for `AgentMetricSingleton.getAgentMetricHandler().globalMetrics`.



##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseHandler agentMetricBaseHandler;
+
+    private AgentMetricSingleton() {
+    }
+
+    public static AgentMetricBaseHandler getAgentMetricHandler() {
+        if (agentMetricBaseHandler == null) {
+            synchronized (AgentJmxMetricHandler.class) {
+                if (agentMetricBaseHandler == null) {
+                    agentMetricBaseHandler = getAgentMetricByConf();
+                    agentMetricBaseHandler.init();
+                }
+            }
+        }
+        return agentMetricBaseHandler;
+    }
+
+    private static AgentMetricBaseHandler getAgentMetricByConf() {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        try {
+            Class<?> handlerClass = ClassUtils
+                    .getClass(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT));
+            Object handlerObject = handlerClass.getDeclaredConstructor().newInstance();
+            //
+            // final MetricListener listener = (MetricListener) listenerObject;

Review Comment:
   Remove commented code?



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseHandler agentMetricBaseHandler;
+
+    private AgentMetricSingleton() {
+    }
+
+    public static AgentMetricBaseHandler getAgentMetricHandler() {
+        if (agentMetricBaseHandler == null) {
+            synchronized (AgentJmxMetricHandler.class) {
+                if (agentMetricBaseHandler == null) {
+                    agentMetricBaseHandler = getAgentMetricByConf();
+                    agentMetricBaseHandler.init();
+                }
+            }
+        }
+        return agentMetricBaseHandler;
+    }
+
+    private static AgentMetricBaseHandler getAgentMetricByConf() {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        try {
+            Class<?> handlerClass = ClassUtils
+                    .getClass(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT));
+            Object handlerObject = handlerClass.getDeclaredConstructor().newInstance();
+            //
+            // final MetricListener listener = (MetricListener) listenerObject;
+            // Constructor<?> constructor =
+            //         Class.forName(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT))
+            //                 .getDeclaredConstructor(AgentMetricBaseHandler.class);
+            // constructor.setAccessible(true);
+            return (AgentMetricBaseHandler) handlerObject;
+        } catch (Exception ex) {
+            LOGGER.error("cannot find AgentMetricBaseHandler, {}", ex.getMessage());
+        }
+        return null;
+    }
+
+    public static void init() {
+        getAgentMetricHandler();

Review Comment:
   But this init() method is not AgentMetricSingleton, but the AgentMetricHandler class
   However, the init() here is a bit redundant and can also be removed



-- 
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] luchunliang commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentPrometheusMetricHandler.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.agent.metrics;
+
+import io.prometheus.client.exporter.HTTPServer;
+import io.prometheus.client.hotspot.DefaultExports;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.metrics.global.PrometheusGlobalMetrics;
+import org.apache.inlong.agent.metrics.job.JobPrometheusMetrics;
+import org.apache.inlong.agent.metrics.task.TaskPrometheusMetrics;
+import org.apache.inlong.common.metric.MetricItemValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.AgentConstants.DEFAULT_PROMETHEUS_EXPORTER_PORT;
+import static org.apache.inlong.agent.constant.AgentConstants.PROMETHEUS_EXPORTER_PORT;
+
+/**
+ * prometheus metric handler
+ */
+public class AgentPrometheusMetricHandler extends AgentMetricBaseHandler {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentPrometheusMetricHandler.class);
+    private static HTTPServer metricsServer;
+
+    static {
+        DefaultExports.initialize();
+    }
+
+    public AgentPrometheusMetricHandler() {
+        jobMetrics = new JobPrometheusMetrics();
+        taskMetrics = new TaskPrometheusMetrics();
+        globalMetrics = new PrometheusGlobalMetrics();
+    }
+
+    @Override
+    public void snapshot(String domain, List<MetricItemValue> itemValues) {
+        // nothing

Review Comment:
   How to snapshot data and report data to metricsServer?



##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseHandler agentMetricBaseHandler;
+
+    private AgentMetricSingleton() {
+    }
+
+    public static AgentMetricBaseHandler getAgentMetricHandler() {
+        if (agentMetricBaseHandler == null) {
+            synchronized (AgentJmxMetricHandler.class) {
+                if (agentMetricBaseHandler == null) {
+                    agentMetricBaseHandler = getAgentMetricByConf();
+                    agentMetricBaseHandler.init();
+                }
+            }
+        }
+        return agentMetricBaseHandler;
+    }
+
+    private static AgentMetricBaseHandler getAgentMetricByConf() {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        try {
+            Class<?> handlerClass = ClassUtils
+                    .getClass(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT));
+            Object handlerObject = handlerClass.getDeclaredConstructor().newInstance();
+            //
+            // final MetricListener listener = (MetricListener) listenerObject;
+            // Constructor<?> constructor =
+            //         Class.forName(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT))
+            //                 .getDeclaredConstructor(AgentMetricBaseHandler.class);
+            // constructor.setAccessible(true);
+            return (AgentMetricBaseHandler) handlerObject;
+        } catch (Exception ex) {
+            LOGGER.error("cannot find AgentMetricBaseHandler, {}", ex.getMessage());
+        }
+        return null;
+    }
+
+    public static void init() {
+        getAgentMetricHandler();

Review Comment:
   The method "init" invoke the method "getAgentMetricHandler".
   The method "getAgentMetricHandler" invoke the method "init".



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -108,18 +108,12 @@ agent.manager.vip.http.host=127.0.0.1
 agent.manager.vip.http.port=8083
 agent.manager.auth.secretId=
 agent.manager.auth.secretKey=
-
-
-
 ############################
-# prometheus config
+# metric config
 ############################
-# whether to enable prometheus
-agent.prometheus.enable=true
-# prometheus exporter server default port
+agent.domainListeners=org.apache.inlong.agent.metrics.AgentPrometheusMetricListener

Review Comment:
   resolve



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -14,9 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #

Review Comment:
   resolve



-- 
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] healchow commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/conf/agent.properties:
##########
@@ -14,9 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #

Review Comment:
   It is recommended not to format the properties file. Appropriate spaces can increase the readability of the file.



-- 
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 #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


-- 
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] pocozh commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestTextFileReader.java:
##########
@@ -56,7 +53,7 @@
 import static org.apache.inlong.agent.constant.JobConstants.JOB_FILE_MAX_WAIT;
 import static org.apache.inlong.agent.constant.JobConstants.JOB_INSTANCE_ID;
 
-@RunWith(PowerMockRunner.class)
+// @RunWith(PowerMockRunner.class)

Review Comment:
   please remove unused code



##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseHandler agentMetricBaseHandler;
+
+    private AgentMetricSingleton() {
+    }
+
+    public static AgentMetricBaseHandler getAgentMetricHandler() {
+        if (agentMetricBaseHandler == null) {
+            synchronized (AgentJmxMetricHandler.class) {
+                if (agentMetricBaseHandler == null) {
+                    agentMetricBaseHandler = getAgentMetricByConf();
+                    agentMetricBaseHandler.init();
+                }
+            }
+        }
+        return agentMetricBaseHandler;
+    }
+
+    private static AgentMetricBaseHandler getAgentMetricByConf() {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        try {
+            Class<?> handlerClass = ClassUtils
+                    .getClass(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT));
+            Object handlerObject = handlerClass.getDeclaredConstructor().newInstance();
+            //
+            // final MetricListener listener = (MetricListener) listenerObject;
+            // Constructor<?> constructor =
+            //         Class.forName(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT))
+            //                 .getDeclaredConstructor(AgentMetricBaseHandler.class);
+            // constructor.setAccessible(true);

Review Comment:
   please remove unused code



##########
inlong-agent/conf/agent.properties:
##########
@@ -112,14 +112,13 @@ agent.manager.auth.secretKey=
 
 
 ############################
-# prometheus config
+# metric config
 ############################
-# whether to enable prometheus
-agent.prometheus.enable=true
+metricDomains=Agent
+metricDomains.Agent.domainListeners=org.apache.inlong.agent.metrics.AgentPrometheusMetricHandler

Review Comment:
   Would it be more intuitive using `xxxxListener` , such as `AgentPrometheusMetricListener`?



##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentPrometheusMetricHandler.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.agent.metrics;
+
+import io.prometheus.client.exporter.HTTPServer;
+import io.prometheus.client.hotspot.DefaultExports;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.metrics.global.PrometheusGlobalMetrics;
+import org.apache.inlong.agent.metrics.job.JobPrometheusMetrics;
+import org.apache.inlong.agent.metrics.task.TaskPrometheusMetrics;
+import org.apache.inlong.common.metric.MetricItemValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.AgentConstants.DEFAULT_PROMETHEUS_EXPORTER_PORT;
+import static org.apache.inlong.agent.constant.AgentConstants.PROMETHEUS_EXPORTER_PORT;
+
+/**
+ * prometheus metric handler
+ */
+public class AgentPrometheusMetricHandler extends AgentMetricBaseHandler {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentPrometheusMetricHandler.class);
+    private static HTTPServer metricsServer;
+
+    static {
+        DefaultExports.initialize();
+    }
+
+    public AgentPrometheusMetricHandler() {
+        jobMetrics = new JobPrometheusMetrics();
+        taskMetrics = new TaskPrometheusMetrics();
+        globalMetrics = new PrometheusGlobalMetrics();
+    }
+
+    @Override
+    public void snapshot(String domain, List<MetricItemValue> itemValues) {
+        // nothing
+    }
+
+    @Override
+    public void init() {
+        // starting metrics server
+        int metricsServerPort = AgentConfiguration.getAgentConf()
+                .getInt(PROMETHEUS_EXPORTER_PORT, DEFAULT_PROMETHEUS_EXPORTER_PORT);

Review Comment:
   This can't handle the scenario where there are multi kinds of listeners in config file. Could you extend it?



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentPrometheusMetricHandler.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.agent.metrics;
+
+import io.prometheus.client.exporter.HTTPServer;
+import io.prometheus.client.hotspot.DefaultExports;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.apache.inlong.agent.metrics.global.PrometheusGlobalMetrics;
+import org.apache.inlong.agent.metrics.job.JobPrometheusMetrics;
+import org.apache.inlong.agent.metrics.task.TaskPrometheusMetrics;
+import org.apache.inlong.common.metric.MetricItemValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.inlong.agent.constant.AgentConstants.DEFAULT_PROMETHEUS_EXPORTER_PORT;
+import static org.apache.inlong.agent.constant.AgentConstants.PROMETHEUS_EXPORTER_PORT;
+
+/**
+ * prometheus metric handler
+ */
+public class AgentPrometheusMetricHandler extends AgentMetricBaseHandler {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentPrometheusMetricHandler.class);
+    private static HTTPServer metricsServer;
+
+    static {
+        DefaultExports.initialize();
+    }
+
+    public AgentPrometheusMetricHandler() {
+        jobMetrics = new JobPrometheusMetrics();
+        taskMetrics = new TaskPrometheusMetrics();
+        globalMetrics = new PrometheusGlobalMetrics();
+    }
+
+    @Override
+    public void snapshot(String domain, List<MetricItemValue> itemValues) {
+        // nothing

Review Comment:
   There are several modes for prometheus registration, such as collector and Gauge. The agent used Gauge before. There are registrations under AgentPrometheusMetricHandler.
   <img width="456" alt="企业微信截图_56982797-03c4-434f-9640-b0a8252d4910" src="https://user-images.githubusercontent.com/20356765/182146332-bc8b622e-9c5e-4aa4-aedc-a3af15589d77.png">
   <img width="624" alt="企业微信截图_5fb68c05-359f-412e-aeac-f187d00483a3" src="https://user-images.githubusercontent.com/20356765/182146348-51752520-dd6a-406d-a33e-de57f9ef7191.png">
   



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/AgentMetricSingleton.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.agent.metrics;
+
+import org.apache.commons.lang3.ClassUtils;
+import org.apache.inlong.agent.conf.AgentConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS;
+import static org.apache.inlong.agent.constant.AgentConstants.AGENT_METRIC_LISTENER_CLASS_DEFAULT;
+
+/**
+ * metric singleton
+ */
+public class AgentMetricSingleton {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AgentMetricSingleton.class);
+    private static AgentMetricBaseHandler agentMetricBaseHandler;
+
+    private AgentMetricSingleton() {
+    }
+
+    public static AgentMetricBaseHandler getAgentMetricHandler() {
+        if (agentMetricBaseHandler == null) {
+            synchronized (AgentJmxMetricHandler.class) {
+                if (agentMetricBaseHandler == null) {
+                    agentMetricBaseHandler = getAgentMetricByConf();
+                    agentMetricBaseHandler.init();
+                }
+            }
+        }
+        return agentMetricBaseHandler;
+    }
+
+    private static AgentMetricBaseHandler getAgentMetricByConf() {
+        AgentConfiguration conf = AgentConfiguration.getAgentConf();
+        try {
+            Class<?> handlerClass = ClassUtils
+                    .getClass(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT));
+            Object handlerObject = handlerClass.getDeclaredConstructor().newInstance();
+            //
+            // final MetricListener listener = (MetricListener) listenerObject;
+            // Constructor<?> constructor =
+            //         Class.forName(conf.get(AGENT_METRIC_LISTENER_CLASS, AGENT_METRIC_LISTENER_CLASS_DEFAULT))
+            //                 .getDeclaredConstructor(AgentMetricBaseHandler.class);
+            // constructor.setAccessible(true);

Review Comment:
   resolve



##########
inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestTextFileReader.java:
##########
@@ -56,7 +53,7 @@
 import static org.apache.inlong.agent.constant.JobConstants.JOB_FILE_MAX_WAIT;
 import static org.apache.inlong.agent.constant.JobConstants.JOB_INSTANCE_ID;
 
-@RunWith(PowerMockRunner.class)
+// @RunWith(PowerMockRunner.class)

Review Comment:
   has been deleted



-- 
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] Greedyu commented on a diff in pull request #5301: [INLONG-4958][Agent] Unify the exposed metrics for the Agent

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


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/channel/MemoryChannel.java:
##########
@@ -54,12 +54,12 @@ public void push(Message message) {
                 if (message instanceof ProxyMessage) {
                     groupId = ((ProxyMessage) message).getInlongGroupId();
                 }
-                GlobalMetrics.incReadNum(groupId);
+                AgentMetricSingleton.getAgentMetricHandler().globalMetrics.incReadNum(groupId);

Review Comment:
   Added AgentConstants.GLOBAL_METRICS global field



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