You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@linkis.apache.org by ca...@apache.org on 2022/06/08 03:47:04 UTC

[incubator-linkis] branch dev-1.1.3 updated: Enable the prometheus for monitoring (#2069)

This is an automated email from the ASF dual-hosted git repository.

casion pushed a commit to branch dev-1.1.3
in repository https://gitbox.apache.org/repos/asf/incubator-linkis.git


The following commit(s) were added to refs/heads/dev-1.1.3 by this push:
     new 2e9813e0d Enable the prometheus for monitoring (#2069)
2e9813e0d is described below

commit 2e9813e0d7d9ad036d962ea707b88a624a67ba77
Author: Leomax_Sun <28...@qq.com>
AuthorDate: Wed Jun 8 11:47:00 2022 +0800

    Enable the prometheus for monitoring (#2069)
    
    
    * Add codecov git action to track test coverage metrics (#1945)
    
    * enable prometheus monitoring
    * Enable to specify the port range of engine conn(EC)
    * Enable prometheus in install script
    
    Co-authored-by: Casion <ca...@gmail.com>
    Co-authored-by: husofskyzy <15...@163.com>
    Co-authored-by: peacewong <wp...@gmail.com>
---
 assembly-combined-package/bin/install.sh           | 12 ++++
 .../deploy-config/linkis-env.sh                    |  3 +
 .../apache/linkis/common/conf/Configuration.scala  |  4 ++
 linkis-commons/linkis-module/pom.xml               |  5 ++
 .../linkis/server/utils/LinkisMainHelper.scala     | 12 +++-
 .../common/conf/GovernaceCommonConf.scala          |  4 +-
 .../linkis-engineconn-manager-core/pom.xml         |  5 ++
 .../ecm/core/launch/ProcessEngineConnLaunch.scala  | 44 ++++++++-------
 .../apache/linkis/ecm/core/utils/PortUtils.scala   | 64 ++++++++++++++++++++++
 .../linkis/ecm/core/utils/PortUtilsTest.scala      | 39 +++++++++++++
 .../linkis-service-discovery/linkis-eureka/pom.xml | 18 ++++++
 tool/dependencies/known-dependencies.txt           |  5 +-
 12 files changed, 189 insertions(+), 26 deletions(-)

diff --git a/assembly-combined-package/bin/install.sh b/assembly-combined-package/bin/install.sh
index 3a2b13b82..bcc33d0b3 100644
--- a/assembly-combined-package/bin/install.sh
+++ b/assembly-combined-package/bin/install.sh
@@ -400,6 +400,18 @@ then
   sed -i ${txt}  "s#spring.server.port.*#spring.server.port=$CS_PORT#g" $cs_conf
 fi
 
+##Eanble prometheus for monitoring
+if [ "true" == "$PROMETHEUS_ENABLE" ]
+then
+  echo "prometheus is enabled"
+  sed -i ${txt} '$a \wds.linkis.prometheus.enable={{ PROMETHEUS_ENABLE }}' $LINKIS_HOME/conf/linkis.properties
+  sed -i ${txt} '$a \wds.linkis.server.user.restful.uri.pass.auth=/actuator/prometheus,' $LINKIS_HOME/conf/linkis.properties
+  sed -i ${txt}  '/eureka:/a \\  instance:\n    metadata-map:\n      prometheus.path: ${prometheus.path:${prometheus.endpoint}}' $LINKIS_HOME/conf/application-linkis.yml
+  sed -i ${txt}  's#include: refresh,info#include: refresh,info,health,metrics,prometheus#g' $LINKIS_HOME/conf/application-linkis.yml
+  sed -i ${txt} '/instance:/a \    metadata-map:\n      prometheus.path: ${prometheus.path:/actuator/prometheus}' $LINKIS_HOME/conf/application-eureka.yml
+  sed -i ${txt} '$a \\nmanagement:\n  endpoints:\n    web:\n      exposure:\n        include: refresh,info,health,metrics,prometheus' $LINKIS_HOME/conf/application-eureka.yml
+fi
+
 echo "preveliges linkis command shells"
 sudo chmod -R 777 $LINKIS_HOME/bin/*
 sudo chmod -R 777 $LINKIS_HOME/sbin/*
diff --git a/assembly-combined-package/deploy-config/linkis-env.sh b/assembly-combined-package/deploy-config/linkis-env.sh
index ac39c2c93..8c8b1c002 100644
--- a/assembly-combined-package/deploy-config/linkis-env.sh
+++ b/assembly-combined-package/deploy-config/linkis-env.sh
@@ -148,6 +148,9 @@ LINKIS_PUBLIC_MODULE=lib/linkis-commons/public-module
 ## If SKYWALKING_AGENT_PATH is set, the Linkis components will be started with Skywalking agent
 #SKYWALKING_AGENT_PATH=/appcom/config/skywalking-agent/skywalking-agent.jar
 
+##If you want to enable prometheus for monitoring linkis, you can set this export PROMETHEUS_ENABLE=true
+export PROMETHEUS_ENABLE=false
+
 #If you want to start metadata related microservices, you can set this export ENABLE_METADATA_MANAGE=true
 export ENABLE_METADATA_MANAGER=false
 
diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/conf/Configuration.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/conf/Configuration.scala
index 65ba7536d..2ba403b56 100644
--- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/conf/Configuration.scala
+++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/conf/Configuration.scala
@@ -30,6 +30,10 @@ object Configuration extends Logging {
 
   val IS_TEST_MODE = CommonVars("wds.linkis.test.mode", false)
 
+  val IS_PROMETHEUS_ENABLE = CommonVars("wds.linkis.prometheus.enable", false)
+
+  val PROMETHEUS_ENDPOINT = CommonVars("wds.linkis.prometheus.endpoint", "/actuator/prometheus")
+
   val LINKIS_HOME = CommonVars("wds.linkis.home", CommonVars("LINKIS_HOME", "/tmp").getValue)
 
   val GATEWAY_URL: CommonVars[String] = CommonVars[String]("wds.linkis.gateway.url", "http://127.0.0.1:9001/")
diff --git a/linkis-commons/linkis-module/pom.xml b/linkis-commons/linkis-module/pom.xml
index 79788b799..2c9139d26 100644
--- a/linkis-commons/linkis-module/pom.xml
+++ b/linkis-commons/linkis-module/pom.xml
@@ -187,6 +187,11 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>io.micrometer</groupId>
+            <artifactId>micrometer-registry-prometheus</artifactId>
+            <scope>compile</scope>
+        </dependency>
 
         <dependency>
             <groupId>org.springframework.cloud</groupId>
diff --git a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/utils/LinkisMainHelper.scala b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/utils/LinkisMainHelper.scala
index 44d5c8e7c..d1aadff79 100644
--- a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/utils/LinkisMainHelper.scala
+++ b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/utils/LinkisMainHelper.scala
@@ -17,7 +17,7 @@
  
 package org.apache.linkis.server.utils
 
-import org.apache.linkis.common.conf.CommonVars
+import org.apache.linkis.common.conf.{CommonVars, Configuration}
 import org.apache.linkis.server.conf.ServerConfiguration
 
 import scala.collection.JavaConverters._
@@ -45,12 +45,18 @@ object LinkisMainHelper {
   }
 
   def getExtraSpringOptions(profilesName: String): Array[String] = {
-   val servletPath=ServerConfiguration.BDP_SERVER_RESTFUL_URI.getValue
-    s"--spring.profiles.active=$profilesName"+:s"--spring.mvc.servlet.path=$servletPath" +: CommonVars.properties.asScala.filter { case (k, v) => k != null && k.startsWith(SPRING_STAR)}
+    val servletPath = ServerConfiguration.BDP_SERVER_RESTFUL_URI.getValue
+    var resArr = s"--spring.profiles.active=$profilesName" +: s"--spring.mvc.servlet.path=$servletPath" +: CommonVars.properties.asScala.filter { case (k, v) => k != null && k.startsWith(SPRING_STAR) }
       .map { case (k, v) =>
         val realKey = k.substring(SPRING_STAR.length)
         s"--$realKey=$v"
       }.toArray
+    if (Configuration.IS_PROMETHEUS_ENABLE.getValue) {
+      var prometheusEndpoint = Configuration.PROMETHEUS_ENDPOINT.getValue
+      if (ServerConfiguration.IS_GATEWAY.getValue.equals("false")) prometheusEndpoint = servletPath + prometheusEndpoint
+      resArr = resArr :+ s"--prometheus.endpoint=$prometheusEndpoint"
+    }
+    return resArr
   }
 
 }
diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/conf/GovernaceCommonConf.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/conf/GovernaceCommonConf.scala
index 4182a96f8..24c888486 100644
--- a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/conf/GovernaceCommonConf.scala
+++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/conf/GovernaceCommonConf.scala
@@ -39,6 +39,8 @@ object GovernanceCommonConf {
 
   val ENGINE_CONN_MANAGER_SPRING_NAME = CommonVars("wds.linkis.engineconn.manager.name", "linkis-cg-engineconnmanager")
 
+  val ENGINE_CONN_PORT_RANGE = CommonVars("wds.linkis.engineconn.port.range", "-")
+
   val MANAGER_SPRING_NAME = CommonVars("wds.linkis.engineconn.manager.name", "linkis-cg-linkismanager")
 
   val ENTRANCE_SPRING_NAME = CommonVars("wds.linkis.entrance.name", "linkis-cg-entrance")
@@ -49,7 +51,7 @@ object GovernanceCommonConf {
 
   val ENGINECONN_ENVKEYS = CommonVars("wds.linkis.engineconn.env.keys", "").getValue
 
-  def getEngineEnvValue(envKey:String): String = {
+  def getEngineEnvValue(envKey: String): String = {
     CommonVars(envKey, "").getValue
   }
 }
diff --git a/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/pom.xml b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/pom.xml
index 4b3f3abf9..46003b250 100644
--- a/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/pom.xml
+++ b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/pom.xml
@@ -30,6 +30,11 @@
     <artifactId>linkis-engineconn-manager-core</artifactId>
 
     <dependencies>
+        <dependency>
+            <groupId>org.apache.linkis</groupId>
+            <artifactId>linkis-module</artifactId>
+            <version>${linkis.version}</version>
+        </dependency>
 
         <dependency>
             <groupId>org.apache.linkis</groupId>
diff --git a/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/launch/ProcessEngineConnLaunch.scala b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/launch/ProcessEngineConnLaunch.scala
index 7c12d2ab0..7f5519ab8 100644
--- a/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/launch/ProcessEngineConnLaunch.scala
+++ b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/launch/ProcessEngineConnLaunch.scala
@@ -17,7 +17,7 @@
  
 package org.apache.linkis.ecm.core.launch
 
-import java.io.{File, InputStream, OutputStream}
+import java.io.{File, IOException, InputStream, OutputStream}
 import java.net.ServerSocket
 import org.apache.linkis.common.conf.{CommonVars, Configuration}
 import org.apache.linkis.common.exception.ErrorException
@@ -33,6 +33,8 @@ import org.apache.linkis.manager.engineplugin.common.launch.process.LaunchConsta
 import org.apache.linkis.manager.engineplugin.common.launch.process.{Environment, ProcessEngineConnLaunchRequest}
 import org.apache.commons.io.{FileUtils, IOUtils}
 import org.apache.commons.lang.StringUtils
+import org.apache.linkis.ecm.core.utils.PortUtils
+import org.apache.linkis.server.conf.ServerConfiguration
 
 import scala.collection.JavaConversions._
 
@@ -87,23 +89,18 @@ trait ProcessEngineConnLaunch extends EngineConnLaunch with Logging {
       case HADOOP_CONF_DIR => putIfExists(HADOOP_CONF_DIR)
       case HIVE_CONF_DIR => putIfExists(HIVE_CONF_DIR)
       case JAVA_HOME => putIfExists(JAVA_HOME)
-      case RANDOM_PORT => environment.put(RANDOM_PORT.toString, findAvailPort().toString)
+      case RANDOM_PORT => environment.put(RANDOM_PORT.toString, PortUtils.findAvailPort().toString)
       case EUREKA_PREFER_IP => environment.put(EUREKA_PREFER_IP.toString, Configuration.EUREKA_PREFER_IP.toString)
       case ENGINECONN_ENVKEYS => environment.put(ENGINECONN_ENVKEYS.toString, GovernanceCommonConf.ENGINECONN_ENVKEYS.toString)
       case _ =>
     }
   }
 
-  private def findAvailPort(): Int = {
-    val socket = new ServerSocket(0)
-    Utils.tryFinally(socket.getLocalPort)(IOUtils.closeQuietly(socket))
-  }
-
   private def setMoreAvailPort(value: String): Unit = {
     val key = RANDOM_PORT.toString + randomPortNum
     // TODO just replace it by sorted RANDOM_PORT, since only one RANDOM_PORT is used now.
     if(value.contains(key)) {
-      processBuilder.setEnv(key, findAvailPort().toString)
+      processBuilder.setEnv(key, PortUtils.findAvailPort().toString)
       randomPortNum += 1
     }
   }
@@ -137,18 +134,23 @@ trait ProcessEngineConnLaunch extends EngineConnLaunch with Logging {
     if (request.creationDesc.properties.exists { case (k, v) => k.contains(" ") || (v != null && v.contains(" ")) })
       throw new ErrorException(30000, "Startup parameters contain spaces!(启动参数中包含空格!)") //TODO exception
     val arguments = EngineConnArgumentsBuilder.newBuilder()
-    engineConnPort = findAvailPort().toString
+    engineConnPort = PortUtils.findAvailPortByRange(GovernanceCommonConf.ENGINE_CONN_PORT_RANGE.getValue).toString
     var springConf = Map("spring.application.name" -> GovernanceCommonConf.ENGINE_CONN_SPRING_NAME.getValue,
       "server.port" -> engineConnPort, "spring.profiles.active" -> "engineconn",
       "logging.config" -> s"classpath:${EnvConfiguration.LOG4J2_XML_FILE.getValue}") ++: discoveryMsgGenerator.generate(engineConnManagerEnv)
 
     val eurekaPreferIp: Boolean = Configuration.EUREKA_PREFER_IP
     logger.info(s"EUREKA_PREFER_IP: " + eurekaPreferIp)
-    if(eurekaPreferIp){
+    if (eurekaPreferIp) {
       springConf = springConf + ("eureka.instance.prefer-ip-address" -> "true")
       springConf = springConf + ("eureka.instance.instance-id" -> "\\${spring.cloud.client.ip-address}:\\${spring.application.name}:\\${server.port}")
     }
-
+    if (Configuration.IS_PROMETHEUS_ENABLE.getValue) {
+      logger.info(s"IS_PROMETHEUS_ENABLE: true")
+      springConf = springConf + ("management.endpoints.web.exposure.include" -> "refresh,info,health,metrics,prometheus")
+      val endpoint = ServerConfiguration.BDP_SERVER_RESTFUL_URI.getValue + Configuration.PROMETHEUS_ENDPOINT.getValue
+      springConf = springConf + ("eureka.instance.metadata-map.prometheus.path" -> ("\\${prometheus.path:" + endpoint + "}"))
+    }
     request.creationDesc.properties.filter(_._1.startsWith("spring.")).foreach { case (k, v) =>
       springConf = springConf += (k -> v)
     }
@@ -161,15 +163,15 @@ trait ProcessEngineConnLaunch extends EngineConnLaunch with Logging {
   }
 
   override def kill(): Unit = {
-    if(process != null){
+    if (process != null) {
       process.destroy()
     }
   }
 
   override def isAlive: Boolean = {
-    if(process != null){
+    if (process != null) {
       process.isAlive
-    }else{
+    } else {
       false
     }
   }
@@ -179,7 +181,7 @@ trait ProcessEngineConnLaunch extends EngineConnLaunch with Logging {
     initializeEnv()
     //TODO env需要考虑顺序问题
     val classPath = request.environment.remove(CLASSPATH.toString)
-    request.environment.foreach{ case (k, v) =>
+    request.environment.foreach { case (k, v) =>
       val value = v.replaceAll(CLASS_PATH_SEPARATOR, File.pathSeparator)
       setMoreAvailPort(value)
       processBuilder.setEnv(k, processBuilder.replaceExpansionMarker(value))
@@ -191,7 +193,7 @@ trait ProcessEngineConnLaunch extends EngineConnLaunch with Logging {
     //set other env
     val engineConnEnvKeyArray = engineConnEnvKeys.split(",")
     engineConnEnvKeyArray.foreach(envKey => {
-      if(null != envKey && !"".equals(envKey.trim)) {
+      if (null != envKey && !"".equals(envKey.trim)) {
         processBuilder.setEnv(envKey, GovernanceCommonConf.getEngineEnvValue(envKey))
       }
     })
@@ -213,17 +215,17 @@ trait ProcessEngineConnLaunch extends EngineConnLaunch with Logging {
   protected def getPreparedExecFile: String = preparedExecFile
 
   def getProcessInputStream: InputStream = {
-    if(process != null){
+    if (process != null) {
       process.getInputStream
-    }else{
+    } else {
       throw new ECMCoreException(ECMErrorCode.PROCESS_WAITFOR_ERROR, "process is not be launch, can not get InputStream!")
     }
   }
 
-  def processWaitFor:Int = {
-    if(process != null){
+  def processWaitFor: Int = {
+    if (process != null) {
       process.waitFor
-    }else{
+    } else {
       throw new ECMCoreException(ECMErrorCode.PROCESS_WAITFOR_ERROR, "process is not be launch, can not get terminated code by wait!")
     }
   }
diff --git a/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/utils/PortUtils.scala b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/utils/PortUtils.scala
new file mode 100644
index 000000000..3e9030620
--- /dev/null
+++ b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/main/scala/org/apache/linkis/ecm/core/utils/PortUtils.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.linkis.ecm.core.utils
+
+import org.apache.commons.io.IOUtils
+import org.apache.commons.lang.StringUtils
+import org.apache.linkis.common.utils.Utils
+
+import java.io.IOException
+import java.net.ServerSocket
+
+object PortUtils {
+
+  /**
+   * portRange: '-' is the separator
+   *
+   * @return
+   */
+  def findAvailPortByRange(portRange: String): Int = {
+    val separator = "-"
+    if (StringUtils.isEmpty(portRange) || portRange.equals(separator)) {
+      this.findAvailPort()
+    } else {
+      // valid user registered port range: 1024-65535
+      // refer https://en.wikipedia.org/wiki/Registered_port
+      var (start, end) = (1024, 65535)
+      val ports = portRange.split(separator, -1)
+      if (!ports(0).isEmpty) start = ports(0).toInt
+      if (!ports(1).isEmpty) end = ports(1).toInt
+      val availablePort = start until end find {
+        port =>
+          try {
+            new ServerSocket(port).close()
+            true
+          } catch {
+            case ioe: IOException => false
+          }
+      }
+      availablePort.getOrElse(throw new IOException("No available port in the portRange: " + portRange))
+    }
+  }
+
+  def findAvailPort(): Int = {
+    val socket = new ServerSocket(0)
+    Utils.tryFinally(socket.getLocalPort)(IOUtils.closeQuietly(socket))
+  }
+
+
+}
diff --git a/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/test/scala/org/apache/linkis/ecm/core/utils/PortUtilsTest.scala b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/test/scala/org/apache/linkis/ecm/core/utils/PortUtilsTest.scala
new file mode 100644
index 000000000..fe2102f4c
--- /dev/null
+++ b/linkis-computation-governance/linkis-engineconn-manager/linkis-engineconn-manager-core/src/test/scala/org/apache/linkis/ecm/core/utils/PortUtilsTest.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.linkis.ecm.core.utils
+
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+
+class PortUtilsTest {
+
+  @Test private[utils] def testFindAvailPortByRange() = {
+    assertTrue(PortUtils.findAvailPortByRange("-") > 0)
+
+    var portRange = "-30000"
+    assertTrue(PortUtils.findAvailPortByRange(portRange) <= 30000)
+
+    portRange = "30000-"
+    assertTrue(PortUtils.findAvailPortByRange(portRange) >= 30000)
+
+    portRange = "30000-40000"
+    val port = PortUtils.findAvailPortByRange(portRange)
+    assertTrue(port >= 30000 && port <= 40000)
+  }
+
+}
diff --git a/linkis-spring-cloud-services/linkis-service-discovery/linkis-eureka/pom.xml b/linkis-spring-cloud-services/linkis-service-discovery/linkis-eureka/pom.xml
index 2ccccc477..ad59362ab 100644
--- a/linkis-spring-cloud-services/linkis-service-discovery/linkis-eureka/pom.xml
+++ b/linkis-spring-cloud-services/linkis-service-discovery/linkis-eureka/pom.xml
@@ -71,6 +71,24 @@
             <artifactId>jackson-databind</artifactId>
             <groupId>com.fasterxml.jackson.core</groupId>
         </dependency>
+
+        <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-actuator</artifactId>
+            <version>${spring.boot.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.springframework.boot</groupId>
+                    <artifactId>spring-boot-starter-logging</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>io.micrometer</groupId>
+            <artifactId>micrometer-registry-prometheus</artifactId>
+            <scope>compile</scope>
+        </dependency>
+
     </dependencies>
     <build>
         <plugins>
diff --git a/tool/dependencies/known-dependencies.txt b/tool/dependencies/known-dependencies.txt
index 23375e976..544fe49f4 100644
--- a/tool/dependencies/known-dependencies.txt
+++ b/tool/dependencies/known-dependencies.txt
@@ -530,4 +530,7 @@ xstream-1.4.19.jar
 zookeeper-3.5.9.jar
 zookeeper-jute-3.5.9.jar
 zstd-jni-1.4.4-7.jar
-zstd-jni-1.4.5-6.jar
\ No newline at end of file
+zstd-jni-1.4.5-6.jar
+micrometer-registry-prometheus-1.5.14.jar
+simpleclient-0.8.1.jar
+simpleclient_common-0.8.1.jar
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@linkis.apache.org
For additional commands, e-mail: commits-help@linkis.apache.org