From 4217a3f7e32438bb1d7b94b328ef620c640baac4 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Mon, 7 Nov 2022 09:41:48 +0800 Subject: [PATCH 01/72] ServiceInstance add registryTime --- .../org/apache/linkis/common/ServiceInstance.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala index 8fcb4af737..9cee5fe329 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala @@ -20,10 +20,13 @@ package org.apache.linkis.common class ServiceInstance { private var applicationName: String = _ private var instance: String = _ + private var registryTimestamp: Long = _ def setApplicationName(applicationName: String): Unit = this.applicationName = applicationName def getApplicationName: String = applicationName def setInstance(instance: String): Unit = this.instance = instance def getInstance: String = instance + def setRegistryTimestamp(registryTimestamp: Long): Unit = this.registryTimestamp = registryTimestamp + def getRegistryTimestamp: Long = registryTimestamp override def equals(other: Any): Boolean = other match { case that: ServiceInstance => @@ -42,7 +45,7 @@ class ServiceInstance { .foldLeft(0)((a, b) => 31 * a + b) } - override def toString: String = s"ServiceInstance($applicationName, $instance)" + override def toString: String = s"ServiceInstance($applicationName, $instance, $registryTimestamp)" } object ServiceInstance { @@ -54,6 +57,14 @@ object ServiceInstance { serviceInstance } + def apply(applicationName: String, instance: String, registryTimestamp: Long): ServiceInstance = { + val serviceInstance = new ServiceInstance + serviceInstance.setApplicationName(applicationName) + serviceInstance.setInstance(instance) + serviceInstance.setRegistryTimestamp(registryTimestamp) + serviceInstance + } + def unapply(serviceInstance: ServiceInstance): Option[(String, String)] = if (serviceInstance != null) { Some(serviceInstance.applicationName, serviceInstance.instance) From 08fc461ed8ed4ac5fd98879f502b7539c1604751 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 9 Nov 2022 18:10:27 +0800 Subject: [PATCH 02/72] gateway edit and entrance api edit --- .../server/conf/ServerConfiguration.scala | 2 + .../protocol/utils/ZuulEntranceUtils.scala | 2 +- .../restful/EntranceLabelRestfulApi.java | 19 ++++++- .../linkis/entrance/EntranceServer.scala | 22 +++++++++ .../entrance/utils/JobHistoryHelper.scala | 17 +++++++ .../linkis-gateway-server-support/pom.xml | 7 +++ .../parser/EntranceRequestGatewayParser.scala | 49 +++++++++++++++++-- 7 files changed, 110 insertions(+), 8 deletions(-) diff --git a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala index 582568e626..6784c5100f 100644 --- a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala +++ b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala @@ -207,4 +207,6 @@ object ServerConfiguration extends Logging { val LINKIS_SERVER_SESSION_PROXY_TICKETID_KEY = CommonVars("wds.linkis.session.proxy.user.ticket.key", "linkis_user_session_proxy_ticket_id_v1") + val LINKIS_SERVER_HEADER_KEY = CommonVars("wds.linkis.session.proxy.user.ticket.key", "job_req_id") + } diff --git a/linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/utils/ZuulEntranceUtils.scala b/linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/utils/ZuulEntranceUtils.scala index 95c7a81873..ad30484c46 100644 --- a/linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/utils/ZuulEntranceUtils.scala +++ b/linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/utils/ZuulEntranceUtils.scala @@ -23,7 +23,7 @@ object ZuulEntranceUtils { private val INSTANCE_SPLIT_TOKEN = "_" - private val EXEC_ID = "exec_id" + val EXEC_ID = "exec_id" private val SPLIT_LEN = 3 diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java index 2ab457747c..03ae97b781 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java @@ -18,6 +18,8 @@ package org.apache.linkis.entrance.restful; import org.apache.linkis.common.conf.Configuration; +import org.apache.linkis.entrance.EntranceServer; +import org.apache.linkis.entrance.context.DefaultEntranceContext; import org.apache.linkis.instance.label.client.InstanceLabelClient; import org.apache.linkis.manager.label.constant.LabelKeyConstant; import org.apache.linkis.manager.label.constant.LabelValueConstant; @@ -26,6 +28,7 @@ import org.apache.linkis.server.Message; import org.apache.linkis.server.utils.ModuleUserUtils; +import org.springframework.beans.factory.annotation.Autowired; import org.springframework.web.bind.annotation.*; import javax.servlet.http.HttpServletRequest; @@ -45,7 +48,13 @@ @RequestMapping(path = "/entrance/operation/label") public class EntranceLabelRestfulApi { - private static final Logger logger = LoggerFactory.getLogger(EntranceLabelRestfulApi.class); + private static final Logger logger = LoggerFactory.getLogger(EntranceLabelRestfulApi.class); + private EntranceServer entranceServer; + + @Autowired + public void setEntranceServer(EntranceServer entranceServer) { + this.entranceServer = entranceServer; + } @ApiOperation(value = "update", notes = "update route label", response = Message.class) @ApiOperationSupport(ignoreParameters = {"jsonNode"}) @@ -79,6 +88,12 @@ public Message updateRouteLabel(HttpServletRequest req) { insLabelRefreshRequest.setServiceInstance(Sender.getThisServiceInstance()); InstanceLabelClient.getInstance().refreshLabelsToInstance(insLabelRefreshRequest); logger.info("Finished to modify the routelabel of entry to offline"); - return Message.ok(); + + logger.info("Prepare to update the instances field for all not execution task to empty string"); + // todo ((DefaultEntranceContext) entranceServer.getEntranceContext()).setOfflineFlag(true); + entranceServer.updateAllNotExecutionTaskInstances(); + logger.info("Finished to update the instances field for all not execution task to empty string"); + + return Message.ok(); } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index f298e54251..c369adcbc3 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -227,6 +227,28 @@ abstract class EntranceServer extends Logging { .toArray } + def getAllNotStartRunningTask(): Array[EntranceJob] = { + val consumers = getEntranceContext + .getOrCreateScheduler() + .getSchedulerContext + .getOrCreateConsumerManager + .listConsumers() + .toSet + + consumers + .flatMap { consumer => + consumer.getConsumeQueue.getWaitingEvents + } + .filter(job => job != null && job.isInstanceOf[EntranceJob]) + .map(_.asInstanceOf[EntranceJob]) + .toArray + } + + def updateAllNotExecutionTaskInstances(): Unit = { + val taskIds = getAllNotStartRunningTask().map(_.getJobRequest.getId) + JobHistoryHelper.updateBatchInstances(taskIds) + } + } object EntranceServer { diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index 0fc7e6e486..080e6913e0 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -123,6 +123,23 @@ object JobHistoryHelper extends Logging { sender.ask(jobReqBatchUpdate) } + /** + * Batch update instances + * + * @param taskIdList + */ + def updateBatchInstances(taskIdList: Array[java.lang.Long]): Unit = { + val jobReqList = new util.ArrayList[JobRequest]() + taskIdList.foreach(taskID => { + val jobRequest = new JobRequest + jobRequest.setId(taskID) + jobRequest.setInstances("") + jobReqList.add(jobRequest) + }) + val jobReqBatchUpdate = JobReqBatchUpdate(jobReqList) + sender.ask(jobReqBatchUpdate) + } + private def getTaskByTaskID(taskID: Long): JobRequest = { val jobRequest = new JobRequest jobRequest.setId(taskID) diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/pom.xml b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/pom.xml index ef4635ae00..38efa93b33 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/pom.xml +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/pom.xml @@ -89,6 +89,13 @@ ${project.version} + + + org.apache.linkis + linkis-jobhistory + ${project.version} + + com.fasterxml.jackson.core jackson-databind diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala index 2ee0f4b023..3eacb8c2b6 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala @@ -17,17 +17,28 @@ package org.apache.linkis.gateway.ujes.parser +import org.apache.commons.lang3.StringUtils import org.apache.linkis.common.ServiceInstance import org.apache.linkis.gateway.config.GatewayConfiguration import org.apache.linkis.gateway.http.GatewayContext import org.apache.linkis.gateway.parser.AbstractGatewayParser import org.apache.linkis.gateway.ujes.parser.EntranceExecutionGatewayParser._ +import org.apache.linkis.jobhistory.entity.JobHistory +import org.apache.linkis.jobhistory.service.JobHistoryQueryService import org.apache.linkis.protocol.utils.ZuulEntranceUtils - +import org.apache.linkis.rpc.interceptor.ServiceInstanceUtils +import org.apache.linkis.server.conf.ServerConfiguration import org.springframework.stereotype.Component +import javax.annotation.Resource + @Component class EntranceRequestGatewayParser extends AbstractGatewayParser { + + + @Resource + private var jobHistoryQueryService: JobHistoryQueryService = _ + override def shouldContainRequestBody(gatewayContext: GatewayContext): Boolean = false override def parse(gatewayContext: GatewayContext): Unit = @@ -36,9 +47,9 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { if (sendResponseWhenNotMatchVersion(gatewayContext, version)) return val serviceInstance = if (execId.startsWith(EntranceRequestGatewayParser.API_REQUEST)) { if ( - gatewayContext.getRequest.getQueryParams.containsKey( - EntranceRequestGatewayParser.INSTANCE - ) + gatewayContext.getRequest.getQueryParams.containsKey( + EntranceRequestGatewayParser.INSTANCE + ) ) { val instances = gatewayContext.getRequest.getQueryParams.get(EntranceRequestGatewayParser.INSTANCE) @@ -50,13 +61,41 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { } else { ServiceInstance(GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue, null) } - } else { + } else if (execId.startsWith(ZuulEntranceUtils.EXEC_ID)) { + // parse by execId ZuulEntranceUtils.parseServiceInstanceByExecID(execId)(0) + } else { + // parse by taskId + val jobHistory = parseJobHistoryByTaskID(execId.toLong, gatewayContext) + // add header + val jobReqId = if (jobHistory == null) "" else jobHistory.getJobReqId + gatewayContext.getRequest.addHeader(ServerConfiguration.LINKIS_SERVER_HEADER_KEY.getValue, Array(jobReqId)) + // select instance + val instance = if (jobHistory == null) null else jobHistory.getInstances + ServiceInstance(GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue, instance) } gatewayContext.getGatewayRoute.setServiceInstance(serviceInstance) case _ => } + def parseJobHistoryByTaskID(taskId: Long, gatewayContext: GatewayContext): JobHistory = { + val histories = jobHistoryQueryService.search(taskId, null, null, null, null, null, null, null) + if (histories.isEmpty) { + sendErrorResponse(s"taskId $taskId is not exists.", gatewayContext) + } + val instances = histories.get(0).getInstances + val activeInstances = ServiceInstanceUtils.getRPCServerLoader.getServiceInstances(GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue) + + if (activeInstances.exists(StringUtils.isNotBlank(instances) && _.getInstance.equals(instances)) && + activeInstances.filter(_.getInstance.equals(instances))(0).getRegistryTimestamp <= histories.get(0).getCreatedTime.getTime + ) { + histories.get(0) + } else { + null + } + + } + } object EntranceRequestGatewayParser { From 17a59318d8595bca1c100a3c334ffb9ad3014873 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 15 Nov 2022 15:58:00 +0800 Subject: [PATCH 03/72] add method of queryFailoverJobs --- .../impl/JobHistoryQueryServiceImpl.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala index 3512d3fbfd..c918ee085c 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala @@ -243,6 +243,37 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { jobResp } + @Receiver + override def queryFailoverJobs(requestFailoverJob: RequestFailoverJob): JobRespProtocol = { + val reqMap = requestFailoverJob.reqMap + val statusList = requestFailoverJob.statusList + val startTimestamp = requestFailoverJob.startTimestamp + val limit = requestFailoverJob.limit + logger.info(s"query failover jobs, start timestamp:${startTimestamp}, limit:${limit}") + val jobResp = new JobRespProtocol + Utils.tryCatch { + val jobList = jobHistoryMapper.selectFailoverJobHistory(reqMap, statusList, startTimestamp, limit) + val jobReqList = jobList.asScala.map(jobHistory2JobRequest).toList + val map = new util.HashMap[String, Object]() + map.put(JobRequestConstants.JOB_HISTORY_LIST, jobReqList) + jobResp.setStatus(0) + jobResp.setData(map) + } { case e: Exception => + logger.error(s"Failed to query failover job, instances ${reqMap.keySet()}", e) + jobResp.setStatus(1) + jobResp.setMsg(ExceptionUtils.getRootCauseMessage(e)) + } + jobResp + } + + /* private def queryTaskList2RequestPersistTaskList(queryTask: java.util.List[QueryTask]): java.util.List[RequestPersistTask] = { + import scala.collection.JavaConversions._ + val tasks = new util.ArrayList[RequestPersistTask] + import org.apache.linkis.jobhistory.conversions.TaskConversions.queryTask2RequestPersistTask + queryTask.foreach(f => tasks.add(f)) + tasks + } */ + override def getJobHistoryByIdAndName(jobId: java.lang.Long, userName: String): JobHistory = { val jobReq = new JobHistory jobReq.setId(jobId) From 7706a6027dc6571bc15bbbd1010090128441b68f Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 15 Nov 2022 15:42:55 +0800 Subject: [PATCH 04/72] entrance ha and failover --- .../scheduler/queue/AbstractGroup.scala | 5 + .../queue/fifoqueue/FIFOUserConsumer.scala | 15 +- .../common/protocol/job/JobReqProcotol.scala | 2 + .../linkis-entrance/pom.xml | 6 + .../conf/EntranceSpringConfiguration.java | 4 +- .../entrance/constant/ServiceNameConsts.java | 2 + .../restful/EntranceLabelRestfulApi.java | 15 +- .../server/DefaultEntranceServer.java | 6 + .../server/EntranceFailoverJobServer.java | 146 +++++++++++++++++ .../linkis/entrance/EntranceServer.scala | 152 +++++++++++++++++- .../entrance/conf/EntranceConfiguration.scala | 21 +++ .../scheduler/EntranceFIFOUserConsumer.scala | 50 ++++++ .../scheduler/EntranceGroupFactory.scala | 88 ++++++---- .../EntranceParallelConsumerManager.scala | 31 ++++ .../scheduler/EntranceSchedulerContext.scala | 5 + .../entrance/utils/JobHistoryHelper.scala | 109 ++++++++++++- .../manager/label/constant/LabelConstant.java | 2 + .../jobhistory/dao/JobHistoryMapper.java | 35 ++++ .../mapper/common/JobHistoryMapper.xml | 23 +++ .../service/JobHistoryQueryService.java | 2 + 20 files changed, 664 insertions(+), 55 deletions(-) create mode 100644 linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java create mode 100644 linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala create mode 100644 linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala index 6e9ecbd26f..cc9577941f 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala @@ -23,11 +23,16 @@ abstract class AbstractGroup extends Group { private var _status: GroupStatus = _ private var maxRunningJobs: Int = _ + private var maxAllowRunningJobs: Int = 0 private var maxAskExecutorTimes: Long = 0L def setMaxRunningJobs(maxRunningJobs: Int): Unit = this.maxRunningJobs = maxRunningJobs def getMaxRunningJobs: Int = maxRunningJobs + def setMaxAllowRunningJobs(maxAllowRunningJobs: Int): Unit = this.maxAllowRunningJobs = maxAllowRunningJobs + def getMaxAllowRunningJobs: Int = + if(maxAllowRunningJobs <= 0) maxRunningJobs else Math.min(maxAllowRunningJobs, maxRunningJobs) + def setMaxAskExecutorTimes(maxAskExecutorTimes: Long): Unit = this.maxAskExecutorTimes = maxAskExecutorTimes diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala index 2a40c2517b..692325b75c 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala @@ -108,8 +108,9 @@ class FIFOUserConsumer( } var event: Option[SchedulerEvent] = getWaitForRetryEvent if (event.isEmpty) { - val completedNums = runningJobs.filter(job => job == null || job.isCompleted) - if (completedNums.length < 1) { + val maxAllowRunningJobs = fifoGroup.getMaxAllowRunningJobs + val currentRunningJobs = runningJobs.filter(e => e != null && !e.isCompleted) + if (maxAllowRunningJobs <= currentRunningJobs) { Utils.tryQuietly(Thread.sleep(1000)) // TODO 还可以优化,通过实现JobListener进行优化 return } @@ -188,6 +189,16 @@ class FIFOUserConsumer( runningJobs(index) = job } + protected def scanAllRetryJobsAndRemove(): Unit = { + for (index <- runningJobs.indices) { + val job = runningJobs(index) + if (job != null && job.isJobCanRetry) { + runningJobs(index) = null + logger.info(s"Job $job can retry, remove from runningJobs") + } + } + } + override def shutdown(): Unit = { future.cancel(true) super.shutdown() diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala index 2e44739787..4d6346c918 100644 --- a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala @@ -51,3 +51,5 @@ class RequestOneJob extends JobReq { } case class RequestAllJob(instance: String) extends JobReq + +case class RequestFailoverJob(reqMap: util.Map[String, java.lang.Long], statusList: util.List[String], startTimestamp: Long, limit: Int = 10) extends JobReq diff --git a/linkis-computation-governance/linkis-entrance/pom.xml b/linkis-computation-governance/linkis-entrance/pom.xml index b9ebec930e..21008708ce 100644 --- a/linkis-computation-governance/linkis-entrance/pom.xml +++ b/linkis-computation-governance/linkis-entrance/pom.xml @@ -102,6 +102,12 @@ ${project.version} + + org.apache.linkis + linkis-ps-common-lock + ${project.version} + + diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/conf/EntranceSpringConfiguration.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/conf/EntranceSpringConfiguration.java index 0bf27a68b3..cf520c3823 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/conf/EntranceSpringConfiguration.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/conf/EntranceSpringConfiguration.java @@ -42,6 +42,7 @@ import org.apache.linkis.entrance.persistence.QueryPersistenceManager; import org.apache.linkis.entrance.persistence.ResultSetEngine; import org.apache.linkis.entrance.scheduler.EntranceGroupFactory; +import org.apache.linkis.entrance.scheduler.EntranceParallelConsumerManager; import org.apache.linkis.entrance.scheduler.EntranceSchedulerContext; import org.apache.linkis.orchestrator.ecm.EngineConnManagerBuilder; import org.apache.linkis.orchestrator.ecm.EngineConnManagerBuilder$; @@ -51,7 +52,6 @@ import org.apache.linkis.scheduler.executer.ExecutorManager; import org.apache.linkis.scheduler.queue.ConsumerManager; import org.apache.linkis.scheduler.queue.GroupFactory; -import org.apache.linkis.scheduler.queue.parallelqueue.ParallelConsumerManager; import org.apache.linkis.scheduler.queue.parallelqueue.ParallelScheduler; import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; @@ -190,7 +190,7 @@ public GroupFactory groupFactory() { @Bean @ConditionalOnMissingBean public ConsumerManager consumerManager() { - return new ParallelConsumerManager( + return new EntranceParallelConsumerManager( ENTRANCE_SCHEDULER_MAX_PARALLELISM_USERS().getValue(), "EntranceJobScheduler"); } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/constant/ServiceNameConsts.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/constant/ServiceNameConsts.java index cb37279c11..bee17b8ed4 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/constant/ServiceNameConsts.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/constant/ServiceNameConsts.java @@ -26,4 +26,6 @@ private ServiceNameConsts() {} public static final String ENTRANCE_SERVER = "entranceServer"; public static final String ENTRANCE_INTERCEPTOR = "entranceInterceptors"; + + public static final String ENTRANCE_FAILOVER_SERVER = "entranceFailoverServer"; } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java index 03ae97b781..e51f66266d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java @@ -19,12 +19,14 @@ import org.apache.linkis.common.conf.Configuration; import org.apache.linkis.entrance.EntranceServer; -import org.apache.linkis.entrance.context.DefaultEntranceContext; +import org.apache.linkis.entrance.scheduler.EntranceSchedulerContext; import org.apache.linkis.instance.label.client.InstanceLabelClient; +import org.apache.linkis.manager.label.constant.LabelConstant; import org.apache.linkis.manager.label.constant.LabelKeyConstant; import org.apache.linkis.manager.label.constant.LabelValueConstant; import org.apache.linkis.protocol.label.InsLabelRefreshRequest; import org.apache.linkis.rpc.Sender; +import org.apache.linkis.scheduler.SchedulerContext; import org.apache.linkis.server.Message; import org.apache.linkis.server.utils.ModuleUserUtils; @@ -89,10 +91,13 @@ public Message updateRouteLabel(HttpServletRequest req) { InstanceLabelClient.getInstance().refreshLabelsToInstance(insLabelRefreshRequest); logger.info("Finished to modify the routelabel of entry to offline"); - logger.info("Prepare to update the instances field for all not execution task to empty string"); - // todo ((DefaultEntranceContext) entranceServer.getEntranceContext()).setOfflineFlag(true); - entranceServer.updateAllNotExecutionTaskInstances(); - logger.info("Finished to update the instances field for all not execution task to empty string"); + logger.info("Prepare to update all not execution task instances to empty string"); + SchedulerContext schedulerContext = entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); + if (schedulerContext instanceof EntranceSchedulerContext) { + ((EntranceSchedulerContext) schedulerContext).setOfflineFlag(true); + } + entranceServer.updateAllNotExecutionTaskInstances(true); + logger.info("Finished to update all not execution task instances to empty string"); return Message.ok(); } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index a050056fe1..999d5cbcbf 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -19,6 +19,7 @@ import org.apache.linkis.entrance.EntranceContext; import org.apache.linkis.entrance.EntranceServer; +import org.apache.linkis.entrance.conf.EntranceConfiguration; import org.apache.linkis.entrance.constant.ServiceNameConsts; import org.apache.linkis.entrance.execute.EntranceJob; import org.apache.linkis.entrance.log.LogReader; @@ -79,6 +80,11 @@ private void shutdownEntrance(ContextClosedEvent event) { if (shutdownFlag) { logger.warn("event has been handled"); } else { + if (EntranceConfiguration.ENTRANCE_SHUTDOWN_FAILOVER_ENABLED()) { + logger.warn("Entrance exit to update all not execution task instances and clean ConsumeQueue"); + updateAllNotExecutionTaskInstances(false); + } + logger.warn("Entrance exit to stop all job"); EntranceJob[] allUndoneJobs = getAllUndoneTask(null); if (null != allUndoneJobs) { diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java new file mode 100644 index 0000000000..7e7e0de69c --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -0,0 +1,146 @@ +/* + * 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.entrance.server; + +import org.apache.commons.compress.utils.Lists; +import org.apache.linkis.common.ServiceInstance; +import org.apache.linkis.common.utils.Utils; +import org.apache.linkis.entrance.EntranceServer; +import org.apache.linkis.entrance.conf.EntranceConfiguration; +import org.apache.linkis.entrance.constant.ServiceNameConsts; +import org.apache.linkis.entrance.scheduler.EntranceSchedulerContext; +import org.apache.linkis.entrance.utils.JobHistoryHelper; +import org.apache.linkis.governance.common.entity.job.JobRequest; +import org.apache.linkis.instance.label.client.InstanceLabelClient; +import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext; +import org.apache.linkis.manager.label.constant.LabelConstant; +import org.apache.linkis.manager.label.constant.LabelKeyConstant; +import org.apache.linkis.manager.label.entity.Label; +import org.apache.linkis.manager.label.entity.route.RouteLabel; +import org.apache.linkis.publicservice.common.lock.entity.CommonLock; +import org.apache.linkis.publicservice.common.lock.service.CommonLockService; +import org.apache.linkis.rpc.Sender; +import org.apache.linkis.scheduler.queue.SchedulerEventState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +import javax.annotation.PostConstruct; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +@Component(ServiceNameConsts.ENTRANCE_FAILOVER_SERVER) +public class EntranceFailoverJobServer { + + private static final Logger logger = LoggerFactory.getLogger(DefaultEntranceServer.class); + + @Autowired + private EntranceServer entranceServer; + + @Autowired + private CommonLockService commonLockService; + + + private static String ENTRANCE_FAILOVER_LOCK = "ENTRANCE_FAILOVER_LOCK"; + + @PostConstruct + public void init() { + failoverTask(); + } + + public void failoverTask() { + if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { + Utils.defaultScheduler().scheduleAtFixedRate( + new Runnable() { + @Override + public void run() { + EntranceSchedulerContext schedulerContext = (EntranceSchedulerContext) entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); + + // entrance do not failover job when it is offline + if (schedulerContext.getOfflineFlag()) return; + + CommonLock commonLock = new CommonLock(); + commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); + Boolean locked = false; + try { + locked = commonLockService.lock(commonLock, 10 * 1000L); + if (!locked) return; + logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); + + // serverInstance to map + Map serverInstanceMap = getActiveServerInstances().stream() + .collect(Collectors.toMap(ServiceInstance::getInstance, ServiceInstance::getRegistryTimestamp, (k1, k2) -> k2)); + if (serverInstanceMap.isEmpty()) return; + + // get failover start time + long startTimestamp = 0L; + if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { + startTimestamp = System.currentTimeMillis() - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); + } + + // get uncompleted status + List statusList = Lists.newArrayList(); + SchedulerEventState.values().filterNot(SchedulerEventState::isCompleted).foreach(state -> statusList.add(state.toString())); + + List jobRequests = JobHistoryHelper.queryWaitForFailoverTask(serverInstanceMap, statusList, startTimestamp, EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); + if (jobRequests.isEmpty()) return; + logger.info("success query failover jobs , job ids: {}", jobRequests.stream().map(JobRequest::getId)); + + // failover to local server + jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); + logger.info("success execute failover jobs, job ids: {}", jobRequests.stream().map(JobRequest::getId)); + + } catch (Exception e) { + logger.error("failover failed", e); + } finally { + if (locked) commonLockService.unlock(commonLock); + } + } + }, + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), + TimeUnit.MILLISECONDS + ); + } + } + + private List getActiveServerInstances() { + // get all entrance server from eureka + ServiceInstance[] serviceInstances = Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); + if (serviceInstances == null || serviceInstances.length <= 0) return Lists.newArrayList(); + + // get all offline label server + RouteLabel routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory() + .createLabel(LabelKeyConstant.ROUTE_KEY, LabelConstant.OFFLINE); + List> labels = Lists.newArrayList(); + labels.add(routeLabel); + List labelInstances = InstanceLabelClient.getInstance().getInstanceFromLabel(labels); + + // get active entrance server + List allInstances = Lists.newArrayList(); + allInstances.addAll(Arrays.asList(serviceInstances)); + allInstances.removeAll(labelInstances); + + return allInstances; + } + +} \ No newline at end of file diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index c369adcbc3..e91cfb3df6 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -35,9 +35,11 @@ import org.apache.linkis.server.conf.ServerConfiguration import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils +import org.apache.linkis.common.log.LogUtils import java.text.MessageFormat -import java.util +import java.{lang, util} +import java.util.Date abstract class EntranceServer extends Logging { @@ -227,7 +229,7 @@ abstract class EntranceServer extends Logging { .toArray } - def getAllNotStartRunningTask(): Array[EntranceJob] = { + def getAllConsumeQueueTask(): Array[EntranceJob] = { val consumers = getEntranceContext .getOrCreateScheduler() .getSchedulerContext @@ -244,9 +246,149 @@ abstract class EntranceServer extends Logging { .toArray } - def updateAllNotExecutionTaskInstances(): Unit = { - val taskIds = getAllNotStartRunningTask().map(_.getJobRequest.getId) - JobHistoryHelper.updateBatchInstances(taskIds) + def clearAllConsumeQueue(): Unit = { + getEntranceContext + .getOrCreateScheduler() + .getSchedulerContext + .getOrCreateConsumerManager + .listConsumers() + .foreach(_.getConsumeQueue.clearAll()) + } + + def updateAllNotExecutionTaskInstances(retryWhenUpdateFail: Boolean): Unit = { + val taskIds = getAllConsumeQueueTask().map(_.getJobRequest.getId).toList + JobHistoryHelper.updateAllConsumeQueueTask(taskIds, retryWhenUpdateFail) + logger.info("Finished to update all not execution task instances") + clearAllConsumeQueue() + logger.info("Finished to clean all ConsumeQueue") + } + + /** + * execute failover job (提交故障转移任务,返回新的execId) + * + * @param jobRequest + */ + def failoverExecute(jobRequest: JobRequest): String = { + + if (null == jobRequest || null == jobRequest.getId || jobRequest.getId <= 0) { + throw new EntranceErrorException( + PERSIST_JOBREQUEST_ERROR.getErrorCode, + PERSIST_JOBREQUEST_ERROR.getErrorDesc + ) + } + + // todo dmp kill ec + + val logAppender = new java.lang.StringBuilder() + // init properties + initJobRequestProperties(jobRequest, logAppender) + // update jobRequest + getEntranceContext + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(jobRequest) + + val job = getEntranceContext.getOrCreateEntranceParser().parseToJob(jobRequest) + Utils.tryThrow { + job.init() + job.setLogListener(getEntranceContext.getOrCreateLogManager()) + job.setProgressListener(getEntranceContext.getOrCreatePersistenceManager()) + job.setJobListener(getEntranceContext.getOrCreatePersistenceManager()) + job match { + case entranceJob: EntranceJob => { + entranceJob.setEntranceListenerBus(getEntranceContext.getOrCreateEventListenerBus) + } + case _ => + } + Utils.tryCatch { + if (logAppender.length() > 0) + job.getLogListener.foreach(_.onLogUpdate(job, logAppender.toString.trim)) + } { t => + logger.error("Failed to write init JobRequest log, reason: ", t) + } + + /** + * job.afterStateChanged() method is only called in job.run(), and job.run() is called only + * after job is scheduled so it suggest that we lack a hook for job init, currently we call + * this to trigger JobListener.onJobinit() + */ + Utils.tryAndWarn(job.getJobListener.foreach(_.onJobInited(job))) + getEntranceContext.getOrCreateScheduler().submit(job) + val msg = s"Job with jobId : ${jobRequest.getId} and execID : ${job.getId()} submitted, success to failover" + logger.info(msg) + + job match { + case entranceJob: EntranceJob => + entranceJob.getJobRequest.setReqId(job.getId()) + if (jobTimeoutManager.timeoutCheck && JobTimeoutManager.hasTimeoutLabel(entranceJob)) + jobTimeoutManager.add(job.getId(), entranceJob) + entranceJob.getLogListener.foreach(_.onLogUpdate(entranceJob, msg)) + case _ => + } + + job.getId() + } { t => + job.onFailure("Submitting the query failed!(提交查询失败!)", t) + val _jobRequest = + getEntranceContext.getOrCreateEntranceParser().parseToJobRequest(job) + getEntranceContext + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(_jobRequest) + t match { + case e: LinkisException => e + case e: LinkisRuntimeException => e + case t: Throwable => + new SubmitFailedException( + SUBMITTING_QUERY_FAILED.getErrorCode, + SUBMITTING_QUERY_FAILED.getErrorDesc + ExceptionUtils.getRootCauseMessage(t), + t + ) + } + } + + } + + private def initJobRequestProperties(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { + + val initInstance = Sender.getThisInstance + val initDate = new Date(System.currentTimeMillis) + val initStatus = SchedulerEventState.Inited.toString + val initProgress = "0.0" + val initReqId = "" + + logAppender.append( + LogUtils.generateInfo(s"Job ${jobRequest.getId} start to failover, Initialize the properties \n") + ) + logAppender.append( + LogUtils.generateInfo(s"the instances ${jobRequest.getInstances} -> ${initInstance} \n") + ) + logAppender.append( + LogUtils.generateInfo(s"the created_time ${jobRequest.getCreatedTime} -> ${initDate} \n") + ) + logAppender.append( + LogUtils.generateInfo(s"the status ${jobRequest.getStatus} -> $initStatus \n") + ) + logAppender.append( + LogUtils.generateInfo(s"the progress ${jobRequest.getProgress} -> $initProgress \n") + ) + logAppender.append( + LogUtils.generateInfo(s"the job_req_id ${jobRequest.getReqId} -> $initReqId \n") + ) + + jobRequest.setInstances(initInstance) + jobRequest.setCreatedTime(initDate) + jobRequest.setStatus(initStatus) + jobRequest.setProgress(initProgress) + jobRequest.setReqId(initReqId) + jobRequest.setErrorCode(0) + jobRequest.setErrorDesc("") + jobRequest.setMetrics(new util.HashMap[String, Object]()) + jobRequest.getMetrics.put(TaskConstant.ENTRANCEJOB_SUBMIT_TIME, initInstance) + + logAppender.append( + LogUtils.generateInfo(s"Job ${jobRequest.getId} success to initialize the properties \n") + ) } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 4b6230299b..62c42cfdd0 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -225,4 +225,25 @@ object EntranceConfiguration { val CREATOR_IP_SWITCH = CommonVars("wds.linkis.entrance.user.creator.ip.interceptor.switch", false) + val ENTRANCE_FAILOVER_ENABLED = CommonVars("linkis.entrance.failover.enable", true).getValue + + val ENTRANCE_FAILOVER_SCAN_INIT_TIME = + CommonVars("linkis.entrance.failover.scan.init.time", 3 * 1000).getValue + + val ENTRANCE_FAILOVER_SCAN_INTERVAL = + CommonVars("linkis.entrance.failover.scan.interval", 3 * 1000).getValue + + val ENTRANCE_FAILOVER_DATA_NUM_LIMIT = CommonVars("linkis.entrance.failover.data.num.limit", 10).getValue + + val ENTRANCE_FAILOVER_DATA_INTERVAL_TIME = CommonVars("linkis.entrance.failover.data.interval.time", new TimeType("7d").toLong).getValue + + // if true, the waitForRetry job in runningJobs can be failover + val ENTRANCE_FAILOVER_RETRY_JOB_ENABLED = CommonVars("linkis.entrance.failover.retry.job.enable", true) + + val ENTRANCE_UPDATE_BATCH_SIZE = CommonVars("linkis.entrance.update.batch.size", 100) + + val ENTRANCE_SHUTDOWN_FAILOVER_ENABLED = CommonVars("linkis.entrance.shutdown.failover.enable", true).getValue + + val ENTRANCE_GROUP_SCAN_ENABLED = CommonVars("linkis.entrance.group.scan.enable", true) + } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala new file mode 100644 index 0000000000..34d3e3042c --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala @@ -0,0 +1,50 @@ +/* + * 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.entrance.scheduler + +import org.apache.linkis.common.utils.Utils +import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.scheduler.SchedulerContext +import org.apache.linkis.scheduler.queue.Group +import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer + +import java.util.concurrent.ExecutorService + +class EntranceFIFOUserConsumer( + schedulerContext: SchedulerContext, + executeService: ExecutorService, + private var group: Group +) extends FIFOUserConsumer(schedulerContext, executeService, group) { + + override def loop(): Unit = { + schedulerContext match { + case entranceSchedulerContext: EntranceSchedulerContext => + if (entranceSchedulerContext.getOfflineFlag && EntranceConfiguration.ENTRANCE_FAILOVER_RETRY_JOB_ENABLED.getValue) { + scanAllRetryJobsAndRemove() + Utils.tryQuietly(Thread.sleep(5000)) + return + } + case _ => + } + + // general logic + super.loop() + + } + +} diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala index 7f16dd2463..a0a644e1d0 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala @@ -31,6 +31,7 @@ import org.apache.linkis.governance.common.protocol.conf.{ import org.apache.linkis.instance.label.client.InstanceLabelClient import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext import org.apache.linkis.manager.label.constant.{LabelKeyConstant, LabelValueConstant} +import org.apache.linkis.governance.common.protocol.conf.{RequestQueryEngineConfigWithGlobalConfig, ResponseQueryConfig} import org.apache.linkis.manager.label.entity.Label import org.apache.linkis.manager.label.entity.engine.{ ConcurrentEngineConnLabel, @@ -38,22 +39,25 @@ import org.apache.linkis.manager.label.entity.engine.{ UserCreatorLabel } import org.apache.linkis.manager.label.entity.route.RouteLabel +import org.apache.linkis.manager.label.entity.engine.{ConcurrentEngineConnLabel, EngineTypeLabel, UserCreatorLabel} import org.apache.linkis.manager.label.utils.LabelUtil import org.apache.linkis.protocol.constants.TaskConstant import org.apache.linkis.protocol.utils.TaskUtils import org.apache.linkis.rpc.Sender import org.apache.linkis.scheduler.queue.{Group, GroupFactory, SchedulerEvent} import org.apache.linkis.scheduler.queue.parallelqueue.ParallelGroup - import org.apache.commons.lang3.StringUtils import java.util import java.util.concurrent.TimeUnit import java.util.regex.Pattern - import scala.collection.JavaConverters._ - import com.google.common.cache.{Cache, CacheBuilder} +import org.apache.linkis.common.ServiceInstance +import org.apache.linkis.instance.label.client.InstanceLabelClient +import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext +import org.apache.linkis.manager.label.constant.{LabelConstant, LabelKeyConstant} +import org.apache.linkis.manager.label.entity.route.RouteLabel class EntranceGroupFactory extends GroupFactory with Logging { @@ -73,6 +77,39 @@ class EntranceGroupFactory extends GroupFactory with Logging { private val GROUP_INIT_CAPACITY = CommonVars("wds.linkis.entrance.init.capacity", 100) + private val GROUP_SCAN_INIT_TIME = CommonVars("linkis.entrance.group.scan.init.time", 3 * 1000) + + private val GROUP_SCAN_INTERVAL = CommonVars("linkis.entrance.group.scan.interval", 60 * 1000) + + if (EntranceConfiguration.ENTRANCE_GROUP_SCAN_ENABLED.getValue) { + Utils.defaultScheduler.scheduleAtFixedRate( + new Runnable { + override def run(): Unit = { + // get all entrance server from eureka + val serviceInstances = Sender.getInstances(Sender.getThisServiceInstance.getApplicationName) + if (null == serviceInstances || serviceInstances.isEmpty) return + + // get all offline label server + val routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory + .createLabel[RouteLabel](LabelKeyConstant.ROUTE_KEY, LabelConstant.OFFLINE) + val labels = new util.ArrayList[Label[_]] + labels.add(routeLabel) + val labelInstances = InstanceLabelClient.getInstance.getInstanceFromLabel(labels) + + // get active entrance server + val allInstances = new util.ArrayList[ServiceInstance]() + allInstances.addAll(serviceInstances.toList.asJava) + allInstances.removeAll(labelInstances) + // refresh all group maxAllowRunningJobs + refreshAllGroupMaxAllowRunningJobs(allInstances.size()) + } + }, + GROUP_SCAN_INIT_TIME.getValue, + GROUP_SCAN_INTERVAL.getValue, + TimeUnit.MILLISECONDS + ) + } + private val specifiedUsernameRegexPattern: Pattern = if (StringUtils.isNotBlank(SPECIFIED_USERNAME_REGEX.getValue)) { Pattern.compile(SPECIFIED_USERNAME_REGEX.getValue) @@ -156,41 +193,22 @@ class EntranceGroupFactory extends GroupFactory with Logging { group } + def refreshAllGroupMaxAllowRunningJobs(activeCount: Int): Unit = { + if (activeCount <= 0) return + groupNameToGroups.asMap().asScala.foreach(item => { + item._2 match { + case group: ParallelGroup => + group.setMaxAllowRunningJobs(Math.round(group.getMaxRunningJobs / activeCount)) + case _ => + } + }) + } + private def getUserMaxRunningJobs(keyAndValue: util.Map[String, String]): Int = { - var userDefinedRunningJobs = EntranceConfiguration.WDS_LINKIS_INSTANCE.getValue(keyAndValue) - var entranceNum = Sender.getInstances(Sender.getThisServiceInstance.getApplicationName).length - val labelList = new util.ArrayList[Label[_]]() - val offlineRouteLabel = LabelBuilderFactoryContext.getLabelBuilderFactory - .createLabel[RouteLabel](LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE) - labelList.add(offlineRouteLabel) - var offlineIns: Array[ServiceInstance] = null - Utils.tryAndWarn { - offlineIns = InstanceLabelClient.getInstance - .getInstanceFromLabel(labelList) - .asScala - .filter(l => - null != l && l.getApplicationName - .equalsIgnoreCase(Sender.getThisServiceInstance.getApplicationName) - ) - .toArray - } - if (null != offlineIns) { - logger.info(s"There are ${offlineIns.length} offlining instance.") - entranceNum = entranceNum - offlineIns.length - } - /* - Sender.getInstances may get 0 instances due to cache in Sender. So this instance is the one instance. - */ - if (0 >= entranceNum) { - logger.error( - s"Got ${entranceNum} ${Sender.getThisServiceInstance.getApplicationName} instances." - ) - entranceNum = 1 - } Math.max( EntranceConfiguration.ENTRANCE_INSTANCE_MIN.getValue, - userDefinedRunningJobs / entranceNum - ); + EntranceConfiguration.WDS_LINKIS_INSTANCE.getValue(keyAndValue) + ) } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala new file mode 100644 index 0000000000..91a7c4aaa6 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -0,0 +1,31 @@ +/* + * 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.entrance.scheduler + +import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer +import org.apache.linkis.scheduler.queue.parallelqueue.ParallelConsumerManager + +class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: String) + extends ParallelConsumerManager(maxParallelismUsers, schedulerName){ + + override protected def createConsumer(groupName: String): FIFOUserConsumer = { + val group = getSchedulerContext.getOrCreateGroupFactory.getGroup(groupName) + new EntranceFIFOUserConsumer(getSchedulerContext, getOrCreateExecutorService, group) + } + +} diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceSchedulerContext.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceSchedulerContext.scala index d5de2cc2da..1638b0fb1c 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceSchedulerContext.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceSchedulerContext.scala @@ -28,6 +28,11 @@ class EntranceSchedulerContext extends SchedulerContext { private var consumerManager: ConsumerManager = _ private var executorManager: ExecutorManager = _ + private var offlineFlag: Boolean = false + + def setOfflineFlag(offlineFlag: Boolean): Unit = this.offlineFlag = offlineFlag + def getOfflineFlag: Boolean = this.offlineFlag + def this( groupFactory: GroupFactory, consumerManager: ConsumerManager, diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index 080e6913e0..a5dbeaab39 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -30,15 +30,13 @@ import org.apache.linkis.protocol.constants.TaskConstant import org.apache.linkis.protocol.query.cache.{CacheTaskResult, RequestReadCache} import org.apache.linkis.rpc.Sender import org.apache.linkis.scheduler.queue.SchedulerEventState - import org.apache.commons.lang3.StringUtils import javax.servlet.http.HttpServletRequest - import java.util import java.util.Date - import scala.collection.JavaConverters._ +import sun.net.util.IPAddressUtil import com.google.common.net.InetAddresses @@ -124,11 +122,50 @@ object JobHistoryHelper extends Logging { } /** - * Batch update instances + * Get all consume queue task and batch update instances(获取所有消费队列中的任务进行批量更新) + * + * @param taskIdList + * @param retryWhenUpdateFail + */ + def updateAllConsumeQueueTask(taskIdList: List[java.lang.Long], retryWhenUpdateFail: Boolean = false): Unit = { + + if (taskIdList.isEmpty) return + + val updateTaskIds = new util.ArrayList[java.lang.Long]() + + if (EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue > 0 && + taskIdList.length > EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue) { + for (i <- 0 until EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue) { + updateTaskIds.add(taskIdList(i)) + } + } else { + updateTaskIds.addAll(taskIdList.asJava) + } + + try { + val successTaskIds = updateBatchInstances(updateTaskIds.asScala.toList) + if (retryWhenUpdateFail) { + taskIdList.asJava.removeAll(successTaskIds.asJava) + } else { + taskIdList.asJava.removeAll(updateTaskIds) + } + } catch { + case e: Exception => + logger.warn("update batch instances failed, wait for retry", e) + Thread.sleep(1000) + } + + updateAllConsumeQueueTask(taskIdList, retryWhenUpdateFail) + + } + + /** + * Batch update instances(批量更新instances字段) * * @param taskIdList + * @return */ - def updateBatchInstances(taskIdList: Array[java.lang.Long]): Unit = { + private def updateBatchInstances(taskIdList: List[java.lang.Long]): List[java.lang.Long] = { val jobReqList = new util.ArrayList[JobRequest]() taskIdList.foreach(taskID => { val jobRequest = new JobRequest @@ -137,7 +174,67 @@ object JobHistoryHelper extends Logging { jobReqList.add(jobRequest) }) val jobReqBatchUpdate = JobReqBatchUpdate(jobReqList) - sender.ask(jobReqBatchUpdate) + Utils.tryCatch { + val response = sender.ask(jobReqBatchUpdate) + response match { + case resp: util.ArrayList[JobRespProtocol] => + resp.asScala.filter(r => r.getStatus == SUCCESS_FLAG && r.getData.containsKey(JobRequestConstants.JOB_ID)) + .map(_.getData.get(JobRequestConstants.JOB_ID).asInstanceOf[java.lang.Long]).toList + case _ => + throw JobHistoryFailedException( + "update batch instances from jobhistory not a correct List type" + ) + } + } { + case errorException: ErrorException => throw errorException + case e: Exception => + val e1 = JobHistoryFailedException(s"update batch instances ${taskIdList.mkString(",")} error") + e1.initCause(e) + throw e + } + } + + /** + * query wait for failover task(获取待故障转移的任务) + * + * @param reqMap + * @param statusList + * @param startTimestamp + * @param limit + * @return + */ + def queryWaitForFailoverTask(reqMap: util.Map[String, java.lang.Long], statusList: util.List[String], startTimestamp: Long, limit: Int): util.List[JobRequest] = { + val requestFailoverJob = RequestFailoverJob(reqMap, statusList, startTimestamp, limit) + val tasks = Utils.tryCatch { + val response = sender.ask(requestFailoverJob) + response match { + case responsePersist: JobRespProtocol => + val status = responsePersist.getStatus + if (status != SUCCESS_FLAG) { + logger.error(s"query from jobHistory status failed, status is $status") + throw JobHistoryFailedException("query from jobHistory status failed") + } + val data = responsePersist.getData + data.get(JobRequestConstants.JOB_HISTORY_LIST) match { + case tasks: util.List[JobRequest] => + tasks + case _ => + throw JobHistoryFailedException( + s"query from jobhistory not a correct List type, instances ${reqMap.keySet()}" + ) + } + case _ => + logger.error("get query response incorrectly") + throw JobHistoryFailedException("get query response incorrectly") + } + } { + case errorException: ErrorException => throw errorException + case e: Exception => + val e1 = JobHistoryFailedException(s"query failover task error, instances ${reqMap.keySet()} ") + e1.initCause(e) + throw e + } + tasks } private def getTaskByTaskID(taskID: Long): JobRequest = { diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java index 4db4bfca40..b43501ed9e 100644 --- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java +++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java @@ -22,4 +22,6 @@ public class LabelConstant { public static final int LABEL_BUILDER_ERROR_CODE = 40001; public static final int LABEL_UTIL_CONVERT_ERROR_CODE = 40002; + + public static final String OFFLINE = "offline"; } diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java b/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java index 1403a29ed0..6568fb838b 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java @@ -23,6 +23,7 @@ import java.util.Date; import java.util.List; +import java.util.Map; public interface JobHistoryMapper { @@ -105,4 +106,38 @@ Integer countUndoneTaskWithCreatorOnly( String selectJobHistoryStatusForUpdate(Long jobId); void updateOberverById(@Param("taskid") Long taskid, @Param("observeInfo") String observeInfo); + + /** + * query wait for failover job + * + * Sql example: + * SELECT a.* FROM linkis_ps_job_history_group_history a + * where (a.instances = '' + * or a.instances is null + * or a.instances not in ('192.168.1.123:9104','192.168.1.124:9104') + * or EXISTS ( + * select 1 from + * ( + * select '192.168.1.123:9104' as instances, 1697775054098 as registryTime + * union all + * select '192.168.1.124:9104' as instances, 1666239054098 as registryTime + * ) b + * where a.instances = b.instances and UNIX_TIMESTAMP(a.created_time) * 1000 < b.registryTime + * ) + * ) + * and + * status in ('Inited','Running','Scheduled','WaitForRetry') + * and UNIX_TIMESTAMP(a.created_time) * 1000 >= 1666239054098 + * limit 10 + * + * @param instancesMap + * @param statusList + * @param startTimestamp + * @param limit + * @return + */ + List selectFailoverJobHistory(@Param("instancesMap") Map instancesMap, + @Param("statusList") List statusList, + @Param("startTimestamp") Long startTimestamp, + @Param("limit") Integer limit); } diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml index 8ac85a7c46..b2fa7f95ba 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml @@ -221,4 +221,27 @@ update linkis_ps_job_history_group_history set observe_info = #{observeInfo} where id = #{taskid} + + diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/JobHistoryQueryService.java b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/JobHistoryQueryService.java index b238738907..ba92d37ecc 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/JobHistoryQueryService.java +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/JobHistoryQueryService.java @@ -36,6 +36,8 @@ public interface JobHistoryQueryService { JobRespProtocol query(JobReqQuery jobReqQuery); + JobRespProtocol queryFailoverJobs(RequestFailoverJob requestFailoverJob); + JobHistory getJobHistoryByIdAndName(Long jobID, String userName); List search(Long jobId, String username, String creator, String status, Date sDate, Date eDate, String engineType, Long startJobId, String instance); From 00777c630c33ee89f1cbedcbea099ea93f010cc9 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Thu, 17 Nov 2022 16:21:02 +0800 Subject: [PATCH 05/72] Interface development of status,log,progress,kill --- .../entrance/restful/EntranceRestfulApi.java | 809 +++++++++++------- .../parser/EntranceRequestGatewayParser.scala | 6 +- 2 files changed, 485 insertions(+), 330 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index 324187fc28..4a946e6d0c 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -35,6 +35,7 @@ import org.apache.linkis.scheduler.queue.Job; import org.apache.linkis.scheduler.queue.SchedulerEventState; import org.apache.linkis.server.Message; +import org.apache.linkis.server.conf.ServerConfiguration; import org.apache.linkis.server.security.SecurityFilter; import org.apache.linkis.server.utils.ModuleUserUtils; @@ -197,193 +198,277 @@ private void pushLog(String log, Job job) { entranceServer.getEntranceContext().getOrCreateLogManager().onLogUpdate(job, log); } - @ApiOperation(value = "status", notes = "get task stats", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = " task id"), - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id ") - }) - @Override - @RequestMapping(path = "/{id}/status", method = RequestMethod.GET) - public Message status( - HttpServletRequest req, - @PathVariable("id") String id, - @RequestParam(value = "taskID", required = false) String taskID) { - Message message = null; - String realId = ZuulEntranceUtils.parseExecID(id)[3]; - ModuleUserUtils.getOperationUser(req, "status realId: " + realId); - Option job = Option.apply(null); - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.warn("获取任务 {} 状态时出现错误", realId, e.getMessage()); - long realTaskID = Long.parseLong(taskID); - String status = JobHistoryHelper.getStatusByTaskID(realTaskID); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/status"); - message.data("status", status).data("execID", id); - return message; - } - if (job.isDefined()) { - if (job.get() instanceof EntranceJob) { - ((EntranceJob) job.get()).updateNewestAccessByClientTimestamp(); - } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/status"); - message.data("status", job.get().getState().toString()).data("execID", id); - } else { - message = - Message.error( - "ID The corresponding job is empty and cannot obtain the corresponding task status.(ID 对应的job为空,不能获取相应的任务状态)"); - } - return message; - } + @ApiOperation(value = "status", notes = "get task stats", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = " task id"), + @ApiImplicitParam(name = "id",required = true, dataType = "String", value = "execute id ") + }) + @Override + @RequestMapping(path = "/{id}/status", method = RequestMethod.GET) + public Message status( + HttpServletRequest req, + @PathVariable("id") String id, + @RequestParam(value = "taskID", required = false) String taskID) { + ModuleUserUtils.getOperationUser(req, "job status"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); + if (StringUtils.isEmpty(jobReqId)){ + logger.warn("The job wait failover, return status is Inited"); + String status = SchedulerEventState.Inited().toString(); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", status).data("execID", "").data("taskID", id); + return message; + } else { + realId = jobReqId; + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } + } - @ApiOperation(value = "progress", notes = "get task progress info", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "exectue id") - }) - @Override - @RequestMapping(path = "/{id}/progress", method = RequestMethod.GET) - public Message progress(HttpServletRequest req, @PathVariable("id") String id) { - Message message = null; - String realId = ZuulEntranceUtils.parseExecID(id)[3]; - ModuleUserUtils.getOperationUser(req, "progress realId: " + realId); - Option job = null; - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.error(e.getMessage()); + Option job = Option.apply(null); + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.warn("get {} status error", realId, e); + if (StringUtils.isEmpty(taskID)) { + message = + Message.error( + "Get job by ID error and cannot obtain the corresponding task status.(获取job时发生异常,不能获取相应的任务状态)"); + return message; + } + long realTaskID = Long.parseLong(taskID); + String status = JobHistoryHelper.getStatusByTaskID(realTaskID); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", status).data("execID", execID); + return message; + } + if (job.isDefined()) { + if (job.get() instanceof EntranceJob) { + ((EntranceJob) job.get()).updateNewestAccessByClientTimestamp(); + } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", job.get().getState().toString()).data("execID", execID); + } else { + message = + Message.error( + "ID The corresponding job is empty and cannot obtain the corresponding task status.(ID 对应的job为空,不能获取相应的任务状态)"); + } + return message; } - if (job != null && job.isDefined()) { - JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); - if (jobProgressInfos == null) { - message = - Message.error( - "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); - message.setMethod("/api/entrance/" + id + "/progress"); - } else { - List> list = new ArrayList<>(); - for (JobProgressInfo jobProgressInfo : jobProgressInfos) { - if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) - || jobProgressInfo.totalTasks() > 0) { - setJobProgressInfos(list, jobProgressInfo); - } + + @ApiOperation(value = "progress", notes = "get task progress info", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id",required = true, dataType = "String", value = "exectue id") + }) + @Override + @RequestMapping(path = "/{id}/progress", method = RequestMethod.GET) + public Message progress(HttpServletRequest req, @PathVariable("id") String id) { + ModuleUserUtils.getOperationUser(req, "job progress"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); + if (StringUtils.isEmpty(jobReqId)){ + logger.warn("The job wait failover, return progress is 0"); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progress"); + message.data("progress", 0) + .data("execID", "") + .data("taskID", id) + .data("progressInfo", new ArrayList<>()); + return message; + } else { + realId = jobReqId; + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/progress"); - message - .data("progress", Math.abs(job.get().getProgress())) - .data("execID", id) - .data("progressInfo", list); - } - } else { - message = - Message.error( - "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); - } - return message; - } + Option job = null; + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.error(e.getMessage()); + } + if (job != null && job.isDefined()) { + JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); + if (jobProgressInfos == null) { + message = + Message.error( + "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); + message.setMethod("/api/entrance/" + id + "/progress"); + } else { + List> list = new ArrayList<>(); + for (JobProgressInfo jobProgressInfo : jobProgressInfos) { + if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) + || jobProgressInfo.totalTasks() > 0) { + setJobProgressInfos(list, jobProgressInfo); + } + } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progress"); - @ApiOperation( - value = "progressWithResource", - notes = "get progress and resource info", - response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") - }) - @Override - @RequestMapping(path = "/{id}/progressWithResource", method = RequestMethod.GET) - public Message progressWithResource(HttpServletRequest req, @PathVariable("id") String id) { - Message message = null; - String realId = ZuulEntranceUtils.parseExecID(id)[3]; - ModuleUserUtils.getOperationUser(req, "progressWithResource realId: " + realId); - Option job = null; - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.error(e.getMessage()); + message.data("progress", Math.abs(job.get().getProgress())) + .data("execID", execID) + .data("progressInfo", list); + } + } else { + message = + Message.error( + "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); + } + return message; } - if (job != null && job.isDefined()) { - JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); - if (jobProgressInfos == null) { - message = - Message.error( - "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); - message.setMethod("/api/entrance/" + id + "/progressWithResource"); - } else { - List> list = new ArrayList<>(); - for (JobProgressInfo jobProgressInfo : jobProgressInfos) { - if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) - || jobProgressInfo.totalTasks() > 0) { - setJobProgressInfos(list, jobProgressInfo); - } + + @ApiOperation(value = "progressWithResource", notes = "get progress and resource info", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") + }) + @Override + @RequestMapping(path = "/{id}/progressWithResource", method = RequestMethod.GET) + public Message progressWithResource(HttpServletRequest req, @PathVariable("id") String id) { + ModuleUserUtils.getOperationUser(req, "job progressWithResource"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); + if (StringUtils.isEmpty(jobReqId)){ + logger.warn("The job wait failover, return progress is 0 and resource is null"); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null) + .data("progress", 0) + .data("execID", "") + .data("taskID", id) + .data("progressInfo", new ArrayList<>()); + return message; + } else { + realId = jobReqId; + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } + } + Option job = null; + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.error(e.getMessage()); } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/progressWithResource"); - - JobRequest jobRequest = ((EntranceJob) job.get()).getJobRequest(); - Map metrics = jobRequest.getMetrics(); - Map metricsVo = new HashMap<>(); - if (metrics.containsKey(TaskConstant.ENTRANCEJOB_YARNRESOURCE)) { - HashMap resourceMap = - (HashMap) - metrics.get(TaskConstant.ENTRANCEJOB_YARNRESOURCE); - ArrayList resoureList = new ArrayList<>(12); - if (null != resourceMap && !resourceMap.isEmpty()) { - resourceMap.forEach( - (applicationId, resource) -> { - resoureList.add(new YarnResourceWithStatusVo(applicationId, resource)); - }); - metricsVo.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, resoureList); - Optional cores = - resourceMap.values().stream() - .map(resource -> resource.queueCores()) - .reduce((x, y) -> x + y); - Optional memory = - resourceMap.values().stream() - .map(resource -> resource.queueMemory()) - .reduce((x, y) -> x + y); - float corePercent = 0.0f; - float memoryPercent = 0.0f; - if (cores.isPresent() && memory.isPresent()) { - corePercent = - cores.get().floatValue() - / EntranceConfiguration.YARN_QUEUE_CORES_MAX().getHotValue(); - memoryPercent = - memory.get().floatValue() - / (EntranceConfiguration.YARN_QUEUE_MEMORY_MAX().getHotValue().longValue() - * 1024 - * 1024 - * 1024); + if (job != null && job.isDefined()) { + JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); + if (jobProgressInfos == null) { + message = + Message.error( + "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + } else { + List> list = new ArrayList<>(); + for (JobProgressInfo jobProgressInfo : jobProgressInfos) { + if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) + || jobProgressInfo.totalTasks() > 0) { + setJobProgressInfos(list, jobProgressInfo); + } + } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + + JobRequest jobRequest = ((EntranceJob) job.get()).getJobRequest(); + Map metrics = jobRequest.getMetrics(); + Map metricsVo = new HashMap<>(); + if (metrics.containsKey(TaskConstant.ENTRANCEJOB_YARNRESOURCE)) { + HashMap resourceMap = + (HashMap) + metrics.get(TaskConstant.ENTRANCEJOB_YARNRESOURCE); + ArrayList resoureList = new ArrayList<>(12); + if (null != resourceMap && !resourceMap.isEmpty()) { + resourceMap.forEach( + (applicationId, resource) -> { + resoureList.add( + new YarnResourceWithStatusVo(applicationId, resource)); + }); + metricsVo.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, resoureList); + Optional cores = + resourceMap.values().stream() + .map(resource -> resource.queueCores()) + .reduce((x, y) -> x + y); + Optional memory = + resourceMap.values().stream() + .map(resource -> resource.queueMemory()) + .reduce((x, y) -> x + y); + float corePercent = 0.0f; + float memoryPercent = 0.0f; + if (cores.isPresent() && memory.isPresent()) { + corePercent = + cores.get().floatValue() + / EntranceConfiguration.YARN_QUEUE_CORES_MAX() + .getValue(); + memoryPercent = + memory.get().floatValue() + / (EntranceConfiguration.YARN_QUEUE_MEMORY_MAX() + .getValue() + .longValue() + * 1024 + * 1024 + * 1024); + } + String coreRGB = RGBUtils.getRGB(corePercent); + String memoryRGB = RGBUtils.getRGB(memoryPercent); + metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_PERCENT, corePercent); + metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_PERCENT, memoryPercent); + metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_RGB, coreRGB); + metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_RGB, memoryRGB); + + message.data(TaskConstant.ENTRANCEJOB_YARN_METRICS, metricsVo); + } else { + message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); + } + } else { + message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); + } + + message.data("progress", Math.abs(job.get().getProgress())) + .data("execID", execID) + .data("progressInfo", list); } - String coreRGB = RGBUtils.getRGB(corePercent); - String memoryRGB = RGBUtils.getRGB(memoryPercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_PERCENT, corePercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_PERCENT, memoryPercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_RGB, coreRGB); - metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_RGB, memoryRGB); - - message.data(TaskConstant.ENTRANCEJOB_YARN_METRICS, metricsVo); - } else { - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); - } } else { - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); + message = + Message.error( + "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); } - - message - .data("progress", Math.abs(job.get().getProgress())) - .data("execID", id) - .data("progressInfo", list); - } - } else { - message = - Message.error( - "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); + return message; } - return message; - } private void setJobProgressInfos( List> list, JobProgressInfo jobProgressInfo) { @@ -396,108 +481,146 @@ private void setJobProgressInfos( list.add(map); } - @ApiOperation(value = "log", notes = "get task log", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") - }) - @Override - @RequestMapping(path = "/{id}/log", method = RequestMethod.GET) - public Message log(HttpServletRequest req, @PathVariable("id") String id) { - String realId = ZuulEntranceUtils.parseExecID(id)[3]; - ModuleUserUtils.getOperationUser(req, "log realId: " + realId); - Option job = Option.apply(null); - Message message = null; - try { - job = entranceServer.getJob(realId); - } catch (final Throwable t) { - message = - Message.error( - "The job you just executed has ended. This interface no longer provides a query. It is recommended that you download the log file for viewing.(您刚刚执行的job已经结束,本接口不再提供查询,建议您下载日志文件进行查看)"); - message.setMethod("/api/entrance/" + id + "/log"); - return message; - } - if (job.isDefined()) { - logger.debug("begin to get log for {}(开始获取 {} 的日志)", job.get().getId(), job.get().getId()); - LogReader logReader = - entranceServer.getEntranceContext().getOrCreateLogManager().getLogReader(realId); - int fromLine = 0; - int size = 100; - boolean distinctLevel = true; - if (req != null) { + @ApiOperation(value = "log", notes = "get task log", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") + }) + @Override + @RequestMapping(path = "/{id}/log", method = RequestMethod.GET) + public Message log(HttpServletRequest req, @PathVariable("id") String id) { + ModuleUserUtils.getOperationUser(req, "get job log"); + Message message = null; + int fromLine = 0; + int size = 100; + boolean distinctLevel = true; String fromLineStr = req.getParameter("fromLine"); String sizeStr = req.getParameter("size"); if (StringUtils.isNotBlank(fromLineStr)) { - fromLine = Math.max(Integer.parseInt(fromLineStr), 0); + fromLine = Math.max(Integer.parseInt(fromLineStr), 0); } if (StringUtils.isNotBlank(sizeStr)) { - size = Integer.parseInt(sizeStr) >= 0 ? Integer.parseInt(sizeStr) : 10000; + size = Integer.parseInt(sizeStr) >= 0 ? Integer.parseInt(sizeStr) : 10000; } String distinctLevelStr = req.getParameter("distinctLevel"); if ("false".equals(distinctLevelStr)) { - distinctLevel = false; + distinctLevel = false; } - } - Object retLog = null; - int retFromLine = 0; - try { - if (distinctLevel) { - String[] logs = new String[4]; - retFromLine = logReader.readArray(logs, fromLine, size); - retLog = new ArrayList(Arrays.asList(logs)); + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; } else { - StringBuilder sb = new StringBuilder(); - retFromLine = logReader.read(sb, fromLine, size); - retLog = sb.toString(); + // taskID + String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); + if (StringUtils.isEmpty(jobReqId)){ + logger.warn("The job wait failover, return customer log"); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + String log = LogUtils.generateInfo("The job will failover soon, please try again later"); + Object retLog; + if (distinctLevel) { + String[] array = new String[4]; + array[2] = log; + array[3] = log; + retLog = new ArrayList(Arrays.asList(array)); + } else { + retLog = log; + } + message.data("log", retLog).data("execID", "").data("taskID", id).data("fromLine", 0); + return message; + } else { + realId = jobReqId; + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } } - } catch (IllegalStateException e) { - logger.debug( - "Failed to get log information for :{}(为 {} 获取日志失败)", - job.get().getId(), - job.get().getId(), - e); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", "").data("execID", id).data("fromLine", retFromLine + fromLine); - } catch (final IllegalArgumentException e) { - logger.debug( - "Failed to get log information for :{}(为 {} 获取日志失败)", - job.get().getId(), - job.get().getId(), - e); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", "").data("execID", id).data("fromLine", retFromLine + fromLine); - return message; - } catch (final Exception e1) { - logger.debug( - "Failed to get log information for :{}(为 {} 获取日志失败)", - job.get().getId(), - job.get().getId(), - e1); - message = Message.error("Failed to get log information(获取日志信息失败)"); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", "").data("execID", id).data("fromLine", retFromLine + fromLine); - return message; - } finally { - if (null != logReader && job.get().isCompleted()) { - IOUtils.closeQuietly(logReader); + + Option job = Option.apply(null); + try { + job = entranceServer.getJob(realId); + } catch (final Throwable t) { + message = + Message.error( + "The job you just executed has ended. This interface no longer provides a query. It is recommended that you download the log file for viewing.(您刚刚执行的job已经结束,本接口不再提供查询,建议您下载日志文件进行查看)"); + message.setMethod("/api/entrance/" + id + "/log"); + return message; } - } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", retLog).data("execID", id).data("fromLine", retFromLine + fromLine); - logger.debug("success to get log for {} (获取 {} 日志成功)", job.get().getId(), job.get().getId()); - } else { - message = - Message.error( - "Can't find execID(不能找到execID): " - + id - + "Corresponding job, can not get the corresponding log(对应的job,不能获得对应的日志)"); - message.setMethod("/api/entrance/" + id + "/log"); + if (job.isDefined()) { + logger.debug( + "begin to get log for {}(开始获取 {} 的日志)", job.get().getId(), job.get().getId()); + LogReader logReader = + entranceServer + .getEntranceContext() + .getOrCreateLogManager() + .getLogReader(realId); + + Object retLog = null; + int retFromLine = 0; + try { + if (distinctLevel) { + String[] logs = new String[4]; + retFromLine = logReader.readArray(logs, fromLine, size); + retLog = new ArrayList(Arrays.asList(logs)); + } else { + StringBuilder sb = new StringBuilder(); + retFromLine = logReader.read(sb, fromLine, size); + retLog = sb.toString(); + } + } catch (IllegalStateException e) { + logger.debug( + "Failed to get log information for :{}(为 {} 获取日志失败)", + job.get().getId(), + job.get().getId(), + e); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); + } catch (final IllegalArgumentException e) { + logger.debug( + "Failed to get log information for :{}(为 {} 获取日志失败)", + job.get().getId(), + job.get().getId(), + e); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); + return message; + } catch (final Exception e1) { + logger.debug( + "Failed to get log information for :{}(为 {} 获取日志失败)", + job.get().getId(), + job.get().getId(), + e1); + message = Message.error("Failed to get log information(获取日志信息失败)"); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); + return message; + } finally { + if (null != logReader && job.get().isCompleted()) { + IOUtils.closeQuietly(logReader); + } + } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", retLog).data("execID", execID).data("fromLine", retFromLine + fromLine); + logger.debug( + "success to get log for {} (获取 {} 日志成功)", job.get().getId(), job.get().getId()); + } else { + message = + Message.error( + "Can't find execID(不能找到execID): " + + id + + "Corresponding job, can not get the corresponding log(对应的job,不能获得对应的日志)"); + message.setMethod("/api/entrance/" + id + "/log"); + } + return message; } - return message; - } @ApiOperation(value = "killJobs", notes = "kill jobs", response = Message.class) @ApiImplicitParams({ @@ -595,71 +718,103 @@ public Message killJobs( return Message.ok("success").data("messages", messages); } - @ApiOperation(value = "kill", notes = "kill", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "excute id"), - @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = "task id") - }) - @Override - @RequestMapping(path = "/{id}/kill", method = RequestMethod.GET) - public Message kill( - HttpServletRequest req, - @PathVariable("id") String id, - @RequestParam(value = "taskID", required = false) Long taskID) { - String realId = ZuulEntranceUtils.parseExecID(id)[3]; - ModuleUserUtils.getOperationUser(req, "kill realId:" + realId); - Option job = Option.apply(null); - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.warn("can not find a job in entranceServer, will force to kill it", e); - // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 - JobHistoryHelper.forceKill(taskID); - Message message = Message.ok("Forced Kill task (强制杀死任务)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(0); - return message; - } - Message message = null; - if (job.isEmpty()) { - logger.warn("can not find a job in entranceServer, will force to kill it"); - // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 - JobHistoryHelper.forceKill(taskID); - message = Message.ok("Forced Kill task (强制杀死任务)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(0); - return message; - } else { - try { - logger.info("begin to kill job {} ", job.get().getId()); - job.get().kill(); - message = Message.ok("Successfully killed the job(成功kill了job)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(0); - message.data("execID", id); - // ensure the job's state is cancelled in database - if (job.get() instanceof EntranceJob) { - EntranceJob entranceJob = (EntranceJob) job.get(); - JobRequest jobReq = entranceJob.getJobRequest(); - entranceJob.updateJobRequestStatus(SchedulerEventState.Cancelled().toString()); - this.entranceServer - .getEntranceContext() - .getOrCreatePersistenceManager() - .createPersistenceEngine() - .updateIfNeeded(jobReq); + @ApiOperation(value = "kill", notes = "kill", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "exec id"), + @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = "task id") + }) + @Override + @RequestMapping(path = "/{id}/kill", method = RequestMethod.GET) + public Message kill( + HttpServletRequest req, + @PathVariable("id") String id, + @RequestParam(value = "taskID", required = false) Long taskID) { + ModuleUserUtils.getOperationUser(req, "kill job"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); + if (StringUtils.isEmpty(jobReqId)){ + logger.warn("The job wait failover, but now force kill"); + // TODO If failover occurs during force kill, the job status may change from Cancelled to Running + long taskId = Long.parseLong(id); + JobHistoryHelper.forceKill(taskId); + message = Message.ok("Forced Kill task (强制杀死任务)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.data("execID", "").data("taskID", id); + return message; + } else { + realId = jobReqId; + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } } - logger.info("end to kill job {} ", job.get().getId()); - } catch (Throwable t) { - logger.error("kill job {} failed ", job.get().getId(), t); - message = - Message.error( - "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(1); - } + + Option job = Option.apply(null); + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.warn("can not find a job in entranceServer, will force to kill it", e); + // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 + if (taskID == null || taskID <= 0) { + message = + Message.error( + "Get job by ID error, kill failed.(获取job时发生异常,kill失败)"); + return message; + } + JobHistoryHelper.forceKill(taskID); + message = Message.ok("Forced Kill task (强制杀死任务)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.setStatus(0); + return message; + } + + if (job.isEmpty()) { + logger.warn("can not find a job in entranceServer, will force to kill it"); + // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 + JobHistoryHelper.forceKill(taskID); + message = Message.ok("Forced Kill task (强制杀死任务)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.setStatus(0); + return message; + } else { + try { + logger.info("begin to kill job {} ", job.get().getId()); + job.get().kill(); + message = Message.ok("Successfully killed the job(成功kill了job)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.data("execID", execID); + // ensure the job's state is cancelled in database + if (job.get() instanceof EntranceJob) { + EntranceJob entranceJob = (EntranceJob) job.get(); + JobRequest jobReq = entranceJob.getJobRequest(); + entranceJob.updateJobRequestStatus(SchedulerEventState.Cancelled().toString()); + this.entranceServer + .getEntranceContext() + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(jobReq); + } + logger.info("end to kill job {} ", job.get().getId()); + } catch (Throwable t) { + logger.error("kill job {} failed ", job.get().getId(), t); + message = + Message.error( + "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败)"); + message.setMethod("/api/entrance/" + id + "/kill"); + } + } + return message; } - return message; - } @ApiOperation(value = "pause ", notes = "puase a task job", response = Message.class) @ApiImplicitParams({ diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala index 3eacb8c2b6..9fb3958ac0 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala @@ -65,8 +65,8 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { // parse by execId ZuulEntranceUtils.parseServiceInstanceByExecID(execId)(0) } else { - // parse by taskId - val jobHistory = parseJobHistoryByTaskID(execId.toLong, gatewayContext) + // check by taskId + val jobHistory = checkJobValidityByTaskID(execId.toLong, gatewayContext) // add header val jobReqId = if (jobHistory == null) "" else jobHistory.getJobReqId gatewayContext.getRequest.addHeader(ServerConfiguration.LINKIS_SERVER_HEADER_KEY.getValue, Array(jobReqId)) @@ -78,7 +78,7 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { case _ => } - def parseJobHistoryByTaskID(taskId: Long, gatewayContext: GatewayContext): JobHistory = { + def checkJobValidityByTaskID(taskId: Long, gatewayContext: GatewayContext): JobHistory = { val histories = jobHistoryQueryService.search(taskId, null, null, null, null, null, null, null) if (histories.isEmpty) { sendErrorResponse(s"taskId $taskId is not exists.", gatewayContext) From 8c5774c02e8eb66971878b009e5d7af939eabd0a Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 22 Nov 2022 11:31:29 +0800 Subject: [PATCH 06/72] failover and kill ec --- .../protocol/constants/TaskConstant.java | 1 + .../server/EntranceFailoverJobServer.java | 35 +++++--- .../linkis/entrance/EntranceServer.scala | 89 ++++++++++++++++++- .../entrance/utils/JobHistoryHelper.scala | 27 ++++-- 4 files changed, 130 insertions(+), 22 deletions(-) diff --git a/linkis-commons/linkis-protocol/src/main/java/org/apache/linkis/protocol/constants/TaskConstant.java b/linkis-commons/linkis-protocol/src/main/java/org/apache/linkis/protocol/constants/TaskConstant.java index 8f5a680089..ea4a30a0b7 100644 --- a/linkis-commons/linkis-protocol/src/main/java/org/apache/linkis/protocol/constants/TaskConstant.java +++ b/linkis-commons/linkis-protocol/src/main/java/org/apache/linkis/protocol/constants/TaskConstant.java @@ -66,6 +66,7 @@ public interface TaskConstant { String TICKET_ID = "ticketId"; String ENGINE_CONN_TASK_ID = "engineConnTaskId"; String ENGINE_CONN_SUBMIT_TIME = "engineConnSubmitTime"; + String FAILOVER_FLAG = "failoverFlag"; String PARAMS_DATA_SOURCE = "dataSources"; diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 7e7e0de69c..175da3be41 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -91,23 +91,34 @@ public void run() { .collect(Collectors.toMap(ServiceInstance::getInstance, ServiceInstance::getRegistryTimestamp, (k1, k2) -> k2)); if (serverInstanceMap.isEmpty()) return; - // get failover start time - long startTimestamp = 0L; - if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { - startTimestamp = System.currentTimeMillis() - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); - } + // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) + long expiredTimestamp = 0L; + if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { + expiredTimestamp = + System.currentTimeMillis() + - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); + } // get uncompleted status List statusList = Lists.newArrayList(); SchedulerEventState.values().filterNot(SchedulerEventState::isCompleted).foreach(state -> statusList.add(state.toString())); - List jobRequests = JobHistoryHelper.queryWaitForFailoverTask(serverInstanceMap, statusList, startTimestamp, EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); - if (jobRequests.isEmpty()) return; - logger.info("success query failover jobs , job ids: {}", jobRequests.stream().map(JobRequest::getId)); - - // failover to local server - jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); - logger.info("success execute failover jobs, job ids: {}", jobRequests.stream().map(JobRequest::getId)); + List jobRequests = + JobHistoryHelper.queryWaitForFailoverTask( + serverInstanceMap, + statusList, + expiredTimestamp, + EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); + if (jobRequests.isEmpty()) return; + logger.info( + "success query failover jobs , job ids: {}", + jobRequests.stream().map(JobRequest::getId)); + + // failover to local server + jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); + logger.info( + "success execute failover jobs, job ids: {}", + jobRequests.stream().map(JobRequest::getId)); } catch (Exception e) { logger.error("failover failed", e); diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index e91cfb3df6..ec8692c84e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -17,6 +17,7 @@ package org.apache.linkis.entrance +import org.apache.linkis.common.ServiceInstance import org.apache.linkis.common.exception.{ErrorException, LinkisException, LinkisRuntimeException} import org.apache.linkis.common.log.LogUtils import org.apache.linkis.common.utils.{Logging, Utils} @@ -27,9 +28,14 @@ import org.apache.linkis.entrance.execute.EntranceJob import org.apache.linkis.entrance.log.LogReader import org.apache.linkis.entrance.timeout.JobTimeoutManager import org.apache.linkis.entrance.utils.JobHistoryHelper +import org.apache.linkis.governance.common.conf.GovernanceCommonConf import org.apache.linkis.governance.common.entity.job.JobRequest +import org.apache.linkis.governance.common.protocol.task.RequestTaskKill +import org.apache.linkis.manager.common.protocol.engine.EngineStopRequest +import org.apache.linkis.manager.label.entity.entrance.ExecuteOnceLabel import org.apache.linkis.protocol.constants.TaskConstant import org.apache.linkis.rpc.Sender +import org.apache.linkis.rpc.conf.RPCConfiguration import org.apache.linkis.scheduler.queue.{Job, SchedulerEventState} import org.apache.linkis.server.conf.ServerConfiguration @@ -41,6 +47,8 @@ import java.text.MessageFormat import java.{lang, util} import java.util.Date +import scala.collection.JavaConverters._ + abstract class EntranceServer extends Logging { private var entranceWebSocketService: Option[EntranceWebSocketService] = None @@ -263,6 +271,68 @@ abstract class EntranceServer extends Logging { logger.info("Finished to clean all ConsumeQueue") } + def killEC(jobRequest: JobRequest): Unit = { + Utils.tryCatch { + if ( + !SchedulerEventState.isRunning(SchedulerEventState.withName(jobRequest.getStatus)) + || !SchedulerEventState.isScheduled(SchedulerEventState.withName(jobRequest.getStatus)) + || jobRequest.getMetrics == null + || !jobRequest.getMetrics.containsKey(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + ) { + logger.info( + s"job ${jobRequest.getId} is not running,scheduled or not have EC info, ignore it" + ) + } + + val engineMap = jobRequest.getMetrics + .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + .asInstanceOf[util.Map[String, Object]] + + val engineInstance = + engineMap.asScala + .map(_._2.asInstanceOf[util.Map[String, Object]]) + .filter(_.containsKey(TaskConstant.ENGINE_INSTANCE)) + .maxBy(_.getOrDefault(TaskConstant.ENGINE_CONN_SUBMIT_TIME, 0).toString) + + if (engineInstance != null || engineInstance.containsKey(TaskConstant.FAILOVER_FLAG)) { + logger.info( + s"job ${jobRequest.getId} do not submit to EC or already failover, not need kill ec" + ) + return + } + engineInstance.put(TaskConstant.FAILOVER_FLAG, "") + + val ecInstance = ServiceInstance( + GovernanceCommonConf.ENGINE_CONN_SPRING_NAME.getValue, + engineInstance.get(TaskConstant.ENGINE_INSTANCE).toString + ) + if (jobRequest.getLabels.asScala.exists(_.isInstanceOf[ExecuteOnceLabel])) { + // kill ec by linkismanager + val engineStopRequest = new EngineStopRequest + engineStopRequest.setServiceInstance(ecInstance) + // send to linkismanager + Sender + .getSender(RPCConfiguration.LINKIS_MANAGER_APPLICATION_NAME.getValue) + .send(engineStopRequest) + logger.info( + s"job ${jobRequest.getId} send EngineStopRequest to linkismanager, kill instance $ecInstance" + ) + } else if (engineInstance.containsKey(TaskConstant.ENGINE_CONN_TASK_ID)) { + // kill ec task + val engineTaskId = engineInstance.get(TaskConstant.ENGINE_CONN_TASK_ID).toString + // send to ec + Sender + .getSender(ecInstance) + .send(RequestTaskKill(engineTaskId)) + logger.info( + s"job ${jobRequest.getId} send RequestTaskKill to kill engineConn $ecInstance, execID $engineTaskId" + ) + } + } { case e: Exception => + logger.error(s"job ${jobRequest.getId} kill ec error", e) + } + } + /** * execute failover job (提交故障转移任务,返回新的execId) * @@ -277,7 +347,8 @@ abstract class EntranceServer extends Logging { ) } - // todo dmp kill ec + // try to kill ec + killEC(jobRequest); val logAppender = new java.lang.StringBuilder() // init properties @@ -376,6 +447,18 @@ abstract class EntranceServer extends Logging { LogUtils.generateInfo(s"the job_req_id ${jobRequest.getReqId} -> $initReqId \n") ) + val metricMap = new util.HashMap[String, Object]() + if ( + jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( + TaskConstant.ENTRANCEJOB_ENGINECONN_MAP + ) + ) { + val oldEngineconnMap = jobRequest.getMetrics + .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + .asInstanceOf[util.Map[String, Object]] + metricMap.put(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP, oldEngineconnMap) + } + jobRequest.setInstances(initInstance) jobRequest.setCreatedTime(initDate) jobRequest.setStatus(initStatus) @@ -383,8 +466,8 @@ abstract class EntranceServer extends Logging { jobRequest.setReqId(initReqId) jobRequest.setErrorCode(0) jobRequest.setErrorDesc("") - jobRequest.setMetrics(new util.HashMap[String, Object]()) - jobRequest.getMetrics.put(TaskConstant.ENTRANCEJOB_SUBMIT_TIME, initInstance) + jobRequest.setMetrics(metricMap) + jobRequest.getMetrics.put(TaskConstant.ENTRANCEJOB_SUBMIT_TIME, initDate) logAppender.append( LogUtils.generateInfo(s"Job ${jobRequest.getId} success to initialize the properties \n") diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index a5dbeaab39..6416fe1d47 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -133,8 +133,10 @@ object JobHistoryHelper extends Logging { val updateTaskIds = new util.ArrayList[java.lang.Long]() - if (EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue > 0 && - taskIdList.length > EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue) { + if ( + EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue > 0 && + taskIdList.length > EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue + ) { for (i <- 0 until EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue) { updateTaskIds.add(taskIdList(i)) } @@ -178,8 +180,12 @@ object JobHistoryHelper extends Logging { val response = sender.ask(jobReqBatchUpdate) response match { case resp: util.ArrayList[JobRespProtocol] => - resp.asScala.filter(r => r.getStatus == SUCCESS_FLAG && r.getData.containsKey(JobRequestConstants.JOB_ID)) - .map(_.getData.get(JobRequestConstants.JOB_ID).asInstanceOf[java.lang.Long]).toList + resp.asScala + .filter(r => + r.getStatus == SUCCESS_FLAG && r.getData.containsKey(JobRequestConstants.JOB_ID) + ) + .map(_.getData.get(JobRequestConstants.JOB_ID).asInstanceOf[java.lang.Long]) + .toList case _ => throw JobHistoryFailedException( "update batch instances from jobhistory not a correct List type" @@ -188,7 +194,8 @@ object JobHistoryHelper extends Logging { } { case errorException: ErrorException => throw errorException case e: Exception => - val e1 = JobHistoryFailedException(s"update batch instances ${taskIdList.mkString(",")} error") + val e1 = + JobHistoryFailedException(s"update batch instances ${taskIdList.mkString(",")} error") e1.initCause(e) throw e } @@ -203,7 +210,12 @@ object JobHistoryHelper extends Logging { * @param limit * @return */ - def queryWaitForFailoverTask(reqMap: util.Map[String, java.lang.Long], statusList: util.List[String], startTimestamp: Long, limit: Int): util.List[JobRequest] = { + def queryWaitForFailoverTask( + reqMap: util.Map[String, java.lang.Long], + statusList: util.List[String], + startTimestamp: Long, + limit: Int + ): util.List[JobRequest] = { val requestFailoverJob = RequestFailoverJob(reqMap, statusList, startTimestamp, limit) val tasks = Utils.tryCatch { val response = sender.ask(requestFailoverJob) @@ -230,7 +242,8 @@ object JobHistoryHelper extends Logging { } { case errorException: ErrorException => throw errorException case e: Exception => - val e1 = JobHistoryFailedException(s"query failover task error, instances ${reqMap.keySet()} ") + val e1 = + JobHistoryFailedException(s"query failover task error, instances ${reqMap.keySet()} ") e1.initCause(e) throw e } From 497f8b62e7a3e2e5e490b3bf18d0046e465d0357 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 22 Nov 2022 19:40:37 +0800 Subject: [PATCH 07/72] failover --- .../scheduler/queue/SchedulerEventState.scala | 4 ++++ .../queue/fifoqueue/FIFOUserConsumer.scala | 2 +- .../server/EntranceFailoverJobServer.java | 18 ++++++++++++------ .../linkis/entrance/EntranceServer.scala | 2 +- 4 files changed, 18 insertions(+), 8 deletions(-) diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala index 4edc1d5d17..a64103628c 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala @@ -38,4 +38,8 @@ object SchedulerEventState extends Enumeration { SchedulerEventState.withName(jobState) ) + def uncompleteStatusArray(): Array[SchedulerEventState] = { + SchedulerEventState.values.filterNot(isCompleted).toArray + } + } diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala index 692325b75c..4483a02a76 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala @@ -109,7 +109,7 @@ class FIFOUserConsumer( var event: Option[SchedulerEvent] = getWaitForRetryEvent if (event.isEmpty) { val maxAllowRunningJobs = fifoGroup.getMaxAllowRunningJobs - val currentRunningJobs = runningJobs.filter(e => e != null && !e.isCompleted) + val currentRunningJobs = runningJobs.count(e => e != null && !e.isCompleted) if (maxAllowRunningJobs <= currentRunningJobs) { Utils.tryQuietly(Thread.sleep(1000)) // TODO 还可以优化,通过实现JobListener进行优化 return diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 175da3be41..1eb29a48fb 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -86,10 +86,15 @@ public void run() { if (!locked) return; logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); - // serverInstance to map - Map serverInstanceMap = getActiveServerInstances().stream() - .collect(Collectors.toMap(ServiceInstance::getInstance, ServiceInstance::getRegistryTimestamp, (k1, k2) -> k2)); - if (serverInstanceMap.isEmpty()) return; + // serverInstance to map + Map serverInstanceMap = + getActiveServerInstances().stream() + .collect( + Collectors.toMap( + ServiceInstance::getInstance, + ServiceInstance::getRegistryTimestamp, + (k1, k2) -> k2)); + if (serverInstanceMap.isEmpty()) return; // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) long expiredTimestamp = 0L; @@ -100,8 +105,9 @@ public void run() { } // get uncompleted status - List statusList = Lists.newArrayList(); - SchedulerEventState.values().filterNot(SchedulerEventState::isCompleted).foreach(state -> statusList.add(state.toString())); + List statusList = + Arrays.stream(SchedulerEventState.uncompleteStatusArray()) + .map(Object::toString).collect(Collectors.toList()); List jobRequests = JobHistoryHelper.queryWaitForFailoverTask( diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index ec8692c84e..b09ef4911a 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -292,7 +292,7 @@ abstract class EntranceServer extends Logging { engineMap.asScala .map(_._2.asInstanceOf[util.Map[String, Object]]) .filter(_.containsKey(TaskConstant.ENGINE_INSTANCE)) - .maxBy(_.getOrDefault(TaskConstant.ENGINE_CONN_SUBMIT_TIME, 0).toString) + .maxBy(_.getOrDefault(TaskConstant.ENGINE_CONN_SUBMIT_TIME, "0").toString) if (engineInstance != null || engineInstance.containsKey(TaskConstant.FAILOVER_FLAG)) { logger.info( From da5b2b7277c2755a62ca78d414a8e9160dfe03d0 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 23 Nov 2022 20:08:27 +0800 Subject: [PATCH 08/72] add log --- .../scheduler/EntranceGroupFactory.scala | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala index a0a644e1d0..2a7432ee6e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala @@ -195,13 +195,19 @@ class EntranceGroupFactory extends GroupFactory with Logging { def refreshAllGroupMaxAllowRunningJobs(activeCount: Int): Unit = { if (activeCount <= 0) return - groupNameToGroups.asMap().asScala.foreach(item => { - item._2 match { - case group: ParallelGroup => - group.setMaxAllowRunningJobs(Math.round(group.getMaxRunningJobs / activeCount)) - case _ => - } - }) + groupNameToGroups + .asMap() + .asScala + .foreach(item => { + item._2 match { + case group: ParallelGroup => + val maxAllowRunningJobs = Math.round(group.getMaxRunningJobs / activeCount) + group.setMaxAllowRunningJobs(maxAllowRunningJobs) + logger + .info(s"group ${group.getGroupName} update maxAllowRunningJobs $maxAllowRunningJobs") + case _ => + } + }) } private def getUserMaxRunningJobs(keyAndValue: util.Map[String, String]): Int = { From b624a04f09907d9bc574d7f3bf55d19e9efa729b Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Thu, 24 Nov 2022 22:59:19 +0800 Subject: [PATCH 09/72] entrance taskID --- .../linkis/common/entity/JobInstance.scala | 26 + .../server/conf/ServerConfiguration.scala | 3 +- .../entrance/restful/EntranceRestfulApi.java | 1068 +++++++++-------- .../entrance/utils/JobHistoryHelper.scala | 11 +- .../parser/EntranceRequestGatewayParser.scala | 57 +- 5 files changed, 658 insertions(+), 507 deletions(-) create mode 100644 linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala new file mode 100644 index 0000000000..aa9db730ee --- /dev/null +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala @@ -0,0 +1,26 @@ +/* + * 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.common.entity + +case class JobInstance( + status: String, + instances: String, + jobReqId: String, + createTimestamp: Long, + instanceRegistryTimestamp: Long +) diff --git a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala index 6784c5100f..8d9f9d65ad 100644 --- a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala +++ b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala @@ -207,6 +207,7 @@ object ServerConfiguration extends Logging { val LINKIS_SERVER_SESSION_PROXY_TICKETID_KEY = CommonVars("wds.linkis.session.proxy.user.ticket.key", "linkis_user_session_proxy_ticket_id_v1") - val LINKIS_SERVER_HEADER_KEY = CommonVars("wds.linkis.session.proxy.user.ticket.key", "job_req_id") + val LINKIS_SERVER_ENTRANCE_HEADER_KEY = + CommonVars("wds.linkis.server.entrance.header.key", "jobInstanceKey") } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index 4a946e6d0c..8b10b9eb52 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -17,6 +17,7 @@ package org.apache.linkis.entrance.restful; +import org.apache.linkis.common.entity.JobInstance; import org.apache.linkis.common.log.LogUtils; import org.apache.linkis.entrance.EntranceServer; import org.apache.linkis.entrance.conf.EntranceConfiguration; @@ -34,6 +35,7 @@ import org.apache.linkis.scheduler.listener.LogListener; import org.apache.linkis.scheduler.queue.Job; import org.apache.linkis.scheduler.queue.SchedulerEventState; +import org.apache.linkis.server.BDPJettyServerHelper; import org.apache.linkis.server.Message; import org.apache.linkis.server.conf.ServerConfiguration; import org.apache.linkis.server.security.SecurityFilter; @@ -61,6 +63,7 @@ import scala.Option; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.github.xiaoymin.knife4j.annotations.ApiOperationSupport; import io.swagger.annotations.Api; @@ -198,277 +201,353 @@ private void pushLog(String log, Job job) { entranceServer.getEntranceContext().getOrCreateLogManager().onLogUpdate(job, log); } - @ApiOperation(value = "status", notes = "get task stats", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = " task id"), - @ApiImplicitParam(name = "id",required = true, dataType = "String", value = "execute id ") - }) - @Override - @RequestMapping(path = "/{id}/status", method = RequestMethod.GET) - public Message status( - HttpServletRequest req, - @PathVariable("id") String id, - @RequestParam(value = "taskID", required = false) String taskID) { - ModuleUserUtils.getOperationUser(req, "job status"); - Message message = null; - String realId; - String execID; - if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { - // execID - realId = ZuulEntranceUtils.parseExecID(id)[3]; - execID = id; - } else { - // taskID - String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); - if (StringUtils.isEmpty(jobReqId)){ - logger.warn("The job wait failover, return status is Inited"); - String status = SchedulerEventState.Inited().toString(); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/status"); - message.data("status", status).data("execID", "").data("taskID", id); - return message; - } else { - realId = jobReqId; - execID = - ZuulEntranceUtils.generateExecID( - realId, - Sender.getThisServiceInstance().getApplicationName(), - new String[] {Sender.getThisInstance()}); - } - } + private JobInstance parseHeaderToJobInstance(HttpServletRequest req) + throws JsonProcessingException { + String jobStr = + req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().getValue()); + return BDPJettyServerHelper.gson().fromJson(jobStr, JobInstance.class); + } - Option job = Option.apply(null); - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.warn("get {} status error", realId, e); - if (StringUtils.isEmpty(taskID)) { - message = - Message.error( - "Get job by ID error and cannot obtain the corresponding task status.(获取job时发生异常,不能获取相应的任务状态)"); - return message; - } - long realTaskID = Long.parseLong(taskID); - String status = JobHistoryHelper.getStatusByTaskID(realTaskID); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/status"); - message.data("status", status).data("execID", execID); - return message; - } - if (job.isDefined()) { - if (job.get() instanceof EntranceJob) { - ((EntranceJob) job.get()).updateNewestAccessByClientTimestamp(); - } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/status"); - message.data("status", job.get().getState().toString()).data("execID", execID); - } else { - message = - Message.error( - "ID The corresponding job is empty and cannot obtain the corresponding task status.(ID 对应的job为空,不能获取相应的任务状态)"); - } + @ApiOperation(value = "status", notes = "get task stats", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = " task id"), + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id ") + }) + @Override + @RequestMapping(path = "/{id}/status", method = RequestMethod.GET) + public Message status( + HttpServletRequest req, + @PathVariable("id") String id, + @RequestParam(value = "taskID", required = false) String taskID) { + ModuleUserUtils.getOperationUser(req, "job status"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + JobInstance jobInstance; + try { + jobInstance = parseHeaderToJobInstance(req); + } catch (JsonProcessingException e) { + logger.error("parse JobInstance json error, id: {}", id); + message = Message.error("parse JobInstance json error"); + message.setMethod("/api/entrance/" + id + "/status"); return message; + } + + // return ok when job complete + if (SchedulerEventState.isCompletedByStr(jobInstance.status())) { + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", jobInstance.status()).data("execID", "").data("taskID", id); + return message; + } else if (jobInstance.instanceRegistryTimestamp() > jobInstance.createTimestamp()) { + logger.warn("The job {} wait failover, return status is Inited", id); + String status = SchedulerEventState.Inited().toString(); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", status).data("execID", "").data("taskID", id); + return message; + } else { + realId = jobInstance.jobReqId(); + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } } - @ApiOperation(value = "progress", notes = "get task progress info", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id",required = true, dataType = "String", value = "exectue id") - }) - @Override - @RequestMapping(path = "/{id}/progress", method = RequestMethod.GET) - public Message progress(HttpServletRequest req, @PathVariable("id") String id) { - ModuleUserUtils.getOperationUser(req, "job progress"); - Message message = null; - String realId; - String execID; - if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { - // execID - realId = ZuulEntranceUtils.parseExecID(id)[3]; - execID = id; - } else { - // taskID - String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); - if (StringUtils.isEmpty(jobReqId)){ - logger.warn("The job wait failover, return progress is 0"); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/progress"); - message.data("progress", 0) - .data("execID", "") - .data("taskID", id) - .data("progressInfo", new ArrayList<>()); - return message; - } else { - realId = jobReqId; - execID = - ZuulEntranceUtils.generateExecID( - realId, - Sender.getThisServiceInstance().getApplicationName(), - new String[] {Sender.getThisInstance()}); - } - } + Option job = Option.apply(null); + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.warn("get {} status error", realId, e); + if (StringUtils.isEmpty(taskID)) { + message = + Message.error( + "Get job by ID error and cannot obtain the corresponding task status.(获取job时发生异常,不能获取相应的任务状态)"); + return message; + } + long realTaskID = Long.parseLong(taskID); + String status = JobHistoryHelper.getStatusByTaskID(realTaskID); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", status).data("execID", execID); + return message; + } + if (job.isDefined()) { + if (job.get() instanceof EntranceJob) { + ((EntranceJob) job.get()).updateNewestAccessByClientTimestamp(); + } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/status"); + message.data("status", job.get().getState().toString()).data("execID", execID); + } else { + message = + Message.error( + "ID The corresponding job is empty and cannot obtain the corresponding task status.(ID 对应的job为空,不能获取相应的任务状态)"); + } + return message; + } - Option job = null; - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.error(e.getMessage()); - } - if (job != null && job.isDefined()) { - JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); - if (jobProgressInfos == null) { - message = - Message.error( - "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); - message.setMethod("/api/entrance/" + id + "/progress"); - } else { - List> list = new ArrayList<>(); - for (JobProgressInfo jobProgressInfo : jobProgressInfos) { - if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) - || jobProgressInfo.totalTasks() > 0) { - setJobProgressInfos(list, jobProgressInfo); - } - } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/progress"); - - message.data("progress", Math.abs(job.get().getProgress())) - .data("execID", execID) - .data("progressInfo", list); - } - } else { - message = - Message.error( - "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); - } + @ApiOperation(value = "progress", notes = "get task progress info", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "exectue id") + }) + @Override + @RequestMapping(path = "/{id}/progress", method = RequestMethod.GET) + public Message progress(HttpServletRequest req, @PathVariable("id") String id) { + ModuleUserUtils.getOperationUser(req, "job progress"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + JobInstance jobInstance; + try { + jobInstance = parseHeaderToJobInstance(req); + } catch (JsonProcessingException e) { + logger.error("parse JobInstance json error, id: {}", id); + message = Message.error("parse JobInstance json error"); + message.setMethod("/api/entrance/" + id + "/progress"); + return message; + } + + // return ok when job complete + if (SchedulerEventState.isCompletedByStr(jobInstance.status())) { + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progress"); + message + .data("progress", "1.0") + .data("execID", "") + .data("taskID", id) + .data("progressInfo", new ArrayList<>()); + return message; + } else if (jobInstance.instanceRegistryTimestamp() > jobInstance.createTimestamp()) { + logger.warn("The job {} wait failover, return progress is 0", id); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progress"); + message + .data("progress", 0) + .data("execID", "") + .data("taskID", id) + .data("progressInfo", new ArrayList<>()); return message; + } else { + realId = jobInstance.jobReqId(); + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } } - @ApiOperation(value = "progressWithResource", notes = "get progress and resource info", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") - }) - @Override - @RequestMapping(path = "/{id}/progressWithResource", method = RequestMethod.GET) - public Message progressWithResource(HttpServletRequest req, @PathVariable("id") String id) { - ModuleUserUtils.getOperationUser(req, "job progressWithResource"); - Message message = null; - String realId; - String execID; - if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { - // execID - realId = ZuulEntranceUtils.parseExecID(id)[3]; - execID = id; - } else { - // taskID - String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); - if (StringUtils.isEmpty(jobReqId)){ - logger.warn("The job wait failover, return progress is 0 and resource is null"); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/progressWithResource"); - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null) - .data("progress", 0) - .data("execID", "") - .data("taskID", id) - .data("progressInfo", new ArrayList<>()); - return message; - } else { - realId = jobReqId; - execID = - ZuulEntranceUtils.generateExecID( - realId, - Sender.getThisServiceInstance().getApplicationName(), - new String[] {Sender.getThisInstance()}); - } + Option job = null; + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.error(e.getMessage()); + } + if (job != null && job.isDefined()) { + JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); + if (jobProgressInfos == null) { + message = + Message.error( + "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); + message.setMethod("/api/entrance/" + id + "/progress"); + } else { + List> list = new ArrayList<>(); + for (JobProgressInfo jobProgressInfo : jobProgressInfos) { + if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) + || jobProgressInfo.totalTasks() > 0) { + setJobProgressInfos(list, jobProgressInfo); + } } - Option job = null; - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.error(e.getMessage()); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progress"); + + message + .data("progress", Math.abs(job.get().getProgress())) + .data("execID", execID) + .data("progressInfo", list); + } + } else { + message = + Message.error( + "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); + } + return message; + } + + @ApiOperation( + value = "progressWithResource", + notes = "get progress and resource info", + response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") + }) + @Override + @RequestMapping(path = "/{id}/progressWithResource", method = RequestMethod.GET) + public Message progressWithResource(HttpServletRequest req, @PathVariable("id") String id) { + ModuleUserUtils.getOperationUser(req, "job progressWithResource"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + JobInstance jobInstance; + try { + jobInstance = parseHeaderToJobInstance(req); + } catch (JsonProcessingException e) { + logger.error("parse JobInstance json error, id: {}", id); + message = Message.error("parse JobInstance json error"); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + return message; + } + + // return ok when job complete + if (SchedulerEventState.isCompletedByStr(jobInstance.status())) { + long realTaskID = Long.parseLong(id); + JobRequest jobRequest = JobHistoryHelper.getTaskByTaskID(realTaskID); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + Map metricsVo = new HashMap<>(); + buildYarnResource(jobRequest, metricsVo, message); + message + .data("progress", "1.0") + .data("execID", "") + .data("taskID", id) + .data("progressInfo", new ArrayList<>()); + return message; + } else if (jobInstance.instanceRegistryTimestamp() > jobInstance.createTimestamp()) { + logger.warn("The job {} wait failover, return progress is 0 and resource is null", id); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + message + .data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null) + .data("progress", 0) + .data("execID", "") + .data("taskID", id) + .data("progressInfo", new ArrayList<>()); + return message; + } else { + realId = jobInstance.jobReqId(); + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } + } + Option job = null; + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.error(e.getMessage()); + } + if (job != null && job.isDefined()) { + JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); + if (jobProgressInfos == null) { + message = + Message.error( + "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + } else { + List> list = new ArrayList<>(); + for (JobProgressInfo jobProgressInfo : jobProgressInfos) { + if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) + || jobProgressInfo.totalTasks() > 0) { + setJobProgressInfos(list, jobProgressInfo); + } } - if (job != null && job.isDefined()) { - JobProgressInfo[] jobProgressInfos = ((EntranceJob) job.get()).getProgressInfo(); - if (jobProgressInfos == null) { - message = - Message.error( - "Can not get the corresponding progress information, it may be that the corresponding progress information has not been generated(不能获取相应的进度信息,可能是相应的进度信息还未生成)"); - message.setMethod("/api/entrance/" + id + "/progressWithResource"); - } else { - List> list = new ArrayList<>(); - for (JobProgressInfo jobProgressInfo : jobProgressInfos) { - if ("true".equals(EntranceConfiguration.PROGRESS_PUSH().getValue()) - || jobProgressInfo.totalTasks() > 0) { - setJobProgressInfos(list, jobProgressInfo); - } - } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/progressWithResource"); - - JobRequest jobRequest = ((EntranceJob) job.get()).getJobRequest(); - Map metrics = jobRequest.getMetrics(); - Map metricsVo = new HashMap<>(); - if (metrics.containsKey(TaskConstant.ENTRANCEJOB_YARNRESOURCE)) { - HashMap resourceMap = - (HashMap) - metrics.get(TaskConstant.ENTRANCEJOB_YARNRESOURCE); - ArrayList resoureList = new ArrayList<>(12); - if (null != resourceMap && !resourceMap.isEmpty()) { - resourceMap.forEach( - (applicationId, resource) -> { - resoureList.add( - new YarnResourceWithStatusVo(applicationId, resource)); - }); - metricsVo.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, resoureList); - Optional cores = - resourceMap.values().stream() - .map(resource -> resource.queueCores()) - .reduce((x, y) -> x + y); - Optional memory = - resourceMap.values().stream() - .map(resource -> resource.queueMemory()) - .reduce((x, y) -> x + y); - float corePercent = 0.0f; - float memoryPercent = 0.0f; - if (cores.isPresent() && memory.isPresent()) { - corePercent = - cores.get().floatValue() - / EntranceConfiguration.YARN_QUEUE_CORES_MAX() - .getValue(); - memoryPercent = - memory.get().floatValue() - / (EntranceConfiguration.YARN_QUEUE_MEMORY_MAX() - .getValue() - .longValue() - * 1024 - * 1024 - * 1024); - } - String coreRGB = RGBUtils.getRGB(corePercent); - String memoryRGB = RGBUtils.getRGB(memoryPercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_PERCENT, corePercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_PERCENT, memoryPercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_RGB, coreRGB); - metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_RGB, memoryRGB); - - message.data(TaskConstant.ENTRANCEJOB_YARN_METRICS, metricsVo); - } else { - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); - } - } else { - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); - } - - message.data("progress", Math.abs(job.get().getProgress())) - .data("execID", execID) - .data("progressInfo", list); - } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/progressWithResource"); + + JobRequest jobRequest = ((EntranceJob) job.get()).getJobRequest(); + Map metricsVo = new HashMap<>(); + buildYarnResource(jobRequest, metricsVo, message); + + message + .data("progress", Math.abs(job.get().getProgress())) + .data("execID", execID) + .data("progressInfo", list); + } + } else { + message = + Message.error( + "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); + } + return message; + } + + private void buildYarnResource( + JobRequest jobRequest, Map metricsVo, Message message) { + try { + Map metrics = jobRequest.getMetrics(); + if (metrics.containsKey(TaskConstant.ENTRANCEJOB_YARNRESOURCE)) { + + HashMap resourceMap = + (HashMap) + metrics.get(TaskConstant.ENTRANCEJOB_YARNRESOURCE); + ArrayList resoureList = new ArrayList<>(12); + if (null != resourceMap && !resourceMap.isEmpty()) { + resourceMap.forEach( + (applicationId, resource) -> { + resoureList.add(new YarnResourceWithStatusVo(applicationId, resource)); + }); + metricsVo.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, resoureList); + Optional cores = + resourceMap.values().stream() + .map(resource -> resource.queueCores()) + .reduce((x, y) -> x + y); + Optional memory = + resourceMap.values().stream() + .map(resource -> resource.queueMemory()) + .reduce((x, y) -> x + y); + float corePercent = 0.0f; + float memoryPercent = 0.0f; + if (cores.isPresent() && memory.isPresent()) { + corePercent = + cores.get().floatValue() / EntranceConfiguration.YARN_QUEUE_CORES_MAX().getValue(); + memoryPercent = + memory.get().floatValue() + / (EntranceConfiguration.YARN_QUEUE_MEMORY_MAX().getValue().longValue() + * 1024 + * 1024 + * 1024); + } + String coreRGB = RGBUtils.getRGB(corePercent); + String memoryRGB = RGBUtils.getRGB(memoryPercent); + metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_PERCENT, corePercent); + metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_PERCENT, memoryPercent); + metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_RGB, coreRGB); + metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_RGB, memoryRGB); + + message.data(TaskConstant.ENTRANCEJOB_YARN_METRICS, metricsVo); } else { - message = - Message.error( - "The job corresponding to the ID is empty, and the corresponding task progress cannot be obtained.(ID 对应的job为空,不能获取相应的任务进度)"); + message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); } - return message; + } else { + message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); + } + } catch (Exception e) { + logger.error("build yarnResource error", e); } + } private void setJobProgressInfos( List> list, JobProgressInfo jobProgressInfo) { @@ -481,146 +560,157 @@ private void setJobProgressInfos( list.add(map); } - @ApiOperation(value = "log", notes = "get task log", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") - }) - @Override - @RequestMapping(path = "/{id}/log", method = RequestMethod.GET) - public Message log(HttpServletRequest req, @PathVariable("id") String id) { - ModuleUserUtils.getOperationUser(req, "get job log"); - Message message = null; - int fromLine = 0; - int size = 100; - boolean distinctLevel = true; - String fromLineStr = req.getParameter("fromLine"); - String sizeStr = req.getParameter("size"); - if (StringUtils.isNotBlank(fromLineStr)) { - fromLine = Math.max(Integer.parseInt(fromLineStr), 0); - } - if (StringUtils.isNotBlank(sizeStr)) { - size = Integer.parseInt(sizeStr) >= 0 ? Integer.parseInt(sizeStr) : 10000; - } - String distinctLevelStr = req.getParameter("distinctLevel"); - if ("false".equals(distinctLevelStr)) { - distinctLevel = false; - } + @ApiOperation(value = "log", notes = "get task log", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "execute id") + }) + @Override + @RequestMapping(path = "/{id}/log", method = RequestMethod.GET) + public Message log(HttpServletRequest req, @PathVariable("id") String id) { + ModuleUserUtils.getOperationUser(req, "get job log"); + Message message = null; + int fromLine = 0; + int size = 100; + boolean distinctLevel = true; + String fromLineStr = req.getParameter("fromLine"); + String sizeStr = req.getParameter("size"); + if (StringUtils.isNotBlank(fromLineStr)) { + fromLine = Math.max(Integer.parseInt(fromLineStr), 0); + } + if (StringUtils.isNotBlank(sizeStr)) { + size = Integer.parseInt(sizeStr) >= 0 ? Integer.parseInt(sizeStr) : 10000; + } + String distinctLevelStr = req.getParameter("distinctLevel"); + if ("false".equals(distinctLevelStr)) { + distinctLevel = false; + } - String realId; - String execID; - if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { - // execID - realId = ZuulEntranceUtils.parseExecID(id)[3]; - execID = id; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + JobInstance jobInstance; + try { + jobInstance = parseHeaderToJobInstance(req); + } catch (JsonProcessingException e) { + logger.error("parse JobInstance json error, id: {}", id); + message = Message.error("parse JobInstance json error"); + message.setMethod("/api/entrance/" + id + "/log"); + return message; + } + + // return ok when job complete + if (SchedulerEventState.isCompletedByStr(jobInstance.status())) { + message = + Message.error( + "The job you just executed has ended. This interface no longer provides a query. It is recommended that you download the log file for viewing.(您刚刚执行的job已经结束,本接口不再提供查询,建议您下载日志文件进行查看)"); + message.setMethod("/api/entrance/" + id + "/log"); + return message; + } else if (jobInstance.instanceRegistryTimestamp() > jobInstance.createTimestamp()) { + logger.warn("The job {} wait failover, return customer log", id); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + String log = LogUtils.generateInfo("The job will failover soon, please try again later"); + Object retLog; + if (distinctLevel) { + String[] array = new String[4]; + array[2] = log; + array[3] = log; + retLog = new ArrayList(Arrays.asList(array)); } else { - // taskID - String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); - if (StringUtils.isEmpty(jobReqId)){ - logger.warn("The job wait failover, return customer log"); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - String log = LogUtils.generateInfo("The job will failover soon, please try again later"); - Object retLog; - if (distinctLevel) { - String[] array = new String[4]; - array[2] = log; - array[3] = log; - retLog = new ArrayList(Arrays.asList(array)); - } else { - retLog = log; - } - message.data("log", retLog).data("execID", "").data("taskID", id).data("fromLine", 0); - return message; - } else { - realId = jobReqId; - execID = - ZuulEntranceUtils.generateExecID( - realId, - Sender.getThisServiceInstance().getApplicationName(), - new String[] {Sender.getThisInstance()}); - } + retLog = log; } + message.data("log", retLog).data("execID", "").data("taskID", id).data("fromLine", 0); + return message; + } else { + realId = jobInstance.jobReqId(); + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } + } - Option job = Option.apply(null); - try { - job = entranceServer.getJob(realId); - } catch (final Throwable t) { - message = - Message.error( - "The job you just executed has ended. This interface no longer provides a query. It is recommended that you download the log file for viewing.(您刚刚执行的job已经结束,本接口不再提供查询,建议您下载日志文件进行查看)"); - message.setMethod("/api/entrance/" + id + "/log"); - return message; - } - if (job.isDefined()) { - logger.debug( - "begin to get log for {}(开始获取 {} 的日志)", job.get().getId(), job.get().getId()); - LogReader logReader = - entranceServer - .getEntranceContext() - .getOrCreateLogManager() - .getLogReader(realId); - - Object retLog = null; - int retFromLine = 0; - try { - if (distinctLevel) { - String[] logs = new String[4]; - retFromLine = logReader.readArray(logs, fromLine, size); - retLog = new ArrayList(Arrays.asList(logs)); - } else { - StringBuilder sb = new StringBuilder(); - retFromLine = logReader.read(sb, fromLine, size); - retLog = sb.toString(); - } - } catch (IllegalStateException e) { - logger.debug( - "Failed to get log information for :{}(为 {} 获取日志失败)", - job.get().getId(), - job.get().getId(), - e); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); - } catch (final IllegalArgumentException e) { - logger.debug( - "Failed to get log information for :{}(为 {} 获取日志失败)", - job.get().getId(), - job.get().getId(), - e); - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); - return message; - } catch (final Exception e1) { - logger.debug( - "Failed to get log information for :{}(为 {} 获取日志失败)", - job.get().getId(), - job.get().getId(), - e1); - message = Message.error("Failed to get log information(获取日志信息失败)"); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); - return message; - } finally { - if (null != logReader && job.get().isCompleted()) { - IOUtils.closeQuietly(logReader); - } - } - message = Message.ok(); - message.setMethod("/api/entrance/" + id + "/log"); - message.data("log", retLog).data("execID", execID).data("fromLine", retFromLine + fromLine); - logger.debug( - "success to get log for {} (获取 {} 日志成功)", job.get().getId(), job.get().getId()); + Option job = Option.apply(null); + try { + job = entranceServer.getJob(realId); + } catch (final Throwable t) { + message = + Message.error( + "The job you just executed has ended. This interface no longer provides a query. It is recommended that you download the log file for viewing.(您刚刚执行的job已经结束,本接口不再提供查询,建议您下载日志文件进行查看)"); + message.setMethod("/api/entrance/" + id + "/log"); + return message; + } + if (job.isDefined()) { + logger.debug("begin to get log for {}(开始获取 {} 的日志)", job.get().getId(), job.get().getId()); + LogReader logReader = + entranceServer.getEntranceContext().getOrCreateLogManager().getLogReader(realId); + + Object retLog = null; + int retFromLine = 0; + try { + if (distinctLevel) { + String[] logs = new String[4]; + retFromLine = logReader.readArray(logs, fromLine, size); + retLog = new ArrayList(Arrays.asList(logs)); } else { - message = - Message.error( - "Can't find execID(不能找到execID): " - + id - + "Corresponding job, can not get the corresponding log(对应的job,不能获得对应的日志)"); - message.setMethod("/api/entrance/" + id + "/log"); + StringBuilder sb = new StringBuilder(); + retFromLine = logReader.read(sb, fromLine, size); + retLog = sb.toString(); } + } catch (IllegalStateException e) { + logger.debug( + "Failed to get log information for :{}(为 {} 获取日志失败)", + job.get().getId(), + job.get().getId(), + e); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); + } catch (final IllegalArgumentException e) { + logger.debug( + "Failed to get log information for :{}(为 {} 获取日志失败)", + job.get().getId(), + job.get().getId(), + e); + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); + return message; + } catch (final Exception e1) { + logger.debug( + "Failed to get log information for :{}(为 {} 获取日志失败)", + job.get().getId(), + job.get().getId(), + e1); + message = Message.error("Failed to get log information(获取日志信息失败)"); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", "").data("execID", execID).data("fromLine", retFromLine + fromLine); return message; + } finally { + if (null != logReader && job.get().isCompleted()) { + IOUtils.closeQuietly(logReader); + } + } + message = Message.ok(); + message.setMethod("/api/entrance/" + id + "/log"); + message.data("log", retLog).data("execID", execID).data("fromLine", retFromLine + fromLine); + logger.debug("success to get log for {} (获取 {} 日志成功)", job.get().getId(), job.get().getId()); + } else { + message = + Message.error( + "Can't find execID(不能找到execID): " + + id + + "Corresponding job, can not get the corresponding log(对应的job,不能获得对应的日志)"); + message.setMethod("/api/entrance/" + id + "/log"); } + return message; + } @ApiOperation(value = "killJobs", notes = "kill jobs", response = Message.class) @ApiImplicitParams({ @@ -718,104 +808,116 @@ public Message killJobs( return Message.ok("success").data("messages", messages); } - @ApiOperation(value = "kill", notes = "kill", response = Message.class) - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "exec id"), - @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = "task id") - }) - @Override - @RequestMapping(path = "/{id}/kill", method = RequestMethod.GET) - public Message kill( - HttpServletRequest req, - @PathVariable("id") String id, - @RequestParam(value = "taskID", required = false) Long taskID) { - ModuleUserUtils.getOperationUser(req, "kill job"); - Message message = null; - String realId; - String execID; - if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { - // execID - realId = ZuulEntranceUtils.parseExecID(id)[3]; - execID = id; - } else { - // taskID - String jobReqId = req.getHeader(ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY().toString()); - if (StringUtils.isEmpty(jobReqId)){ - logger.warn("The job wait failover, but now force kill"); - // TODO If failover occurs during force kill, the job status may change from Cancelled to Running - long taskId = Long.parseLong(id); - JobHistoryHelper.forceKill(taskId); - message = Message.ok("Forced Kill task (强制杀死任务)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.data("execID", "").data("taskID", id); - return message; - } else { - realId = jobReqId; - execID = - ZuulEntranceUtils.generateExecID( - realId, - Sender.getThisServiceInstance().getApplicationName(), - new String[] {Sender.getThisInstance()}); - } - } + @ApiOperation(value = "kill", notes = "kill", response = Message.class) + @ApiImplicitParams({ + @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "exec id"), + @ApiImplicitParam(name = "taskID", required = false, dataType = "String", value = "task id") + }) + @Override + @RequestMapping(path = "/{id}/kill", method = RequestMethod.GET) + public Message kill( + HttpServletRequest req, + @PathVariable("id") String id, + @RequestParam(value = "taskID", required = false) Long taskID) { + ModuleUserUtils.getOperationUser(req, "kill job"); + Message message = null; + String realId; + String execID; + if (id.startsWith(ZuulEntranceUtils.EXEC_ID())) { + // execID + realId = ZuulEntranceUtils.parseExecID(id)[3]; + execID = id; + } else { + // taskID + JobInstance jobInstance; + try { + jobInstance = parseHeaderToJobInstance(req); + } catch (JsonProcessingException e) { + logger.error("parse JobInstance json error, id: {}", id); + message = Message.error("parse JobInstance json error"); + message.setMethod("/api/entrance/" + id + "/kill"); + return message; + } - Option job = Option.apply(null); - try { - job = entranceServer.getJob(realId); - } catch (Exception e) { - logger.warn("can not find a job in entranceServer, will force to kill it", e); - // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 - if (taskID == null || taskID <= 0) { - message = - Message.error( - "Get job by ID error, kill failed.(获取job时发生异常,kill失败)"); - return message; - } - JobHistoryHelper.forceKill(taskID); - message = Message.ok("Forced Kill task (强制杀死任务)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(0); - return message; - } + // return ok when job complete + if (SchedulerEventState.isCompletedByStr(jobInstance.status())) { + message = Message.error("The job already completed. Do not support kill.(任务已经结束,不支持kill)"); + message.setMethod("/api/entrance/" + id + "/kill"); + return message; + } else if (jobInstance.instanceRegistryTimestamp() > jobInstance.createTimestamp()) { + logger.warn("The job {} wait failover, but now force kill", id); + // TODO If failover during force kill, the job status may change from Cancelled to Running + long taskId = Long.parseLong(id); + JobHistoryHelper.forceKill(taskId); + message = Message.ok("Forced Kill task (强制杀死任务)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.data("execID", "").data("taskID", id); + return message; + } else { + realId = jobInstance.jobReqId(); + execID = + ZuulEntranceUtils.generateExecID( + realId, + Sender.getThisServiceInstance().getApplicationName(), + new String[] {Sender.getThisInstance()}); + } + } - if (job.isEmpty()) { - logger.warn("can not find a job in entranceServer, will force to kill it"); - // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 - JobHistoryHelper.forceKill(taskID); - message = Message.ok("Forced Kill task (强制杀死任务)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(0); - return message; - } else { - try { - logger.info("begin to kill job {} ", job.get().getId()); - job.get().kill(); - message = Message.ok("Successfully killed the job(成功kill了job)"); - message.setMethod("/api/entrance/" + id + "/kill"); - message.data("execID", execID); - // ensure the job's state is cancelled in database - if (job.get() instanceof EntranceJob) { - EntranceJob entranceJob = (EntranceJob) job.get(); - JobRequest jobReq = entranceJob.getJobRequest(); - entranceJob.updateJobRequestStatus(SchedulerEventState.Cancelled().toString()); - this.entranceServer - .getEntranceContext() - .getOrCreatePersistenceManager() - .createPersistenceEngine() - .updateIfNeeded(jobReq); - } - logger.info("end to kill job {} ", job.get().getId()); - } catch (Throwable t) { - logger.error("kill job {} failed ", job.get().getId(), t); - message = - Message.error( - "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败)"); - message.setMethod("/api/entrance/" + id + "/kill"); - } - } + Option job = Option.apply(null); + try { + job = entranceServer.getJob(realId); + } catch (Exception e) { + logger.warn("can not find a job in entranceServer, will force to kill it", e); + // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 + if (taskID == null || taskID <= 0) { + message = Message.error("Get job by ID error, kill failed.(获取job时发生异常,kill失败)"); return message; + } + JobHistoryHelper.forceKill(taskID); + message = Message.ok("Forced Kill task (强制杀死任务)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.setStatus(0); + return message; } + if (job.isEmpty()) { + logger.warn("can not find a job in entranceServer, will force to kill it"); + // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 + JobHistoryHelper.forceKill(taskID); + message = Message.ok("Forced Kill task (强制杀死任务)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.setStatus(0); + return message; + } else { + try { + logger.info("begin to kill job {} ", job.get().getId()); + job.get().kill(); + message = Message.ok("Successfully killed the job(成功kill了job)"); + message.setMethod("/api/entrance/" + id + "/kill"); + message.data("execID", execID); + // ensure the job's state is cancelled in database + if (job.get() instanceof EntranceJob) { + EntranceJob entranceJob = (EntranceJob) job.get(); + JobRequest jobReq = entranceJob.getJobRequest(); + entranceJob.updateJobRequestStatus(SchedulerEventState.Cancelled().toString()); + this.entranceServer + .getEntranceContext() + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(jobReq); + } + logger.info("end to kill job {} ", job.get().getId()); + } catch (Throwable t) { + logger.error("kill job {} failed ", job.get().getId(), t); + message = + Message.error( + "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败)"); + message.setMethod("/api/entrance/" + id + "/kill"); + } + } + return message; + } + @ApiOperation(value = "pause ", notes = "puase a task job", response = Message.class) @ApiImplicitParams({ @ApiImplicitParam(name = "id", required = true, dataType = "String", value = "excete id") diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index 6416fe1d47..811af8fce5 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -67,6 +67,11 @@ object JobHistoryHelper extends Logging { else task.getStatus } + def getProgressByTaskID(taskID: Long): String = { + val task = getTaskByTaskID(taskID) + if (task == null) "0" else task.getProgress + } + def getRequestIpAddr(req: HttpServletRequest): String = { val addrList = List( Option(req.getHeader("x-forwarded-for")).getOrElse("").split(",")(0), @@ -228,8 +233,8 @@ object JobHistoryHelper extends Logging { } val data = responsePersist.getData data.get(JobRequestConstants.JOB_HISTORY_LIST) match { - case tasks: util.List[JobRequest] => - tasks + case tasks: List[JobRequest] => + tasks.asJava case _ => throw JobHistoryFailedException( s"query from jobhistory not a correct List type, instances ${reqMap.keySet()}" @@ -250,7 +255,7 @@ object JobHistoryHelper extends Logging { tasks } - private def getTaskByTaskID(taskID: Long): JobRequest = { + def getTaskByTaskID(taskID: Long): JobRequest = { val jobRequest = new JobRequest jobRequest.setId(taskID) jobRequest.setSource(null) diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala index 9fb3958ac0..a1be26de87 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala @@ -19,14 +19,16 @@ package org.apache.linkis.gateway.ujes.parser import org.apache.commons.lang3.StringUtils import org.apache.linkis.common.ServiceInstance +import org.apache.linkis.common.entity.JobInstance +import org.apache.linkis.common.utils.JsonUtils import org.apache.linkis.gateway.config.GatewayConfiguration import org.apache.linkis.gateway.http.GatewayContext import org.apache.linkis.gateway.parser.AbstractGatewayParser import org.apache.linkis.gateway.ujes.parser.EntranceExecutionGatewayParser._ -import org.apache.linkis.jobhistory.entity.JobHistory import org.apache.linkis.jobhistory.service.JobHistoryQueryService import org.apache.linkis.protocol.utils.ZuulEntranceUtils import org.apache.linkis.rpc.interceptor.ServiceInstanceUtils +import org.apache.linkis.server.BDPJettyServerHelper import org.apache.linkis.server.conf.ServerConfiguration import org.springframework.stereotype.Component @@ -65,35 +67,50 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { // parse by execId ZuulEntranceUtils.parseServiceInstanceByExecID(execId)(0) } else { - // check by taskId - val jobHistory = checkJobValidityByTaskID(execId.toLong, gatewayContext) - // add header - val jobReqId = if (jobHistory == null) "" else jobHistory.getJobReqId - gatewayContext.getRequest.addHeader(ServerConfiguration.LINKIS_SERVER_HEADER_KEY.getValue, Array(jobReqId)) - // select instance - val instance = if (jobHistory == null) null else jobHistory.getInstances - ServiceInstance(GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue, instance) + // build JobInstance by taskId + val jobInstance = buildJobInstance(execId.toLong, gatewayContext) + if (jobInstance == null) return + val str = BDPJettyServerHelper.gson.toJson(jobInstance) + gatewayContext.getRequest.addHeader( + ServerConfiguration.LINKIS_SERVER_ENTRANCE_HEADER_KEY.getValue, + Array(str) + ) + + ServiceInstance(GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue, jobInstance.instances) } gatewayContext.getGatewayRoute.setServiceInstance(serviceInstance) case _ => } - def checkJobValidityByTaskID(taskId: Long, gatewayContext: GatewayContext): JobHistory = { + def buildJobInstance(taskId: Long, gatewayContext: GatewayContext): JobInstance = { val histories = jobHistoryQueryService.search(taskId, null, null, null, null, null, null, null) if (histories.isEmpty) { sendErrorResponse(s"taskId $taskId is not exists.", gatewayContext) + return null } - val instances = histories.get(0).getInstances - val activeInstances = ServiceInstanceUtils.getRPCServerLoader.getServiceInstances(GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue) - - if (activeInstances.exists(StringUtils.isNotBlank(instances) && _.getInstance.equals(instances)) && - activeInstances.filter(_.getInstance.equals(instances))(0).getRegistryTimestamp <= histories.get(0).getCreatedTime.getTime - ) { - histories.get(0) - } else { - null + val history = histories.get(0) + if (StringUtils.isEmpty(history.getInstances)) { + return JobInstance( + history.getStatus, + null, + history.getJobReqId, + history.getCreatedTime.getTime, + Long.MaxValue + ) } - + val activeInstances = ServiceInstanceUtils.getRPCServerLoader.getServiceInstances( + GatewayConfiguration.ENTRANCE_SPRING_NAME.getValue + ) + val instance = activeInstances + .find(_.getInstance.equals(history.getInstances)) + .getOrElse(ServiceInstance("", "", Long.MaxValue)) + JobInstance( + history.getStatus, + history.getInstances, + history.getJobReqId, + history.getCreatedTime.getTime, + instance.getRegistryTimestamp + ) } } From cbfbdff07b244fc0d4e140af468b277bb6aa7ecb Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Thu, 24 Nov 2022 23:02:08 +0800 Subject: [PATCH 10/72] update status limit for failover --- .../governance/common/entity/job/JobRequest.java | 10 ++++++++++ .../org/apache/linkis/entrance/EntranceServer.scala | 1 + .../service/impl/JobHistoryQueryServiceImpl.scala | 4 ++-- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java index d5d97aa364..01f9df3f5d 100644 --- a/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java @@ -51,6 +51,8 @@ public class JobRequest { private String observeInfo; + private Boolean updateLimitFlag = true; + private Map metrics = new HashMap<>(); public Long getId() { @@ -205,6 +207,14 @@ public void setObserveInfo(String observeInfo) { this.observeInfo = observeInfo; } + public Boolean getUpdateLimitFlag() { + return updateLimitFlag; + } + + public void setUpdateLimitFlag(Boolean updateLimitFlag) { + this.updateLimitFlag = updateLimitFlag; + } + @Override public String toString() { return "JobRequest{" diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index b09ef4911a..42c7f8ea67 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -468,6 +468,7 @@ abstract class EntranceServer extends Logging { jobRequest.setErrorDesc("") jobRequest.setMetrics(metricMap) jobRequest.getMetrics.put(TaskConstant.ENTRANCEJOB_SUBMIT_TIME, initDate) + jobRequest.setUpdateLimitFlag(false) logAppender.append( LogUtils.generateInfo(s"Job ${jobRequest.getId} success to initialize the properties \n") diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala index c918ee085c..bb90fee2dc 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala @@ -109,7 +109,7 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { logger.info(s"${jobReq.getErrorDesc}") } } - if (jobReq.getStatus != null) { + if (jobReq.getUpdateLimitFlag && jobReq.getStatus != null) { val oldStatus: String = jobHistoryMapper.selectJobHistoryStatusForUpdate(jobReq.getId) if (oldStatus != null && !shouldUpdate(oldStatus, jobReq.getStatus)) { throw new QueryException( @@ -174,7 +174,7 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { logger.info(s"${jobReq.getErrorDesc}") } } - if (jobReq.getStatus != null) { + if (jobReq.getUpdateLimitFlag && jobReq.getStatus != null) { val oldStatus: String = jobHistoryMapper.selectJobHistoryStatusForUpdate(jobReq.getId) if (oldStatus != null && !shouldUpdate(oldStatus, jobReq.getStatus)) { throw new QueryException( From d7eb30227d674a4339934951d8f1e475153351d7 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 25 Nov 2022 15:30:38 +0800 Subject: [PATCH 11/72] [bug-fix] failover logic --- .../scala/org/apache/linkis/entrance/EntranceServer.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 42c7f8ea67..ad386be806 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -282,6 +282,7 @@ abstract class EntranceServer extends Logging { logger.info( s"job ${jobRequest.getId} is not running,scheduled or not have EC info, ignore it" ) + return } val engineMap = jobRequest.getMetrics @@ -294,7 +295,7 @@ abstract class EntranceServer extends Logging { .filter(_.containsKey(TaskConstant.ENGINE_INSTANCE)) .maxBy(_.getOrDefault(TaskConstant.ENGINE_CONN_SUBMIT_TIME, "0").toString) - if (engineInstance != null || engineInstance.containsKey(TaskConstant.FAILOVER_FLAG)) { + if (engineInstance == null || engineInstance.containsKey(TaskConstant.FAILOVER_FLAG)) { logger.info( s"job ${jobRequest.getId} do not submit to EC or already failover, not need kill ec" ) @@ -328,8 +329,8 @@ abstract class EntranceServer extends Logging { s"job ${jobRequest.getId} send RequestTaskKill to kill engineConn $ecInstance, execID $engineTaskId" ) } - } { case e: Exception => - logger.error(s"job ${jobRequest.getId} kill ec error", e) + } { t => + logger.error(s"job ${jobRequest.getId} kill ec error", t) } } From 88017f36df3cabb72b6d0c9c72e4723b9229757a Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 25 Nov 2022 15:31:33 +0800 Subject: [PATCH 12/72] add failover log --- .../server/EntranceFailoverJobServer.java | 9 ++-- .../linkis/entrance/EntranceServer.scala | 45 +++++++++++++++---- 2 files changed, 39 insertions(+), 15 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 1eb29a48fb..cebb7c68b5 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -116,15 +116,12 @@ public void run() { expiredTimestamp, EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); if (jobRequests.isEmpty()) return; - logger.info( - "success query failover jobs , job ids: {}", - jobRequests.stream().map(JobRequest::getId)); + Object[] ids = jobRequests.stream().map(JobRequest::getId).toArray(); + logger.info("success query failover jobs , job ids: {}", ids); // failover to local server jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); - logger.info( - "success execute failover jobs, job ids: {}", - jobRequests.stream().map(JobRequest::getId)); + logger.info("success execute failover jobs, job ids: {}", ids); } catch (Exception e) { logger.error("failover failed", e); diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index ad386be806..4ba011a5c3 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -43,8 +43,8 @@ import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.linkis.common.log.LogUtils -import java.text.MessageFormat import java.{lang, util} +import java.text.{MessageFormat, SimpleDateFormat} import java.util.Date import scala.collection.JavaConverters._ @@ -360,6 +360,8 @@ abstract class EntranceServer extends Logging { .createPersistenceEngine() .updateIfNeeded(jobRequest) + logger.info(s"job ${jobRequest.getId} update JobRequest success") + val job = getEntranceContext.getOrCreateEntranceParser().parseToJob(jobRequest) Utils.tryThrow { job.init() @@ -386,7 +388,9 @@ abstract class EntranceServer extends Logging { */ Utils.tryAndWarn(job.getJobListener.foreach(_.onJobInited(job))) getEntranceContext.getOrCreateScheduler().submit(job) - val msg = s"Job with jobId : ${jobRequest.getId} and execID : ${job.getId()} submitted, success to failover" + val msg = LogUtils.generateInfo( + s"Job with jobId : ${jobRequest.getId} and execID : ${job.getId()} submitted, success to failover" + ) logger.info(msg) job match { @@ -421,22 +425,36 @@ abstract class EntranceServer extends Logging { } - private def initJobRequestProperties(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { - + private def initJobRequestProperties( + jobRequest: JobRequest, + logAppender: lang.StringBuilder + ): Unit = { + logger.info(s"Job ${jobRequest.getId} start to initialize the properties") + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") val initInstance = Sender.getThisInstance val initDate = new Date(System.currentTimeMillis) val initStatus = SchedulerEventState.Inited.toString val initProgress = "0.0" val initReqId = "" + logAppender.append("\n\n") logAppender.append( - LogUtils.generateInfo(s"Job ${jobRequest.getId} start to failover, Initialize the properties \n") + LogUtils + .generateInfo( + s"*************************************FAILOVER************************************** \n" + ) + ) + logAppender.append( + LogUtils + .generateInfo(s"Job ${jobRequest.getId} start to failover, Initialize the properties \n") ) logAppender.append( LogUtils.generateInfo(s"the instances ${jobRequest.getInstances} -> ${initInstance} \n") ) logAppender.append( - LogUtils.generateInfo(s"the created_time ${jobRequest.getCreatedTime} -> ${initDate} \n") + LogUtils.generateInfo( + s"the created_time ${sdf.format(jobRequest.getCreatedTime)} -> ${sdf.format(initDate)} \n" + ) ) logAppender.append( LogUtils.generateInfo(s"the status ${jobRequest.getStatus} -> $initStatus \n") @@ -444,9 +462,6 @@ abstract class EntranceServer extends Logging { logAppender.append( LogUtils.generateInfo(s"the progress ${jobRequest.getProgress} -> $initProgress \n") ) - logAppender.append( - LogUtils.generateInfo(s"the job_req_id ${jobRequest.getReqId} -> $initReqId \n") - ) val metricMap = new util.HashMap[String, Object]() if ( @@ -460,6 +475,17 @@ abstract class EntranceServer extends Logging { metricMap.put(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP, oldEngineconnMap) } + if ( + jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( + TaskConstant.ENTRANCEJOB_YARNRESOURCE + ) + ) { + val oldResourceMap = jobRequest.getMetrics + .get(TaskConstant.ENTRANCEJOB_YARNRESOURCE) + .asInstanceOf[util.Map[String, Object]] + metricMap.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, oldResourceMap) + } + jobRequest.setInstances(initInstance) jobRequest.setCreatedTime(initDate) jobRequest.setStatus(initStatus) @@ -474,6 +500,7 @@ abstract class EntranceServer extends Logging { logAppender.append( LogUtils.generateInfo(s"Job ${jobRequest.getId} success to initialize the properties \n") ) + logger.info(s"Job ${jobRequest.getId} success to initialize the properties") } } From e32118958d339749ef5b414ce4abfd1bda4882ab Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 25 Nov 2022 16:34:07 +0800 Subject: [PATCH 13/72] push log to entrance --- .../linkis/entrance/EntranceServer.scala | 101 ++++++++++-------- .../entrance/conf/EntranceConfiguration.scala | 6 ++ 2 files changed, 63 insertions(+), 44 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 4ba011a5c3..b18441549a 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -21,6 +21,7 @@ import org.apache.linkis.common.ServiceInstance import org.apache.linkis.common.exception.{ErrorException, LinkisException, LinkisRuntimeException} import org.apache.linkis.common.log.LogUtils import org.apache.linkis.common.utils.{Logging, Utils} +import org.apache.linkis.entrance.conf.EntranceConfiguration import org.apache.linkis.entrance.cs.CSEntranceHelper import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary._ import org.apache.linkis.entrance.exception.{EntranceErrorException, SubmitFailedException} @@ -271,17 +272,25 @@ abstract class EntranceServer extends Logging { logger.info("Finished to clean all ConsumeQueue") } - def killEC(jobRequest: JobRequest): Unit = { + def killEC(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { Utils.tryCatch { if ( !SchedulerEventState.isRunning(SchedulerEventState.withName(jobRequest.getStatus)) || !SchedulerEventState.isScheduled(SchedulerEventState.withName(jobRequest.getStatus)) - || jobRequest.getMetrics == null + ) { + val msg = s"job ${jobRequest.getId} status is not running or scheduled, ignore it" + logger.info(msg) + logAppender.append(LogUtils.generateInfo(msg) + "\n") + return + } + + if ( + jobRequest.getMetrics == null || !jobRequest.getMetrics.containsKey(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) ) { - logger.info( - s"job ${jobRequest.getId} is not running,scheduled or not have EC info, ignore it" - ) + val msg = s"job ${jobRequest.getId} not have EC info, ignore it" + logger.info(msg) + logAppender.append(LogUtils.generateInfo(msg) + "\n") return } @@ -296,9 +305,10 @@ abstract class EntranceServer extends Logging { .maxBy(_.getOrDefault(TaskConstant.ENGINE_CONN_SUBMIT_TIME, "0").toString) if (engineInstance == null || engineInstance.containsKey(TaskConstant.FAILOVER_FLAG)) { - logger.info( + val msg = s"job ${jobRequest.getId} do not submit to EC or already failover, not need kill ec" - ) + logger.info(msg) + logAppender.append(LogUtils.generateInfo(msg) + "\n") return } engineInstance.put(TaskConstant.FAILOVER_FLAG, "") @@ -315,9 +325,10 @@ abstract class EntranceServer extends Logging { Sender .getSender(RPCConfiguration.LINKIS_MANAGER_APPLICATION_NAME.getValue) .send(engineStopRequest) - logger.info( + val msg = s"job ${jobRequest.getId} send EngineStopRequest to linkismanager, kill instance $ecInstance" - ) + logger.info(msg) + logAppender.append(LogUtils.generateInfo(msg) + "\n") } else if (engineInstance.containsKey(TaskConstant.ENGINE_CONN_TASK_ID)) { // kill ec task val engineTaskId = engineInstance.get(TaskConstant.ENGINE_CONN_TASK_ID).toString @@ -325,9 +336,10 @@ abstract class EntranceServer extends Logging { Sender .getSender(ecInstance) .send(RequestTaskKill(engineTaskId)) - logger.info( + val msg = s"job ${jobRequest.getId} send RequestTaskKill to kill engineConn $ecInstance, execID $engineTaskId" - ) + logger.info(msg) + logAppender.append(LogUtils.generateInfo(msg) + "\n") } } { t => logger.error(s"job ${jobRequest.getId} kill ec error", t) @@ -347,11 +359,15 @@ abstract class EntranceServer extends Logging { PERSIST_JOBREQUEST_ERROR.getErrorDesc ) } - - // try to kill ec - killEC(jobRequest); - val logAppender = new java.lang.StringBuilder() + logAppender.append( + LogUtils + .generateInfo( + s"\n\n *************************************FAILOVER************************************** \n" + ) + ) + // try to kill ec + killEC(jobRequest, logAppender); // init properties initJobRequestProperties(jobRequest, logAppender) // update jobRequest @@ -429,7 +445,7 @@ abstract class EntranceServer extends Logging { jobRequest: JobRequest, logAppender: lang.StringBuilder ): Unit = { - logger.info(s"Job ${jobRequest.getId} start to initialize the properties") + logger.info(s"job ${jobRequest.getId} start to initialize the properties") val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") val initInstance = Sender.getThisInstance val initDate = new Date(System.currentTimeMillis) @@ -437,16 +453,9 @@ abstract class EntranceServer extends Logging { val initProgress = "0.0" val initReqId = "" - logAppender.append("\n\n") logAppender.append( LogUtils - .generateInfo( - s"*************************************FAILOVER************************************** \n" - ) - ) - logAppender.append( - LogUtils - .generateInfo(s"Job ${jobRequest.getId} start to failover, Initialize the properties \n") + .generateInfo(s"job ${jobRequest.getId} start to failover, Initialize the properties \n") ) logAppender.append( LogUtils.generateInfo(s"the instances ${jobRequest.getInstances} -> ${initInstance} \n") @@ -464,26 +473,30 @@ abstract class EntranceServer extends Logging { ) val metricMap = new util.HashMap[String, Object]() - if ( - jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( - TaskConstant.ENTRANCEJOB_ENGINECONN_MAP - ) - ) { - val oldEngineconnMap = jobRequest.getMetrics - .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) - .asInstanceOf[util.Map[String, Object]] - metricMap.put(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP, oldEngineconnMap) + if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_ENGINE_CONN_ENABLED.getValue) { + if ( + jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( + TaskConstant.ENTRANCEJOB_ENGINECONN_MAP + ) + ) { + val oldEngineconnMap = jobRequest.getMetrics + .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + .asInstanceOf[util.Map[String, Object]] + metricMap.put(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP, oldEngineconnMap) + } } - if ( - jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( - TaskConstant.ENTRANCEJOB_YARNRESOURCE - ) - ) { - val oldResourceMap = jobRequest.getMetrics - .get(TaskConstant.ENTRANCEJOB_YARNRESOURCE) - .asInstanceOf[util.Map[String, Object]] - metricMap.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, oldResourceMap) + if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_YARN_RESOURCE_ENABLED.getValue) { + if ( + jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( + TaskConstant.ENTRANCEJOB_YARNRESOURCE + ) + ) { + val oldResourceMap = jobRequest.getMetrics + .get(TaskConstant.ENTRANCEJOB_YARNRESOURCE) + .asInstanceOf[util.Map[String, Object]] + metricMap.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, oldResourceMap) + } } jobRequest.setInstances(initInstance) @@ -498,9 +511,9 @@ abstract class EntranceServer extends Logging { jobRequest.setUpdateLimitFlag(false) logAppender.append( - LogUtils.generateInfo(s"Job ${jobRequest.getId} success to initialize the properties \n") + LogUtils.generateInfo(s"job ${jobRequest.getId} success to initialize the properties \n") ) - logger.info(s"Job ${jobRequest.getId} success to initialize the properties") + logger.info(s"job ${jobRequest.getId} success to initialize the properties") } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 62c42cfdd0..ada2048097 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -246,4 +246,10 @@ object EntranceConfiguration { val ENTRANCE_GROUP_SCAN_ENABLED = CommonVars("linkis.entrance.group.scan.enable", true) + val ENTRANCE_FAILOVER_RETAIN_ENGINE_CONN_ENABLED = + CommonVars("linkis.entrance.failover.retain.engine.conn.enable", true) + + val ENTRANCE_FAILOVER_RETAIN_YARN_RESOURCE_ENABLED = + CommonVars("linkis.entrance.failover.retain.yarn.resource.enable", true) + } From 2dc0c7297a8d18051c24f74b5c3397238f0399ca Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 25 Nov 2022 20:56:55 +0800 Subject: [PATCH 14/72] add entrance log --- .../org/apache/linkis/entrance/EntranceServer.scala | 8 ++++++-- .../entrance/scheduler/EntranceGroupFactory.scala | 10 ++++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index b18441549a..354dafa118 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -274,6 +274,10 @@ abstract class EntranceServer extends Logging { def killEC(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { Utils.tryCatch { + logAppender.append( + LogUtils + .generateInfo(s"job ${jobRequest.getId} start to kill ec \n") + ) if ( !SchedulerEventState.isRunning(SchedulerEventState.withName(jobRequest.getStatus)) || !SchedulerEventState.isScheduled(SchedulerEventState.withName(jobRequest.getStatus)) @@ -363,7 +367,7 @@ abstract class EntranceServer extends Logging { logAppender.append( LogUtils .generateInfo( - s"\n\n *************************************FAILOVER************************************** \n" + s"\n\n*************************************FAILOVER************************************** \n\n" ) ) // try to kill ec @@ -455,7 +459,7 @@ abstract class EntranceServer extends Logging { logAppender.append( LogUtils - .generateInfo(s"job ${jobRequest.getId} start to failover, Initialize the properties \n") + .generateInfo(s"job ${jobRequest.getId} start to Initialize the properties \n") ) logAppender.append( LogUtils.generateInfo(s"the instances ${jobRequest.getInstances} -> ${initInstance} \n") diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala index 2a7432ee6e..c38fae5e4a 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala @@ -193,18 +193,20 @@ class EntranceGroupFactory extends GroupFactory with Logging { group } - def refreshAllGroupMaxAllowRunningJobs(activeCount: Int): Unit = { - if (activeCount <= 0) return + def refreshAllGroupMaxAllowRunningJobs(validInsCount: Int): Unit = { + if (validInsCount <= 0) return groupNameToGroups .asMap() .asScala .foreach(item => { item._2 match { case group: ParallelGroup => - val maxAllowRunningJobs = Math.round(group.getMaxRunningJobs / activeCount) + val maxAllowRunningJobs = Math.round(group.getMaxRunningJobs / validInsCount) group.setMaxAllowRunningJobs(maxAllowRunningJobs) logger - .info(s"group ${group.getGroupName} update maxAllowRunningJobs $maxAllowRunningJobs") + .info( + s"group ${group.getGroupName} refresh maxAllowRunningJobs => ${group.getMaxRunningJobs}/$validInsCount=$maxAllowRunningJobs" + ) case _ => } }) From fe501c65eb9e7f29ba84ebf559bdae0226d52bd6 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Sun, 27 Nov 2022 20:51:10 +0800 Subject: [PATCH 15/72] update failover scan interval --- .../server/EntranceFailoverJobServer.java | 184 +++++++++--------- .../entrance/conf/EntranceConfiguration.scala | 2 +- 2 files changed, 97 insertions(+), 89 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index cebb7c68b5..a2bf900536 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -67,94 +67,102 @@ public void init() { failoverTask(); } - public void failoverTask() { - if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { - Utils.defaultScheduler().scheduleAtFixedRate( - new Runnable() { - @Override - public void run() { - EntranceSchedulerContext schedulerContext = (EntranceSchedulerContext) entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); - - // entrance do not failover job when it is offline - if (schedulerContext.getOfflineFlag()) return; - - CommonLock commonLock = new CommonLock(); - commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); - Boolean locked = false; - try { - locked = commonLockService.lock(commonLock, 10 * 1000L); - if (!locked) return; - logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); - - // serverInstance to map - Map serverInstanceMap = - getActiveServerInstances().stream() - .collect( - Collectors.toMap( - ServiceInstance::getInstance, - ServiceInstance::getRegistryTimestamp, - (k1, k2) -> k2)); - if (serverInstanceMap.isEmpty()) return; - - // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) - long expiredTimestamp = 0L; - if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { - expiredTimestamp = - System.currentTimeMillis() - - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); - } - - // get uncompleted status - List statusList = - Arrays.stream(SchedulerEventState.uncompleteStatusArray()) - .map(Object::toString).collect(Collectors.toList()); - - List jobRequests = - JobHistoryHelper.queryWaitForFailoverTask( - serverInstanceMap, - statusList, - expiredTimestamp, - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); - if (jobRequests.isEmpty()) return; - Object[] ids = jobRequests.stream().map(JobRequest::getId).toArray(); - logger.info("success query failover jobs , job ids: {}", ids); - - // failover to local server - jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); - logger.info("success execute failover jobs, job ids: {}", ids); - - } catch (Exception e) { - logger.error("failover failed", e); - } finally { - if (locked) commonLockService.unlock(commonLock); - } - } - }, - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), - TimeUnit.MILLISECONDS - ); - } - } - - private List getActiveServerInstances() { - // get all entrance server from eureka - ServiceInstance[] serviceInstances = Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); - if (serviceInstances == null || serviceInstances.length <= 0) return Lists.newArrayList(); - - // get all offline label server - RouteLabel routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory() - .createLabel(LabelKeyConstant.ROUTE_KEY, LabelConstant.OFFLINE); - List> labels = Lists.newArrayList(); - labels.add(routeLabel); - List labelInstances = InstanceLabelClient.getInstance().getInstanceFromLabel(labels); - - // get active entrance server - List allInstances = Lists.newArrayList(); - allInstances.addAll(Arrays.asList(serviceInstances)); - allInstances.removeAll(labelInstances); - - return allInstances; + public void failoverTask() { + if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { + Utils.defaultScheduler() + .scheduleWithFixedDelay( + () -> { + EntranceSchedulerContext schedulerContext = + (EntranceSchedulerContext) + entranceServer + .getEntranceContext() + .getOrCreateScheduler() + .getSchedulerContext(); + + // entrance do not failover job when it is offline + if (schedulerContext.getOfflineFlag()) return; + + CommonLock commonLock = new CommonLock(); + commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); + Boolean locked = false; + try { + locked = commonLockService.lock(commonLock, 10 * 1000L); + if (!locked) return; + logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); + + // serverInstance to map + Map serverInstanceMap = + getActiveServerInstances().stream() + .collect( + Collectors.toMap( + ServiceInstance::getInstance, + ServiceInstance::getRegistryTimestamp, + (k1, k2) -> k2)); + if (serverInstanceMap.isEmpty()) return; + + // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) + long expiredTimestamp = 0L; + if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { + expiredTimestamp = + System.currentTimeMillis() + - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); + } + + // get uncompleted status + List statusList = + Arrays.stream(SchedulerEventState.uncompleteStatusArray()) + .map(Object::toString) + .collect(Collectors.toList()); + + List jobRequests = + JobHistoryHelper.queryWaitForFailoverTask( + serverInstanceMap, + statusList, + expiredTimestamp, + EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); + if (jobRequests.isEmpty()) return; + Object[] ids = jobRequests.stream().map(JobRequest::getId).toArray(); + logger.info("success query failover jobs , job ids: {}", ids); + + // failover to local server + jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); + logger.info("success execute failover jobs, job ids: {}", ids); + + } catch (Exception e) { + logger.error("failover failed", e); + } finally { + if (locked) commonLockService.unlock(commonLock); + } + }, + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), + TimeUnit.MILLISECONDS); } + } + + private List getActiveServerInstances() { + // get all entrance server from eureka + ServiceInstance[] serviceInstances = + Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); + if (serviceInstances == null || serviceInstances.length <= 0) return Lists.newArrayList(); + + // get all offline label server + RouteLabel routeLabel = + LabelBuilderFactoryContext.getLabelBuilderFactory() + .createLabel(LabelKeyConstant.ROUTE_KEY, LabelConstant.OFFLINE); + List> labels = Lists.newArrayList(); + labels.add(routeLabel); + List labelInstances = + InstanceLabelClient.getInstance().getInstanceFromLabel(labels); + if (labelInstances == null) labelInstances = Lists.newArrayList(); + + // get active entrance server + List allInstances = Lists.newArrayList(); + allInstances.addAll(Arrays.asList(serviceInstances)); + allInstances.removeAll(labelInstances); + + return allInstances; + } +} } \ No newline at end of file diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index ada2048097..907b67e89e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -231,7 +231,7 @@ object EntranceConfiguration { CommonVars("linkis.entrance.failover.scan.init.time", 3 * 1000).getValue val ENTRANCE_FAILOVER_SCAN_INTERVAL = - CommonVars("linkis.entrance.failover.scan.interval", 3 * 1000).getValue + CommonVars("linkis.entrance.failover.scan.interval", 30 * 1000).getValue val ENTRANCE_FAILOVER_DATA_NUM_LIMIT = CommonVars("linkis.entrance.failover.data.num.limit", 10).getValue From 170c0c9849ded01667b997df9246a0260ec9688b Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Mon, 28 Nov 2022 14:09:17 +0800 Subject: [PATCH 16/72] [Bug-fix] gateway choose instance --- .../gateway/ujes/parser/EntranceRequestGatewayParser.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala index a1be26de87..883f252d70 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala @@ -103,10 +103,10 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { ) val instance = activeInstances .find(_.getInstance.equals(history.getInstances)) - .getOrElse(ServiceInstance("", "", Long.MaxValue)) + .getOrElse(ServiceInstance(null, null, Long.MaxValue)) JobInstance( history.getStatus, - history.getInstances, + instance.getInstance, history.getJobReqId, history.getCreatedTime.getTime, instance.getRegistryTimestamp From c91a6e8656644702b8bf851253a78d8f85d5db8c Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Mon, 28 Nov 2022 14:12:56 +0800 Subject: [PATCH 17/72] batch update instance --- .../server/DefaultEntranceServer.java | 15 ++-- .../linkis/entrance/EntranceServer.scala | 75 +++++++++++++++++-- .../entrance/utils/JobHistoryHelper.scala | 50 +++++++------ 3 files changed, 106 insertions(+), 34 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index 999d5cbcbf..443feb2a81 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -81,16 +81,19 @@ private void shutdownEntrance(ContextClosedEvent event) { logger.warn("event has been handled"); } else { if (EntranceConfiguration.ENTRANCE_SHUTDOWN_FAILOVER_ENABLED()) { - logger.warn("Entrance exit to update all not execution task instances and clean ConsumeQueue"); + logger.warn("Entrance exit to update and clean all ConsumeQueue task instances"); updateAllNotExecutionTaskInstances(false); } logger.warn("Entrance exit to stop all job"); - EntranceJob[] allUndoneJobs = getAllUndoneTask(null); - if (null != allUndoneJobs) { - for (EntranceJob job : allUndoneJobs) { - job.onFailure( - "Entrance exits the automatic cleanup task and can be rerun(服务退出自动清理任务,可以重跑)", null); + EntranceJob[] allUndoneTask = getAllUndoneTask(null); + if (null != allUndoneTask) { + String msg = "Entrance exits the automatic cleanup task and can be rerun(服务退出自动清理任务,可以重跑)"; + for (EntranceJob job : allUndoneTask) { + if (job.getLogListener().isDefined()) { + job.getLogListener().get().onLogUpdate(job, msg); + } + job.onFailure(msg, null); } } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 354dafa118..0c43bc8159 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -44,6 +44,8 @@ import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.linkis.common.log.LogUtils +import org.springframework.beans.BeanUtils + import java.{lang, util} import java.text.{MessageFormat, SimpleDateFormat} import java.util.Date @@ -265,11 +267,14 @@ abstract class EntranceServer extends Logging { } def updateAllNotExecutionTaskInstances(retryWhenUpdateFail: Boolean): Unit = { - val taskIds = getAllConsumeQueueTask().map(_.getJobRequest.getId).toList - JobHistoryHelper.updateAllConsumeQueueTask(taskIds, retryWhenUpdateFail) - logger.info("Finished to update all not execution task instances") - clearAllConsumeQueue() - logger.info("Finished to clean all ConsumeQueue") + val consumeQueueTasks = getAllConsumeQueueTask() + if (consumeQueueTasks != null && consumeQueueTasks.length > 0) { + val taskIds = consumeQueueTasks.map(_.getJobRequest.getId.asInstanceOf[Long]).toList + clearAllConsumeQueue() + logger.info("Finished to clean all ConsumeQueue") + JobHistoryHelper.updateAllConsumeQueueTask(taskIds.asJava, retryWhenUpdateFail) + logger.info("Finished to update all not execution task instances") + } } def killEC(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { @@ -355,14 +360,18 @@ abstract class EntranceServer extends Logging { * * @param jobRequest */ - def failoverExecute(jobRequest: JobRequest): String = { + def failoverExecute(jobReq: JobRequest): String = { - if (null == jobRequest || null == jobRequest.getId || jobRequest.getId <= 0) { + if (null == jobReq || null == jobReq.getId || jobReq.getId <= 0) { throw new EntranceErrorException( PERSIST_JOBREQUEST_ERROR.getErrorCode, PERSIST_JOBREQUEST_ERROR.getErrorDesc ) } + + var jobRequest = new JobRequest + BeanUtils.copyProperties(jobReq, jobRequest) + val logAppender = new java.lang.StringBuilder() logAppender.append( LogUtils @@ -370,10 +379,62 @@ abstract class EntranceServer extends Logging { s"\n\n*************************************FAILOVER************************************** \n\n" ) ) + // try to kill ec killEC(jobRequest, logAppender); + + // if status is Inited, need to deal by all Interceptors, such as log_path + if (jobRequest.getStatus.equals(SchedulerEventState.Inited.toString)) { + Utils.tryThrow( + getEntranceContext + .getOrCreateEntranceInterceptors() + .foreach(int => jobRequest = int.apply(jobRequest, logAppender)) + ) { t => + val error = t match { + case error: ErrorException => error + case t1: Throwable => + val exception = new EntranceErrorException( + FAILED_ANALYSIS_TASK.getErrorCode, + MessageFormat.format( + FAILED_ANALYSIS_TASK.getErrorDesc, + ExceptionUtils.getRootCauseMessage(t) + ) + ) + exception.initCause(t1) + exception + case _ => + new EntranceErrorException( + FAILED_ANALYSIS_TASK.getErrorCode, + MessageFormat.format( + FAILED_ANALYSIS_TASK.getErrorDesc, + ExceptionUtils.getRootCauseMessage(t) + ) + ) + } + jobRequest match { + case t: JobRequest => + t.setErrorCode(error.getErrCode) + t.setErrorDesc(error.getDesc) + t.setStatus(SchedulerEventState.Failed.toString) + t.setProgress(EntranceJob.JOB_COMPLETED_PROGRESS.toString) + val infoMap = new util.HashMap[String, Object] + infoMap.put(TaskConstant.ENGINE_INSTANCE, "NULL") + infoMap.put(TaskConstant.TICKET_ID, "") + infoMap.put("message", "Task interception failed and cannot be retried") + JobHistoryHelper.updateJobRequestMetrics(jobRequest, null, infoMap) + case _ => + } + getEntranceContext + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(jobRequest) + error + } + } + // init properties initJobRequestProperties(jobRequest, logAppender) + // update jobRequest getEntranceContext .getOrCreatePersistenceManager() diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index 811af8fce5..7a55124f75 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -132,37 +132,40 @@ object JobHistoryHelper extends Logging { * @param taskIdList * @param retryWhenUpdateFail */ - def updateAllConsumeQueueTask(taskIdList: List[java.lang.Long], retryWhenUpdateFail: Boolean = false): Unit = { + def updateAllConsumeQueueTask( + taskIdList: util.List[Long], + retryWhenUpdateFail: Boolean = false + ): Unit = { if (taskIdList.isEmpty) return - val updateTaskIds = new util.ArrayList[java.lang.Long]() + val updateTaskIds = new util.ArrayList[Long]() if ( EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue > 0 && - taskIdList.length > EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue + taskIdList.size() > EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue ) { for (i <- 0 until EntranceConfiguration.ENTRANCE_UPDATE_BATCH_SIZE.getValue) { - updateTaskIds.add(taskIdList(i)) + updateTaskIds.add(taskIdList.get(i)) } } else { - updateTaskIds.addAll(taskIdList.asJava) + updateTaskIds.addAll(taskIdList) } - + val list = new util.ArrayList[Long]() + list.addAll(taskIdList) try { - val successTaskIds = updateBatchInstances(updateTaskIds.asScala.toList) + val successTaskIds = updateBatchInstances(updateTaskIds) if (retryWhenUpdateFail) { - taskIdList.asJava.removeAll(successTaskIds.asJava) + list.removeAll(successTaskIds) } else { - taskIdList.asJava.removeAll(updateTaskIds) + list.removeAll(updateTaskIds) } } catch { case e: Exception => logger.warn("update batch instances failed, wait for retry", e) Thread.sleep(1000) } - - updateAllConsumeQueueTask(taskIdList, retryWhenUpdateFail) + updateAllConsumeQueueTask(list, retryWhenUpdateFail) } @@ -172,9 +175,9 @@ object JobHistoryHelper extends Logging { * @param taskIdList * @return */ - private def updateBatchInstances(taskIdList: List[java.lang.Long]): List[java.lang.Long] = { + private def updateBatchInstances(taskIdList: util.List[Long]): util.List[Long] = { val jobReqList = new util.ArrayList[JobRequest]() - taskIdList.foreach(taskID => { + taskIdList.asScala.foreach(taskID => { val jobRequest = new JobRequest jobRequest.setId(taskID) jobRequest.setInstances("") @@ -184,13 +187,16 @@ object JobHistoryHelper extends Logging { Utils.tryCatch { val response = sender.ask(jobReqBatchUpdate) response match { - case resp: util.ArrayList[JobRespProtocol] => - resp.asScala - .filter(r => - r.getStatus == SUCCESS_FLAG && r.getData.containsKey(JobRequestConstants.JOB_ID) - ) - .map(_.getData.get(JobRequestConstants.JOB_ID).asInstanceOf[java.lang.Long]) - .toList + case resp: util.List[JobRespProtocol] => + // todo filter success data, rpc have bug +// resp.asScala +// .filter(r => +// r.getStatus == SUCCESS_FLAG && r.getData.containsKey(JobRequestConstants.JOB_ID) +// ) +// .map(_.getData.get(JobRequestConstants.JOB_ID).asInstanceOf[java.lang.Long]) +// .toList + + taskIdList case _ => throw JobHistoryFailedException( "update batch instances from jobhistory not a correct List type" @@ -200,7 +206,9 @@ object JobHistoryHelper extends Logging { case errorException: ErrorException => throw errorException case e: Exception => val e1 = - JobHistoryFailedException(s"update batch instances ${taskIdList.mkString(",")} error") + JobHistoryFailedException( + s"update batch instances ${taskIdList.asScala.mkString(",")} error" + ) e1.initCause(e) throw e } From 8c55b180893ea2e8ba5b62d9ef162832e8c96d70 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 2 Dec 2022 11:12:34 +0800 Subject: [PATCH 18/72] failover status changed from Running to Cancelled --- .../linkis/entrance/EntranceServer.scala | 158 ++++++++++++------ .../entrance/conf/EntranceConfiguration.scala | 3 + 2 files changed, 106 insertions(+), 55 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 0c43bc8159..afeb23e820 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -19,6 +19,7 @@ package org.apache.linkis.entrance import org.apache.linkis.common.ServiceInstance import org.apache.linkis.common.exception.{ErrorException, LinkisException, LinkisRuntimeException} +import org.apache.linkis.common.io.FsPath import org.apache.linkis.common.log.LogUtils import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.entrance.conf.EntranceConfiguration @@ -26,7 +27,7 @@ import org.apache.linkis.entrance.cs.CSEntranceHelper import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary._ import org.apache.linkis.entrance.exception.{EntranceErrorException, SubmitFailedException} import org.apache.linkis.entrance.execute.EntranceJob -import org.apache.linkis.entrance.log.LogReader +import org.apache.linkis.entrance.log.{Cache, HDFSCacheLogWriter, LogReader} import org.apache.linkis.entrance.timeout.JobTimeoutManager import org.apache.linkis.entrance.utils.JobHistoryHelper import org.apache.linkis.governance.common.conf.GovernanceCommonConf @@ -39,6 +40,7 @@ import org.apache.linkis.rpc.Sender import org.apache.linkis.rpc.conf.RPCConfiguration import org.apache.linkis.scheduler.queue.{Job, SchedulerEventState} import org.apache.linkis.server.conf.ServerConfiguration +import org.apache.linkis.storage.utils.StorageUtils import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils @@ -355,23 +357,107 @@ abstract class EntranceServer extends Logging { } } + def dealInitedJobRequest(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { + Utils.tryThrow( + getEntranceContext + .getOrCreateEntranceInterceptors() + .foreach(int => int.apply(jobRequest, logAppender)) + ) { t => + val error = t match { + case error: ErrorException => error + case t1: Throwable => + val exception = new EntranceErrorException( + FAILED_ANALYSIS_TASK.getErrorCode, + MessageFormat.format( + FAILED_ANALYSIS_TASK.getErrorDesc, + ExceptionUtils.getRootCauseMessage(t) + ) + ) + exception.initCause(t1) + exception + case _ => + new EntranceErrorException( + FAILED_ANALYSIS_TASK.getErrorCode, + MessageFormat.format( + FAILED_ANALYSIS_TASK.getErrorDesc, + ExceptionUtils.getRootCauseMessage(t) + ) + ) + } + jobRequest match { + case t: JobRequest => + t.setErrorCode(error.getErrCode) + t.setErrorDesc(error.getDesc) + t.setStatus(SchedulerEventState.Failed.toString) + t.setProgress(EntranceJob.JOB_COMPLETED_PROGRESS.toString) + val infoMap = new util.HashMap[String, Object] + infoMap.put(TaskConstant.ENGINE_INSTANCE, "NULL") + infoMap.put(TaskConstant.TICKET_ID, "") + infoMap.put("message", "Task interception failed and cannot be retried") + JobHistoryHelper.updateJobRequestMetrics(jobRequest, null, infoMap) + case _ => + } + getEntranceContext + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(jobRequest) + error + } + + } + + def dealRunningJobRequest(jobRequest: JobRequest): Unit = { + Utils.tryCatch { + // init jobRequest properties + jobRequest.setStatus(SchedulerEventState.Cancelled.toString) + jobRequest.setProgress("1.0") + jobRequest.setInstances(Sender.getThisInstance) + + // update jobRequest + getEntranceContext + .getOrCreatePersistenceManager() + .createPersistenceEngine() + .updateIfNeeded(jobRequest) + + // append log + val logPath = jobRequest.getLogPath + if (StringUtils.isNotBlank(logPath)) { + val fsLogPath = new FsPath(logPath) + if (StorageUtils.HDFS == fsLogPath.getFsType) { + val logWriter = new HDFSCacheLogWriter( + logPath, + EntranceConfiguration.DEFAULT_LOG_CHARSET.getValue, + Cache(1), + jobRequest.getExecuteUser + ) + + val msg = + s"Job ${jobRequest.getId} failover, status changed from Running to Cancelled (任务故障转移,状态从Running变更为Cancelled)" + logWriter.write(msg) + logWriter.flush() + logWriter.close() + } + } + } { case e: Exception => + logger.error(s"Job ${jobRequest.getId} failover, change status error", e) + } + + } + /** * execute failover job (提交故障转移任务,返回新的execId) * * @param jobRequest */ - def failoverExecute(jobReq: JobRequest): String = { + def failoverExecute(jobRequest: JobRequest): Unit = { - if (null == jobReq || null == jobReq.getId || jobReq.getId <= 0) { + if (null == jobRequest || null == jobRequest.getId || jobRequest.getId <= 0) { throw new EntranceErrorException( PERSIST_JOBREQUEST_ERROR.getErrorCode, PERSIST_JOBREQUEST_ERROR.getErrorDesc ) } - var jobRequest = new JobRequest - BeanUtils.copyProperties(jobReq, jobRequest) - val logAppender = new java.lang.StringBuilder() logAppender.append( LogUtils @@ -383,53 +469,18 @@ abstract class EntranceServer extends Logging { // try to kill ec killEC(jobRequest, logAppender); - // if status is Inited, need to deal by all Interceptors, such as log_path + // deal Inited jobRequest, if status is Inited, need to deal by all Interceptors, such as log_path if (jobRequest.getStatus.equals(SchedulerEventState.Inited.toString)) { - Utils.tryThrow( - getEntranceContext - .getOrCreateEntranceInterceptors() - .foreach(int => jobRequest = int.apply(jobRequest, logAppender)) - ) { t => - val error = t match { - case error: ErrorException => error - case t1: Throwable => - val exception = new EntranceErrorException( - FAILED_ANALYSIS_TASK.getErrorCode, - MessageFormat.format( - FAILED_ANALYSIS_TASK.getErrorDesc, - ExceptionUtils.getRootCauseMessage(t) - ) - ) - exception.initCause(t1) - exception - case _ => - new EntranceErrorException( - FAILED_ANALYSIS_TASK.getErrorCode, - MessageFormat.format( - FAILED_ANALYSIS_TASK.getErrorDesc, - ExceptionUtils.getRootCauseMessage(t) - ) - ) - } - jobRequest match { - case t: JobRequest => - t.setErrorCode(error.getErrCode) - t.setErrorDesc(error.getDesc) - t.setStatus(SchedulerEventState.Failed.toString) - t.setProgress(EntranceJob.JOB_COMPLETED_PROGRESS.toString) - val infoMap = new util.HashMap[String, Object] - infoMap.put(TaskConstant.ENGINE_INSTANCE, "NULL") - infoMap.put(TaskConstant.TICKET_ID, "") - infoMap.put("message", "Task interception failed and cannot be retried") - JobHistoryHelper.updateJobRequestMetrics(jobRequest, null, infoMap) - case _ => - } - getEntranceContext - .getOrCreatePersistenceManager() - .createPersistenceEngine() - .updateIfNeeded(jobRequest) - error - } + dealInitedJobRequest(jobRequest, logAppender) + } + + // deal Running jobRequest, if enabled, status changed from Running to Cancelled + if ( + EntranceConfiguration.ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED.getValue && + jobRequest.getStatus.equals(SchedulerEventState.Running.toString) + ) { + dealRunningJobRequest(jobRequest) + return } // init properties @@ -482,8 +533,6 @@ abstract class EntranceServer extends Logging { entranceJob.getLogListener.foreach(_.onLogUpdate(entranceJob, msg)) case _ => } - - job.getId() } { t => job.onFailure("Submitting the query failed!(提交查询失败!)", t) val _jobRequest = @@ -503,7 +552,6 @@ abstract class EntranceServer extends Logging { ) } } - } private def initJobRequestProperties( diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 907b67e89e..10db3715fe 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -252,4 +252,7 @@ object EntranceConfiguration { val ENTRANCE_FAILOVER_RETAIN_YARN_RESOURCE_ENABLED = CommonVars("linkis.entrance.failover.retain.yarn.resource.enable", true) + val ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED = + CommonVars("linkis.entrance.failover.running.kill.enable", true) + } From 7971904007b4d879df5b8fe8d020f55a04c3cec8 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 6 Dec 2022 19:38:35 +0800 Subject: [PATCH 19/72] entrance refactor failoverExecute --- .../service/TaskExecutionServiceImpl.scala | 2 +- .../errorcode/EntranceErrorCodeSummary.java | 6 +- .../entrance/restful/EntranceRestfulApi.java | 11 +- .../linkis/entrance/EntranceServer.scala | 159 ++++++++++-------- 4 files changed, 105 insertions(+), 73 deletions(-) diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/TaskExecutionServiceImpl.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/TaskExecutionServiceImpl.scala index 039c1060c4..50110088b0 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/TaskExecutionServiceImpl.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/TaskExecutionServiceImpl.scala @@ -184,7 +184,7 @@ class TaskExecutionServiceImpl if (!lockService.isLockExist(requestTask.getLock)) { logger.error(s"Lock ${requestTask.getLock} not exist, cannot execute.") return ErrorExecuteResponse( - "Lock not exixt", + "Lock not exist", new EngineConnExecutorErrorException( EngineConnExecutorErrorCode.INVALID_LOCK, "Lock : " + requestTask.getLock + " not exist(您的锁无效,请重新获取后再提交)." diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/errorcode/EntranceErrorCodeSummary.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/errorcode/EntranceErrorCodeSummary.java index 2f045a1760..b5f90e3070 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/errorcode/EntranceErrorCodeSummary.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/errorcode/EntranceErrorCodeSummary.java @@ -71,7 +71,11 @@ public enum EntranceErrorCodeSummary implements LinkisErrorCode { SHELL_BLACKLISTED_CODE(50081, "Shell code contains blacklisted code(shell中包含黑名单代码)"), JOB_HISTORY_FAILED_ID(50081, ""), - LOGPATH_NOT_NULL(20301, "The logPath cannot be empty(日志路径不能为空)"); + LOGPATH_NOT_NULL(20301, "The logPath cannot be empty(日志路径不能为空)"), + + FAILOVER_RUNNING_TO_CANCELLED( + 30001, + "Job {0} failover, status changed from Running to Cancelled (任务故障转移,状态从Running变更为Cancelled)"); /** (errorCode)错误码 */ private final int errorCode; diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index 8b10b9eb52..b32923cc0d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -778,11 +778,12 @@ public Message killJobs( if (null != logListener) { logListener.onLogUpdate( entranceJob, - "Job " - + jobReq.getId() - + " was kill by user successfully(任务" - + jobReq.getId() - + "已成功取消)"); + LogUtils.generateInfo( + "Job " + + jobReq.getId() + + " was kill by user successfully(任务" + + jobReq.getId() + + "已成功取消)")); } this.entranceServer .getEntranceContext() diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index afeb23e820..81c701720e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -24,10 +24,12 @@ import org.apache.linkis.common.log.LogUtils import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.entrance.conf.EntranceConfiguration import org.apache.linkis.entrance.cs.CSEntranceHelper +import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary._ import org.apache.linkis.entrance.exception.{EntranceErrorException, SubmitFailedException} import org.apache.linkis.entrance.execute.EntranceJob -import org.apache.linkis.entrance.log.{Cache, HDFSCacheLogWriter, LogReader} +import org.apache.linkis.entrance.log.{Cache, CacheLogWriter, HDFSCacheLogWriter, LogReader} +import org.apache.linkis.entrance.parser.ParserUtils import org.apache.linkis.entrance.timeout.JobTimeoutManager import org.apache.linkis.entrance.utils.JobHistoryHelper import org.apache.linkis.governance.common.conf.GovernanceCommonConf @@ -279,7 +281,49 @@ abstract class EntranceServer extends Logging { } } - def killEC(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { + /** + * execute failover job (提交故障转移任务,返回新的execId) + * + * @param jobRequest + */ + def failoverExecute(jobRequest: JobRequest): Unit = { + + if (null == jobRequest || null == jobRequest.getId || jobRequest.getId <= 0) { + throw new EntranceErrorException( + PERSIST_JOBREQUEST_ERROR.getErrorCode, + PERSIST_JOBREQUEST_ERROR.getErrorDesc + ) + } + + val logAppender = new java.lang.StringBuilder() + logAppender.append( + LogUtils + .generateInfo( + s"\n\n*************************************FAILOVER************************************** \n\n" + ) + ) + + // try to kill ec + killOldEC(jobRequest, logAppender); + + // deal Inited jobRequest, if status is Inited, need to deal by all Interceptors, such as set log_path + if (jobRequest.getStatus.equals(SchedulerEventState.Inited.toString)) { + dealInitedJobRequest(jobRequest, logAppender) + } + + if ( + EntranceConfiguration.ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED.getValue && + jobRequest.getStatus.equals(SchedulerEventState.Running.toString) + ) { + // deal Running jobRequest, if enabled, status changed from Running to Cancelled + dealRunningJobRequest(jobRequest, logAppender) + } else { + // init and submit + initAndSubmitJobRequest(jobRequest, logAppender) + } + } + + def killOldEC(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { Utils.tryCatch { logAppender.append( LogUtils @@ -332,18 +376,18 @@ abstract class EntranceServer extends Logging { // kill ec by linkismanager val engineStopRequest = new EngineStopRequest engineStopRequest.setServiceInstance(ecInstance) - // send to linkismanager + // send to linkismanager kill ec Sender .getSender(RPCConfiguration.LINKIS_MANAGER_APPLICATION_NAME.getValue) .send(engineStopRequest) val msg = - s"job ${jobRequest.getId} send EngineStopRequest to linkismanager, kill instance $ecInstance" + s"job ${jobRequest.getId} send EngineStopRequest to linkismanager, kill EC instance $ecInstance" logger.info(msg) logAppender.append(LogUtils.generateInfo(msg) + "\n") } else if (engineInstance.containsKey(TaskConstant.ENGINE_CONN_TASK_ID)) { - // kill ec task + // get ec taskId val engineTaskId = engineInstance.get(TaskConstant.ENGINE_CONN_TASK_ID).toString - // send to ec + // send to ec kill task Sender .getSender(ecInstance) .send(RequestTaskKill(engineTaskId)) @@ -403,15 +447,22 @@ abstract class EntranceServer extends Logging { .updateIfNeeded(jobRequest) error } - } - def dealRunningJobRequest(jobRequest: JobRequest): Unit = { + def dealRunningJobRequest(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { Utils.tryCatch { + // error_msg + val msg = + MessageFormat.format( + EntranceErrorCodeSummary.FAILOVER_RUNNING_TO_CANCELLED.getErrorDesc, + jobRequest.getId + ) // init jobRequest properties jobRequest.setStatus(SchedulerEventState.Cancelled.toString) jobRequest.setProgress("1.0") jobRequest.setInstances(Sender.getThisInstance) + jobRequest.setErrorCode(EntranceErrorCodeSummary.FAILOVER_RUNNING_TO_CANCELLED.getErrorCode) + jobRequest.setErrorDesc(msg) // update jobRequest getEntranceContext @@ -419,70 +470,46 @@ abstract class EntranceServer extends Logging { .createPersistenceEngine() .updateIfNeeded(jobRequest) - // append log - val logPath = jobRequest.getLogPath - if (StringUtils.isNotBlank(logPath)) { - val fsLogPath = new FsPath(logPath) - if (StorageUtils.HDFS == fsLogPath.getFsType) { - val logWriter = new HDFSCacheLogWriter( - logPath, - EntranceConfiguration.DEFAULT_LOG_CHARSET.getValue, - Cache(1), - jobRequest.getExecuteUser - ) - - val msg = - s"Job ${jobRequest.getId} failover, status changed from Running to Cancelled (任务故障转移,状态从Running变更为Cancelled)" - logWriter.write(msg) - logWriter.flush() - logWriter.close() - } + // getOrGenerate log_path + var logPath = jobRequest.getLogPath + if (StringUtils.isBlank(logPath)) { + ParserUtils.generateLogPath(jobRequest, null) + logPath = jobRequest.getLogPath + logAppender.append( + LogUtils.generateInfo(s"job ${jobRequest.getId} generate new logPath $logPath \n") + ) } - } { case e: Exception => - logger.error(s"Job ${jobRequest.getId} failover, change status error", e) - } - - } - - /** - * execute failover job (提交故障转移任务,返回新的execId) - * - * @param jobRequest - */ - def failoverExecute(jobRequest: JobRequest): Unit = { - - if (null == jobRequest || null == jobRequest.getId || jobRequest.getId <= 0) { - throw new EntranceErrorException( - PERSIST_JOBREQUEST_ERROR.getErrorCode, - PERSIST_JOBREQUEST_ERROR.getErrorDesc - ) - } - - val logAppender = new java.lang.StringBuilder() - logAppender.append( - LogUtils - .generateInfo( - s"\n\n*************************************FAILOVER************************************** \n\n" + val fsLogPath = new FsPath(logPath) + val cache = Cache(EntranceConfiguration.DEFAULT_CACHE_MAX.getHotValue()) + val logWriter = if (StorageUtils.HDFS == fsLogPath.getFsType) { + new HDFSCacheLogWriter( + logPath, + EntranceConfiguration.DEFAULT_LOG_CHARSET.getValue, + cache, + jobRequest.getExecuteUser ) - ) - - // try to kill ec - killEC(jobRequest, logAppender); + } else { + new CacheLogWriter( + logPath, + EntranceConfiguration.DEFAULT_LOG_CHARSET.getValue, + cache, + jobRequest.getExecuteUser + ) + } + if (logAppender.length() > 0) { + logWriter.write(logAppender.toString.trim) + } - // deal Inited jobRequest, if status is Inited, need to deal by all Interceptors, such as log_path - if (jobRequest.getStatus.equals(SchedulerEventState.Inited.toString)) { - dealInitedJobRequest(jobRequest, logAppender) - } + logWriter.write(LogUtils.generateInfo(msg) + "\n") + logWriter.flush() + logWriter.close() - // deal Running jobRequest, if enabled, status changed from Running to Cancelled - if ( - EntranceConfiguration.ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED.getValue && - jobRequest.getStatus.equals(SchedulerEventState.Running.toString) - ) { - dealRunningJobRequest(jobRequest) - return + } { case e: Exception => + logger.error(s"Job ${jobRequest.getId} failover, change status error", e) } + } + def initAndSubmitJobRequest(jobRequest: JobRequest, logAppender: lang.StringBuilder): Unit = { // init properties initJobRequestProperties(jobRequest, logAppender) From ae9a172015333ae9ec93b8ee87e55db067858625 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 7 Dec 2022 18:20:08 +0800 Subject: [PATCH 20/72] change updateOrderFlag --- .../governance/common/entity/job/JobRequest.java | 12 ++++++------ .../org/apache/linkis/entrance/EntranceServer.scala | 2 +- .../service/impl/JobHistoryQueryServiceImpl.scala | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java index 01f9df3f5d..75134bd84a 100644 --- a/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java @@ -49,9 +49,9 @@ public class JobRequest { /** result location */ private String resultLocation; - private String observeInfo; + private Boolean updateOrderFlag = true; - private Boolean updateLimitFlag = true; + private String observeInfo; private Map metrics = new HashMap<>(); @@ -207,12 +207,12 @@ public void setObserveInfo(String observeInfo) { this.observeInfo = observeInfo; } - public Boolean getUpdateLimitFlag() { - return updateLimitFlag; + public Boolean getUpdateOrderFlag() { + return updateOrderFlag; } - public void setUpdateLimitFlag(Boolean updateLimitFlag) { - this.updateLimitFlag = updateLimitFlag; + public void setUpdateOrderFlag(Boolean updateOrderFlag) { + this.updateOrderFlag = updateOrderFlag; } @Override diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 81c701720e..d2a504100d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -648,7 +648,7 @@ abstract class EntranceServer extends Logging { jobRequest.setErrorDesc("") jobRequest.setMetrics(metricMap) jobRequest.getMetrics.put(TaskConstant.ENTRANCEJOB_SUBMIT_TIME, initDate) - jobRequest.setUpdateLimitFlag(false) + jobRequest.setUpdateOrderFlag(false) logAppender.append( LogUtils.generateInfo(s"job ${jobRequest.getId} success to initialize the properties \n") diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala index bb90fee2dc..22084f88a6 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala @@ -109,7 +109,7 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { logger.info(s"${jobReq.getErrorDesc}") } } - if (jobReq.getUpdateLimitFlag && jobReq.getStatus != null) { + if (jobReq.getUpdateOrderFlag && jobReq.getStatus != null) { val oldStatus: String = jobHistoryMapper.selectJobHistoryStatusForUpdate(jobReq.getId) if (oldStatus != null && !shouldUpdate(oldStatus, jobReq.getStatus)) { throw new QueryException( @@ -174,7 +174,7 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { logger.info(s"${jobReq.getErrorDesc}") } } - if (jobReq.getUpdateLimitFlag && jobReq.getStatus != null) { + if (jobReq.getUpdateOrderFlag && jobReq.getStatus != null) { val oldStatus: String = jobHistoryMapper.selectJobHistoryStatusForUpdate(jobReq.getId) if (oldStatus != null && !shouldUpdate(oldStatus, jobReq.getStatus)) { throw new QueryException( From a4a41d6d6a2f51c541d1e40749798c3fd7019e15 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 7 Dec 2022 18:22:55 +0800 Subject: [PATCH 21/72] edit failoverJobServer --- .../server/EntranceFailoverJobServer.java | 185 ++++++++++-------- 1 file changed, 100 insertions(+), 85 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index a2bf900536..b5f0688626 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -17,7 +17,6 @@ package org.apache.linkis.entrance.server; -import org.apache.commons.compress.utils.Lists; import org.apache.linkis.common.ServiceInstance; import org.apache.linkis.common.utils.Utils; import org.apache.linkis.entrance.EntranceServer; @@ -28,115 +27,133 @@ import org.apache.linkis.governance.common.entity.job.JobRequest; import org.apache.linkis.instance.label.client.InstanceLabelClient; import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext; -import org.apache.linkis.manager.label.constant.LabelConstant; import org.apache.linkis.manager.label.constant.LabelKeyConstant; +import org.apache.linkis.manager.label.constant.LabelValueConstant; import org.apache.linkis.manager.label.entity.Label; import org.apache.linkis.manager.label.entity.route.RouteLabel; import org.apache.linkis.publicservice.common.lock.entity.CommonLock; import org.apache.linkis.publicservice.common.lock.service.CommonLockService; import org.apache.linkis.rpc.Sender; import org.apache.linkis.scheduler.queue.SchedulerEventState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import org.apache.commons.compress.utils.Lists; + import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import javax.annotation.PostConstruct; + import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + @Component(ServiceNameConsts.ENTRANCE_FAILOVER_SERVER) public class EntranceFailoverJobServer { - private static final Logger logger = LoggerFactory.getLogger(DefaultEntranceServer.class); + private static final Logger logger = LoggerFactory.getLogger(EntranceFailoverJobServer.class); - @Autowired - private EntranceServer entranceServer; + @Autowired private EntranceServer entranceServer; - @Autowired - private CommonLockService commonLockService; + @Autowired private CommonLockService commonLockService; + private static String ENTRANCE_FAILOVER_LOCK = "ENTRANCE_FAILOVER_LOCK"; - private static String ENTRANCE_FAILOVER_LOCK = "ENTRANCE_FAILOVER_LOCK"; + private ScheduledExecutorService scheduledExecutor; - @PostConstruct - public void init() { - failoverTask(); - } + @PostConstruct + public void init() { + this.scheduledExecutor = + Executors.newSingleThreadScheduledExecutor( + Utils.threadFactory("Linkis-Failover-Scheduler-Thread-", true)); + failoverTask(); + } public void failoverTask() { if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { - Utils.defaultScheduler() - .scheduleWithFixedDelay( - () -> { - EntranceSchedulerContext schedulerContext = - (EntranceSchedulerContext) - entranceServer - .getEntranceContext() - .getOrCreateScheduler() - .getSchedulerContext(); - - // entrance do not failover job when it is offline - if (schedulerContext.getOfflineFlag()) return; - - CommonLock commonLock = new CommonLock(); - commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); - Boolean locked = false; - try { - locked = commonLockService.lock(commonLock, 10 * 1000L); - if (!locked) return; - logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); - - // serverInstance to map - Map serverInstanceMap = - getActiveServerInstances().stream() - .collect( - Collectors.toMap( - ServiceInstance::getInstance, - ServiceInstance::getRegistryTimestamp, - (k1, k2) -> k2)); - if (serverInstanceMap.isEmpty()) return; - - // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) - long expiredTimestamp = 0L; - if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { - expiredTimestamp = - System.currentTimeMillis() - - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); - } - - // get uncompleted status - List statusList = - Arrays.stream(SchedulerEventState.uncompleteStatusArray()) - .map(Object::toString) - .collect(Collectors.toList()); - - List jobRequests = - JobHistoryHelper.queryWaitForFailoverTask( - serverInstanceMap, - statusList, - expiredTimestamp, - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); - if (jobRequests.isEmpty()) return; - Object[] ids = jobRequests.stream().map(JobRequest::getId).toArray(); - logger.info("success query failover jobs , job ids: {}", ids); - - // failover to local server - jobRequests.forEach(jobRequest -> entranceServer.failoverExecute(jobRequest)); - logger.info("success execute failover jobs, job ids: {}", ids); - - } catch (Exception e) { - logger.error("failover failed", e); - } finally { - if (locked) commonLockService.unlock(commonLock); - } - }, - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), - TimeUnit.MILLISECONDS); + scheduledExecutor.scheduleWithFixedDelay( + () -> { + EntranceSchedulerContext schedulerContext = + (EntranceSchedulerContext) + entranceServer + .getEntranceContext() + .getOrCreateScheduler() + .getSchedulerContext(); + + // entrance do not failover job when it is offline + if (schedulerContext.getOfflineFlag()) return; + + CommonLock commonLock = new CommonLock(); + commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); + Boolean locked = false; + try { + locked = commonLockService.lock(commonLock, 10 * 1000L); + if (!locked) return; + logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); + + // serverInstance to map + Map serverInstanceMap = + getActiveServerInstances().stream() + .collect( + Collectors.toMap( + ServiceInstance::getInstance, + ServiceInstance::getRegistryTimestamp, + (k1, k2) -> k2)); + if (serverInstanceMap.isEmpty()) return; + + // It is very important to avoid repeated execute job + // when failover self job, if self instance is empty, the job can be repeated execute + if (!serverInstanceMap.containsKey(Sender.getThisInstance())) { + logger.warn( + "server has just started and has not get self info, it does not failover"); + return; + } + + // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) + long expiredTimestamp = 0L; + if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { + expiredTimestamp = + System.currentTimeMillis() + - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); + } + + // get uncompleted status + List statusList = + Arrays.stream(SchedulerEventState.uncompleteStatusArray()) + .map(Object::toString) + .collect(Collectors.toList()); + + List jobRequests = + JobHistoryHelper.queryWaitForFailoverTask( + serverInstanceMap, + statusList, + expiredTimestamp, + EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); + if (jobRequests.isEmpty()) return; + Object[] ids = jobRequests.stream().map(JobRequest::getId).toArray(); + logger.info("success query failover jobs , job ids: {}", ids); + + // failover to local server + for (JobRequest jobRequest : jobRequests) { + entranceServer.failoverExecute(jobRequest); + } + logger.info("finished execute failover jobs, job ids: {}", ids); + + } catch (Exception e) { + logger.error("failover failed", e); + } finally { + if (locked) commonLockService.unlock(commonLock); + } + }, + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), + TimeUnit.MILLISECONDS); } } @@ -149,7 +166,7 @@ private List getActiveServerInstances() { // get all offline label server RouteLabel routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory() - .createLabel(LabelKeyConstant.ROUTE_KEY, LabelConstant.OFFLINE); + .createLabel(LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE); List> labels = Lists.newArrayList(); labels.add(routeLabel); List labelInstances = @@ -164,5 +181,3 @@ private List getActiveServerInstances() { return allInstances; } } - -} \ No newline at end of file From f0755376d4c9db08cd9a71c27c8ab0b6b4b18fd1 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 14 Dec 2022 15:59:43 +0800 Subject: [PATCH 22/72] =?UTF-8?q?1.=E4=BF=AE=E6=94=B9=E8=8E=B7=E5=8F=96fai?= =?UTF-8?q?lover=E5=AE=9E=E4=BE=8B=E5=88=97=E8=A1=A8=EF=BC=8C=E4=B8=8D?= =?UTF-8?q?=E8=83=BD=E5=8E=BB=E9=99=A4offline=20lable=E5=AE=9E=E4=BE=8B=20?= =?UTF-8?q?2.=E7=A7=BB=E9=99=A4retry=E4=BB=BB=E5=8A=A1=E6=97=B6=EF=BC=8C?= =?UTF-8?q?=E8=A6=81=E6=9B=B4=E6=96=B0=E6=95=B0=E6=8D=AE=E5=BA=93instance?= =?UTF-8?q?=E4=B8=BA=E7=A9=BA?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../queue/fifoqueue/FIFOUserConsumer.scala | 6 ++- .../server/DefaultEntranceServer.java | 2 +- .../server/EntranceFailoverJobServer.java | 40 +++---------------- .../entrance/conf/EntranceConfiguration.scala | 3 +- .../scheduler/EntranceFIFOUserConsumer.scala | 26 +++++++++--- .../entrance/utils/JobHistoryHelper.scala | 2 +- 6 files changed, 36 insertions(+), 43 deletions(-) diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala index 4483a02a76..ea4f4ce6df 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/fifoqueue/FIFOUserConsumer.scala @@ -27,6 +27,7 @@ import org.apache.linkis.scheduler.executer.Executor import org.apache.linkis.scheduler.future.{BDPFuture, BDPFutureTask} import org.apache.linkis.scheduler.queue._ +import java.util import java.util.concurrent.{ExecutorService, Future} import scala.beans.BeanProperty @@ -189,14 +190,17 @@ class FIFOUserConsumer( runningJobs(index) = job } - protected def scanAllRetryJobsAndRemove(): Unit = { + protected def scanAllRetryJobsAndRemove(): util.List[Job] = { + val jobs = new util.ArrayList[Job]() for (index <- runningJobs.indices) { val job = runningJobs(index) if (job != null && job.isJobCanRetry) { + jobs.add(job) runningJobs(index) = null logger.info(s"Job $job can retry, remove from runningJobs") } } + jobs } override def shutdown(): Unit = { diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index 443feb2a81..ea920f4c11 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -80,7 +80,7 @@ private void shutdownEntrance(ContextClosedEvent event) { if (shutdownFlag) { logger.warn("event has been handled"); } else { - if (EntranceConfiguration.ENTRANCE_SHUTDOWN_FAILOVER_ENABLED()) { + if (EntranceConfiguration.ENTRANCE_SHUTDOWN_FAILOVER_CONSUME_QUEUE_ENABLED()) { logger.warn("Entrance exit to update and clean all ConsumeQueue task instances"); updateAllNotExecutionTaskInstances(false); } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index b5f0688626..1c2f906a9d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -25,19 +25,11 @@ import org.apache.linkis.entrance.scheduler.EntranceSchedulerContext; import org.apache.linkis.entrance.utils.JobHistoryHelper; import org.apache.linkis.governance.common.entity.job.JobRequest; -import org.apache.linkis.instance.label.client.InstanceLabelClient; -import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext; -import org.apache.linkis.manager.label.constant.LabelKeyConstant; -import org.apache.linkis.manager.label.constant.LabelValueConstant; -import org.apache.linkis.manager.label.entity.Label; -import org.apache.linkis.manager.label.entity.route.RouteLabel; import org.apache.linkis.publicservice.common.lock.entity.CommonLock; import org.apache.linkis.publicservice.common.lock.service.CommonLockService; import org.apache.linkis.rpc.Sender; import org.apache.linkis.scheduler.queue.SchedulerEventState; -import org.apache.commons.compress.utils.Lists; - import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; @@ -97,15 +89,19 @@ public void failoverTask() { if (!locked) return; logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); + // get all entrance server from eureka + ServiceInstance[] serviceInstances = + Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); + if (serviceInstances == null || serviceInstances.length <= 0) return; + // serverInstance to map Map serverInstanceMap = - getActiveServerInstances().stream() + Arrays.stream(serviceInstances) .collect( Collectors.toMap( ServiceInstance::getInstance, ServiceInstance::getRegistryTimestamp, (k1, k2) -> k2)); - if (serverInstanceMap.isEmpty()) return; // It is very important to avoid repeated execute job // when failover self job, if self instance is empty, the job can be repeated execute @@ -156,28 +152,4 @@ public void failoverTask() { TimeUnit.MILLISECONDS); } } - - private List getActiveServerInstances() { - // get all entrance server from eureka - ServiceInstance[] serviceInstances = - Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); - if (serviceInstances == null || serviceInstances.length <= 0) return Lists.newArrayList(); - - // get all offline label server - RouteLabel routeLabel = - LabelBuilderFactoryContext.getLabelBuilderFactory() - .createLabel(LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE); - List> labels = Lists.newArrayList(); - labels.add(routeLabel); - List labelInstances = - InstanceLabelClient.getInstance().getInstanceFromLabel(labels); - if (labelInstances == null) labelInstances = Lists.newArrayList(); - - // get active entrance server - List allInstances = Lists.newArrayList(); - allInstances.addAll(Arrays.asList(serviceInstances)); - allInstances.removeAll(labelInstances); - - return allInstances; - } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 10db3715fe..3b606cfe3e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -242,7 +242,8 @@ object EntranceConfiguration { val ENTRANCE_UPDATE_BATCH_SIZE = CommonVars("linkis.entrance.update.batch.size", 100) - val ENTRANCE_SHUTDOWN_FAILOVER_ENABLED = CommonVars("linkis.entrance.shutdown.failover.enable", true).getValue + val ENTRANCE_SHUTDOWN_FAILOVER_CONSUME_QUEUE_ENABLED = + CommonVars("linkis.entrance.shutdown.failover.consume.queue.enable", true).getValue val ENTRANCE_GROUP_SCAN_ENABLED = CommonVars("linkis.entrance.group.scan.enable", true) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala index 34d3e3042c..2ff42d1eb8 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala @@ -19,23 +19,39 @@ package org.apache.linkis.entrance.scheduler import org.apache.linkis.common.utils.Utils import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.entrance.execute.EntranceJob +import org.apache.linkis.entrance.utils.JobHistoryHelper import org.apache.linkis.scheduler.SchedulerContext import org.apache.linkis.scheduler.queue.Group import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer +import java.util import java.util.concurrent.ExecutorService +import scala.collection.JavaConverters.collectionAsScalaIterableConverter + class EntranceFIFOUserConsumer( - schedulerContext: SchedulerContext, - executeService: ExecutorService, - private var group: Group + schedulerContext: SchedulerContext, + executeService: ExecutorService, + private var group: Group ) extends FIFOUserConsumer(schedulerContext, executeService, group) { override def loop(): Unit = { schedulerContext match { case entranceSchedulerContext: EntranceSchedulerContext => - if (entranceSchedulerContext.getOfflineFlag && EntranceConfiguration.ENTRANCE_FAILOVER_RETRY_JOB_ENABLED.getValue) { - scanAllRetryJobsAndRemove() + if ( + entranceSchedulerContext.getOfflineFlag && EntranceConfiguration.ENTRANCE_FAILOVER_RETRY_JOB_ENABLED.getValue + ) { + val jobs = scanAllRetryJobsAndRemove() + if (!jobs.isEmpty) { + val ids = new util.ArrayList[Long]() + jobs.asScala.foreach { + case entranceJob: EntranceJob => + ids.add(entranceJob.getJobRequest.getId) + case _ => + } + JobHistoryHelper.updateBatchInstances(ids) + } Utils.tryQuietly(Thread.sleep(5000)) return } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index 7a55124f75..714f1d77de 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -175,7 +175,7 @@ object JobHistoryHelper extends Logging { * @param taskIdList * @return */ - private def updateBatchInstances(taskIdList: util.List[Long]): util.List[Long] = { + def updateBatchInstances(taskIdList: util.List[Long]): util.List[Long] = { val jobReqList = new util.ArrayList[JobRequest]() taskIdList.asScala.foreach(taskID => { val jobRequest = new JobRequest From db2f0eda512024f4d9b1ce47ab3ae6f653e4a7b3 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 14 Dec 2022 16:20:24 +0800 Subject: [PATCH 23/72] =?UTF-8?q?1.=E6=97=A5=E5=BF=97=E4=BF=A1=E6=81=AF?= =?UTF-8?q?=E5=A2=9E=E5=8A=A0=E4=B8=AD=E6=96=87=E6=8F=8F=E8=BF=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/apache/linkis/entrance/restful/EntranceRestfulApi.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index b32923cc0d..afa4aeb06c 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -614,7 +614,7 @@ public Message log(HttpServletRequest req, @PathVariable("id") String id) { logger.warn("The job {} wait failover, return customer log", id); message = Message.ok(); message.setMethod("/api/entrance/" + id + "/log"); - String log = LogUtils.generateInfo("The job will failover soon, please try again later"); + String log = LogUtils.generateInfo("The job will failover soon, please try again later.(job很快就会failover,请稍后再试)"); Object retLog; if (distinctLevel) { String[] array = new String[4]; From dfb531afc19411532d2eecef04305156363def1d Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 16 Dec 2022 09:19:52 +0800 Subject: [PATCH 24/72] =?UTF-8?q?failover=E6=97=B6=EF=BC=8C=E4=BB=BB?= =?UTF-8?q?=E5=8A=A1=E4=B8=BB=E5=8A=A8=E5=88=B7=E6=97=A5=E5=BF=97hdfs?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../server/DefaultEntranceServer.java | 7 +++--- .../linkis/entrance/EntranceServer.scala | 24 ++++++++++++++----- .../scheduler/EntranceFIFOUserConsumer.scala | 6 ++--- 3 files changed, 25 insertions(+), 12 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index ea920f4c11..54b855ffbd 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -22,9 +22,12 @@ import org.apache.linkis.entrance.conf.EntranceConfiguration; import org.apache.linkis.entrance.constant.ServiceNameConsts; import org.apache.linkis.entrance.execute.EntranceJob; +import org.apache.linkis.entrance.job.EntranceExecutionJob; import org.apache.linkis.entrance.log.LogReader; import org.apache.linkis.rpc.Sender; +import org.apache.commons.io.IOUtils; + import org.springframework.beans.factory.annotation.Autowired; import org.springframework.context.event.ContextClosedEvent; import org.springframework.context.event.EventListener; @@ -90,10 +93,8 @@ private void shutdownEntrance(ContextClosedEvent event) { if (null != allUndoneTask) { String msg = "Entrance exits the automatic cleanup task and can be rerun(服务退出自动清理任务,可以重跑)"; for (EntranceJob job : allUndoneTask) { - if (job.getLogListener().isDefined()) { - job.getLogListener().get().onLogUpdate(job, msg); - } job.onFailure(msg, null); + IOUtils.closeQuietly(((EntranceExecutionJob) job).getLogWriter().get()); } } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index d2a504100d..b5563262b8 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -43,17 +43,16 @@ import org.apache.linkis.rpc.conf.RPCConfiguration import org.apache.linkis.scheduler.queue.{Job, SchedulerEventState} import org.apache.linkis.server.conf.ServerConfiguration import org.apache.linkis.storage.utils.StorageUtils - import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.linkis.common.log.LogUtils import org.springframework.beans.BeanUtils +import org.apache.linkis.entrance.job.EntranceExecutionJob import java.{lang, util} import java.text.{MessageFormat, SimpleDateFormat} import java.util.Date - import scala.collection.JavaConverters._ abstract class EntranceServer extends Logging { @@ -272,11 +271,24 @@ abstract class EntranceServer extends Logging { def updateAllNotExecutionTaskInstances(retryWhenUpdateFail: Boolean): Unit = { val consumeQueueTasks = getAllConsumeQueueTask() + + clearAllConsumeQueue() + logger.info("Finished to clean all ConsumeQueue") + if (consumeQueueTasks != null && consumeQueueTasks.length > 0) { - val taskIds = consumeQueueTasks.map(_.getJobRequest.getId.asInstanceOf[Long]).toList - clearAllConsumeQueue() - logger.info("Finished to clean all ConsumeQueue") - JobHistoryHelper.updateAllConsumeQueueTask(taskIds.asJava, retryWhenUpdateFail) + val taskIds = new util.ArrayList[Long]() + consumeQueueTasks.foreach(job => { + taskIds.add(job.getJobRequest.getId.asInstanceOf[Long]) + job match { + case entranceExecutionJob : EntranceExecutionJob => + val msg = LogUtils.generateWarn(s"job ${job.getJobRequest.getId} clean from ConsumeQueue, wait for failover") + entranceExecutionJob.getLogListener.foreach(_.onLogUpdate(entranceExecutionJob, msg)) + entranceExecutionJob.getLogWriter.foreach(_.close()) + case _ => + } + }) + + JobHistoryHelper.updateAllConsumeQueueTask(taskIds, retryWhenUpdateFail) logger.info("Finished to update all not execution task instances") } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala index 2ff42d1eb8..2404db51dc 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala @@ -19,7 +19,7 @@ package org.apache.linkis.entrance.scheduler import org.apache.linkis.common.utils.Utils import org.apache.linkis.entrance.conf.EntranceConfiguration -import org.apache.linkis.entrance.execute.EntranceJob +import org.apache.linkis.entrance.job.EntranceExecutionJob import org.apache.linkis.entrance.utils.JobHistoryHelper import org.apache.linkis.scheduler.SchedulerContext import org.apache.linkis.scheduler.queue.Group @@ -27,7 +27,6 @@ import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer import java.util import java.util.concurrent.ExecutorService - import scala.collection.JavaConverters.collectionAsScalaIterableConverter class EntranceFIFOUserConsumer( @@ -46,7 +45,8 @@ class EntranceFIFOUserConsumer( if (!jobs.isEmpty) { val ids = new util.ArrayList[Long]() jobs.asScala.foreach { - case entranceJob: EntranceJob => + case entranceJob: EntranceExecutionJob => + entranceJob.getLogWriter.foreach(_.close()) ids.add(entranceJob.getJobRequest.getId) case _ => } From 8ab841c8baf12b516a330cd73ae342013a9bebfa Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Fri, 16 Dec 2022 16:18:12 +0800 Subject: [PATCH 25/72] =?UTF-8?q?=E8=BF=81=E7=A7=BB=E5=88=B7=E6=96=B0maxAl?= =?UTF-8?q?lowRunningJobs=E6=9C=8D=E5=8A=A1=EF=BC=8C=E6=94=BE=E5=88=B0cons?= =?UTF-8?q?umeManager=E9=87=8C=EF=BC=8C=E5=88=B7=E6=96=B0consumer=E9=87=8C?= =?UTF-8?q?=E7=9A=84group,=E8=80=8C=E4=B8=8D=E6=98=AF=E5=88=B7=E6=96=B0gro?= =?UTF-8?q?upFactory=E9=87=8C=E7=9A=84?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../entrance/conf/EntranceConfiguration.scala | 4 ++ .../scheduler/EntranceGroupFactory.scala | 54 +-------------- .../EntranceParallelConsumerManager.scala | 65 ++++++++++++++++++- 3 files changed, 69 insertions(+), 54 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 3b606cfe3e..959d8c68bc 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -247,6 +247,10 @@ object EntranceConfiguration { val ENTRANCE_GROUP_SCAN_ENABLED = CommonVars("linkis.entrance.group.scan.enable", true) + val ENTRANCE_GROUP_SCAN_INIT_TIME = CommonVars("linkis.entrance.group.scan.init.time", 3 * 1000) + + val ENTRANCE_GROUP_SCAN_INTERVAL = CommonVars("linkis.entrance.group.scan.interval", 60 * 1000) + val ENTRANCE_FAILOVER_RETAIN_ENGINE_CONN_ENABLED = CommonVars("linkis.entrance.failover.retain.engine.conn.enable", true) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala index c38fae5e4a..4bd0caca1b 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala @@ -17,7 +17,6 @@ package org.apache.linkis.entrance.scheduler -import org.apache.linkis.common.ServiceInstance import org.apache.linkis.common.conf.{CommonVars, Configuration} import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.entrance.conf.EntranceConfiguration @@ -46,6 +45,7 @@ import org.apache.linkis.protocol.utils.TaskUtils import org.apache.linkis.rpc.Sender import org.apache.linkis.scheduler.queue.{Group, GroupFactory, SchedulerEvent} import org.apache.linkis.scheduler.queue.parallelqueue.ParallelGroup + import org.apache.commons.lang3.StringUtils import java.util @@ -77,39 +77,6 @@ class EntranceGroupFactory extends GroupFactory with Logging { private val GROUP_INIT_CAPACITY = CommonVars("wds.linkis.entrance.init.capacity", 100) - private val GROUP_SCAN_INIT_TIME = CommonVars("linkis.entrance.group.scan.init.time", 3 * 1000) - - private val GROUP_SCAN_INTERVAL = CommonVars("linkis.entrance.group.scan.interval", 60 * 1000) - - if (EntranceConfiguration.ENTRANCE_GROUP_SCAN_ENABLED.getValue) { - Utils.defaultScheduler.scheduleAtFixedRate( - new Runnable { - override def run(): Unit = { - // get all entrance server from eureka - val serviceInstances = Sender.getInstances(Sender.getThisServiceInstance.getApplicationName) - if (null == serviceInstances || serviceInstances.isEmpty) return - - // get all offline label server - val routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory - .createLabel[RouteLabel](LabelKeyConstant.ROUTE_KEY, LabelConstant.OFFLINE) - val labels = new util.ArrayList[Label[_]] - labels.add(routeLabel) - val labelInstances = InstanceLabelClient.getInstance.getInstanceFromLabel(labels) - - // get active entrance server - val allInstances = new util.ArrayList[ServiceInstance]() - allInstances.addAll(serviceInstances.toList.asJava) - allInstances.removeAll(labelInstances) - // refresh all group maxAllowRunningJobs - refreshAllGroupMaxAllowRunningJobs(allInstances.size()) - } - }, - GROUP_SCAN_INIT_TIME.getValue, - GROUP_SCAN_INTERVAL.getValue, - TimeUnit.MILLISECONDS - ) - } - private val specifiedUsernameRegexPattern: Pattern = if (StringUtils.isNotBlank(SPECIFIED_USERNAME_REGEX.getValue)) { Pattern.compile(SPECIFIED_USERNAME_REGEX.getValue) @@ -193,25 +160,6 @@ class EntranceGroupFactory extends GroupFactory with Logging { group } - def refreshAllGroupMaxAllowRunningJobs(validInsCount: Int): Unit = { - if (validInsCount <= 0) return - groupNameToGroups - .asMap() - .asScala - .foreach(item => { - item._2 match { - case group: ParallelGroup => - val maxAllowRunningJobs = Math.round(group.getMaxRunningJobs / validInsCount) - group.setMaxAllowRunningJobs(maxAllowRunningJobs) - logger - .info( - s"group ${group.getGroupName} refresh maxAllowRunningJobs => ${group.getMaxRunningJobs}/$validInsCount=$maxAllowRunningJobs" - ) - case _ => - } - }) - } - private def getUserMaxRunningJobs(keyAndValue: util.Map[String, String]): Int = { Math.max( EntranceConfiguration.ENTRANCE_INSTANCE_MIN.getValue, diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index 91a7c4aaa6..f114981c5c 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -17,8 +17,22 @@ package org.apache.linkis.entrance.scheduler +import org.apache.linkis.common.ServiceInstance +import org.apache.linkis.common.conf.CommonVars +import org.apache.linkis.common.utils.Utils +import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.instance.label.client.InstanceLabelClient +import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext +import org.apache.linkis.manager.label.constant.{LabelKeyConstant, LabelValueConstant} +import org.apache.linkis.manager.label.entity.Label +import org.apache.linkis.manager.label.entity.route.RouteLabel +import org.apache.linkis.rpc.Sender import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer -import org.apache.linkis.scheduler.queue.parallelqueue.ParallelConsumerManager +import org.apache.linkis.scheduler.queue.parallelqueue.{ParallelConsumerManager, ParallelGroup} + +import java.util +import java.util.concurrent.TimeUnit +import scala.collection.JavaConverters._ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: String) extends ParallelConsumerManager(maxParallelismUsers, schedulerName){ @@ -28,4 +42,53 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S new EntranceFIFOUserConsumer(getSchedulerContext, getOrCreateExecutorService, group) } + if (EntranceConfiguration.ENTRANCE_GROUP_SCAN_ENABLED.getValue) { + Utils.defaultScheduler.scheduleAtFixedRate( + new Runnable { + override def run(): Unit = { + logger.info("start refresh consumer group maxAllowRunningJobs") + // get all entrance server from eureka + val serviceInstances = + Sender.getInstances(Sender.getThisServiceInstance.getApplicationName) + if (null == serviceInstances || serviceInstances.isEmpty) return + + // get all offline label server + val routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory + .createLabel[RouteLabel](LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE) + val labels = new util.ArrayList[Label[_]] + labels.add(routeLabel) + val labelInstances = InstanceLabelClient.getInstance.getInstanceFromLabel(labels) + + // get active entrance server + val allInstances = new util.ArrayList[ServiceInstance]() + allInstances.addAll(serviceInstances.toList.asJava) + allInstances.removeAll(labelInstances) + // refresh all group maxAllowRunningJobs + refreshAllGroupMaxAllowRunningJobs(allInstances.size()) + logger.info("Finished to refresh consumer group maxAllowRunningJobs") + } + }, + EntranceConfiguration.ENTRANCE_GROUP_SCAN_INIT_TIME.getValue, + EntranceConfiguration.ENTRANCE_GROUP_SCAN_INTERVAL.getValue, + TimeUnit.MILLISECONDS + ) + } + + def refreshAllGroupMaxAllowRunningJobs(validInsCount: Int): Unit = { + if (validInsCount <= 0) return + listConsumers() + .foreach(item => { + item.getGroup match { + case group: ParallelGroup => + val maxAllowRunningJobs = Math.round(group.getMaxRunningJobs / validInsCount) + group.setMaxAllowRunningJobs(maxAllowRunningJobs) + logger + .info( + s"group ${group.getGroupName} refresh maxAllowRunningJobs => ${group.getMaxRunningJobs}/$validInsCount=$maxAllowRunningJobs" + ) + case _ => + } + }) + } + } From 1cf6321ef5721f93096cae256a7721eec819d179 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Mon, 19 Dec 2022 11:40:40 +0800 Subject: [PATCH 26/72] update name --- .../linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala | 2 +- .../org/apache/linkis/entrance/utils/JobHistoryHelper.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala index 2404db51dc..1977fa68ac 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala @@ -50,7 +50,7 @@ class EntranceFIFOUserConsumer( ids.add(entranceJob.getJobRequest.getId) case _ => } - JobHistoryHelper.updateBatchInstances(ids) + JobHistoryHelper.updateBatchInstancesEmpty(ids) } Utils.tryQuietly(Thread.sleep(5000)) return diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index 714f1d77de..df7b846a7d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -154,7 +154,7 @@ object JobHistoryHelper extends Logging { val list = new util.ArrayList[Long]() list.addAll(taskIdList) try { - val successTaskIds = updateBatchInstances(updateTaskIds) + val successTaskIds = updateBatchInstancesEmpty(updateTaskIds) if (retryWhenUpdateFail) { list.removeAll(successTaskIds) } else { @@ -175,7 +175,7 @@ object JobHistoryHelper extends Logging { * @param taskIdList * @return */ - def updateBatchInstances(taskIdList: util.List[Long]): util.List[Long] = { + def updateBatchInstancesEmpty(taskIdList: util.List[Long]): util.List[Long] = { val jobReqList = new util.ArrayList[JobRequest]() taskIdList.asScala.foreach(taskID => { val jobRequest = new JobRequest From 789404fa26877959ca85e739d118925683dbbe2f Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 27 Dec 2022 10:40:21 +0800 Subject: [PATCH 27/72] add header --- .../springcloud/http/SpringCloudGatewayHttpRequest.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/scala/org/apache/linkis/gateway/springcloud/http/SpringCloudGatewayHttpRequest.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/scala/org/apache/linkis/gateway/springcloud/http/SpringCloudGatewayHttpRequest.scala index d591e5ce94..929ed6ae62 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/scala/org/apache/linkis/gateway/springcloud/http/SpringCloudGatewayHttpRequest.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/scala/org/apache/linkis/gateway/springcloud/http/SpringCloudGatewayHttpRequest.scala @@ -87,8 +87,10 @@ class SpringCloudGatewayHttpRequest(request: AbstractServerHttpRequest) extends override def getHeaders: JMap[String, Array[String]] = headers - override def addHeader(headerName: String, headers: Array[String]): Unit = + override def addHeader(headerName: String, headers: Array[String]): Unit = { + this.headers.put(headerName, headers) addHeaders.put(headerName, headers) + } override def addCookie(cookieName: String, cookies: Array[Cookie]): Unit = { this.cookies.put(cookieName, cookies) From 2a39fa38099515df4b39c59e6b370878e404aec2 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 8 Feb 2023 16:32:58 +0800 Subject: [PATCH 28/72] bug - print failover ids size --- .../linkis/entrance/server/EntranceFailoverJobServer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 1c2f906a9d..6d8e2971c1 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -132,8 +132,8 @@ public void failoverTask() { expiredTimestamp, EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); if (jobRequests.isEmpty()) return; - Object[] ids = jobRequests.stream().map(JobRequest::getId).toArray(); - logger.info("success query failover jobs , job ids: {}", ids); + List ids = jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); + logger.info("success query failover jobs , job size: {}, ids: {}", ids.size(), ids); // failover to local server for (JobRequest jobRequest : jobRequests) { From a33ad78b0827cc7cc95f378e0f066ef6fe7bb285 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 8 Feb 2023 17:48:13 +0800 Subject: [PATCH 29/72] update failover log --- .../org/apache/linkis/entrance/EntranceServer.scala | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index b5563262b8..9125abf4c6 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -308,12 +308,7 @@ abstract class EntranceServer extends Logging { } val logAppender = new java.lang.StringBuilder() - logAppender.append( - LogUtils - .generateInfo( - s"\n\n*************************************FAILOVER************************************** \n\n" - ) - ) + logAppender.append("*************************************FAILOVER**************************************") // try to kill ec killOldEC(jobRequest, logAppender); @@ -339,7 +334,7 @@ abstract class EntranceServer extends Logging { Utils.tryCatch { logAppender.append( LogUtils - .generateInfo(s"job ${jobRequest.getId} start to kill ec \n") + .generateInfo(s"job ${jobRequest.getId} start to kill old ec \n") ) if ( !SchedulerEventState.isRunning(SchedulerEventState.withName(jobRequest.getStatus)) From c172f326a0b8cfdf8055e79592aa25b0334a3822 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 8 Feb 2023 17:57:26 +0800 Subject: [PATCH 30/72] update log --- .../main/scala/org/apache/linkis/entrance/EntranceServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 9125abf4c6..efd5e76a45 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -462,7 +462,7 @@ abstract class EntranceServer extends Logging { val msg = MessageFormat.format( EntranceErrorCodeSummary.FAILOVER_RUNNING_TO_CANCELLED.getErrorDesc, - jobRequest.getId + jobRequest.getId.toString ) // init jobRequest properties jobRequest.setStatus(SchedulerEventState.Cancelled.toString) From 5e1712d0f437055cd536a983803a7a4c84ed49bb Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 28 Feb 2023 13:48:59 +0800 Subject: [PATCH 31/72] code format --- .../linkis/common/ServiceInstance.scala | 9 +++-- .../scheduler/queue/AbstractGroup.scala | 6 ++-- .../common/protocol/job/JobReqProcotol.scala | 7 +++- .../restful/EntranceLabelRestfulApi.java | 20 +++++------ .../entrance/restful/EntranceRestfulApi.java | 4 ++- .../server/EntranceFailoverJobServer.java | 3 +- .../linkis/entrance/EntranceServer.scala | 15 +++++--- .../entrance/conf/EntranceConfiguration.scala | 9 +++-- .../scheduler/EntranceFIFOUserConsumer.scala | 1 + .../scheduler/EntranceGroupFactory.scala | 13 ++----- .../EntranceParallelConsumerManager.scala | 4 +-- .../entrance/utils/JobHistoryHelper.scala | 12 ++++--- .../manager/label/constant/LabelConstant.java | 2 -- .../jobhistory/dao/JobHistoryMapper.java | 35 +++++++------------ .../impl/JobHistoryQueryServiceImpl.scala | 11 ++---- .../parser/EntranceRequestGatewayParser.scala | 15 ++++---- 16 files changed, 82 insertions(+), 84 deletions(-) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala index 9cee5fe329..f9e4718472 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/ServiceInstance.scala @@ -25,7 +25,10 @@ class ServiceInstance { def getApplicationName: String = applicationName def setInstance(instance: String): Unit = this.instance = instance def getInstance: String = instance - def setRegistryTimestamp(registryTimestamp: Long): Unit = this.registryTimestamp = registryTimestamp + + def setRegistryTimestamp(registryTimestamp: Long): Unit = this.registryTimestamp = + registryTimestamp + def getRegistryTimestamp: Long = registryTimestamp override def equals(other: Any): Boolean = other match { @@ -45,7 +48,9 @@ class ServiceInstance { .foldLeft(0)((a, b) => 31 * a + b) } - override def toString: String = s"ServiceInstance($applicationName, $instance, $registryTimestamp)" + override def toString: String = + s"ServiceInstance($applicationName, $instance, $registryTimestamp)" + } object ServiceInstance { diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala index cc9577941f..b123682b56 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/AbstractGroup.scala @@ -29,9 +29,11 @@ abstract class AbstractGroup extends Group { def setMaxRunningJobs(maxRunningJobs: Int): Unit = this.maxRunningJobs = maxRunningJobs def getMaxRunningJobs: Int = maxRunningJobs - def setMaxAllowRunningJobs(maxAllowRunningJobs: Int): Unit = this.maxAllowRunningJobs = maxAllowRunningJobs + def setMaxAllowRunningJobs(maxAllowRunningJobs: Int): Unit = this.maxAllowRunningJobs = + maxAllowRunningJobs + def getMaxAllowRunningJobs: Int = - if(maxAllowRunningJobs <= 0) maxRunningJobs else Math.min(maxAllowRunningJobs, maxRunningJobs) + if (maxAllowRunningJobs <= 0) maxRunningJobs else Math.min(maxAllowRunningJobs, maxRunningJobs) def setMaxAskExecutorTimes(maxAskExecutorTimes: Long): Unit = this.maxAskExecutorTimes = maxAskExecutorTimes diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala index 4d6346c918..df197ddb2c 100644 --- a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/protocol/job/JobReqProcotol.scala @@ -52,4 +52,9 @@ class RequestOneJob extends JobReq { case class RequestAllJob(instance: String) extends JobReq -case class RequestFailoverJob(reqMap: util.Map[String, java.lang.Long], statusList: util.List[String], startTimestamp: Long, limit: Int = 10) extends JobReq +case class RequestFailoverJob( + reqMap: util.Map[String, java.lang.Long], + statusList: util.List[String], + startTimestamp: Long, + limit: Int = 10 +) extends JobReq diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java index e51f66266d..841a6a3fb0 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceLabelRestfulApi.java @@ -21,7 +21,6 @@ import org.apache.linkis.entrance.EntranceServer; import org.apache.linkis.entrance.scheduler.EntranceSchedulerContext; import org.apache.linkis.instance.label.client.InstanceLabelClient; -import org.apache.linkis.manager.label.constant.LabelConstant; import org.apache.linkis.manager.label.constant.LabelKeyConstant; import org.apache.linkis.manager.label.constant.LabelValueConstant; import org.apache.linkis.protocol.label.InsLabelRefreshRequest; @@ -50,13 +49,13 @@ @RequestMapping(path = "/entrance/operation/label") public class EntranceLabelRestfulApi { - private static final Logger logger = LoggerFactory.getLogger(EntranceLabelRestfulApi.class); - private EntranceServer entranceServer; + private static final Logger logger = LoggerFactory.getLogger(EntranceLabelRestfulApi.class); + private EntranceServer entranceServer; - @Autowired - public void setEntranceServer(EntranceServer entranceServer) { - this.entranceServer = entranceServer; - } + @Autowired + public void setEntranceServer(EntranceServer entranceServer) { + this.entranceServer = entranceServer; + } @ApiOperation(value = "update", notes = "update route label", response = Message.class) @ApiOperationSupport(ignoreParameters = {"jsonNode"}) @@ -92,13 +91,14 @@ public Message updateRouteLabel(HttpServletRequest req) { logger.info("Finished to modify the routelabel of entry to offline"); logger.info("Prepare to update all not execution task instances to empty string"); - SchedulerContext schedulerContext = entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); + SchedulerContext schedulerContext = + entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); if (schedulerContext instanceof EntranceSchedulerContext) { - ((EntranceSchedulerContext) schedulerContext).setOfflineFlag(true); + ((EntranceSchedulerContext) schedulerContext).setOfflineFlag(true); } entranceServer.updateAllNotExecutionTaskInstances(true); logger.info("Finished to update all not execution task instances to empty string"); - return Message.ok(); + return Message.ok(); } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index afa4aeb06c..71b0df4250 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -614,7 +614,9 @@ public Message log(HttpServletRequest req, @PathVariable("id") String id) { logger.warn("The job {} wait failover, return customer log", id); message = Message.ok(); message.setMethod("/api/entrance/" + id + "/log"); - String log = LogUtils.generateInfo("The job will failover soon, please try again later.(job很快就会failover,请稍后再试)"); + String log = + LogUtils.generateInfo( + "The job will failover soon, please try again later.(job很快就会failover,请稍后再试)"); Object retLog; if (distinctLevel) { String[] array = new String[4]; diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 6d8e2971c1..77e85cba69 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -132,7 +132,8 @@ public void failoverTask() { expiredTimestamp, EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); if (jobRequests.isEmpty()) return; - List ids = jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); + List ids = + jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); logger.info("success query failover jobs , job size: {}, ids: {}", ids.size(), ids); // failover to local server diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index efd5e76a45..8ef5c268b5 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -28,6 +28,7 @@ import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary._ import org.apache.linkis.entrance.exception.{EntranceErrorException, SubmitFailedException} import org.apache.linkis.entrance.execute.EntranceJob +import org.apache.linkis.entrance.job.EntranceExecutionJob import org.apache.linkis.entrance.log.{Cache, CacheLogWriter, HDFSCacheLogWriter, LogReader} import org.apache.linkis.entrance.parser.ParserUtils import org.apache.linkis.entrance.timeout.JobTimeoutManager @@ -43,16 +44,16 @@ import org.apache.linkis.rpc.conf.RPCConfiguration import org.apache.linkis.scheduler.queue.{Job, SchedulerEventState} import org.apache.linkis.server.conf.ServerConfiguration import org.apache.linkis.storage.utils.StorageUtils + import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils -import org.apache.linkis.common.log.LogUtils import org.springframework.beans.BeanUtils -import org.apache.linkis.entrance.job.EntranceExecutionJob import java.{lang, util} import java.text.{MessageFormat, SimpleDateFormat} import java.util.Date + import scala.collection.JavaConverters._ abstract class EntranceServer extends Logging { @@ -280,8 +281,10 @@ abstract class EntranceServer extends Logging { consumeQueueTasks.foreach(job => { taskIds.add(job.getJobRequest.getId.asInstanceOf[Long]) job match { - case entranceExecutionJob : EntranceExecutionJob => - val msg = LogUtils.generateWarn(s"job ${job.getJobRequest.getId} clean from ConsumeQueue, wait for failover") + case entranceExecutionJob: EntranceExecutionJob => + val msg = LogUtils.generateWarn( + s"job ${job.getJobRequest.getId} clean from ConsumeQueue, wait for failover" + ) entranceExecutionJob.getLogListener.foreach(_.onLogUpdate(entranceExecutionJob, msg)) entranceExecutionJob.getLogWriter.foreach(_.close()) case _ => @@ -308,7 +311,9 @@ abstract class EntranceServer extends Logging { } val logAppender = new java.lang.StringBuilder() - logAppender.append("*************************************FAILOVER**************************************") + logAppender.append( + "*************************************FAILOVER**************************************" + ) // try to kill ec killOldEC(jobRequest, logAppender); diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 959d8c68bc..13db69700f 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -233,12 +233,15 @@ object EntranceConfiguration { val ENTRANCE_FAILOVER_SCAN_INTERVAL = CommonVars("linkis.entrance.failover.scan.interval", 30 * 1000).getValue - val ENTRANCE_FAILOVER_DATA_NUM_LIMIT = CommonVars("linkis.entrance.failover.data.num.limit", 10).getValue + val ENTRANCE_FAILOVER_DATA_NUM_LIMIT = + CommonVars("linkis.entrance.failover.data.num.limit", 10).getValue - val ENTRANCE_FAILOVER_DATA_INTERVAL_TIME = CommonVars("linkis.entrance.failover.data.interval.time", new TimeType("7d").toLong).getValue + val ENTRANCE_FAILOVER_DATA_INTERVAL_TIME = + CommonVars("linkis.entrance.failover.data.interval.time", new TimeType("7d").toLong).getValue // if true, the waitForRetry job in runningJobs can be failover - val ENTRANCE_FAILOVER_RETRY_JOB_ENABLED = CommonVars("linkis.entrance.failover.retry.job.enable", true) + val ENTRANCE_FAILOVER_RETRY_JOB_ENABLED = + CommonVars("linkis.entrance.failover.retry.job.enable", true) val ENTRANCE_UPDATE_BATCH_SIZE = CommonVars("linkis.entrance.update.batch.size", 100) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala index 1977fa68ac..faee683fbf 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceFIFOUserConsumer.scala @@ -27,6 +27,7 @@ import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer import java.util import java.util.concurrent.ExecutorService + import scala.collection.JavaConverters.collectionAsScalaIterableConverter class EntranceFIFOUserConsumer( diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala index 4bd0caca1b..0f31351b48 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceGroupFactory.scala @@ -27,18 +27,12 @@ import org.apache.linkis.governance.common.protocol.conf.{ RequestQueryEngineConfigWithGlobalConfig, ResponseQueryConfig } -import org.apache.linkis.instance.label.client.InstanceLabelClient -import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext -import org.apache.linkis.manager.label.constant.{LabelKeyConstant, LabelValueConstant} -import org.apache.linkis.governance.common.protocol.conf.{RequestQueryEngineConfigWithGlobalConfig, ResponseQueryConfig} import org.apache.linkis.manager.label.entity.Label import org.apache.linkis.manager.label.entity.engine.{ ConcurrentEngineConnLabel, EngineTypeLabel, UserCreatorLabel } -import org.apache.linkis.manager.label.entity.route.RouteLabel -import org.apache.linkis.manager.label.entity.engine.{ConcurrentEngineConnLabel, EngineTypeLabel, UserCreatorLabel} import org.apache.linkis.manager.label.utils.LabelUtil import org.apache.linkis.protocol.constants.TaskConstant import org.apache.linkis.protocol.utils.TaskUtils @@ -51,13 +45,10 @@ import org.apache.commons.lang3.StringUtils import java.util import java.util.concurrent.TimeUnit import java.util.regex.Pattern + import scala.collection.JavaConverters._ + import com.google.common.cache.{Cache, CacheBuilder} -import org.apache.linkis.common.ServiceInstance -import org.apache.linkis.instance.label.client.InstanceLabelClient -import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext -import org.apache.linkis.manager.label.constant.{LabelConstant, LabelKeyConstant} -import org.apache.linkis.manager.label.entity.route.RouteLabel class EntranceGroupFactory extends GroupFactory with Logging { diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index f114981c5c..a067d65829 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -18,7 +18,6 @@ package org.apache.linkis.entrance.scheduler import org.apache.linkis.common.ServiceInstance -import org.apache.linkis.common.conf.CommonVars import org.apache.linkis.common.utils.Utils import org.apache.linkis.entrance.conf.EntranceConfiguration import org.apache.linkis.instance.label.client.InstanceLabelClient @@ -32,10 +31,11 @@ import org.apache.linkis.scheduler.queue.parallelqueue.{ParallelConsumerManager, import java.util import java.util.concurrent.TimeUnit + import scala.collection.JavaConverters._ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: String) - extends ParallelConsumerManager(maxParallelismUsers, schedulerName){ + extends ParallelConsumerManager(maxParallelismUsers, schedulerName) { override protected def createConsumer(groupName: String): FIFOUserConsumer = { val group = getSchedulerContext.getOrCreateGroupFactory.getGroup(groupName) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala index df7b846a7d..3fed0f78be 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/JobHistoryHelper.scala @@ -30,13 +30,15 @@ import org.apache.linkis.protocol.constants.TaskConstant import org.apache.linkis.protocol.query.cache.{CacheTaskResult, RequestReadCache} import org.apache.linkis.rpc.Sender import org.apache.linkis.scheduler.queue.SchedulerEventState + import org.apache.commons.lang3.StringUtils import javax.servlet.http.HttpServletRequest + import java.util import java.util.Date + import scala.collection.JavaConverters._ -import sun.net.util.IPAddressUtil import com.google.common.net.InetAddresses @@ -316,15 +318,15 @@ object JobHistoryHelper extends Logging { val ecResourceMap = if (resourceInfo == null) new util.HashMap[String, ResourceWithStatus] else resourceInfo if (resourceMap != null) { - resourceMap.asInstanceOf[util.HashMap[String, ResourceWithStatus]].putAll(ecResourceMap) + resourceMap.asInstanceOf[util.Map[String, ResourceWithStatus]].putAll(ecResourceMap) } else { metricsMap.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, ecResourceMap) } - var engineInstanceMap: util.HashMap[String, AnyRef] = null + var engineInstanceMap: util.Map[String, AnyRef] = null if (metricsMap.containsKey(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP)) { engineInstanceMap = metricsMap .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) - .asInstanceOf[util.HashMap[String, AnyRef]] + .asInstanceOf[util.Map[String, AnyRef]] } else { engineInstanceMap = new util.HashMap[String, AnyRef]() metricsMap.put(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP, engineInstanceMap) @@ -334,7 +336,7 @@ object JobHistoryHelper extends Logging { val ticketId = infoMap.get(TaskConstant.TICKET_ID).asInstanceOf[String] val engineExtraInfoMap = engineInstanceMap .getOrDefault(ticketId, new util.HashMap[String, AnyRef]) - .asInstanceOf[util.HashMap[String, AnyRef]] + .asInstanceOf[util.Map[String, AnyRef]] engineExtraInfoMap.putAll(infoMap) engineInstanceMap.put(ticketId, engineExtraInfoMap) } else { diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java index b43501ed9e..4db4bfca40 100644 --- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java +++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelConstant.java @@ -22,6 +22,4 @@ public class LabelConstant { public static final int LABEL_BUILDER_ERROR_CODE = 40001; public static final int LABEL_UTIL_CONVERT_ERROR_CODE = 40002; - - public static final String OFFLINE = "offline"; } diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java b/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java index 6568fb838b..88267e4800 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java @@ -110,25 +110,13 @@ Integer countUndoneTaskWithCreatorOnly( /** * query wait for failover job * - * Sql example: - * SELECT a.* FROM linkis_ps_job_history_group_history a - * where (a.instances = '' - * or a.instances is null - * or a.instances not in ('192.168.1.123:9104','192.168.1.124:9104') - * or EXISTS ( - * select 1 from - * ( - * select '192.168.1.123:9104' as instances, 1697775054098 as registryTime - * union all - * select '192.168.1.124:9104' as instances, 1666239054098 as registryTime - * ) b - * where a.instances = b.instances and UNIX_TIMESTAMP(a.created_time) * 1000 < b.registryTime - * ) - * ) - * and - * status in ('Inited','Running','Scheduled','WaitForRetry') - * and UNIX_TIMESTAMP(a.created_time) * 1000 >= 1666239054098 - * limit 10 + *

Sql example: SELECT a.* FROM linkis_ps_job_history_group_history a where (a.instances = '' + * or a.instances is null or a.instances not in ('192.168.1.123:9104','192.168.1.124:9104') or + * EXISTS ( select 1 from ( select '192.168.1.123:9104' as instances, 1697775054098 as + * registryTime union all select '192.168.1.124:9104' as instances, 1666239054098 as registryTime + * ) b where a.instances = b.instances and UNIX_TIMESTAMP(a.created_time) * 1000 < b.registryTime + * ) ) and status in ('Inited','Running','Scheduled','WaitForRetry') and + * UNIX_TIMESTAMP(a.created_time) * 1000 >= 1666239054098 limit 10 * * @param instancesMap * @param statusList @@ -136,8 +124,9 @@ Integer countUndoneTaskWithCreatorOnly( * @param limit * @return */ - List selectFailoverJobHistory(@Param("instancesMap") Map instancesMap, - @Param("statusList") List statusList, - @Param("startTimestamp") Long startTimestamp, - @Param("limit") Integer limit); + List selectFailoverJobHistory( + @Param("instancesMap") Map instancesMap, + @Param("statusList") List statusList, + @Param("startTimestamp") Long startTimestamp, + @Param("limit") Integer limit); } diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala index 22084f88a6..a44cd0e262 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/scala/org/apache/linkis/jobhistory/service/impl/JobHistoryQueryServiceImpl.scala @@ -252,7 +252,8 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { logger.info(s"query failover jobs, start timestamp:${startTimestamp}, limit:${limit}") val jobResp = new JobRespProtocol Utils.tryCatch { - val jobList = jobHistoryMapper.selectFailoverJobHistory(reqMap, statusList, startTimestamp, limit) + val jobList = + jobHistoryMapper.selectFailoverJobHistory(reqMap, statusList, startTimestamp, limit) val jobReqList = jobList.asScala.map(jobHistory2JobRequest).toList val map = new util.HashMap[String, Object]() map.put(JobRequestConstants.JOB_HISTORY_LIST, jobReqList) @@ -266,14 +267,6 @@ class JobHistoryQueryServiceImpl extends JobHistoryQueryService with Logging { jobResp } - /* private def queryTaskList2RequestPersistTaskList(queryTask: java.util.List[QueryTask]): java.util.List[RequestPersistTask] = { - import scala.collection.JavaConversions._ - val tasks = new util.ArrayList[RequestPersistTask] - import org.apache.linkis.jobhistory.conversions.TaskConversions.queryTask2RequestPersistTask - queryTask.foreach(f => tasks.add(f)) - tasks - } */ - override def getJobHistoryByIdAndName(jobId: java.lang.Long, userName: String): JobHistory = { val jobReq = new JobHistory jobReq.setId(jobId) diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala index 883f252d70..930bfac73a 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala @@ -17,10 +17,8 @@ package org.apache.linkis.gateway.ujes.parser -import org.apache.commons.lang3.StringUtils import org.apache.linkis.common.ServiceInstance import org.apache.linkis.common.entity.JobInstance -import org.apache.linkis.common.utils.JsonUtils import org.apache.linkis.gateway.config.GatewayConfiguration import org.apache.linkis.gateway.http.GatewayContext import org.apache.linkis.gateway.parser.AbstractGatewayParser @@ -30,6 +28,9 @@ import org.apache.linkis.protocol.utils.ZuulEntranceUtils import org.apache.linkis.rpc.interceptor.ServiceInstanceUtils import org.apache.linkis.server.BDPJettyServerHelper import org.apache.linkis.server.conf.ServerConfiguration + +import org.apache.commons.lang3.StringUtils + import org.springframework.stereotype.Component import javax.annotation.Resource @@ -37,7 +38,6 @@ import javax.annotation.Resource @Component class EntranceRequestGatewayParser extends AbstractGatewayParser { - @Resource private var jobHistoryQueryService: JobHistoryQueryService = _ @@ -49,9 +49,9 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { if (sendResponseWhenNotMatchVersion(gatewayContext, version)) return val serviceInstance = if (execId.startsWith(EntranceRequestGatewayParser.API_REQUEST)) { if ( - gatewayContext.getRequest.getQueryParams.containsKey( - EntranceRequestGatewayParser.INSTANCE - ) + gatewayContext.getRequest.getQueryParams.containsKey( + EntranceRequestGatewayParser.INSTANCE + ) ) { val instances = gatewayContext.getRequest.getQueryParams.get(EntranceRequestGatewayParser.INSTANCE) @@ -83,7 +83,8 @@ class EntranceRequestGatewayParser extends AbstractGatewayParser { } def buildJobInstance(taskId: Long, gatewayContext: GatewayContext): JobInstance = { - val histories = jobHistoryQueryService.search(taskId, null, null, null, null, null, null, null) + val histories = + jobHistoryQueryService.search(taskId, null, null, null, null, null, null, null, null) if (histories.isEmpty) { sendErrorResponse(s"taskId $taskId is not exists.", gatewayContext) return null From 9e27aec833d3d20e43ff2f6c6c3d1d107860364d Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 28 Feb 2023 14:44:05 +0800 Subject: [PATCH 32/72] set default value --- .../server/DefaultEntranceServer.java | 13 +- .../server/EntranceFailoverJobServer.java | 178 +++++++++--------- .../entrance/conf/EntranceConfiguration.scala | 8 +- 3 files changed, 95 insertions(+), 104 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index 54b855ffbd..b077ab37bb 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -17,6 +17,7 @@ package org.apache.linkis.entrance.server; +import org.apache.commons.io.IOUtils; import org.apache.linkis.entrance.EntranceContext; import org.apache.linkis.entrance.EntranceServer; import org.apache.linkis.entrance.conf.EntranceConfiguration; @@ -25,9 +26,8 @@ import org.apache.linkis.entrance.job.EntranceExecutionJob; import org.apache.linkis.entrance.log.LogReader; import org.apache.linkis.rpc.Sender; - -import org.apache.commons.io.IOUtils; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.context.event.ContextClosedEvent; import org.springframework.context.event.EventListener; @@ -35,9 +35,6 @@ import javax.annotation.PostConstruct; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** Description: */ @Component(ServiceNameConsts.ENTRANCE_SERVER) public class DefaultEntranceServer extends EntranceServer { @@ -91,9 +88,9 @@ private void shutdownEntrance(ContextClosedEvent event) { logger.warn("Entrance exit to stop all job"); EntranceJob[] allUndoneTask = getAllUndoneTask(null); if (null != allUndoneTask) { - String msg = "Entrance exits the automatic cleanup task and can be rerun(服务退出自动清理任务,可以重跑)"; for (EntranceJob job : allUndoneTask) { - job.onFailure(msg, null); + job.onFailure( + "Entrance exits the automatic cleanup task and can be rerun(服务退出自动清理任务,可以重跑)", null); IOUtils.closeQuietly(((EntranceExecutionJob) job).getLogWriter().get()); } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 77e85cba69..73c91f6a36 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -29,12 +29,12 @@ import org.apache.linkis.publicservice.common.lock.service.CommonLockService; import org.apache.linkis.rpc.Sender; import org.apache.linkis.scheduler.queue.SchedulerEventState; - +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import javax.annotation.PostConstruct; - import java.util.Arrays; import java.util.List; import java.util.Map; @@ -43,9 +43,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - @Component(ServiceNameConsts.ENTRANCE_FAILOVER_SERVER) public class EntranceFailoverJobServer { @@ -61,96 +58,93 @@ public class EntranceFailoverJobServer { @PostConstruct public void init() { - this.scheduledExecutor = - Executors.newSingleThreadScheduledExecutor( - Utils.threadFactory("Linkis-Failover-Scheduler-Thread-", true)); - failoverTask(); + if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { + this.scheduledExecutor = + Executors.newSingleThreadScheduledExecutor( + Utils.threadFactory("Linkis-Failover-Scheduler-Thread-", true)); + failoverTask(); + } } public void failoverTask() { - if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { - scheduledExecutor.scheduleWithFixedDelay( - () -> { - EntranceSchedulerContext schedulerContext = - (EntranceSchedulerContext) - entranceServer - .getEntranceContext() - .getOrCreateScheduler() - .getSchedulerContext(); - - // entrance do not failover job when it is offline - if (schedulerContext.getOfflineFlag()) return; - - CommonLock commonLock = new CommonLock(); - commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); - Boolean locked = false; - try { - locked = commonLockService.lock(commonLock, 10 * 1000L); - if (!locked) return; - logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); - - // get all entrance server from eureka - ServiceInstance[] serviceInstances = - Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); - if (serviceInstances == null || serviceInstances.length <= 0) return; - - // serverInstance to map - Map serverInstanceMap = - Arrays.stream(serviceInstances) - .collect( - Collectors.toMap( - ServiceInstance::getInstance, - ServiceInstance::getRegistryTimestamp, - (k1, k2) -> k2)); - - // It is very important to avoid repeated execute job - // when failover self job, if self instance is empty, the job can be repeated execute - if (!serverInstanceMap.containsKey(Sender.getThisInstance())) { - logger.warn( - "server has just started and has not get self info, it does not failover"); - return; - } - - // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) - long expiredTimestamp = 0L; - if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { - expiredTimestamp = - System.currentTimeMillis() - - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); - } - - // get uncompleted status - List statusList = - Arrays.stream(SchedulerEventState.uncompleteStatusArray()) - .map(Object::toString) - .collect(Collectors.toList()); - - List jobRequests = - JobHistoryHelper.queryWaitForFailoverTask( - serverInstanceMap, - statusList, - expiredTimestamp, - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); - if (jobRequests.isEmpty()) return; - List ids = - jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); - logger.info("success query failover jobs , job size: {}, ids: {}", ids.size(), ids); - - // failover to local server - for (JobRequest jobRequest : jobRequests) { - entranceServer.failoverExecute(jobRequest); - } - logger.info("finished execute failover jobs, job ids: {}", ids); - - } catch (Exception e) { - logger.error("failover failed", e); - } finally { - if (locked) commonLockService.unlock(commonLock); + scheduledExecutor.scheduleWithFixedDelay( + () -> { + EntranceSchedulerContext schedulerContext = + (EntranceSchedulerContext) + entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); + + // entrance do not failover job when it is offline + if (schedulerContext.getOfflineFlag()) return; + + CommonLock commonLock = new CommonLock(); + commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); + Boolean locked = false; + try { + locked = commonLockService.lock(commonLock, 30 * 1000L); + if (!locked) return; + logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); + + // get all entrance server from eureka + ServiceInstance[] serviceInstances = + Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); + if (serviceInstances == null || serviceInstances.length <= 0) return; + + // serverInstance to map + Map serverInstanceMap = + Arrays.stream(serviceInstances) + .collect( + Collectors.toMap( + ServiceInstance::getInstance, + ServiceInstance::getRegistryTimestamp, + (k1, k2) -> k2)); + + // It is very important to avoid repeated execute job + // when failover self job, if self instance is empty, the job can be repeated execute + if (!serverInstanceMap.containsKey(Sender.getThisInstance())) { + logger.warn( + "server has just started and has not get self info, it does not failover"); + return; } - }, - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), - TimeUnit.MILLISECONDS); - } + + // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) + long expiredTimestamp = 0L; + if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { + expiredTimestamp = + System.currentTimeMillis() + - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); + } + + // get uncompleted status + List statusList = + Arrays.stream(SchedulerEventState.uncompleteStatusArray()) + .map(Object::toString) + .collect(Collectors.toList()); + + List jobRequests = + JobHistoryHelper.queryWaitForFailoverTask( + serverInstanceMap, + statusList, + expiredTimestamp, + EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); + if (jobRequests.isEmpty()) return; + List ids = + jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); + logger.info("success query failover jobs , job size: {}, ids: {}", ids.size(), ids); + + // failover to local server + for (JobRequest jobRequest : jobRequests) { + entranceServer.failoverExecute(jobRequest); + } + logger.info("finished execute failover jobs, job ids: {}", ids); + + } catch (Exception e) { + logger.error("failover failed", e); + } finally { + if (locked) commonLockService.unlock(commonLock); + } + }, + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), + TimeUnit.MILLISECONDS); } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 13db69700f..d8248620b7 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -241,7 +241,7 @@ object EntranceConfiguration { // if true, the waitForRetry job in runningJobs can be failover val ENTRANCE_FAILOVER_RETRY_JOB_ENABLED = - CommonVars("linkis.entrance.failover.retry.job.enable", true) + CommonVars("linkis.entrance.failover.retry.job.enable", false) val ENTRANCE_UPDATE_BATCH_SIZE = CommonVars("linkis.entrance.update.batch.size", 100) @@ -255,12 +255,12 @@ object EntranceConfiguration { val ENTRANCE_GROUP_SCAN_INTERVAL = CommonVars("linkis.entrance.group.scan.interval", 60 * 1000) val ENTRANCE_FAILOVER_RETAIN_ENGINE_CONN_ENABLED = - CommonVars("linkis.entrance.failover.retain.engine.conn.enable", true) + CommonVars("linkis.entrance.failover.retain.engine.conn.enable", false) val ENTRANCE_FAILOVER_RETAIN_YARN_RESOURCE_ENABLED = - CommonVars("linkis.entrance.failover.retain.yarn.resource.enable", true) + CommonVars("linkis.entrance.failover.retain.yarn.resource.enable", false) val ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED = - CommonVars("linkis.entrance.failover.running.kill.enable", true) + CommonVars("linkis.entrance.failover.running.kill.enable", false) } From 3dda863d37aa94326b77c52a33251bf934e588d3 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 28 Feb 2023 15:27:17 +0800 Subject: [PATCH 33/72] add config to properties --- .../linkis/entrance/server/DefaultEntranceServer.java | 9 ++++++--- .../entrance/server/EntranceFailoverJobServer.java | 7 +++++-- .../org/apache/linkis/entrance/EntranceServer.scala | 2 +- linkis-dist/package/conf/linkis-cg-entrance.properties | 7 ++++++- linkis-dist/package/conf/linkis-mg-gateway.properties | 4 ++-- 5 files changed, 20 insertions(+), 9 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index b077ab37bb..14bea60435 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -17,7 +17,6 @@ package org.apache.linkis.entrance.server; -import org.apache.commons.io.IOUtils; import org.apache.linkis.entrance.EntranceContext; import org.apache.linkis.entrance.EntranceServer; import org.apache.linkis.entrance.conf.EntranceConfiguration; @@ -26,8 +25,9 @@ import org.apache.linkis.entrance.job.EntranceExecutionJob; import org.apache.linkis.entrance.log.LogReader; import org.apache.linkis.rpc.Sender; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import org.apache.commons.io.IOUtils; + import org.springframework.beans.factory.annotation.Autowired; import org.springframework.context.event.ContextClosedEvent; import org.springframework.context.event.EventListener; @@ -35,6 +35,9 @@ import javax.annotation.PostConstruct; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** Description: */ @Component(ServiceNameConsts.ENTRANCE_SERVER) public class DefaultEntranceServer extends EntranceServer { diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index 73c91f6a36..d7f5ce5951 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -29,12 +29,12 @@ import org.apache.linkis.publicservice.common.lock.service.CommonLockService; import org.apache.linkis.rpc.Sender; import org.apache.linkis.scheduler.queue.SchedulerEventState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; import javax.annotation.PostConstruct; + import java.util.Arrays; import java.util.List; import java.util.Map; @@ -43,6 +43,9 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + @Component(ServiceNameConsts.ENTRANCE_FAILOVER_SERVER) public class EntranceFailoverJobServer { diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 8ef5c268b5..8e9bbeeac0 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -390,7 +390,7 @@ abstract class EntranceServer extends Logging { engineStopRequest.setServiceInstance(ecInstance) // send to linkismanager kill ec Sender - .getSender(RPCConfiguration.LINKIS_MANAGER_APPLICATION_NAME.getValue) + .getSender(RPCConfiguration.LINKIS_MANAGER_SERVICE_NAME.getValue) .send(engineStopRequest) val msg = s"job ${jobRequest.getId} send EngineStopRequest to linkismanager, kill EC instance $ecInstance" diff --git a/linkis-dist/package/conf/linkis-cg-entrance.properties b/linkis-dist/package/conf/linkis-cg-entrance.properties index e89ced2159..639256d5cf 100644 --- a/linkis-dist/package/conf/linkis-cg-entrance.properties +++ b/linkis-dist/package/conf/linkis-cg-entrance.properties @@ -33,4 +33,9 @@ spring.server.port=9104 wds.linkis.entrance.user.creator.ip.interceptor.switch=false ## you may set service version if you want to distinguish different configuration version -spring.eureka.instance.metadata-map.linkis.conf.version=v1 \ No newline at end of file +spring.eureka.instance.metadata-map.linkis.conf.version=v1 + + +wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml +wds.linkis.server.mybatis.BasePackage=org.apache.linkis.publicservice.common.lock.dao +wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.publicservice.common.lock.entity \ No newline at end of file diff --git a/linkis-dist/package/conf/linkis-mg-gateway.properties b/linkis-dist/package/conf/linkis-mg-gateway.properties index 84be3d897d..27656f7f31 100644 --- a/linkis-dist/package/conf/linkis-mg-gateway.properties +++ b/linkis-dist/package/conf/linkis-mg-gateway.properties @@ -21,8 +21,8 @@ wds.linkis.gateway.conf.url.pass.auth=/dss/ wds.linkis.gateway.conf.enable.token.auth=true wds.linkis.is.gateway=true wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml -wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity -wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao +wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity,org.apache.linkis.jobhistory.entity +wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao,org.apache.linkis.jobhistory.dao wds.linkis.label.entity.packages=org.apache.linkis.gateway.ujes.route.label wds.linkis.login_encrypt.enable=false ##LDAP From ff2871919e7903877e175f14979d797e20706dd6 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Tue, 28 Feb 2023 21:09:58 +0800 Subject: [PATCH 34/72] change HashMap to Map --- .../main/scala/org/apache/linkis/entrance/EntranceServer.scala | 2 +- .../linkis/entrance/interceptor/impl/CustomVariableUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 8e9bbeeac0..55be20fd4d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -312,7 +312,7 @@ abstract class EntranceServer extends Logging { val logAppender = new java.lang.StringBuilder() logAppender.append( - "*************************************FAILOVER**************************************" + "*************************************FAILOVER**************************************\n" ) // try to kill ec diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala index 7a7cb7463a..a40c3fa35d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CustomVariableUtils.scala @@ -63,7 +63,7 @@ object CustomVariableUtils extends Logging { } val variableMap = TaskUtils .getVariableMap(jobRequest.getParams) - .asInstanceOf[util.HashMap[String, String]] + .asInstanceOf[util.Map[String, String]] variables.putAll(variableMap) if (!variables.containsKey("user")) { variables.put("user", jobRequest.getExecuteUser) From 39d45d3b40d27427a2aa28d334f2bc88189b55e5 Mon Sep 17 00:00:00 2001 From: guoshupei <719126Liyuelynn> Date: Wed, 1 Mar 2023 11:03:56 +0800 Subject: [PATCH 35/72] update default value --- .../main/scala/org/apache/linkis/entrance/EntranceServer.scala | 2 +- .../org/apache/linkis/entrance/conf/EntranceConfiguration.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 55be20fd4d..5560cc716d 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -312,7 +312,7 @@ abstract class EntranceServer extends Logging { val logAppender = new java.lang.StringBuilder() logAppender.append( - "*************************************FAILOVER**************************************\n" + "*************************************FAILOVER************************************** \n" ) // try to kill ec diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index d8248620b7..17f2dffd9c 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -237,7 +237,7 @@ object EntranceConfiguration { CommonVars("linkis.entrance.failover.data.num.limit", 10).getValue val ENTRANCE_FAILOVER_DATA_INTERVAL_TIME = - CommonVars("linkis.entrance.failover.data.interval.time", new TimeType("7d").toLong).getValue + CommonVars("linkis.entrance.failover.data.interval.time", new TimeType("1d").toLong).getValue // if true, the waitForRetry job in runningJobs can be failover val ENTRANCE_FAILOVER_RETRY_JOB_ENABLED = From fdc54d45cb2b9f6f88a1eeef9e06ba3424a24fa5 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Thu, 2 Mar 2023 12:08:08 +0800 Subject: [PATCH 36/72] Optimal refresh consumer group maxAllowRunningJobs logic --- .../entrance/scheduler/EntranceParallelConsumerManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index a067d65829..a6e24388a6 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -75,7 +75,7 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S } def refreshAllGroupMaxAllowRunningJobs(validInsCount: Int): Unit = { - if (validInsCount <= 0) return + if (validInsCount <= 1) return listConsumers() .foreach(item => { item.getGroup match { From 0b0ef7917eeccc2707754b88eb74421c1d3e3913 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Sun, 5 Mar 2023 18:44:38 +0800 Subject: [PATCH 37/72] rename config key --- .../org/apache/linkis/server/conf/ServerConfiguration.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala index 8d9f9d65ad..3c6a25a343 100644 --- a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala +++ b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/conf/ServerConfiguration.scala @@ -208,6 +208,6 @@ object ServerConfiguration extends Logging { CommonVars("wds.linkis.session.proxy.user.ticket.key", "linkis_user_session_proxy_ticket_id_v1") val LINKIS_SERVER_ENTRANCE_HEADER_KEY = - CommonVars("wds.linkis.server.entrance.header.key", "jobInstanceKey") + CommonVars("linkis.server.entrance.header.key", "jobInstanceKey") } From 75eddde7591a59c2b721a92afd8b5dc88b1143e3 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Mon, 6 Mar 2023 11:46:01 +0800 Subject: [PATCH 38/72] rename metric config key --- .../scala/org/apache/linkis/entrance/EntranceServer.scala | 4 ++-- .../linkis/entrance/conf/EntranceConfiguration.scala | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 5560cc716d..e9c3da2cda 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -625,7 +625,7 @@ abstract class EntranceServer extends Logging { ) val metricMap = new util.HashMap[String, Object]() - if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_ENGINE_CONN_ENABLED.getValue) { + if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_METRIC_ENGINE_CONN_ENABLED.getValue) { if ( jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( TaskConstant.ENTRANCEJOB_ENGINECONN_MAP @@ -638,7 +638,7 @@ abstract class EntranceServer extends Logging { } } - if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_YARN_RESOURCE_ENABLED.getValue) { + if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_METRIC_YARN_RESOURCE_ENABLED.getValue) { if ( jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( TaskConstant.ENTRANCEJOB_YARNRESOURCE diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 17f2dffd9c..617584f278 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -254,11 +254,11 @@ object EntranceConfiguration { val ENTRANCE_GROUP_SCAN_INTERVAL = CommonVars("linkis.entrance.group.scan.interval", 60 * 1000) - val ENTRANCE_FAILOVER_RETAIN_ENGINE_CONN_ENABLED = - CommonVars("linkis.entrance.failover.retain.engine.conn.enable", false) + val ENTRANCE_FAILOVER_RETAIN_METRIC_ENGINE_CONN_ENABLED = + CommonVars("linkis.entrance.failover.retain.metric.engine.conn.enable", false) - val ENTRANCE_FAILOVER_RETAIN_YARN_RESOURCE_ENABLED = - CommonVars("linkis.entrance.failover.retain.yarn.resource.enable", false) + val ENTRANCE_FAILOVER_RETAIN_METRIC_YARN_RESOURCE_ENABLED = + CommonVars("linkis.entrance.failover.retain.metric.yarn.resource.enable", false) val ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED = CommonVars("linkis.entrance.failover.running.kill.enable", false) From 6fee59f67d637dad555fc96715164e327c1eee08 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Mar 2023 11:02:34 +0800 Subject: [PATCH 39/72] - failover server close - use logger template --- .../server/EntranceFailoverJobServer.java | 182 ++++++++++-------- .../EntranceParallelConsumerManager.scala | 10 +- 2 files changed, 108 insertions(+), 84 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index d7f5ce5951..d162be0820 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -31,6 +31,8 @@ import org.apache.linkis.scheduler.queue.SchedulerEventState; import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.event.ContextClosedEvent; +import org.springframework.context.event.EventListener; import org.springframework.stereotype.Component; import javax.annotation.PostConstruct; @@ -38,9 +40,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.*; import java.util.stream.Collectors; import org.slf4j.Logger; @@ -59,6 +59,8 @@ public class EntranceFailoverJobServer { private ScheduledExecutorService scheduledExecutor; + private Future future; + @PostConstruct public void init() { if (EntranceConfiguration.ENTRANCE_FAILOVER_ENABLED()) { @@ -69,85 +71,101 @@ public void init() { } } + @EventListener + private void shutdownFailover(ContextClosedEvent event) { + if (future != null && !future.isDone()) { + future.cancel(true); + } + if (scheduledExecutor != null) { + scheduledExecutor.shutdown(); + logger.info("Entrance Failover Server exit!"); + } + } + public void failoverTask() { - scheduledExecutor.scheduleWithFixedDelay( - () -> { - EntranceSchedulerContext schedulerContext = - (EntranceSchedulerContext) - entranceServer.getEntranceContext().getOrCreateScheduler().getSchedulerContext(); - - // entrance do not failover job when it is offline - if (schedulerContext.getOfflineFlag()) return; - - CommonLock commonLock = new CommonLock(); - commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); - Boolean locked = false; - try { - locked = commonLockService.lock(commonLock, 30 * 1000L); - if (!locked) return; - logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); - - // get all entrance server from eureka - ServiceInstance[] serviceInstances = - Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); - if (serviceInstances == null || serviceInstances.length <= 0) return; - - // serverInstance to map - Map serverInstanceMap = - Arrays.stream(serviceInstances) - .collect( - Collectors.toMap( - ServiceInstance::getInstance, - ServiceInstance::getRegistryTimestamp, - (k1, k2) -> k2)); - - // It is very important to avoid repeated execute job - // when failover self job, if self instance is empty, the job can be repeated execute - if (!serverInstanceMap.containsKey(Sender.getThisInstance())) { - logger.warn( - "server has just started and has not get self info, it does not failover"); - return; - } - - // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) - long expiredTimestamp = 0L; - if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { - expiredTimestamp = - System.currentTimeMillis() - - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); - } - - // get uncompleted status - List statusList = - Arrays.stream(SchedulerEventState.uncompleteStatusArray()) - .map(Object::toString) - .collect(Collectors.toList()); - - List jobRequests = - JobHistoryHelper.queryWaitForFailoverTask( - serverInstanceMap, - statusList, - expiredTimestamp, - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); - if (jobRequests.isEmpty()) return; - List ids = - jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); - logger.info("success query failover jobs , job size: {}, ids: {}", ids.size(), ids); - - // failover to local server - for (JobRequest jobRequest : jobRequests) { - entranceServer.failoverExecute(jobRequest); - } - logger.info("finished execute failover jobs, job ids: {}", ids); - - } catch (Exception e) { - logger.error("failover failed", e); - } finally { - if (locked) commonLockService.unlock(commonLock); - } - }, - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), - EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), - TimeUnit.MILLISECONDS); + future = + scheduledExecutor.scheduleWithFixedDelay( + () -> { + EntranceSchedulerContext schedulerContext = + (EntranceSchedulerContext) + entranceServer + .getEntranceContext() + .getOrCreateScheduler() + .getSchedulerContext(); + + // entrance do not failover job when it is offline + if (schedulerContext.getOfflineFlag()) return; + + CommonLock commonLock = new CommonLock(); + commonLock.setLockObject(ENTRANCE_FAILOVER_LOCK); + Boolean locked = false; + try { + locked = commonLockService.lock(commonLock, 30 * 1000L); + if (!locked) return; + logger.info("success locked {}", ENTRANCE_FAILOVER_LOCK); + + // get all entrance server from eureka + ServiceInstance[] serviceInstances = + Sender.getInstances(Sender.getThisServiceInstance().getApplicationName()); + if (serviceInstances == null || serviceInstances.length <= 0) return; + + // serverInstance to map + Map serverInstanceMap = + Arrays.stream(serviceInstances) + .collect( + Collectors.toMap( + ServiceInstance::getInstance, + ServiceInstance::getRegistryTimestamp, + (k1, k2) -> k2)); + + // It is very important to avoid repeated execute job + // when failover self job, if self instance is empty, the job can be repeated + // execute + if (!serverInstanceMap.containsKey(Sender.getThisInstance())) { + logger.warn( + "server has just started and has not get self info, it does not failover"); + return; + } + + // get failover job expired time (获取任务故障转移过期时间,配置为0表示不过期, 过期则不处理) + long expiredTimestamp = 0L; + if (EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME() > 0) { + expiredTimestamp = + System.currentTimeMillis() + - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); + } + + // get uncompleted status + List statusList = + Arrays.stream(SchedulerEventState.uncompleteStatusArray()) + .map(Object::toString) + .collect(Collectors.toList()); + + List jobRequests = + JobHistoryHelper.queryWaitForFailoverTask( + serverInstanceMap, + statusList, + expiredTimestamp, + EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); + if (jobRequests.isEmpty()) return; + List ids = + jobRequests.stream().map(JobRequest::getId).collect(Collectors.toList()); + logger.info("success query failover jobs , job size: {}, ids: {}", ids.size(), ids); + + // failover to local server + for (JobRequest jobRequest : jobRequests) { + entranceServer.failoverExecute(jobRequest); + } + logger.info("finished execute failover jobs, job ids: {}", ids); + + } catch (Exception e) { + logger.error("failover failed", e); + } finally { + if (locked) commonLockService.unlock(commonLock); + } + }, + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INIT_TIME(), + EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), + TimeUnit.MILLISECONDS); } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index a6e24388a6..060fcbdd65 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -30,10 +30,12 @@ import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer import org.apache.linkis.scheduler.queue.parallelqueue.{ParallelConsumerManager, ParallelGroup} import java.util -import java.util.concurrent.TimeUnit +import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.JavaConverters._ +import com.sun.javafx.util.Logging + class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: String) extends ParallelConsumerManager(maxParallelismUsers, schedulerName) { @@ -84,7 +86,11 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S group.setMaxAllowRunningJobs(maxAllowRunningJobs) logger .info( - s"group ${group.getGroupName} refresh maxAllowRunningJobs => ${group.getMaxRunningJobs}/$validInsCount=$maxAllowRunningJobs" + "group {} refresh maxAllowRunningJobs => {}/{}={}", + group.getGroupName, + group.getMaxRunningJobs, + validInsCount, + maxAllowRunningJobs ) case _ => } From 37567a86bbdac42952727497737fb6fc5f596843 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Mar 2023 11:41:03 +0800 Subject: [PATCH 40/72] Remove useless references --- .../entrance/scheduler/EntranceParallelConsumerManager.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index 060fcbdd65..d30f53a8f5 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -30,12 +30,10 @@ import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer import org.apache.linkis.scheduler.queue.parallelqueue.{ParallelConsumerManager, ParallelGroup} import java.util -import java.util.concurrent.{ScheduledFuture, TimeUnit} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ -import com.sun.javafx.util.Logging - class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: String) extends ParallelConsumerManager(maxParallelismUsers, schedulerName) { From 71d3e089dcd455027bc36026711e26a4abeb8f40 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Mar 2023 12:36:25 +0800 Subject: [PATCH 41/72] cast string when use logger template --- .../scheduler/EntranceParallelConsumerManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index d30f53a8f5..726d93c500 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -86,9 +86,9 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S .info( "group {} refresh maxAllowRunningJobs => {}/{}={}", group.getGroupName, - group.getMaxRunningJobs, - validInsCount, - maxAllowRunningJobs + group.getMaxRunningJobs.toString, + validInsCount.toString, + maxAllowRunningJobs.toString ) case _ => } From 8ae8a3de4b3d2741459adcd63d309ff4e54dfbc5 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Mar 2023 15:29:07 +0800 Subject: [PATCH 42/72] use logger template --- .../scheduler/EntranceParallelConsumerManager.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index 726d93c500..afaf6b16e7 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -85,10 +85,12 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S logger .info( "group {} refresh maxAllowRunningJobs => {}/{}={}", - group.getGroupName, - group.getMaxRunningJobs.toString, - validInsCount.toString, - maxAllowRunningJobs.toString + Array( + group.getGroupName, + group.getMaxRunningJobs, + validInsCount, + maxAllowRunningJobs + ) ) case _ => } From cb048534aa2a7180365b3c1f15b3cacc053c461e Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Mar 2023 16:47:07 +0800 Subject: [PATCH 43/72] use logger template --- .../entrance/scheduler/EntranceParallelConsumerManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index afaf6b16e7..5e74d48939 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -90,7 +90,7 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S group.getMaxRunningJobs, validInsCount, maxAllowRunningJobs - ) + ): _* ) case _ => } From 800074e400c834735fb04299f4422d35803cc3bf Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Mar 2023 18:01:33 +0800 Subject: [PATCH 44/72] use logger template --- .../scheduler/EntranceParallelConsumerManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index 5e74d48939..6d756ad1a8 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -87,9 +87,9 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S "group {} refresh maxAllowRunningJobs => {}/{}={}", Array( group.getGroupName, - group.getMaxRunningJobs, - validInsCount, - maxAllowRunningJobs + group.getMaxRunningJobs.toString, + validInsCount.toString, + maxAllowRunningJobs.toString ): _* ) case _ => From 2d4f7848754f5a439f1d870350863d3f19883bce Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Tue, 28 Mar 2023 15:26:07 +0800 Subject: [PATCH 45/72] Update the maximum concurrency of orchestrator from 200 to 1000 --- .../linkis/orchestrator/conf/OrchestratorConfiguration.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-orchestrator/linkis-orchestrator-core/src/main/scala/org/apache/linkis/orchestrator/conf/OrchestratorConfiguration.scala b/linkis-orchestrator/linkis-orchestrator-core/src/main/scala/org/apache/linkis/orchestrator/conf/OrchestratorConfiguration.scala index 50dbef632c..10f3a64d13 100644 --- a/linkis-orchestrator/linkis-orchestrator-core/src/main/scala/org/apache/linkis/orchestrator/conf/OrchestratorConfiguration.scala +++ b/linkis-orchestrator/linkis-orchestrator-core/src/main/scala/org/apache/linkis/orchestrator/conf/OrchestratorConfiguration.scala @@ -48,7 +48,7 @@ object OrchestratorConfiguration { CommonVars("wds.linkis.orchestrator.execution.task.max.parallelism", 5) val TASK_RUNNER_MAX_SIZE = - CommonVars("wds.linkis.orchestrator.execution.task.runner.max.size", 200) + CommonVars("wds.linkis.orchestrator.execution.task.runner.max.size", 1000) val EXEC_RUNNER_FACTORY_CLASS = CommonVars("wds.linkis.orchestrator.exec.task.runner.factory.class", "") From 8dafb2cdc421bba440fae45abad83c733dc3cf2b Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Tue, 28 Mar 2023 17:21:19 +0800 Subject: [PATCH 46/72] - moved JobInstance from linkis-common to linkis-protocol - add isInitedStr,isRunningStr method and remove uncompleteStatusArray method in SchedulerEventState --- .../linkis/protocol/engine}/JobInstance.scala | 2 +- .../scheduler/queue/SchedulerEventState.scala | 6 ++--- .../common/entity/job/JobRequest.java | 1 + .../entrance/restful/EntranceRestfulApi.java | 2 +- .../server/EntranceFailoverJobServer.java | 23 +++++++++++++------ .../linkis/entrance/EntranceServer.scala | 4 ++-- .../parser/EntranceRequestGatewayParser.scala | 2 +- 7 files changed, 25 insertions(+), 15 deletions(-) rename linkis-commons/{linkis-common/src/main/scala/org/apache/linkis/common/entity => linkis-protocol/src/main/scala/org/apache/linkis/protocol/engine}/JobInstance.scala (95%) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala b/linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/engine/JobInstance.scala similarity index 95% rename from linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala rename to linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/engine/JobInstance.scala index aa9db730ee..5e2eb10a59 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/entity/JobInstance.scala +++ b/linkis-commons/linkis-protocol/src/main/scala/org/apache/linkis/protocol/engine/JobInstance.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.common.entity +package org.apache.linkis.protocol.engine case class JobInstance( status: String, diff --git a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala index a64103628c..26087d99f0 100644 --- a/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala +++ b/linkis-commons/linkis-scheduler/src/main/scala/org/apache/linkis/scheduler/queue/SchedulerEventState.scala @@ -38,8 +38,8 @@ object SchedulerEventState extends Enumeration { SchedulerEventState.withName(jobState) ) - def uncompleteStatusArray(): Array[SchedulerEventState] = { - SchedulerEventState.values.filterNot(isCompleted).toArray - } + def isInitedByStr(jobState: String): Boolean = SchedulerEventState.withName(jobState) == Inited + + def isRunningByStr(jobState: String): Boolean = isRunning(SchedulerEventState.withName(jobState)) } diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java index 75134bd84a..46fa8a69ef 100644 --- a/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/entity/job/JobRequest.java @@ -49,6 +49,7 @@ public class JobRequest { /** result location */ private String resultLocation; + /** Task status updates is ordered, if false, not checked */ private Boolean updateOrderFlag = true; private String observeInfo; diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index 71b0df4250..6dcfcdc4b7 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -17,7 +17,6 @@ package org.apache.linkis.entrance.restful; -import org.apache.linkis.common.entity.JobInstance; import org.apache.linkis.common.log.LogUtils; import org.apache.linkis.entrance.EntranceServer; import org.apache.linkis.entrance.conf.EntranceConfiguration; @@ -29,6 +28,7 @@ import org.apache.linkis.governance.common.entity.job.JobRequest; import org.apache.linkis.manager.common.protocol.resource.ResourceWithStatus; import org.apache.linkis.protocol.constants.TaskConstant; +import org.apache.linkis.protocol.engine.JobInstance; import org.apache.linkis.protocol.engine.JobProgressInfo; import org.apache.linkis.protocol.utils.ZuulEntranceUtils; import org.apache.linkis.rpc.Sender; diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java index d162be0820..4e66da5cc3 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/EntranceFailoverJobServer.java @@ -37,12 +37,16 @@ import javax.annotation.PostConstruct; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.*; import java.util.stream.Collectors; +import scala.Enumeration; +import scala.collection.Iterator; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -135,16 +139,10 @@ public void failoverTask() { - EntranceConfiguration.ENTRANCE_FAILOVER_DATA_INTERVAL_TIME(); } - // get uncompleted status - List statusList = - Arrays.stream(SchedulerEventState.uncompleteStatusArray()) - .map(Object::toString) - .collect(Collectors.toList()); - List jobRequests = JobHistoryHelper.queryWaitForFailoverTask( serverInstanceMap, - statusList, + getUnCompleteStatus(), expiredTimestamp, EntranceConfiguration.ENTRANCE_FAILOVER_DATA_NUM_LIMIT()); if (jobRequests.isEmpty()) return; @@ -168,4 +166,15 @@ public void failoverTask() { EntranceConfiguration.ENTRANCE_FAILOVER_SCAN_INTERVAL(), TimeUnit.MILLISECONDS); } + + private List getUnCompleteStatus() { + List status = new ArrayList<>(); + Enumeration.ValueSet values = SchedulerEventState.values(); + Iterator iterator = values.iterator(); + while (iterator.hasNext()) { + Enumeration.Value next = iterator.next(); + if (!SchedulerEventState.isCompleted(next)) status.add(next.toString()); + } + return status; + } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index e9c3da2cda..45be36287b 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -319,13 +319,13 @@ abstract class EntranceServer extends Logging { killOldEC(jobRequest, logAppender); // deal Inited jobRequest, if status is Inited, need to deal by all Interceptors, such as set log_path - if (jobRequest.getStatus.equals(SchedulerEventState.Inited.toString)) { + if (SchedulerEventState.isInitedByStr(jobRequest.getStatus)) { dealInitedJobRequest(jobRequest, logAppender) } if ( EntranceConfiguration.ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED.getValue && - jobRequest.getStatus.equals(SchedulerEventState.Running.toString) + SchedulerEventState.isRunningByStr(jobRequest.getStatus) ) { // deal Running jobRequest, if enabled, status changed from Running to Cancelled dealRunningJobRequest(jobRequest, logAppender) diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala index 930bfac73a..04f206d6f6 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-gateway-server-support/src/main/scala/org/apache/linkis/gateway/ujes/parser/EntranceRequestGatewayParser.scala @@ -18,12 +18,12 @@ package org.apache.linkis.gateway.ujes.parser import org.apache.linkis.common.ServiceInstance -import org.apache.linkis.common.entity.JobInstance import org.apache.linkis.gateway.config.GatewayConfiguration import org.apache.linkis.gateway.http.GatewayContext import org.apache.linkis.gateway.parser.AbstractGatewayParser import org.apache.linkis.gateway.ujes.parser.EntranceExecutionGatewayParser._ import org.apache.linkis.jobhistory.service.JobHistoryQueryService +import org.apache.linkis.protocol.engine.JobInstance import org.apache.linkis.protocol.utils.ZuulEntranceUtils import org.apache.linkis.rpc.interceptor.ServiceInstanceUtils import org.apache.linkis.server.BDPJettyServerHelper From e624b373359f9c35bdd27ad5a77ff49962f01e24 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Tue, 28 Mar 2023 17:57:46 +0800 Subject: [PATCH 47/72] Add description --- .../org/apache/linkis/entrance/conf/EntranceConfiguration.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala index 617584f278..839b3123cc 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/conf/EntranceConfiguration.scala @@ -245,6 +245,7 @@ object EntranceConfiguration { val ENTRANCE_UPDATE_BATCH_SIZE = CommonVars("linkis.entrance.update.batch.size", 100) + // if true, the job in ConsumeQueue can be failover val ENTRANCE_SHUTDOWN_FAILOVER_CONSUME_QUEUE_ENABLED = CommonVars("linkis.entrance.shutdown.failover.consume.queue.enable", true).getValue @@ -260,6 +261,7 @@ object EntranceConfiguration { val ENTRANCE_FAILOVER_RETAIN_METRIC_YARN_RESOURCE_ENABLED = CommonVars("linkis.entrance.failover.retain.metric.yarn.resource.enable", false) + // if true, job whose status is running will be set to Cancelled val ENTRANCE_FAILOVER_RUNNING_KILL_ENABLED = CommonVars("linkis.entrance.failover.running.kill.enable", false) From 94f3ec15a1995ee00614058ef7fb867d4035c654 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Fri, 7 Apr 2023 21:49:37 +0800 Subject: [PATCH 48/72] replace constant --- .../entrance/restful/EntranceRestfulApi.java | 23 +++++++++---------- .../server/DefaultEntranceServer.java | 2 +- .../linkis/entrance/EntranceServer.scala | 18 +++++++-------- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index 3335eec90f..90a1bdbd2b 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -439,7 +439,7 @@ public Message progressWithResource(HttpServletRequest req, @PathVariable("id") message = Message.ok(); message.setMethod("/api/entrance/" + id + "/progressWithResource"); message - .data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null) + .data(TaskConstant.JOB_YARNRESOURCE, null) .data("progress", 0) .data("execID", "") .data("taskID", id) @@ -499,18 +499,17 @@ private void buildYarnResource( JobRequest jobRequest, Map metricsVo, Message message) { try { Map metrics = jobRequest.getMetrics(); - if (metrics.containsKey(TaskConstant.ENTRANCEJOB_YARNRESOURCE)) { + if (metrics.containsKey(TaskConstant.JOB_YARNRESOURCE)) { HashMap resourceMap = - (HashMap) - metrics.get(TaskConstant.ENTRANCEJOB_YARNRESOURCE); + (HashMap) metrics.get(TaskConstant.JOB_YARNRESOURCE); ArrayList resoureList = new ArrayList<>(12); if (null != resourceMap && !resourceMap.isEmpty()) { resourceMap.forEach( (applicationId, resource) -> { resoureList.add(new YarnResourceWithStatusVo(applicationId, resource)); }); - metricsVo.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, resoureList); + metricsVo.put(TaskConstant.JOB_YARNRESOURCE, resoureList); Optional cores = resourceMap.values().stream() .map(resource -> resource.queueCores()) @@ -533,17 +532,17 @@ private void buildYarnResource( } String coreRGB = RGBUtils.getRGB(corePercent); String memoryRGB = RGBUtils.getRGB(memoryPercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_PERCENT, corePercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_PERCENT, memoryPercent); - metricsVo.put(TaskConstant.ENTRANCEJOB_CORE_RGB, coreRGB); - metricsVo.put(TaskConstant.ENTRANCEJOB_MEMORY_RGB, memoryRGB); + metricsVo.put(TaskConstant.JOB_CORE_PERCENT, corePercent); + metricsVo.put(TaskConstant.JOB_MEMORY_PERCENT, memoryPercent); + metricsVo.put(TaskConstant.JOB_CORE_RGB, coreRGB); + metricsVo.put(TaskConstant.JOB_MEMORY_RGB, memoryRGB); - message.data(TaskConstant.ENTRANCEJOB_YARN_METRICS, metricsVo); + message.data(TaskConstant.JOB_YARN_METRICS, metricsVo); } else { - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); + message.data(TaskConstant.JOB_YARNRESOURCE, null); } } else { - message.data(TaskConstant.ENTRANCEJOB_YARNRESOURCE, null); + message.data(TaskConstant.JOB_YARNRESOURCE, null); } } catch (Exception e) { logger.error("build yarnResource error", e); diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index 24d077068f..66a241026c 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -20,8 +20,8 @@ import org.apache.linkis.common.ServiceInstance; import org.apache.linkis.entrance.EntranceContext; import org.apache.linkis.entrance.EntranceServer; -import org.apache.linkis.entrance.conf.EntranceConfiguration$; import org.apache.linkis.entrance.conf.EntranceConfiguration; +import org.apache.linkis.entrance.conf.EntranceConfiguration$; import org.apache.linkis.entrance.constant.ServiceNameConsts; import org.apache.linkis.entrance.execute.EntranceJob; import org.apache.linkis.entrance.job.EntranceExecutionJob; diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 45be36287b..c44eb07922 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -353,7 +353,7 @@ abstract class EntranceServer extends Logging { if ( jobRequest.getMetrics == null - || !jobRequest.getMetrics.containsKey(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + || !jobRequest.getMetrics.containsKey(TaskConstant.JOB_ENGINECONN_MAP) ) { val msg = s"job ${jobRequest.getId} not have EC info, ignore it" logger.info(msg) @@ -362,7 +362,7 @@ abstract class EntranceServer extends Logging { } val engineMap = jobRequest.getMetrics - .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + .get(TaskConstant.JOB_ENGINECONN_MAP) .asInstanceOf[util.Map[String, Object]] val engineInstance = @@ -628,26 +628,26 @@ abstract class EntranceServer extends Logging { if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_METRIC_ENGINE_CONN_ENABLED.getValue) { if ( jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( - TaskConstant.ENTRANCEJOB_ENGINECONN_MAP + TaskConstant.JOB_ENGINECONN_MAP ) ) { val oldEngineconnMap = jobRequest.getMetrics - .get(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP) + .get(TaskConstant.JOB_ENGINECONN_MAP) .asInstanceOf[util.Map[String, Object]] - metricMap.put(TaskConstant.ENTRANCEJOB_ENGINECONN_MAP, oldEngineconnMap) + metricMap.put(TaskConstant.JOB_ENGINECONN_MAP, oldEngineconnMap) } } if (EntranceConfiguration.ENTRANCE_FAILOVER_RETAIN_METRIC_YARN_RESOURCE_ENABLED.getValue) { if ( jobRequest.getMetrics != null && jobRequest.getMetrics.containsKey( - TaskConstant.ENTRANCEJOB_YARNRESOURCE + TaskConstant.JOB_YARNRESOURCE ) ) { val oldResourceMap = jobRequest.getMetrics - .get(TaskConstant.ENTRANCEJOB_YARNRESOURCE) + .get(TaskConstant.JOB_YARNRESOURCE) .asInstanceOf[util.Map[String, Object]] - metricMap.put(TaskConstant.ENTRANCEJOB_YARNRESOURCE, oldResourceMap) + metricMap.put(TaskConstant.JOB_YARNRESOURCE, oldResourceMap) } } @@ -659,7 +659,7 @@ abstract class EntranceServer extends Logging { jobRequest.setErrorCode(0) jobRequest.setErrorDesc("") jobRequest.setMetrics(metricMap) - jobRequest.getMetrics.put(TaskConstant.ENTRANCEJOB_SUBMIT_TIME, initDate) + jobRequest.getMetrics.put(TaskConstant.JOB_SUBMIT_TIME, initDate) jobRequest.setUpdateOrderFlag(false) logAppender.append( From acc91db7d6dac9132faf0287d43761807df31e3c Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Mon, 10 Apr 2023 09:56:11 +0800 Subject: [PATCH 49/72] replace Option.apply to null --- .../entrance/restful/EntranceRestfulApi.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java index 90a1bdbd2b..873f5fd8a4 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceRestfulApi.java @@ -263,7 +263,7 @@ public Message status( } } - Option job = Option.apply(null); + Option job = null; try { job = entranceServer.getJob(realId); } catch (Exception e) { @@ -281,7 +281,7 @@ public Message status( message.data("status", status).data("execID", execID); return message; } - if (job.isDefined()) { + if (job != null && job.isDefined()) { if (job.get() instanceof EntranceJob) { ((EntranceJob) job.get()).updateNewestAccessByClientTimestamp(); } @@ -638,7 +638,7 @@ public Message log(HttpServletRequest req, @PathVariable("id") String id) { } } - Option job = Option.apply(null); + Option job = null; try { job = entranceServer.getJob(realId); } catch (final Throwable t) { @@ -648,7 +648,7 @@ public Message log(HttpServletRequest req, @PathVariable("id") String id) { message.setMethod("/api/entrance/" + id + "/log"); return message; } - if (job.isDefined()) { + if (job != null && job.isDefined()) { logger.debug("begin to get log for {}(开始获取 {} 的日志)", job.get().getId(), job.get().getId()); LogReader logReader = entranceServer.getEntranceContext().getOrCreateLogManager().getLogReader(realId); @@ -741,7 +741,7 @@ public Message killJobs( String id = idNode.get(i).asText(); Long taskID = taskIDNode.get(i).asLong(); String realId = ZuulEntranceUtils.parseExecID(id)[3]; - Option job = Option.apply(null); + Option job = null; try { job = entranceServer.getJob(realId); } catch (Exception e) { @@ -755,7 +755,7 @@ public Message killJobs( continue; } Message message = null; - if (job.isEmpty()) { + if (job == null || job.isEmpty()) { logger.warn("can not find a job in entranceServer, will force to kill it"); waitToForceKill.add(taskID); message = Message.ok("Forced Kill task (强制杀死任务)"); @@ -877,7 +877,7 @@ public Message kill( } } - Option job = Option.apply(null); + Option job = null; try { job = entranceServer.getJob(realId); } catch (Exception e) { @@ -894,7 +894,7 @@ public Message kill( return message; } - if (job.isEmpty()) { + if (job == null || job.isEmpty()) { logger.warn("can not find a job in entranceServer, will force to kill it"); // 如果在内存中找不到该任务,那么该任务可能已经完成了,或者就是重启导致的 JobHistoryHelper.forceKill(taskID); From caeeab9fc13489adad51a06cf86bd0733a761e05 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Mon, 14 Aug 2023 11:23:30 +0800 Subject: [PATCH 50/72] sql optimize and bug fix --- .../linkis/entrance/EntranceServer.scala | 3 ++ .../EntranceParallelConsumerManager.scala | 38 ++++++++++--------- .../jobhistory/dao/JobHistoryMapper.java | 6 +-- .../mapper/common/JobHistoryMapper.xml | 4 +- 4 files changed, 28 insertions(+), 23 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index c44eb07922..fc73887534 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -531,6 +531,9 @@ abstract class EntranceServer extends Logging { .createPersistenceEngine() .updateIfNeeded(jobRequest) + // reset `UpdateOrderFlag` + jobRequest.setUpdateOrderFlag(true) + logger.info(s"job ${jobRequest.getId} update JobRequest success") val job = getEntranceContext.getOrCreateEntranceParser().parseToJob(jobRequest) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index 6d756ad1a8..0f86c2a335 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -46,26 +46,28 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S Utils.defaultScheduler.scheduleAtFixedRate( new Runnable { override def run(): Unit = { - logger.info("start refresh consumer group maxAllowRunningJobs") - // get all entrance server from eureka - val serviceInstances = - Sender.getInstances(Sender.getThisServiceInstance.getApplicationName) - if (null == serviceInstances || serviceInstances.isEmpty) return + Utils.tryAndError { + logger.info("start refresh consumer group maxAllowRunningJobs") + // get all entrance server from eureka + val serviceInstances = + Sender.getInstances(Sender.getThisServiceInstance.getApplicationName) + if (null == serviceInstances || serviceInstances.isEmpty) return - // get all offline label server - val routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory - .createLabel[RouteLabel](LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE) - val labels = new util.ArrayList[Label[_]] - labels.add(routeLabel) - val labelInstances = InstanceLabelClient.getInstance.getInstanceFromLabel(labels) + // get all offline label server + val routeLabel = LabelBuilderFactoryContext.getLabelBuilderFactory + .createLabel[RouteLabel](LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE) + val labels = new util.ArrayList[Label[_]] + labels.add(routeLabel) + val labelInstances = InstanceLabelClient.getInstance.getInstanceFromLabel(labels) - // get active entrance server - val allInstances = new util.ArrayList[ServiceInstance]() - allInstances.addAll(serviceInstances.toList.asJava) - allInstances.removeAll(labelInstances) - // refresh all group maxAllowRunningJobs - refreshAllGroupMaxAllowRunningJobs(allInstances.size()) - logger.info("Finished to refresh consumer group maxAllowRunningJobs") + // get active entrance server + val allInstances = new util.ArrayList[ServiceInstance]() + allInstances.addAll(serviceInstances.toList.asJava) + allInstances.removeAll(labelInstances) + // refresh all group maxAllowRunningJobs + refreshAllGroupMaxAllowRunningJobs(allInstances.size()) + logger.info("Finished to refresh consumer group maxAllowRunningJobs") + } } }, EntranceConfiguration.ENTRANCE_GROUP_SCAN_INIT_TIME.getValue, diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java b/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java index 64e76de0f0..806d8ec70c 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/java/org/apache/linkis/jobhistory/dao/JobHistoryMapper.java @@ -117,9 +117,9 @@ void updateJobHistoryCancelById( * or a.instances is null or a.instances not in ('192.168.1.123:9104','192.168.1.124:9104') or * EXISTS ( select 1 from ( select '192.168.1.123:9104' as instances, 1697775054098 as * registryTime union all select '192.168.1.124:9104' as instances, 1666239054098 as registryTime - * ) b where a.instances = b.instances and UNIX_TIMESTAMP(a.created_time) * 1000 < b.registryTime - * ) ) and status in ('Inited','Running','Scheduled','WaitForRetry') and - * UNIX_TIMESTAMP(a.created_time) * 1000 >= 1666239054098 limit 10 + * ) b where a.instances = b.instances and a.created_time < FROM_UNIXTIME(b.registryTime/1000) ) ) + * and status in ('Inited','Running','Scheduled','WaitForRetry') and a.created_time >= + * FROM_UNIXTIME(1666239054098/1000) limit 10 * * @param instancesMap * @param statusList diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml index a99dbf3c87..9d76d27ddf 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/common/JobHistoryMapper.xml @@ -244,12 +244,12 @@ select #{key} as instances, #{val} as registryTime ) b - where a.instances = b.instances and UNIX_TIMESTAMP(a.created_time) * 1000 b.registryTime + where a.instances = b.instances and a.created_time FROM_UNIXTIME(b.registryTime/1000) ) ) and status in #{status} - and UNIX_TIMESTAMP(a.created_time) * 1000 >= #{startTimestamp} + and a.created_time >= FROM_UNIXTIME(#{startTimestamp}/1000) limit #{limit} From 3d681fb8a74142417c510687527438deb2e31577 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Mon, 14 Aug 2023 15:05:45 +0800 Subject: [PATCH 51/72] merge master --- .../server/DefaultEntranceServer.java | 3 ++- .../linkis/entrance/EntranceServer.scala | 2 +- .../conf/linkis-cg-entrance.properties | 1 - .../package/conf/linkis-mg-gateway.properties | 1 - .../mapper/postgresql/JobHistoryMapper.xml | 22 +++++++++++++++++++ 5 files changed, 25 insertions(+), 4 deletions(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java index af2bbaf19c..94531cd5fe 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/server/DefaultEntranceServer.java @@ -108,7 +108,8 @@ private void shutdownEntrance(ContextClosedEvent event) { if (null != allUndoneTask) { for (EntranceJob job : allUndoneTask) { job.onFailure( - "Your job will be marked as canceled because the Entrance service restarted(因为Entrance服务重启,您的任务将被标记为取消)", null); + "Your job will be marked as canceled because the Entrance service restarted(因为Entrance服务重启,您的任务将被标记为取消)", + null); IOUtils.closeQuietly(((EntranceExecutionJob) job).getLogWriter().get()); } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 53cf0256e6..2fa5ff23c4 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -36,9 +36,9 @@ import org.apache.linkis.entrance.utils.JobHistoryHelper import org.apache.linkis.governance.common.conf.GovernanceCommonConf import org.apache.linkis.governance.common.entity.job.JobRequest import org.apache.linkis.governance.common.protocol.task.RequestTaskKill +import org.apache.linkis.governance.common.utils.LoggerUtils import org.apache.linkis.manager.common.protocol.engine.EngineStopRequest import org.apache.linkis.manager.label.entity.entrance.ExecuteOnceLabel -import org.apache.linkis.governance.common.utils.LoggerUtils import org.apache.linkis.protocol.constants.TaskConstant import org.apache.linkis.rpc.Sender import org.apache.linkis.rpc.conf.RPCConfiguration diff --git a/linkis-dist/package/conf/linkis-cg-entrance.properties b/linkis-dist/package/conf/linkis-cg-entrance.properties index 579ac25a18..c0568288a5 100644 --- a/linkis-dist/package/conf/linkis-cg-entrance.properties +++ b/linkis-dist/package/conf/linkis-cg-entrance.properties @@ -39,6 +39,5 @@ spring.eureka.instance.metadata-map.linkis.conf.version=v1 linkis.entrance.auto.clean.dirty.data.enable=true -wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml wds.linkis.server.mybatis.BasePackage=org.apache.linkis.publicservice.common.lock.dao wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.publicservice.common.lock.entity \ No newline at end of file diff --git a/linkis-dist/package/conf/linkis-mg-gateway.properties b/linkis-dist/package/conf/linkis-mg-gateway.properties index 27656f7f31..1f1d2416b4 100644 --- a/linkis-dist/package/conf/linkis-mg-gateway.properties +++ b/linkis-dist/package/conf/linkis-mg-gateway.properties @@ -20,7 +20,6 @@ wds.linkis.gateway.conf.enable.proxy.user=false wds.linkis.gateway.conf.url.pass.auth=/dss/ wds.linkis.gateway.conf.enable.token.auth=true wds.linkis.is.gateway=true -wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity,org.apache.linkis.jobhistory.entity wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao,org.apache.linkis.jobhistory.dao wds.linkis.label.entity.packages=org.apache.linkis.gateway.ujes.route.label diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml index 30e4e85b34..e194a2e4cd 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml @@ -229,4 +229,26 @@ #{id} + From bdc3e0eb72ce543db841bb937bb5d96fcbd7a005 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Tue, 22 Aug 2023 16:08:52 +0800 Subject: [PATCH 52/72] add comment --- .../main/scala/org/apache/linkis/entrance/EntranceServer.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala index 2fa5ff23c4..4931659742 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceServer.scala @@ -669,6 +669,7 @@ abstract class EntranceServer extends Logging { jobRequest.setErrorDesc("") jobRequest.setMetrics(metricMap) jobRequest.getMetrics.put(TaskConstant.JOB_SUBMIT_TIME, initDate) + // Allow task status updates to be unordered jobRequest.setUpdateOrderFlag(false) logAppender.append( From 5fa73f2576cc159dfd394c7b6bdc3ad57f777380 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Tue, 22 Aug 2023 21:57:10 +0800 Subject: [PATCH 53/72] add mybatis config --- linkis-dist/package/conf/linkis-cg-entrance.properties | 2 +- linkis-dist/package/conf/linkis-mg-gateway.properties | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/linkis-dist/package/conf/linkis-cg-entrance.properties b/linkis-dist/package/conf/linkis-cg-entrance.properties index c0568288a5..62b1de5d5e 100644 --- a/linkis-dist/package/conf/linkis-cg-entrance.properties +++ b/linkis-dist/package/conf/linkis-cg-entrance.properties @@ -38,6 +38,6 @@ spring.eureka.instance.metadata-map.linkis.conf.version=v1 ## clean dirty data when the entrance start linkis.entrance.auto.clean.dirty.data.enable=true - +wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml wds.linkis.server.mybatis.BasePackage=org.apache.linkis.publicservice.common.lock.dao wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.publicservice.common.lock.entity \ No newline at end of file diff --git a/linkis-dist/package/conf/linkis-mg-gateway.properties b/linkis-dist/package/conf/linkis-mg-gateway.properties index 1f1d2416b4..27656f7f31 100644 --- a/linkis-dist/package/conf/linkis-mg-gateway.properties +++ b/linkis-dist/package/conf/linkis-mg-gateway.properties @@ -20,6 +20,7 @@ wds.linkis.gateway.conf.enable.proxy.user=false wds.linkis.gateway.conf.url.pass.auth=/dss/ wds.linkis.gateway.conf.enable.token.auth=true wds.linkis.is.gateway=true +wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity,org.apache.linkis.jobhistory.entity wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao,org.apache.linkis.jobhistory.dao wds.linkis.label.entity.packages=org.apache.linkis.gateway.ujes.route.label From 42435c50c9282a1b5e47ecd934f0abadc08fc734 Mon Sep 17 00:00:00 2001 From: peacewong Date: Mon, 4 Sep 2023 19:19:07 +0800 Subject: [PATCH 54/72] bak --- linkis-extensions/linkis-et-monitor/pom.xml | 128 ++++++++ .../src/main/assembly/distribution.xml | 297 ++++++++++++++++++ .../app/LinkisDirtyDataCleanApplication.java | 111 +++++++ .../app/LinkisJobHistoryScanApplication.java | 172 ++++++++++ .../app/LinksMonitorResourceApplication.java | 32 ++ .../app/instance/dao/InsLabelRelationDao.java | 35 +++ .../app/instance/dao/InstanceInfoDao.java | 36 +++ .../app/instance/dao/InstanceLabelDao.java | 41 +++ .../instance/entity/InsPersistenceLabel.java | 82 +++++ .../entity/InsPersistenceLabelValue.java | 59 ++++ .../app/instance/entity/InstanceInfo.java | 92 ++++++ .../scan/app/jobhistory/QueryUtils.java | 31 ++ .../app/jobhistory/dao/JobHistoryMapper.java | 68 ++++ .../app/jobhistory/entity/JobHistory.java | 221 +++++++++++++ .../exception/AnomalyScannerException.java | 37 +++ .../exception/DirtyDataCleanException.java | 37 +++ .../app/monitor/config/ListenerConfig.java | 46 +++ .../app/monitor/config/MonitorConfig.java | 71 +++++ .../app/monitor/config/ScheduledConfig.java | 39 +++ .../app/monitor/entity/ChatbotEntity.java | 87 +++++ .../scan/app/monitor/entity/EngineEntity.java | 53 ++++ .../app/monitor/entity/EntranceEntity.java | 63 ++++ .../scan/app/monitor/entity/IndexEntity.java | 85 +++++ .../scan/app/monitor/scheduled/BmlClear.java | 47 +++ .../app/monitor/scheduled/EcRecordClear.java | 53 ++++ .../monitor/scheduled/JobHistoryClear.java | 53 ++++ .../monitor/scheduled/JobHistoryMonitor.java | 264 ++++++++++++++++ .../monitor/scheduled/ResourceMonitor.java | 151 +++++++++ .../app/monitor/scheduled/TaskLogClear.java | 53 ++++ .../app/monitor/scheduled/TaskMonitor.java | 183 +++++++++++ .../monitor/scheduled/UserModeMonitor.java | 156 +++++++++ .../app/monitor/scheduled/ValidatorClear.java | 51 +++ .../scan/app/monitor/until/CacheUtils.java | 35 +++ .../scan/app/monitor/until/HttpsUntils.java | 183 +++++++++++ .../scan/app/monitor/until/ThreadUtils.java | 68 ++++ .../mapper/common/InsLabelRelationMapper.xml | 67 ++++ .../mapper/common/InstanceInfoMapper.xml | 53 ++++ .../mapper/common/InstanceLabelMapper.xml | 57 ++++ .../mapper/common/JobHistoryMapper.xml | 172 ++++++++++ ...kisJobHistoryScanSpringConfiguration.scala | 49 +++ .../scan/app/factory/MapperFactory.scala | 64 ++++ .../jobhistory/JobHistoryDataFetcher.scala | 111 +++++++ .../errorcode/JobHistoryErrCodeHitEvent.scala | 23 ++ .../errorcode/JobHistoryErrCodeRule.scala | 80 +++++ .../JobHistoryErrorCodeAlertSender.scala | 97 ++++++ .../jobtime/JobTimeExceedAlertSender.scala | 112 +++++++ .../jobtime/JobTimeExceedHitEvent.scala | 23 ++ .../jobtime/JobTimeExceedRule.scala | 103 ++++++ .../labels/JobHistoryLabelsAlertSender.scala | 69 ++++ .../labels/JobHistoryLabelsHitEvent.scala | 22 ++ .../labels/JobHistoryLabelsRule.scala | 112 +++++++ .../runtime/CommonJobRunTimeRule.scala | 79 +++++ .../runtime/CommonRunTimeAlertSender.scala | 88 ++++++ .../runtime/CommonRunTimeHitEvent.scala | 22 ++ .../JobHistoryRunTimeAlertSender.scala | 73 +++++ .../runtime/JobHistoryRunTimeHitEvent.scala | 22 ++ .../runtime/JobHistoryRunTimeRule.scala | 82 +++++ .../scan/client/MonitorHTTPClient.scala | 117 +++++++ .../client/MonitorHTTPClientClientImpl.scala | 39 +++ .../scan/client/MonitorResourceClient.scala | 112 +++++++ .../client/MonitorResourceClientImpl.scala | 39 +++ .../monitor/scan/constants/Constants.scala | 113 +++++++ .../scan/constants/ScanOperatorEnum.scala | 24 ++ .../linkis/monitor/scan/core/ob/Event.scala | 29 ++ .../monitor/scan/core/ob/Observer.scala | 27 ++ .../scan/core/ob/SingleObserverEvent.java | 43 +++ .../scan/core/pac/AbstractDataFetcher.scala | 30 ++ .../scan/core/pac/AbstractScanRule.scala | 52 +++ .../scan/core/pac/BaseScannedData.scala | 27 ++ .../monitor/scan/core/pac/DataFetcher.scala | 40 +++ .../monitor/scan/core/pac/ScanBuffer.scala | 43 +++ .../monitor/scan/core/pac/ScanRule.scala | 50 +++ .../monitor/scan/core/pac/ScannedData.scala | 27 ++ .../scan/core/scanner/AbstractScanner.scala | 168 ++++++++++ .../scan/core/scanner/AnomalyScanner.scala | 95 ++++++ .../scan/core/scanner/DefaultScanner.scala | 29 ++ .../monitor/scan/request/EmsListAction.scala | 74 +++++ .../scan/request/EntranceTaskAction.scala | 76 +++++ .../scan/request/MonitorResourceAction.scala | 22 ++ .../monitor/scan/request/UserAction.scala | 26 ++ .../scan/response/EntranceTaskResult.scala | 36 +++ .../scan/response/MonitorResourceResult.scala | 33 ++ .../linkis/monitor/scan/utils/ScanUtils.java | 71 +++++ .../monitor/scan/utils/alert/AlertDesc.scala | 26 ++ .../scan/utils/alert/AlertSender.scala | 30 ++ .../scan/utils/alert/PooledAlertSender.scala | 112 +++++++ .../scan/utils/alert/ims/ImsAlertDesc.scala | 155 +++++++++ .../scan/utils/alert/ims/ImsAlertLevel.scala | 29 ++ .../scan/utils/alert/ims/ImsAlertWay.scala | 27 ++ .../scan/utils/alert/ims/ImsRequest.scala | 36 +++ ...storyScanImsAlertPropFileParserUtils.scala | 132 ++++++++ .../utils/alert/ims/MonitorAlertUtils.scala | 229 ++++++++++++++ .../alert/ims/PooledImsAlertSender.scala | 110 +++++++ .../utils/alert/ims/PooledImsAlertUtils.scala | 115 +++++++ .../utils/alert/ims/UserLabelAlertUtils.scala | 126 ++++++++ .../monitor/scan/utils/log/LogUtils.scala | 25 ++ ...ryScanImsAlertPropFileParserUtilsTest.java | 48 +++ .../LinkisJobHistoryScanApplicationTest.java | 38 +++ .../utils/alert/PooledImsAlertSenderTest.java | 67 ++++ .../utils/alert/PooledImsAlertUtilsTest.java | 49 +++ .../linkis-et-jobhistory-scan-ims.properties | 6 + .../linkis-et-jobhistory-scan.properties | 14 + .../src/test/resources/log4j2-console.xml | 46 +++ linkis-extensions/pom.xml | 1 + 104 files changed, 7663 insertions(+) create mode 100644 linkis-extensions/linkis-et-monitor/pom.xml create mode 100644 linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml create mode 100644 linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml create mode 100644 linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml create mode 100644 linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsHitEvent.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeHitEvent.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeHitEvent.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClientClientImpl.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClient.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClientImpl.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/SingleObserverEvent.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/DataFetcher.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanRule.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EmsListAction.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/MonitorResourceAction.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/UserAction.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/EntranceTaskResult.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/MonitorResourceResult.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/ScanUtils.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala create mode 100644 linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java create mode 100644 linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java create mode 100644 linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java create mode 100644 linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java create mode 100644 linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties create mode 100644 linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties create mode 100644 linkis-extensions/linkis-et-monitor/src/test/resources/log4j2-console.xml diff --git a/linkis-extensions/linkis-et-monitor/pom.xml b/linkis-extensions/linkis-et-monitor/pom.xml new file mode 100644 index 0000000000..5e62fa60b5 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/pom.xml @@ -0,0 +1,128 @@ + + + + 4.0.0 + + org.apache.linkis + linkis + ${revision} + ../../pom.xml + + + linkis-et-monitor + + + + org.apache.linkis + linkis-httpclient + ${project.version} + + + org.apache.linkis + linkis-mybatis + ${project.version} + + + + org.apache.linkis + linkis-wedatasphere-common + ${project.version} + + + + org.apache.linkis + linkis-module + ${project.version} + provided + + + + org.apache.linkis + linkis-gateway-httpclient-support + ${project.version} + + + + org.apache.linkis + linkis-common + ${project.version} + + + + org.springframework + spring-test + test + + + + junit + junit + 4.12 + test + + + org.apache.linkis + linkis-computation-client + ${project.version} + + + org.apache.linkis + linkis-bml-cleaner + ${project.version} + + + org.apache.linkis + linkis-storage + ${project.version} + provided + + + org.apache.linkis + linkis-rpc + ${project.version} + + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-assembly-plugin + false + + false + out + false + false + + src/main/assembly/distribution.xml + + + + + make-assembly + + single + + package + + + + + + + diff --git a/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml b/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml new file mode 100644 index 0000000000..8cc9e7ea7b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml @@ -0,0 +1,297 @@ + + + + + linkis-et-monitor + + dir + + false + + + + + + lib + true + true + false + false + true + + + antlr:antlr:jar + aopalliance:aopalliance:jar + asm:asm:jar + cglib:cglib:jar + com.amazonaws:aws-java-sdk-autoscaling:jar + com.amazonaws:aws-java-sdk-core:jar + com.amazonaws:aws-java-sdk-ec2:jar + com.amazonaws:aws-java-sdk-route53:jar + com.amazonaws:aws-java-sdk-sts:jar + com.amazonaws:jmespath-java:jar + com.fasterxml.jackson.core:jackson-annotations:jar + com.fasterxml.jackson.core:jackson-core:jar + com.fasterxml.jackson.core:jackson-databind:jar + com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar + com.fasterxml.jackson.datatype:jackson-datatype-jdk8:jar + com.fasterxml.jackson.datatype:jackson-datatype-jsr310:jar + com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:jar + com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:jar + com.fasterxml.jackson.module:jackson-module-jaxb-annotations:jar + com.fasterxml.jackson.module:jackson-module-parameter-names:jar + com.fasterxml.jackson.module:jackson-module-paranamer:jar + com.fasterxml.jackson.module:jackson-module-scala_2.11:jar + com.github.andrewoma.dexx:dexx-collections:jar + com.github.vlsi.compactmap:compactmap:jar + com.google.code.findbugs:annotations:jar + com.google.code.findbugs:jsr305:jar + com.google.code.gson:gson:jar + com.google.guava:guava:jar + com.google.inject:guice:jar + com.google.protobuf:protobuf-java:jar + com.netflix.archaius:archaius-core:jar + com.netflix.eureka:eureka-client:jar + com.netflix.eureka:eureka-core:jar + com.netflix.hystrix:hystrix-core:jar + com.netflix.netflix-commons:netflix-commons-util:jar + com.netflix.netflix-commons:netflix-eventbus:jar + com.netflix.netflix-commons:netflix-infix:jar + com.netflix.netflix-commons:netflix-statistics:jar + com.netflix.ribbon:ribbon:jar + com.netflix.ribbon:ribbon-core:jar + com.netflix.ribbon:ribbon-eureka:jar + com.netflix.ribbon:ribbon-httpclient:jar + com.netflix.ribbon:ribbon-loadbalancer:jar + com.netflix.ribbon:ribbon-transport:jar + com.netflix.servo:servo-core:jar + com.ning:async-http-client:jar + com.sun.jersey.contribs:jersey-apache-client4:jar + com.sun.jersey:jersey-client:jar + com.sun.jersey:jersey-core:jar + com.sun.jersey:jersey-json:jar + com.sun.jersey:jersey-server:jar + com.sun.jersey:jersey-servlet:jar + com.sun.xml.bind:jaxb-impl:jar + com.thoughtworks.paranamer:paranamer:jar + com.thoughtworks.xstream:xstream:jar + org.apache.linkis:linkis-common:jar + org.apache.linkis:linkis-module:jar + commons-beanutils:commons-beanutils:jar + commons-beanutils:commons-beanutils-core:jar + commons-cli:commons-cli:jar + commons-codec:commons-codec:jar + commons-collections:commons-collections:jar + commons-configuration:commons-configuration:jar + commons-daemon:commons-daemon:jar + commons-dbcp:commons-dbcp:jar + commons-digester:commons-digester:jar + commons-httpclient:commons-httpclient:jar + commons-io:commons-io:jar + commons-jxpath:commons-jxpath:jar + commons-lang:commons-lang:jar + commons-logging:commons-logging:jar + commons-net:commons-net:jar + commons-pool:commons-pool:jar + io.micrometer:micrometer-core:jar + io.netty:netty:jar + io.netty:netty-all:jar + io.netty:netty-buffer:jar + io.netty:netty-codec:jar + io.netty:netty-codec-http:jar + io.netty:netty-common:jar + io.netty:netty-handler:jar + io.netty:netty-transport:jar + io.netty:netty-transport-native-epoll:jar + io.reactivex:rxjava:jar + io.reactivex:rxnetty:jar + io.reactivex:rxnetty-contexts:jar + io.reactivex:rxnetty-servo:jar + javax.activation:activation:jar + javax.annotation:javax.annotation-api:jar + javax.inject:javax.inject:jar + javax.servlet:javax.servlet-api:jar + javax.servlet.jsp:jsp-api:jar + javax.validation:validation-api:jar + javax.websocket:javax.websocket-api:jar + javax.ws.rs:javax.ws.rs-api:jar + javax.xml.bind:jaxb-api:jar + javax.xml.stream:stax-api:jar + joda-time:joda-time:jar + log4j:log4j:jar + mysql:mysql-connector-java:jar + net.databinder.dispatch:dispatch-core_2.11:jar + net.databinder.dispatch:dispatch-json4s-jackson_2.11:jar + org.antlr:antlr-runtime:jar + org.antlr:stringtemplate:jar + org.apache.commons:commons-compress:jar + org.apache.commons:commons-math:jar + org.apache.commons:commons-math3:jar + org.apache.curator:curator-client:jar + org.apache.curator:curator-framework:jar + org.apache.curator:curator-recipes:jar + org.apache.directory.api:api-asn1-api:jar + org.apache.directory.api:api-util:jar + org.apache.directory.server:apacheds-i18n:jar + org.apache.directory.server:apacheds-kerberos-codec:jar + org.apache.hadoop:hadoop-annotations:jar + org.apache.hadoop:hadoop-auth:jar + org.apache.hadoop:hadoop-common:jar + org.apache.hadoop:hadoop-hdfs:jar + org.apache.htrace:htrace-core:jar + org.apache.httpcomponents:httpclient:jar + org.apache.httpcomponents:httpcore:jar + org.apache.logging.log4j:log4j-api:jar + org.apache.logging.log4j:log4j-core:jar + org.apache.logging.log4j:log4j-jul:jar + org.apache.logging.log4j:log4j-slf4j-impl:jar + org.apache.zookeeper:zookeeper:jar + org.aspectj:aspectjweaver:jar + org.bouncycastle:bcpkix-jdk15on:jar + org.bouncycastle:bcprov-jdk15on:jar + org.codehaus.jackson:jackson-jaxrs:jar + org.codehaus.jackson:jackson-xc:jar + org.codehaus.jettison:jettison:jar + org.codehaus.woodstox:stax2-api:jar + org.codehaus.woodstox:woodstox-core-asl:jar + org.eclipse.jetty:jetty-annotations:jar + org.eclipse.jetty:jetty-client:jar + org.eclipse.jetty:jetty-continuation:jar + org.eclipse.jetty:jetty-http:jar + org.eclipse.jetty:jetty-io:jar + org.eclipse.jetty:jetty-jndi:jar + org.eclipse.jetty:jetty-plus:jar + org.eclipse.jetty:jetty-security:jar + org.eclipse.jetty:jetty-server:jar + org.eclipse.jetty:jetty-servlet:jar + org.eclipse.jetty:jetty-servlets:jar + org.eclipse.jetty:jetty-util:jar + org.eclipse.jetty:jetty-webapp:jar + org.eclipse.jetty:jetty-xml:jar + org.eclipse.jetty.websocket:javax-websocket-client-impl:jar + org.eclipse.jetty.websocket:javax-websocket-server-impl:jar + org.eclipse.jetty.websocket:websocket-api:jar + org.eclipse.jetty.websocket:websocket-client:jar + org.eclipse.jetty.websocket:websocket-common:jar + org.eclipse.jetty.websocket:websocket-server:jar + org.eclipse.jetty.websocket:websocket-servlet:jar + org.fusesource.leveldbjni:leveldbjni-all:jar + org.glassfish.hk2:class-model:jar + org.glassfish.hk2:config-types:jar + org.glassfish.hk2.external:aopalliance-repackaged:jar + org.glassfish.hk2.external:asm-all-repackaged:jar + org.glassfish.hk2.external:bean-validator:jar + org.glassfish.hk2.external:javax.inject:jar + org.glassfish.hk2:hk2:jar + org.glassfish.hk2:hk2-api:jar + org.glassfish.hk2:hk2-config:jar + org.glassfish.hk2:hk2-core:jar + org.glassfish.hk2:hk2-locator:jar + org.glassfish.hk2:hk2-runlevel:jar + org.glassfish.hk2:hk2-utils:jar + org.glassfish.hk2:osgi-resource-locator:jar + org.glassfish.hk2:spring-bridge:jar + org.glassfish.jersey.bundles:jaxrs-ri:jar + org.glassfish.jersey.bundles.repackaged:jersey-guava:jar + org.glassfish.jersey.containers:jersey-container-servlet:jar + org.glassfish.jersey.containers:jersey-container-servlet-core:jar + org.glassfish.jersey.core:jersey-client:jar + org.glassfish.jersey.core:jersey-common:jar + org.glassfish.jersey.core:jersey-server:jar + org.glassfish.jersey.ext:jersey-entity-filtering:jar + org.glassfish.jersey.ext:jersey-spring3:jar + org.glassfish.jersey.media:jersey-media-jaxb:jar + org.glassfish.jersey.media:jersey-media-json-jackson:jar + org.glassfish.jersey.media:jersey-media-multipart:jar + org.hdrhistogram:HdrHistogram:jar + org.javassist:javassist:jar + org.json4s:json4s-ast_2.11:jar + org.json4s:json4s-core_2.11:jar + org.json4s:json4s-jackson_2.11:jar + org.jsoup:jsoup:jar + org.jvnet.mimepull:mimepull:jar + org.jvnet:tiger-types:jar + org.latencyutils:LatencyUtils:jar + org.mortbay.jasper:apache-el:jar + org.mortbay.jetty:jetty:jar + org.mortbay.jetty:jetty-util:jar + org.ow2.asm:asm-analysis:jar + org.ow2.asm:asm-commons:jar + org.ow2.asm:asm-tree:jar + org.reflections:reflections:jar + org.scala-lang.modules:scala-parser-combinators_2.11:jar + org.scala-lang.modules:scala-xml_2.11:jar + org.scala-lang:scala-compiler:jar + org.scala-lang:scala-library:jar + org.scala-lang:scala-reflect:jar + org.scala-lang:scalap:jar + org.slf4j:jul-to-slf4j:jar + org.slf4j:slf4j-api:jar + org.springframework.boot:spring-boot:jar + org.springframework.boot:spring-boot-actuator:jar + org.springframework.boot:spring-boot-actuator-autoconfigure:jar + org.springframework.boot:spring-boot-autoconfigure:jar + org.springframework.boot:spring-boot-starter:jar + org.springframework.boot:spring-boot-starter-actuator:jar + org.springframework.boot:spring-boot-starter-aop:jar + org.springframework.boot:spring-boot-starter-jetty:jar + org.springframework.boot:spring-boot-starter-json:jar + org.springframework.boot:spring-boot-starter-log4j2:jar + org.springframework.boot:spring-boot-starter-web:jar + org.springframework.cloud:spring-cloud-commons:jar + org.springframework.cloud:spring-cloud-config-client:jar + org.springframework.cloud:spring-cloud-context:jar + org.springframework.cloud:spring-cloud-netflix-archaius:jar + org.springframework.cloud:spring-cloud-netflix-core:jar + org.springframework.cloud:spring-cloud-netflix-eureka-client:jar + org.springframework.cloud:spring-cloud-netflix-ribbon:jar + org.springframework.cloud:spring-cloud-starter:jar + org.springframework.cloud:spring-cloud-starter-config:jar + org.springframework.cloud:spring-cloud-starter-eureka:jar + org.springframework.cloud:spring-cloud-starter-netflix-archaius:jar + org.springframework.cloud:spring-cloud-starter-netflix-eureka-client:jar + org.springframework.cloud:spring-cloud-starter-netflix-ribbon:jar + org.springframework.security:spring-security-crypto:jar + org.springframework.security:spring-security-rsa:jar + org.springframework:spring-aop:jar + org.springframework:spring-beans:jar + org.springframework:spring-context:jar + org.springframework:spring-core:jar + org.springframework:spring-expression:jar + org.springframework:spring-jcl:jar + org.springframework:spring-web:jar + org.springframework:spring-webmvc:jar + org.tukaani:xz:jar + org.yaml:snakeyaml:jar + software.amazon.ion:ion-java:jar + xerces:xercesImpl:jar + xmlenc:xmlenc:jar + xmlpull:xmlpull:jar + xpp3:xpp3_min:jar + + + + + + + + + diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java new file mode 100644 index 0000000000..3530ef5139 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java @@ -0,0 +1,111 @@ +/* + * 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.monitor.scan.app; + +import org.apache.linkis.monitor.scan.app.dirtydata.entrance.EntranceDirtyDataHandler; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.server.utils.LinkisMainHelper; + +import org.apache.commons.lang3.StringUtils; + +import org.springframework.context.annotation.AnnotationConfigApplicationContext; +import org.springframework.context.support.AbstractApplicationContext; + +import org.slf4j.Logger; + +public class LinkisDirtyDataCleanApplication { + private static final Logger logger = LogUtils.stdOutLogger(); + + /** @param args: args[0]: host args[1] port */ + public static void main(String[] args) throws ReflectiveOperationException { + if (args.length == 0 + || StringUtils.equalsIgnoreCase(args[0], "help") + || StringUtils.equalsIgnoreCase(args[0], "--help")) { + LogUtils.stdOutLogger() + .info( + "[help-message]this app cleans entrance dirty-data. args[0]: command-type (entrance/help/...) args[1]: entrance-hostname(not null), args[2]: entrance-port(can be null)"); + return; + } + String serviceName = System.getProperty(LinkisMainHelper.SERVER_NAME_KEY()); + LinkisMainHelper.formatPropertyFiles(serviceName); + + if (StringUtils.equalsIgnoreCase(args[0], "entrance")) { + AbstractApplicationContext context = + new AnnotationConfigApplicationContext(LinkisJobHistoryScanSpringConfiguration.class); + + String host = ""; + String port = ""; + if (args.length > 1) { + host = args[1]; + } + if (args.length > 2) { + port = args[2]; + } + if (args.length > 3) { + printIllegalInput("wrong number of arguments"); + return; + } + try { + removeDirtyEurekaInstance(host, port); + } catch (Exception e) { + LogUtils.stdOutLogger().error("Failed to remove dirty eureka-instance", e); + } + try { + removeDbDirtyData(host, port); + } catch (Exception e) { + LogUtils.stdOutLogger().error("Failed to remove dirty db-data", e); + } + + context.close(); + } else { + LogUtils.stdOutLogger().error("Upsupported command type: " + args[0]); + } + } + + private static void printIllegalInput(String msg) { + LogUtils.stdOutLogger().error("illegal input: " + msg); + LogUtils.stdOutLogger() + .info( + "[help-message] this app cleans entrance dirty-data. args[0]: entrance-hostname, args[1]: entrance-port"); + return; + } + + private static void removeDirtyEurekaInstance(String host, String port) { + if (StringUtils.isBlank(host)) { + printIllegalInput("host cannot be blank"); + return; + } + if (StringUtils.isBlank(port)) { + EntranceDirtyDataHandler.handleEurekaDirtyData(host); + } else { + EntranceDirtyDataHandler.handleEurekaDirtyData(host, port); + } + } + + private static void removeDbDirtyData(String host, String port) { + if (StringUtils.isBlank(host)) { + printIllegalInput("host cannot be blank"); + return; + } + if (StringUtils.isBlank(port)) { + EntranceDirtyDataHandler.handleDbDirtData(host); + } else { + EntranceDirtyDataHandler.handleDbDirtData(host, port); + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java new file mode 100644 index 0000000000..8b4366805e --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java @@ -0,0 +1,172 @@ +/* + * 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.monitor.scan.app; + +import org.apache.linkis.monitor.scan.app.factory.MapperFactory; +import org.apache.linkis.monitor.scan.app.jobhistory.JobHistoryDataFetcher; +import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrCodeRule; +import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrorCodeAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedRule; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.core.pac.DataFetcher; +import org.apache.linkis.monitor.scan.core.scanner.AnomalyScanner; +import org.apache.linkis.monitor.scan.core.scanner.DefaultScanner; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.server.utils.LinkisMainHelper; + +import org.apache.commons.lang3.exception.ExceptionUtils; + +import org.springframework.context.annotation.AnnotationConfigApplicationContext; +import org.springframework.context.support.AbstractApplicationContext; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; + +public class LinkisJobHistoryScanApplication { + private static final Logger logger = LogUtils.stdOutLogger(); + + /** @param args: args[0]: startTime, args[1] endTime */ + public static void main(String[] args) throws ReflectiveOperationException { + + String serviceName = System.getProperty(LinkisMainHelper.SERVER_NAME_KEY()); + LinkisMainHelper.formatPropertyFiles(serviceName); + + long intervalMs = Constants.SCAN_INTERVALS_SECONDS() * 1000; + long maxIntervalMs = Constants.MAX_INTERVALS_SECONDS() * 1000; + long endTime = System.currentTimeMillis(); + long startTime = endTime - intervalMs; + /** parse input into timestamp */ + if (args != null && args.length == 2) { + SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHH"); + Long tmpStart; + Long tmpEnd; + try { + tmpStart = format.parse(args[0]).getTime(); + tmpEnd = format.parse(args[1]).getTime(); + } catch (ParseException e) { + logger.info( + "Failed to parse input into startTime and endTime." + ExceptionUtils.getMessage(e)); + return; + } + if (endTime <= startTime) { + logger.info("[error] startTime larger than endTime"); + return; + } + if (tmpStart != null && tmpEnd != null) { + startTime = tmpStart; + endTime = tmpEnd; + } + } + long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + + runApp(startTime, endTime, realIntervals, maxIntervalMs); + } + + private static void runApp(long startTime, long endTime, long realIntervals, long maxIntervalMs) { + + AbstractApplicationContext context = + new AnnotationConfigApplicationContext(LinkisJobHistoryScanSpringConfiguration.class); + + AnomalyScanner scanner = new DefaultScanner(); + boolean shouldStart = false; + + List fetchers = generateFetchers(startTime, endTime, maxIntervalMs); + if (fetchers == null) { + logger.warn("generated 0 dataFetchers, plz check input"); + return; + } + + Map errorCodeAlerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + if (errorCodeAlerts == null || errorCodeAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 errorcode alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} errorcode alert from alert-rules properties file.", + errorCodeAlerts.size()); + shouldStart = true; + addIntervalToImsAlerts(errorCodeAlerts, realIntervals); + JobHistoryErrCodeRule jobHistoryErrCodeRule = + new JobHistoryErrCodeRule( + errorCodeAlerts.keySet(), new JobHistoryErrorCodeAlertSender(errorCodeAlerts)); + scanner.addScanRule(jobHistoryErrCodeRule); + } + + Map jobTimeAlerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts( + Constants.SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC()); + if (jobTimeAlerts == null || jobTimeAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 alerts jobtime alert-rule from alert properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobtime alert-rules from alert properties file.", + jobTimeAlerts.size()); + shouldStart = true; + addIntervalToImsAlerts(jobTimeAlerts, realIntervals); + JobTimeExceedRule jobTimeExceedRule = + new JobTimeExceedRule( + jobTimeAlerts.keySet(), new JobTimeExceedAlertSender(jobTimeAlerts)); + scanner.addScanRule(jobTimeExceedRule); + } + if (shouldStart) { + scanner.addDataFetchers(fetchers); + scanner.run(); + scanner.shutdown(); // wait all alert to be send + } + context.close(); + } + + private static List generateFetchers( + long startTime, long endTime, long maxIntervalMs) { + List ret = new ArrayList<>(); + long pe = endTime; + long ps; + while (pe > startTime) { + ps = pe - maxIntervalMs > startTime ? pe - maxIntervalMs : startTime; + String[] fetcherArgs = new String[] {String.valueOf(ps), String.valueOf(pe)}; + ret.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); + logger.info( + "Generated dataFetcher for startTime: " + + new Date(ps).toString() + + ". EndTime: " + + new Date(pe).toString()); + pe = pe - maxIntervalMs; + } + return ret; + } + + private static void addIntervalToImsAlerts(Map alerts, long realIntervals) { + for (AlertDesc alert : alerts.values()) { + if (!(alert instanceof ImsAlertDesc)) { + logger.info("[warn] ignore wrong alert" + alert); + } else { + ((ImsAlertDesc) alert).hitIntervalMs_$eq(realIntervals); + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java new file mode 100644 index 0000000000..405d94037b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java @@ -0,0 +1,32 @@ +/* + * 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.monitor.scan.app; + +import org.apache.linkis.LinkisBaseServerApp; + +import org.springframework.boot.autoconfigure.SpringBootApplication; +import org.springframework.scheduling.annotation.EnableScheduling; + +@EnableScheduling +@SpringBootApplication +public class LinksMonitorResourceApplication { + + public static void main(String[] args) throws ReflectiveOperationException { + LinkisBaseServerApp.main(args); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java new file mode 100644 index 0000000000..3705288e66 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java @@ -0,0 +1,35 @@ +/* + * 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.monitor.scan.app.instance.dao; + +import org.apache.linkis.monitor.scan.app.instance.entity.InsPersistenceLabel; + +import java.util.List; + +public interface InsLabelRelationDao { + + /** + * Search labels + * + * @param instance instance value (http:port) + * @return + */ + List searchLabelsByInstance(String instance); + + void dropRelationsByInstance(String instance); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java new file mode 100644 index 0000000000..73a7166a5f --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java @@ -0,0 +1,36 @@ +/* + * 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.monitor.scan.app.instance.dao; + +import org.apache.linkis.common.ServiceInstance; +import org.apache.linkis.monitor.scan.app.instance.entity.InstanceInfo; + +import org.apache.ibatis.annotations.Param; + +import java.util.List; + +public interface InstanceInfoDao { + + void removeInstance(ServiceInstance instance); + + List getInstanceInfoByApplicationNameAndHostnameFuzzy( + @Param("hostname") String hostname, @Param("applicationName") String applicationName); + + List getInstanceInfoByApplicationNameAndInstanceName( + @Param("instanceName") String instanceName, @Param("applicationName") String applicationName); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java new file mode 100644 index 0000000000..8f73f0349a --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java @@ -0,0 +1,41 @@ +/* + * 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.monitor.scan.app.instance.dao; + +import org.apache.linkis.monitor.scan.app.instance.entity.InsPersistenceLabel; + +import java.util.List; + + +public interface InstanceLabelDao { + /** + * Remove label + * + * @param label label entity + */ + void remove(InsPersistenceLabel label); + + /** + * Remove key -> value map from label id + * + * @param labelId + */ + void doRemoveKeyValues(Integer labelId); + + void doRemoveKeyValuesBatch(List labelIds); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java new file mode 100644 index 0000000000..e828328b43 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java @@ -0,0 +1,82 @@ +/* + * 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.monitor.scan.app.instance.entity; + +import java.util.Date; + +public class InsPersistenceLabel { + private Integer id; + private int labelValueSize = -1; + private String stringValue; + private Boolean modifiable = false; + + private String labelKey; + + private String feature; + + private Date updateTime; + private Date createTime; + + public Boolean getModifiable() { + return modifiable; + } + + public void setModifiable(Boolean modifiable) { + this.modifiable = modifiable; + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + public int getLabelValueSize() { + return labelValueSize; + } + + public void setLabelValueSize(int labelValueSize) { + this.labelValueSize = labelValueSize; + } + + public String getStringValue() { + return stringValue; + } + + public void setStringValue(String stringValue) { + this.stringValue = stringValue; + } + + public Date getUpdateTime() { + return updateTime; + } + + public void setUpdateTime(Date updateTime) { + this.updateTime = updateTime; + } + + public Date getCreateTime() { + return createTime; + } + + public void setCreateTime(Date createTime) { + this.createTime = createTime; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java new file mode 100644 index 0000000000..53ab4bd806 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java @@ -0,0 +1,59 @@ +/* + * 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.monitor.scan.app.instance.entity; + +public class InsPersistenceLabelValue { + + private Integer labelId; + + private String valueKey; + + private String valueContent; + + public InsPersistenceLabelValue() {} + + public InsPersistenceLabelValue(Integer labelId, String key, String content) { + this.labelId = labelId; + this.valueKey = key; + this.valueContent = content; + } + + public String getValueKey() { + return valueKey; + } + + public void setValueKey(String valueKey) { + this.valueKey = valueKey; + } + + public String getValueContent() { + return valueContent; + } + + public void setValueContent(String valueContent) { + this.valueContent = valueContent; + } + + public Integer getLabelId() { + return labelId; + } + + public void setLabelId(Integer labelId) { + this.labelId = labelId; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java new file mode 100644 index 0000000000..a74cfaabb0 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java @@ -0,0 +1,92 @@ +/* + * 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.monitor.scan.app.instance.entity; + +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + +public class InstanceInfo { + /** Automatic increment */ + private Integer id; + + private String applicationName; + + private String instance; + + private Date updateTime; + + private Date createTime; + /** Labels related */ + private List labels = new ArrayList<>(); + + public InstanceInfo() {} + + public InstanceInfo(String applicationName, String instance) { + this.applicationName = applicationName; + this.instance = instance; + } + + public List getLabels() { + return labels; + } + + public void setLabels(List labels) { + this.labels = labels; + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + public String getApplicationName() { + return applicationName; + } + + public void setApplicationName(String applicationName) { + this.applicationName = applicationName; + } + + public String getInstance() { + return instance; + } + + public void setInstance(String instance) { + this.instance = instance; + } + + public Date getUpdateTime() { + return updateTime; + } + + public void setUpdateTime(Date updateTime) { + this.updateTime = updateTime; + } + + public Date getCreateTime() { + return createTime; + } + + public void setCreateTime(Date createTime) { + this.createTime = createTime; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java new file mode 100644 index 0000000000..799eae89a6 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java @@ -0,0 +1,31 @@ +/* + * 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.monitor.scan.app.jobhistory; + +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.Date; + +public class QueryUtils { + + private static DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + + public static String dateToString(Date date) { + return dateFormat.format(date); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java new file mode 100644 index 0000000000..b3774860af --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java @@ -0,0 +1,68 @@ +/* + * 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.monitor.scan.app.jobhistory.dao; + +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory; + +import org.apache.ibatis.annotations.Param; + +import java.util.Date; +import java.util.List; + + +public interface JobHistoryMapper { + + List selectJobHistory(JobHistory jobReq); + + List search( + @Param("id") Long id, + @Param("umUser") String username, + @Param("status") List status, + @Param("startDate") Date startDate, + @Param("endDate") Date endDate, + @Param("engineType") String engineType); + + void updateIncompleteJobStatusGivenIDList( + @Param("idList") List idList, @Param("targetStatus") String targetStatus); + + void updateJobStatusForInstanceGivenStatusList( + @Param("instanceName") String instanceName, + @Param("statusList") List statusList, + @Param("targetStatus") String targetStatus, + @Param("startDate") Date startDate); + + List searchByCache( + @Param("id") Long id, + @Param("umUser") String username, + @Param("status") List status, + @Param("startDate") Date startDate, + @Param("endDate") Date endDate, + @Param("engineType") String engineType); + + List searchByCacheAndUpdateTime( + @Param("id") Long id, + @Param("umUser") String username, + @Param("status") List status, + @Param("startDate") Date startDate, + @Param("endDate") Date endDate, + @Param("engineType") String engineType); + + Long selectIdByHalfDay(@Param("id") long beginId); + + Long selectMaxId(); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java new file mode 100644 index 0000000000..937eff410b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java @@ -0,0 +1,221 @@ +/* + * 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.monitor.scan.app.jobhistory.entity; + +import org.apache.linkis.monitor.scan.app.jobhistory.QueryUtils; + +import java.util.Date; + +public class JobHistory { + + private Long id; + + private String jobReqId; + + private String submitUser; + + private String executeUser; + + private String source; + + private String labels; + + private String params; + + private String progress; + + private String status; + + private String logPath; + + private Integer errorCode; + + private String errorDesc; + + private Date createdTime; + + private Date updatedTime; + + private String updateTimeMills; + + private String instances; + + private String metrics; + + private String engineType; + + private String executionCode; + + private String observeInfo; + + public Long getId() { + return id; + } + + public void setId(Long id) { + this.id = id; + } + + public String getJobReqId() { + return jobReqId; + } + + public void setJobReqId(String jobReqId) { + this.jobReqId = jobReqId; + } + + public String getSubmitUser() { + return submitUser; + } + + public void setSubmitUser(String submitUser) { + this.submitUser = submitUser; + } + + public String getExecuteUser() { + return executeUser; + } + + public void setExecuteUser(String executeUser) { + this.executeUser = executeUser; + } + + public String getSource() { + return source; + } + + public void setSource(String source) { + this.source = source; + } + + public String getLabels() { + return labels; + } + + public void setLabels(String labels) { + this.labels = labels; + } + + public String getParams() { + return params; + } + + public void setParams(String params) { + this.params = params; + } + + public String getProgress() { + return progress; + } + + public void setProgress(String progress) { + this.progress = progress; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + public String getLogPath() { + return logPath; + } + + public void setLogPath(String logPath) { + this.logPath = logPath; + } + + public Integer getErrorCode() { + return errorCode; + } + + public void setErrorCode(Integer errorCode) { + this.errorCode = errorCode; + } + + public String getErrorDesc() { + return errorDesc; + } + + public void setErrorDesc(String errorDesc) { + this.errorDesc = errorDesc; + } + + public Date getCreatedTime() { + return createdTime; + } + + public void setCreatedTime(Date createdTime) { + this.createdTime = createdTime; + } + + public Date getUpdatedTime() { + return updatedTime; + } + + public void setUpdatedTime(Date updatedTime) { + this.updatedTime = updatedTime; + } + + public String getInstances() { + return instances; + } + + public void setInstances(String instances) { + this.instances = instances; + } + + public String getMetrics() { + return metrics; + } + + public void setMetrics(String metrics) { + this.metrics = metrics; + } + + public String getEngineType() { + return engineType; + } + + public void setEngineType(String engineType) { + this.engineType = engineType; + } + + public String getExecutionCode() { + return executionCode; + } + + public void setExecutionCode(String executionCode) { + this.executionCode = executionCode; + } + + public String getUpdateTimeMills() { + return QueryUtils.dateToString(getUpdatedTime()); + } + + public String getObserveInfo() { + return observeInfo; + } + + public void setObserveInfo(String observeInfo) { + this.observeInfo = observeInfo; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java new file mode 100644 index 0000000000..0e3dd10d6b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java @@ -0,0 +1,37 @@ +/* + * 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.monitor.scan.app.jobhistory.exception; + +import org.apache.linkis.common.exception.ErrorException; +import org.apache.linkis.common.exception.ExceptionLevel; + +public class AnomalyScannerException extends ErrorException { + public AnomalyScannerException(int errCode, String desc) { + super(errCode, desc); + } + + public AnomalyScannerException( + int errCode, String desc, String ip, int port, String serviceKind) { + super(errCode, desc, ip, port, serviceKind); + } + + @Override + public ExceptionLevel getLevel() { + return super.getLevel(); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java new file mode 100644 index 0000000000..93b6065f75 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java @@ -0,0 +1,37 @@ +/* + * 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.monitor.scan.app.jobhistory.exception; + +import org.apache.linkis.common.exception.ErrorException; +import org.apache.linkis.common.exception.ExceptionLevel; + +public class DirtyDataCleanException extends ErrorException { + public DirtyDataCleanException(int errCode, String desc) { + super(errCode, desc); + } + + public DirtyDataCleanException( + int errCode, String desc, String ip, int port, String serviceKind) { + super(errCode, desc, ip, port, serviceKind); + } + + @Override + public ExceptionLevel getLevel() { + return super.getLevel(); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java new file mode 100644 index 0000000000..fae16b63a3 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java @@ -0,0 +1,46 @@ +/* + * 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.monitor.scan.app.monitor.config; + +import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.context.annotation.Configuration; +import org.springframework.context.event.ContextClosedEvent; +import org.springframework.context.event.EventListener; + +import java.io.IOException; + +import org.slf4j.Logger; + +@Configuration +public class ListenerConfig { + + private static final Logger logger = LogUtils.stdOutLogger(); + + @EventListener + private void shutdownEntrance(ContextClosedEvent event) { + try { + ThreadUtils.executors.shutdown(); + HttpsUntils.client.close(); + } catch (IOException e) { + logger.error("ListenerConfig error msg {}", e.getMessage()); + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java new file mode 100644 index 0000000000..fab9a5cbe7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java @@ -0,0 +1,71 @@ +/* + * 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.monitor.scan.app.monitor.config; + +import org.apache.linkis.common.conf.CommonVars; +import org.apache.linkis.common.conf.Configuration; + +public class MonitorConfig { + + public static final String shellPath = Configuration.getLinkisHome() + "/admin/"; + + public static final CommonVars GATEWAY_URL = CommonVars.apply("wds.linkis.gateway.url"); + + public static final CommonVars USER_MODE_TIMEOUT = + CommonVars.apply("linkis.monitor.user.timeOut", 300); + public static final CommonVars USER_MODE_AUTHTOKEN = + CommonVars.apply("linkis.monitor.user.authToken","VALIDATOR-AUTH"); + public static final CommonVars USER_MODE_ENGINE = + CommonVars.apply("linkis.monitor.user.enginelist","[]"); + + public static final CommonVars ECM_TASK_MAJOR = + CommonVars.apply("linkis.monitor.ecmResourceTask.major", 0.03); + public static final CommonVars ECM_TASK_MINOR = + CommonVars.apply("linkis.monitor.ecmResourceTask.minor", 0.1); + public static final CommonVars ECM_TASK_IMURL = + CommonVars.apply("linkis.monitor.metrics.imsUrl"); + public static final CommonVars ECM_TASK_USER_AUTHKEY = + CommonVars.apply("linkis.monitor.metrics.userAuthKey"); + + public static final CommonVars JOB_HISTORY_TIME_EXCEED = + CommonVars.apply("linkis.monitor.jobhistory.id.timeExceed",0L); + + public static final CommonVars ENTRANCE_TASK_USERTOTAL = + CommonVars.apply("linkis.monitor.entranceTask.userTotalTask", 1000); + public static final CommonVars ENTRANCE_TASK_TOTAL_MAJOR = + CommonVars.apply("linkis.monitor.entranceTask.linkisTotalTaskMajor", 50000); + public static final CommonVars ENTRANCE_TASK_TOTAL_MINOR = + CommonVars.apply("linkis.monitor.entranceTask.linkisTotalTaskMinor", 10000); + public static final CommonVars ENTRANCE_TASK_USERLIST = + CommonVars.apply("linkis.monitor.entranceTask.userlist","[]"); + + public static final CommonVars SCHEDULED_CONFIG_NUM = + CommonVars.apply("linkis.monitor.scheduled.pool.cores.num", 10); + + public static final CommonVars SHELL_TIMEOUT = + CommonVars.apply("linkis.monitor.shell.time.out.minute", 30); + + public static final CommonVars USER_MODE_INTERFACE_TIMEOUT = + CommonVars.apply("linkis.monitor.user.mode.time.out", 30*1000); + + public static final CommonVars CHATBOT_KEY_ID = CommonVars.apply("linkis.monitor.chatbot.key.id","23e6afad1b78a0c5eed67e4d24de7063"); + public static final CommonVars CHATBOT_TYPE = CommonVars.apply("linkis.monitor.chatbot.type","text"); + public static final CommonVars CHATBOT_SERVICE_NAME= CommonVars.apply("linkis.monitor.chatbot.serviceName","大数据生产助手(BDP_PRD)"); + public static final CommonVars CHATBOT_URL= CommonVars.apply("linkis.monitor.chatbot.url","http://172.21.3.43:1377/pros-chatbot/yuanfang/sendEMsg"); + public static final CommonVars SOLUTION_URL = CommonVars.apply("linkis.monitor.jobhistory.solution.url", "http://kn.dss.weoa.com/linkis/qa"); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java new file mode 100644 index 0000000000..30495a87e1 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java @@ -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.monitor.scan.app.monitor.config; + +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.scheduling.annotation.SchedulingConfigurer; +import org.springframework.scheduling.config.ScheduledTaskRegistrar; + +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +@Configuration +public class ScheduledConfig implements SchedulingConfigurer { + @Bean + public Executor taskExecutor() { + return Executors.newScheduledThreadPool(MonitorConfig.SCHEDULED_CONFIG_NUM.getValue()); + } + + @Override + public void configureTasks(ScheduledTaskRegistrar scheduledTaskRegistrar) { + scheduledTaskRegistrar.setScheduler(taskExecutor()); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java new file mode 100644 index 0000000000..3cf288adbb --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java @@ -0,0 +1,87 @@ +/* + * 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.monitor.scan.app.monitor.entity; + +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; + +public class ChatbotEntity { + private String keyid; + private String content; + private String type; + private String userName; + private String serviceName; + + public ChatbotEntity(String content, String userName) { + this.keyid = MonitorConfig.CHATBOT_KEY_ID.getValue(); + this.content = content; + this.type = MonitorConfig.CHATBOT_TYPE.getValue(); + this.userName = userName; + this.serviceName = MonitorConfig.CHATBOT_SERVICE_NAME.getValue(); + } + + public String getKeyid() { + return keyid; + } + + public void setKeyid(String keyid) { + this.keyid = keyid; + } + + public String getContent() { + return content; + } + + public void setContent(String content) { + this.content = content; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getServiceName() { + return serviceName; + } + + public void setServiceName(String serviceNameuserName) { + this.serviceName = serviceNameuserName; + } + + @Override + public String toString() { + return "ChatbotEntity{" + + "keyid='" + keyid + '\'' + + ", content='" + content + '\'' + + ", type='" + type + '\'' + + ", userName='" + userName + '\'' + + ", serviceName='" + serviceName + '\'' + + '}'; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java new file mode 100644 index 0000000000..650fa081cc --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java @@ -0,0 +1,53 @@ +/* + * 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.monitor.scan.app.monitor.entity; + +import java.io.Serializable; + +public class EngineEntity implements Serializable { + + private String engineType; + + private String code; + + private String runType; + + public String getEngineType() { + return engineType; + } + + public void setEngineType(String engineType) { + this.engineType = engineType; + } + + public String getCode() { + return code; + } + + public void setCode(String code) { + this.code = code; + } + + public String getRunType() { + return runType; + } + + public void setRunType(String runType) { + this.runType = runType; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java new file mode 100644 index 0000000000..6fa9441474 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java @@ -0,0 +1,63 @@ +/* + * 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.monitor.scan.app.monitor.entity; + +import java.io.Serializable; + +public class EntranceEntity implements Serializable { + + private Integer runningtasks; + + private Integer queuedtasks; + + private String alteruser; + + private String username; + + public Integer getQueuedtasks() { + return queuedtasks; + } + + public void setQueuedtasks(Integer queuedtasks) { + this.queuedtasks = queuedtasks; + } + + public String getAlteruser() { + return alteruser; + } + + public void setAlteruser(String alteruser) { + this.alteruser = alteruser; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public Integer getRunningtasks() { + return runningtasks; + } + + public void setRunningtasks(Integer runningtasks) { + this.runningtasks = runningtasks; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java new file mode 100644 index 0000000000..fbea8886e8 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java @@ -0,0 +1,85 @@ +/* + * 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.monitor.scan.app.monitor.entity; + +import org.apache.linkis.monitor.scan.constants.Constants; + +public class IndexEntity { + + private final String subsystemId = Constants.ALERT_SUB_SYSTEM_ID(); + private String interfaceName; + private String attrGroup; + private String attrName; + private String hostIp; + private String metricValue; + + public String getSubsystemId() { + return subsystemId; + } + + public String getInterfaceName() { + return interfaceName; + } + + public void setInterfaceName(String interfaceName) { + this.interfaceName = interfaceName; + } + + public String getAttrGroup() { + return attrGroup; + } + + public void setAttrGroup(String attrGroup) { + this.attrGroup = attrGroup; + } + + public String getAttrName() { + return attrName; + } + + public void setAttrName(String attrName) { + this.attrName = attrName; + } + + public String getHostIp() { + return hostIp; + } + + public void setHostIp(String hostIp) { + this.hostIp = hostIp; + } + + public String getMetricValue() { + return metricValue; + } + + public void setMetricValue(String metricValue) { + this.metricValue = metricValue; + } + + public IndexEntity() {} + + public IndexEntity( + String interfaceName, String attrGroup, String attrName, String hostIp, String metricValue) { + this.interfaceName = interfaceName; + this.attrGroup = attrGroup; + this.attrName = attrName; + this.hostIp = hostIp; + this.metricValue = metricValue; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java new file mode 100644 index 0000000000..75d415ac0d --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java @@ -0,0 +1,47 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.bml.cleaner.service.CleanerService; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.util.*; + +import org.slf4j.Logger; + + +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class BmlClear { + + private static final Logger logger = LogUtils.stdOutLogger(); + + @Autowired private CleanerService cleanerServices; + + @Scheduled(cron = "${linkis.monitor.bml.clear.historyVersion.cron}") + public void jobHistoryScanTask() { + logger.info("start to clear bml history version"); + cleanerServices.run(); + logger.info("end to clear bml history version"); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java new file mode 100644 index 0000000000..dced6117b1 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java @@ -0,0 +1,53 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; + +/*** + * Task: clean up linkis_cg_ec_resource_info_record data + */ +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class EcRecordClear { + + private static final Logger logger = LogUtils.stdOutLogger(); + + @Scheduled(cron = "${linkis.monitor.clear.ecRecord.cron}") + public void ecRecordClear() { + logger.info("Start to clear_ec_record shell"); + List cmdlist = new ArrayList<>(); + cmdlist.add("sh"); + cmdlist.add(MonitorConfig.shellPath + "clear_ec_record.sh"); + logger.info("clear_ec_record shell command {}", cmdlist); + String exec = ThreadUtils.run(cmdlist, "clear_ec_record.sh"); + logger.info("shell log {}", exec); + logger.info("End to clear_ec_record shell "); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java new file mode 100644 index 0000000000..4880a600cc --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java @@ -0,0 +1,53 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; + +/*** + * Task: clean up linkis_ps_job_history_group_history data + */ +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class JobHistoryClear { + + private static final Logger logger = LogUtils.stdOutLogger(); + + @Scheduled(cron = "${linkis.monitor.clear.historyTask.cron}") + public void historyTaskClear() { + logger.info("Start to clear_history_task shell"); + List cmdlist = new ArrayList<>(); + cmdlist.add("sh"); + cmdlist.add(MonitorConfig.shellPath + "clear_history_task.sh"); + logger.info("clear_history_task shell command {}", cmdlist); + String exec = ThreadUtils.run(cmdlist, "clear_history_task.sh"); + logger.info("shell log {}", exec); + logger.info("End to clear_history_task shell "); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java new file mode 100644 index 0000000000..01873a988b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java @@ -0,0 +1,264 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.monitor.scan.app.factory.MapperFactory; +import org.apache.linkis.monitor.scan.app.jobhistory.JobHistoryDataFetcher; +import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrCodeRule; +import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrorCodeAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedRule; +import org.apache.linkis.monitor.scan.app.jobhistory.labels.JobHistoryLabelsAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.labels.JobHistoryLabelsRule; +import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonRunTimeAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonJobRunTimeRule; +import org.apache.linkis.monitor.scan.app.jobhistory.runtime.JobHistoryRunTimeAlertSender; +import org.apache.linkis.monitor.scan.app.jobhistory.runtime.JobHistoryRunTimeRule; +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.core.pac.DataFetcher; +import org.apache.linkis.monitor.scan.core.scanner.AnomalyScanner; +import org.apache.linkis.monitor.scan.core.scanner.DefaultScanner; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.UserLabelAlertUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.util.*; + +import org.slf4j.Logger; + +/*** + * jobHistory monitor + * 1.errorCode: Monitor the error code + * 2.userLabel: tenant label monitoring, scan the execution data within the first 20 minutes, and judge the labels field of the data + * 3.jobResultRunTime: Scan the execution data within the first 20 minutes, and judge the completed tasks. If the parm field in the jobhistory contains (task.notification.conditions) and the result of executing the task is (Succeed, Failed, Canceled, Timeout, ALL) any one of them, an alarm is triggered and the result of the job is that it has ended. All three are indispensable + * 4.jobResultRunTimeForDSS: Scan the execution data within the first 20 minutes, scan the tasks that have been marked for notification, if the task has ended, a notification will be initiated + * 5.jobHistoryUnfinishedScan: monitor the status of the execution task, scan the data outside 12 hours and within 24 hours + */ +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class JobHistoryMonitor { + + private static final Logger logger = LogUtils.stdOutLogger(); + private static final long backtrackNum = 1000000L; + + /** + * Scan tasks completed within 20 minutes + */ + @Scheduled(cron = "0 0/10 * * * ?") + public void jobHistoryFinishedScan() { + long intervalMs = 20 * 60 * 1000; // 20分钟 + long maxIntervalMs = Constants.MAX_INTERVALS_SECONDS() * 1000; + long endTime = System.currentTimeMillis(); + long startTime = endTime - intervalMs; + long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + AnomalyScanner scanner = new DefaultScanner(); + boolean shouldStart = false; + long id; + if (null == CacheUtils.cacheBuilder.getIfPresent("jobHistoryId")) { + //20230206:新增获取最大值-100W 作为初始id进行查询,防止慢查询 + long maxId = MapperFactory.getJobHistoryMapper().selectMaxId(); + long beginId = 0L; + if (maxId > backtrackNum) { + beginId = maxId - backtrackNum; + } + id = MapperFactory.getJobHistoryMapper().selectIdByHalfDay(beginId); + CacheUtils.cacheBuilder.put("jobHistoryId", id); + } else { + id = CacheUtils.cacheBuilder.getIfPresent("jobHistoryId"); + } + List fetchers = generateFetchersfortime(startTime, endTime, id, "updated_time"); + if (fetchers == null) { + logger.warn("generated 0 dataFetchers, plz check input"); + return; + } + // errorCode + try { + Map errorCodeAlerts = JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + + if (errorCodeAlerts == null || errorCodeAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 errorcode alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} errorcode alert from alert-rules properties file.", + errorCodeAlerts.size()); + shouldStart = true; + addIntervalToImsAlerts(errorCodeAlerts, realIntervals); + JobHistoryErrCodeRule jobHistoryErrCodeRule = + new JobHistoryErrCodeRule( + errorCodeAlerts.keySet(), new JobHistoryErrorCodeAlertSender(errorCodeAlerts)); + scanner.addScanRule(jobHistoryErrCodeRule); + } + } catch (Exception e) { + logger.warn("Jobhistory Monitor ErrorCode Faily: "+ e.getMessage()); + } + // userLabel + try { + Map userLabelAlerts = + UserLabelAlertUtils.getAlerts(Constants.USER_LABEL_MONITOR(), ""); + if (userLabelAlerts == null || userLabelAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 alerts userLabel alert-rule from alert properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts userLabel alert-rules from alert properties file.", + userLabelAlerts.size()); + shouldStart = true; + JobHistoryLabelsRule jobHistoryLabelsRule = + new JobHistoryLabelsRule(new JobHistoryLabelsAlertSender()); + scanner.addScanRule(jobHistoryLabelsRule); + } + } catch (Exception e) { + logger.warn("Jobhistory Monitor UserLabel Faily: "+ e.getMessage()); + } + // jobResultRunTime + try { + Map jobResultAlerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + if (jobResultAlerts == null || jobResultAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 jobResult alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobResult alert-rules from alert properties file.", + jobResultAlerts.size()); + shouldStart = true; + JobHistoryRunTimeRule jobHistoryRunTimeRule = + new JobHistoryRunTimeRule(new JobHistoryRunTimeAlertSender()); + scanner.addScanRule(jobHistoryRunTimeRule); + } + } catch (Exception e) { + logger.warn("Jobhistory Monitor JobResultRunTime Faily: "+ e.getMessage()); + } + // jobResultRunTimeForDSS + try { + Map dssJobResultAlerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + if (dssJobResultAlerts == null || dssJobResultAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 jobResult alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobResult alert-rules from alert properties file.", + dssJobResultAlerts.size()); + shouldStart = true; + + CommonJobRunTimeRule commonJobRunTimeRule = + new CommonJobRunTimeRule(new CommonRunTimeAlertSender()); + scanner.addScanRule(commonJobRunTimeRule); + } + } catch (Exception e) { + logger.warn("Jobhistory JobResultRunTimeForDSS ErrorCode Faily: "+ e.getMessage()); + } + run(scanner, fetchers, shouldStart); + } + + /*** + * Whether scanning data within 12 hours has timed out + */ + @Scheduled(cron = "${linkis.monitor.jobHistory.ScanTask.cron}") + public void jobHistoryUnfinishedScan() { + long id = + Optional.ofNullable(CacheUtils.cacheBuilder.getIfPresent("jobhistoryScan")) + .orElse(MonitorConfig.JOB_HISTORY_TIME_EXCEED.getValue()); + long intervalMs = Constants.SCAN_INTERVALS_SECONDS() * 1000; + long maxIntervalMs = Constants.MAX_INTERVALS_SECONDS() * 1000; + long endTime = System.currentTimeMillis(); + long startTime = endTime - intervalMs; + long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + AnomalyScanner scanner = new DefaultScanner(); + boolean shouldStart = false; + List fetchers = generateFetchers(startTime, endTime, maxIntervalMs, id, "created_time"); + if (fetchers == null) { + logger.warn("generated 0 dataFetchers, plz check input"); + return; + } + Map jobTimeAlerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts( + Constants.SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC()); + if (jobTimeAlerts == null || jobTimeAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 alerts jobtime alert-rule from alert properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobtime alert-rules from alert properties file.", + jobTimeAlerts.size()); + shouldStart = true; + addIntervalToImsAlerts(jobTimeAlerts, realIntervals); + JobTimeExceedRule jobTimeExceedRule = + new JobTimeExceedRule( + jobTimeAlerts.keySet(), new JobTimeExceedAlertSender(jobTimeAlerts)); + scanner.addScanRule(jobTimeExceedRule); + } + run(scanner, fetchers, shouldStart); + } + + public static void run(AnomalyScanner scanner, List fetchers, Boolean shouldStart) { + if (shouldStart) { + scanner.addDataFetchers(fetchers); + scanner.run(); + // scanner.shutdown(); // wait all alert to be send + } + } + + private static List generateFetchers( + long startTime, long endTime, long maxIntervalMs, long id, String timeType) { + List ret = new ArrayList<>(); + long pe = endTime; + long ps; + while (pe > startTime) { + ps = pe - maxIntervalMs > startTime ? pe - maxIntervalMs : startTime; + String[] fetcherArgs = + new String[]{String.valueOf(ps), String.valueOf(pe), String.valueOf(id), timeType}; + ret.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); + logger.info( + "Generated dataFetcher for startTime: " + + new Date(ps).toString() + + ". EndTime: " + + new Date(pe).toString()); + pe = pe - maxIntervalMs; + } + return ret; + } + + private static List generateFetchersfortime(long startTime, long endTime, long id, String timeType) { + List fetchers = new ArrayList<>(); + String[] fetcherArgs = + new String[]{String.valueOf(startTime), String.valueOf(endTime), String.valueOf(id), timeType}; + fetchers.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); + logger.info( + "Generated dataFetcher for startTime: " + + new Date(startTime).toString() + + ". EndTime: " + + new Date(endTime).toString()); + return fetchers; + } + + private static void addIntervalToImsAlerts(Map alerts, long realIntervals) { + for (AlertDesc alert : alerts.values()) { + if (!(alert instanceof ImsAlertDesc)) { + logger.info("[warn] ignore wrong alert" + alert); + } else { + ((ImsAlertDesc) alert).hitIntervalMs_$eq(realIntervals); + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java new file mode 100644 index 0000000000..b7066ba420 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java @@ -0,0 +1,151 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.common.utils.ByteTimeUtils; +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.entity.IndexEntity; +import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; + +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; + +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/*** + * Monitor the usage of ECM resources for monitoring and metrics reporting + */ +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class ResourceMonitor { + + private static final Logger logger = LoggerFactory.getLogger(ResourceMonitor.class); + + @Scheduled(cron = "${linkis.monitor.ecmResourceTask.cron}") + public void ecmResourceTask() { + Map resultmap = null; + AtomicReference tenant = new AtomicReference<>("租户标签:公共资源"); + AtomicReference totalMemory = new AtomicReference<>(0.0); + AtomicReference totalInstance = new AtomicReference<>(0.0); + AtomicReference totalCores = new AtomicReference<>(0.0); + try { + resultmap = HttpsUntils.sendHttp(null, null); + logger.info("ResourceMonitor response {}:", resultmap); + } catch (IOException e) { + logger.warn("failed to get EcmResource data"); + } + // got interface data + Map>> data = MapUtils.getMap(resultmap, "data"); + List> emNodeVoList = data.getOrDefault("EMs", new ArrayList<>()); + StringJoiner minor = new StringJoiner(","); + StringJoiner major = new StringJoiner(","); + // deal ecm resource + emNodeVoList.forEach( + emNodeVo -> { + Map leftResource = MapUtils.getMap(emNodeVo, "leftResource"); + Map maxResource = MapUtils.getMap(emNodeVo, "maxResource"); + // 新增 ECM资源告警,需补充此ECM所属租户 + List> labels = (List>) emNodeVo.get("labels"); + labels.stream() + .filter(labelmap -> labelmap.containsKey("tenant")) + .forEach(map -> tenant.set("租户标签:" + map.get("stringValue").toString())); + String leftmemory = ByteTimeUtils.bytesToString((long) leftResource.getOrDefault("memory",0)); + String maxmemory = ByteTimeUtils.bytesToString((long) maxResource.getOrDefault("memory",0)); + + String leftmemoryStr = leftmemory.split(" ")[0]; + String maxmemoryStr = maxmemory.split(" ")[0]; + + BigDecimal leftMemory = new BigDecimal(leftmemoryStr); + BigDecimal leftCores = new BigDecimal((int) leftResource.get("cores")); + BigDecimal leftInstance = new BigDecimal((int) leftResource.get("instance")); + totalMemory.set(totalMemory.get() + leftMemory.doubleValue()); + totalInstance.set(totalInstance.get() + leftInstance.doubleValue()); + totalCores.set(totalCores.get() + leftCores.doubleValue()); + + BigDecimal maxMemory = new BigDecimal(maxmemoryStr); + BigDecimal maxCores = new BigDecimal((int) maxResource.get("cores")); + BigDecimal maxInstance = new BigDecimal((int) maxResource.get("instance")); + double memorydouble = leftMemory.divide(maxMemory,2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); + double coresdouble = leftCores.divide(maxCores,2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); + double instancedouble = leftInstance.divide(maxInstance,2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); + Double majorValue = MonitorConfig.ECM_TASK_MAJOR.getValue(); + Double minorValue = MonitorConfig.ECM_TASK_MINOR.getValue(); + if (((memorydouble) <= majorValue) + || ((coresdouble) <= majorValue) + || ((instancedouble) <= majorValue)) { + // major告警 + major.add(emNodeVo.get("instance").toString()); + } else if (((memorydouble) < minorValue) + || ((coresdouble) < minorValue) + || ((instancedouble) < minorValue)) { + // minor告警 + minor.add(emNodeVo.get("instance").toString()); + } + HashMap replaceParm = new HashMap<>(); + replaceParm.put("$tenant", tenant.get()); + if (StringUtils.isNotBlank(major.toString())) { + replaceParm.put("$instance", major.toString()); + replaceParm.put("$ratio", majorValue.toString()); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), replaceParm); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12004")); + } + if (StringUtils.isNotBlank(minor.toString())) { + replaceParm.put("$instance", minor.toString()); + replaceParm.put("$ratio", minorValue.toString()); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), replaceParm); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12003")); + } + // ECM资源占比上报 + resourceSendToIms(coresdouble, memorydouble, instancedouble, HttpsUntils.localHost,"USED"); + }); + //ECM 剩余资源总数上报 + resourceSendToIms(totalCores.get(), totalMemory.get(), totalInstance.get(), HttpsUntils.localHost,"TOTAL"); + } + + private void resourceSendToIms(Double coresdouble, Double memorydouble, Double instancedouble, String loaclhost, String name) { + List list = new ArrayList<>(); + logger.info("ResourceMonitor send index "); + String core ="ECM_CPU_"; + String memory ="ECM_MEMORY_"; + String instance ="ECM_INSTANCE_"; + list.add(new IndexEntity(core.concat(name), "CPU", "INDEX", loaclhost, String.valueOf(coresdouble))); + list.add(new IndexEntity(memory.concat(name), "MEMORY", "INDEX", loaclhost, String.valueOf(memorydouble))); + list.add(new IndexEntity(instance.concat(name), "INSTANCE", "INDEX", loaclhost, String.valueOf(instancedouble))); + try { + HttpsUntils.sendIndex(list); + } catch (IOException e) { + logger.warn("failed to send EcmResource index"); + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java new file mode 100644 index 0000000000..bae3dc53f7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java @@ -0,0 +1,53 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; + +/*** + * Task: clean up logs, file data of ec materials + */ +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class TaskLogClear { + + private static final Logger logger = LogUtils.stdOutLogger(); + + @Scheduled(cron = "${linkis.monitor.clear.taskLog.cron}") + public void taskLogClear() { + logger.info("Start to linkis_task_res_log_clear shell"); + List cmdlist = new ArrayList<>(); + cmdlist.add("sh"); + cmdlist.add(MonitorConfig.shellPath + "linkis_task_res_log_clear.sh"); + logger.info("linkis_task_res_log_clear shell command {}", cmdlist); + String exec = ThreadUtils.run(cmdlist, "linkis_task_res_log_clear.sh"); + logger.info("shell log {}", exec); + logger.info("End to linkis_task_res_log_clear shell "); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java new file mode 100644 index 0000000000..87bc6d6e22 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java @@ -0,0 +1,183 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.common.ServiceInstance; +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.entity.IndexEntity; +import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.rpc.Sender; +import org.apache.linkis.server.BDPJettyServerHelper; + +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; + +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.*; + +import com.google.gson.internal.LinkedTreeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/*** + * Entrance monitors the number of tasks for specified users and systems. + * If the configured threshold is exceeded, an alarm will be triggered. + */ +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class TaskMonitor { + + private static final Logger logger = LoggerFactory.getLogger(TaskMonitor.class); + + private static final String ENTRANCE_RUNNING_TASK = "entrance_running_task"; + private static final String ENTRANCE_QUEUED_TASK = "entrance_queued_task"; + + + @Scheduled(cron = "${linkis.monitor.entranceTask.cron}") + public void entranceTask() throws IOException { + List> userlist = new ArrayList<>(); + String value = MonitorConfig.ENTRANCE_TASK_USERLIST.getValue(); + if (StringUtils.isNotBlank(value)) { + userlist = BDPJettyServerHelper.gson().fromJson(value, ArrayList.class); + } + // 用户监控 + userlist.forEach( + entranceEntity -> { + Map data = new HashMap<>(); + try { + data = + MapUtils.getMap( + HttpsUntils.getEntranceTask(null, entranceEntity.get("username"),null), "data"); + logger.info("TaskMonitor userlist response {}:", data); + } catch (IOException e) { + logger.warn("failed to get EntranceTask data"); + } + + int runningNumber = + null != entranceEntity.get("runningtasks") + ? Integer.parseInt(entranceEntity.get("runningtasks")) + : 0; + int queuedNumber = + null != entranceEntity.get("queuedtasks") + ? Integer.parseInt(entranceEntity.get("queuedtasks")) + : 0; + + BigDecimal runningtotal = new BigDecimal((int) data.get("runningNumber")); + BigDecimal queuedtotal = new BigDecimal((int) data.get("queuedNumber")); + BigDecimal total = runningtotal.add(queuedtotal); + HashMap parms = new HashMap<>(); + parms.put("$username", entranceEntity.get("username")); + parms.put("$alteruser", entranceEntity.get("alteruser")); + parms.put("$url", MonitorConfig.GATEWAY_URL.getValue()); + // 获取标准阈值 + if (runningtotal.intValue() > runningNumber) { + // 触发告警 用户运行任务满 + parms.put("$runningtask", String.valueOf(runningNumber)); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12005")); + } + if (queuedtotal.intValue() > queuedNumber) { + // 触发告警 用户排队任务满 + parms.put("$queuedtask", String.valueOf(queuedNumber)); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12006")); + } + int usertotalTask = MonitorConfig.ENTRANCE_TASK_USERTOTAL.getValue(); + if (total.intValue() > usertotalTask) { + // 触发告警 用户任务总数满 + parms.put("$tasktotal", String.valueOf(usertotalTask)); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12007")); + } + }); + Map likisData = null; + try { + likisData = MapUtils.getMap(HttpsUntils.getEntranceTask(null, "hadoop",null), "data"); + logger.info("TaskMonitor hadoop response {}:", likisData); + } catch (IOException e) { + logger.warn("failed to get EntranceTask data"); + } + // 系统监控 + BigDecimal runningNumber = new BigDecimal((int) likisData.get("runningNumber")); + BigDecimal queuedNumber = new BigDecimal((int) likisData.get("queuedNumber")); + BigDecimal total = runningNumber.add(queuedNumber); + + HashMap parms = new HashMap<>(); + parms.put("$url", MonitorConfig.GATEWAY_URL.getValue()); + int linkisTotalMajor = MonitorConfig.ENTRANCE_TASK_TOTAL_MAJOR.getValue(); + int linkisTotalMinor = MonitorConfig.ENTRANCE_TASK_TOTAL_MINOR.getValue(); + if (total.intValue() >= linkisTotalMajor) { + // 触发告警Major + parms.put("$taskmajor", String.valueOf(linkisTotalMajor)); + logger.info("TaskMonitor parms {}:", parms); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12009")); + + } else if (total.intValue() >= linkisTotalMinor) { + // 触发告警Minor + parms.put("$taskminor", String.valueOf(linkisTotalMinor)); + logger.info("TaskMonitor parms {}:", parms); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12008")); + } + //指标上报 + resourceSendToIms(); + } + + public static void resourceSendToIms() { + //获取所有的entrance实例,逐个上送IMS + ServiceInstance[] instances = Sender.getInstances(Constants.DIRTY_DATA_ENTRANCE_APPLICATIONNAME()); + if (null != instances) { + for (ServiceInstance instance : instances) { + String serviceInstance = instance.getInstance(); + try { + Map instanceData = MapUtils.getMap(HttpsUntils.getEntranceTask(null, "hadoop", serviceInstance), "data"); + int runningNumber = 0; + int queuedNumber = 0; + if (instanceData.containsKey("runningNumber")) { + runningNumber = (int) instanceData.get("runningNumber"); + } + if (instanceData.containsKey("queuedNumber")) { + queuedNumber = (int) instanceData.get("queuedNumber"); + } + logger.info("ResourceMonitor send index "); + List list = new ArrayList<>(); + list.add(new IndexEntity(serviceInstance, "entrance", ENTRANCE_RUNNING_TASK, HttpsUntils.localHost, String.valueOf(runningNumber))); + list.add(new IndexEntity(serviceInstance, "entrance", ENTRANCE_QUEUED_TASK, HttpsUntils.localHost, String.valueOf(queuedNumber))); + HttpsUntils.sendIndex(list); + } catch (IOException e) { + logger.warn("failed to send EcmResource index :" + e); + } + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java new file mode 100644 index 0000000000..03db536344 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java @@ -0,0 +1,156 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.common.utils.Utils; +import org.apache.linkis.governance.common.entity.task.RequestPersistTask; +import org.apache.linkis.httpclient.dws.config.DWSClientConfig; +import org.apache.linkis.manager.label.constant.LabelKeyConstant; +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.server.BDPJettyServerHelper; +import org.apache.linkis.ujes.client.UJESClient; +import org.apache.linkis.ujes.client.UJESClientImpl; +import org.apache.linkis.ujes.client.request.GetTableStatisticInfoAction; +import org.apache.linkis.ujes.client.request.JobSubmitAction; +import org.apache.linkis.ujes.client.response.GetTableStatisticInfoResult; +import org.apache.linkis.ujes.client.response.JobExecuteResult; +import org.apache.linkis.ujes.client.response.JobInfoResult; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; +import java.net.SocketTimeoutException; +import java.util.*; +import java.util.concurrent.TimeUnit; + +import com.google.gson.internal.LinkedTreeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/*** + * User mode monitoring: regularly trigger scripts to monitor whether the engine status is running normally + */ +@Component +public class UserModeMonitor { + + private static final Logger logger = LoggerFactory.getLogger(UserModeMonitor.class); + + private static final DWSClientConfig clientConfig = HttpsUntils.dwsClientConfig; + + private static final UJESClient client = new UJESClientImpl(clientConfig); + + @Scheduled(cron = "${linkis.monitor.user.cron}") + public void job() { + Optional.ofNullable(MonitorConfig.USER_MODE_ENGINE.getValue()).ifPresent(configStr -> { + ArrayList> userModeStr = + BDPJettyServerHelper.gson().fromJson(configStr, ArrayList.class); + userModeStr.forEach(engine -> { + // 3. build job and execute + JobExecuteResult jobExecuteResult = toSubmit(engine); + logger.info( + "start run engineType: {},job id : {}", + engine.get("engineType"), + jobExecuteResult.taskID()); + HashMap parms = new HashMap<>(); + parms.put("$engineType", engine.get("engineType")); + parms.put("$url", MonitorConfig.GATEWAY_URL.getValue()); + parms.put("$jobId", jobExecuteResult.taskID()); + Utils.sleepQuietly(MonitorConfig.USER_MODE_TIMEOUT.getValue() * 1000); + JobInfoResult jobInfo = client.getJobInfo(jobExecuteResult); + if (jobInfo.isCompleted()) { + if (jobInfo.getJobStatus().equals("Failed")) { + logger.info("run fail engineType: {},job id : {}", engine.get("engineType"), jobExecuteResult.taskID()); + RequestPersistTask requestPersistTask = jobInfo.getRequestPersistTask(); + parms.put("$errorCode", String.valueOf(requestPersistTask.getErrCode())); + parms.put("$errorMsg", requestPersistTask.getErrDesc()); + Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(failedAlerts.get("12012")); + } + } else { + logger.info("run timeout engineType: {},job id : {}", engine.get("engineType"), jobExecuteResult.taskID()); + Map alerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(alerts.get("12011")); + } + } + ); + }); + } + + private static JobExecuteResult toSubmit(LinkedTreeMap engine) { + // 1. build params + // set label map :EngineTypeLabel/UserCreatorLabel/EngineRunTypeLabel/Tenant + Map labels = new HashMap(); + labels.put(LabelKeyConstant.ENGINE_TYPE_KEY, engine.get("engineType")); // required engineType Label + labels.put(LabelKeyConstant.USER_CREATOR_TYPE_KEY, engine.get("executeUser") + "-IDE");// required execute user and creator eg:hadoop-IDE + labels.put(LabelKeyConstant.CODE_TYPE_KEY, engine.get("runType")); // required codeType + Map startupMap = new HashMap(16); + // setting linkis params + //startupMap.put("wds.linkis.rm.yarnqueue", "dws"); + // 2. build jobSubmitAction + JobSubmitAction jobSubmitAction = JobSubmitAction.builder() + .addExecuteCode(engine.get("code")) + .setStartupParams(startupMap) + .setUser(engine.get("executeUser")) //submit user + .addExecuteUser(engine.get("executeUser")) // execute user + .setLabels(labels) + .build(); + // 3. to execute + return client.submit(jobSubmitAction); + } + + @Scheduled(cron = "${linkis.monitor.user.db.cron:0 0/10 * * * ?}") + public void dbJob() { + Map properties= new HashMap<>(); + properties.put("readTimeout",MonitorConfig.USER_MODE_INTERFACE_TIMEOUT.getValue()); + DWSClientConfig clientConfig = HttpsUntils.createClientConfig(null, properties); + UJESClientImpl ujesClient = new UJESClientImpl(clientConfig); + GetTableStatisticInfoAction builder = GetTableStatisticInfoAction + .builder() + .setUser("hadoop") + .setDatabase("default") + .setTable("dual") + .builder(); + HashMap parms = new HashMap<>(); + try { + GetTableStatisticInfoResult tableStatisticInfo = ujesClient.getTableStatisticInfo(builder); + if (tableStatisticInfo.getStatus() != 0) { + logger.info("元数据查询服务用户态,执行失败,异常信息:"+tableStatisticInfo.getMessage()); +// parms.put("$msg", tableStatisticInfo.getMessage()); +// Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); +// PooledImsAlertUtils.addAlert(failedAlerts.get("12017")); + } + } catch (Exception e) { + if(e instanceof SocketTimeoutException){ + Integer timeoutValue = MonitorConfig.USER_MODE_INTERFACE_TIMEOUT.getValue(); + long timeout = TimeUnit.MILLISECONDS.toSeconds(timeoutValue); + logger.info("元数据查询服务用户态,执行超时:"+timeout+"秒"); +// parms.put("$timeout", String.valueOf(timeout)); +// Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); +// PooledImsAlertUtils.addAlert(failedAlerts.get("12018")); + } else { + logger.error("元数据查询服务用户态,执行异常:"+ e); +// parms.put("$msg", e.getMessage()); +// Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); +// PooledImsAlertUtils.addAlert(failedAlerts.get("12017")); + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java new file mode 100644 index 0000000000..440b7a6bc4 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java @@ -0,0 +1,51 @@ +/* + * 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.monitor.scan.app.monitor.scheduled; + +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.slf4j.Logger; +import org.springframework.context.annotation.PropertySource; +import org.springframework.scheduling.annotation.Scheduled; +import org.springframework.stereotype.Component; + +import java.util.ArrayList; +import java.util.List; +/*** + * Task: clean up linkis_et_validator_checkinfo data + */ + +@Component +@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") +public class ValidatorClear { + + private static final Logger logger = LogUtils.stdOutLogger(); + + @Scheduled(cron = "${linkis.monitor.clear.validator.cron}") + public void ValidatorClear() { + logger.info("Start to clear_validator_record shell"); + List cmdlist = new ArrayList<>(); + cmdlist.add("sh"); + cmdlist.add(MonitorConfig.shellPath + "clear_validator_record.sh"); + logger.info("clear_validator_record shell command {}", cmdlist); + String exec = ThreadUtils.run(cmdlist, "clear_validator_record.sh"); + logger.info("shell log {}", exec); + logger.info("End to clear_validator_record shell "); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java new file mode 100644 index 0000000000..258d81183c --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java @@ -0,0 +1,35 @@ +/* + * 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.monitor.scan.app.monitor.until; + +import java.util.concurrent.TimeUnit; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; + +public class CacheUtils { + + public static Cache cacheBuilder = + CacheBuilder.newBuilder() + .concurrencyLevel(5) + .expireAfterAccess(1, TimeUnit.DAYS) + .initialCapacity(20) + .maximumSize(1000) + .recordStats() + .build(); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java new file mode 100644 index 0000000000..1208a6ef64 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java @@ -0,0 +1,183 @@ +/* + * 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.monitor.scan.app.monitor.until; + +import org.apache.linkis.common.conf.Configuration; +import org.apache.linkis.common.utils.Utils; +import org.apache.linkis.httpclient.dws.authentication.TokenAuthenticationStrategy; +import org.apache.linkis.httpclient.dws.config.DWSClientConfig; +import org.apache.linkis.httpclient.dws.config.DWSClientConfigBuilder; +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.app.monitor.entity.ChatbotEntity; +import org.apache.linkis.monitor.scan.app.monitor.entity.IndexEntity; +import org.apache.linkis.monitor.scan.client.MonitorHTTPClient; +import org.apache.linkis.monitor.scan.client.MonitorHTTPClientClientImpl; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.request.EmsListAction; +import org.apache.linkis.monitor.scan.request.EntranceTaskAction; +import org.apache.linkis.monitor.scan.response.EntranceTaskResult; +import org.apache.linkis.server.BDPJettyServerHelper; + +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; + +import org.apache.linkis.ujes.client.response.EmsListResult; +import org.springframework.util.Assert; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class HttpsUntils { + private static final Logger logger = LoggerFactory.getLogger(HttpsUntils.class); + + public static DWSClientConfig dwsClientConfig = createClientConfig(null, null); + // IOUtils.closeQuietly(client); + public static MonitorHTTPClient client = new MonitorHTTPClientClientImpl(dwsClientConfig); + public static final String localHost = Utils.getLocalHostname(); + + public static Map sendHttp(String url, Map properties) + throws IOException { + if (null == dwsClientConfig) { + dwsClientConfig = createClientConfig(url, properties); + } + if (null == client) { + client = new MonitorHTTPClientClientImpl(dwsClientConfig); + } + EmsListAction build = EmsListAction.newBuilder().setUser("hadoop").build(); + EmsListResult result = client.list(build); + return result.getResultMap(); + } + + public static DWSClientConfig createClientConfig(String url, Map properties) { + String realUrl = ""; + if (StringUtils.isBlank(url)) { + realUrl = Configuration.getGateWayURL(); + } else { + realUrl = url; + } + Map parms = new HashMap<>(); + if (MapUtils.isNotEmpty(properties)) { + parms = properties; + } + int maxConnection = + (int) + parms.getOrDefault( + Constants.CONNECTION_MAX_SIZE_SHORT_NAME(), + Constants.CONNECTION_MAX_SIZE().getValue()); + int connectTimeout = + (int) + parms.getOrDefault( + Constants.CONNECTION_TIMEOUT_SHORT_NAME(), + Constants.CONNECTION_TIMEOUT().getValue()); + int readTimeout = + (int) + parms.getOrDefault( + Constants.CONNECTION_READ_TIMEOUT_SHORT_NAME(), + Constants.CONNECTION_READ_TIMEOUT().getValue()); + String tokenKey = + (String) + parms.getOrDefault( + Constants.AUTH_TOKEN_KEY_SHORT_NAME(), Constants.AUTH_TOKEN_KEY().getValue()); + String tokenValue = + (String) + parms.getOrDefault( + Constants.AUTH_TOKEN_VALUE_SHORT_NAME(), Constants.AUTH_TOKEN_VALUE().getValue()); + + DWSClientConfig clientConfig = + ((DWSClientConfigBuilder) + (DWSClientConfigBuilder.newBuilder() + .addServerUrl(realUrl) + .connectionTimeout(connectTimeout) + .discoveryEnabled(false) + .discoveryFrequency(1, TimeUnit.MINUTES) + .loadbalancerEnabled(false) + .maxConnectionSize(maxConnection) + .retryEnabled(false) + .readTimeout(readTimeout) + .setAuthenticationStrategy(new TokenAuthenticationStrategy()) + .setAuthTokenKey(tokenKey) + .setAuthTokenValue(tokenValue))) + .setDWSVersion("v1") + .build(); + + return clientConfig; + } + + public static Map getEntranceTask(String url, String user,String Instance) throws IOException { + if (null == dwsClientConfig) { + dwsClientConfig = createClientConfig(null, null); + } + if (null == client) { + client = new MonitorHTTPClientClientImpl(dwsClientConfig); + } + EntranceTaskAction build = EntranceTaskAction.newBuilder().setUser(user).setInstance(Instance).build(); + EntranceTaskResult result = client.entranList(build); + return result.getResultMap(); + } + + public static void sendIndex(List list) throws IOException { + Map parm = new HashMap<>(); + parm.put("userAuthKey", MonitorConfig.ECM_TASK_USER_AUTHKEY.getValue()); + parm.put("metricDataList", list); + String json = BDPJettyServerHelper.gson().toJson(parm); + + RequestConfig requestConfig = RequestConfig.DEFAULT; + StringEntity entity = + new StringEntity( + json, ContentType.create(ContentType.APPLICATION_JSON.getMimeType(), "UTF-8")); + entity.setContentEncoding("UTF-8"); + + HttpPost httpPost = new HttpPost(MonitorConfig.ECM_TASK_IMURL.getValue()); + httpPost.setConfig(requestConfig); + httpPost.setEntity(entity); + + CloseableHttpClient httpClient = HttpClients.createDefault(); + CloseableHttpResponse execute = httpClient.execute(httpPost); + String responseStr = EntityUtils.toString(execute.getEntity(), "UTF-8"); + Map map = BDPJettyServerHelper.gson().fromJson(responseStr, Map.class); + logger.info("send index response :{}", map); + Assert.isTrue(!"0".equals(map.get("resultCode")), map.get("resultMsg")); + } + + public static void sendChatbot(ChatbotEntity chatbotEntity) throws IOException { + String json = BDPJettyServerHelper.gson().toJson(chatbotEntity); + StringEntity entity = new StringEntity(json, ContentType.create(ContentType.APPLICATION_JSON.getMimeType(), "UTF-8")); + entity.setContentEncoding("UTF-8"); + HttpPost httpPost = new HttpPost(MonitorConfig.CHATBOT_URL.getValue()); + httpPost.setConfig(RequestConfig.DEFAULT); + httpPost.setEntity(entity); + CloseableHttpResponse execute = HttpClients.createDefault().execute(httpPost); + String responseStr = EntityUtils.toString(execute.getEntity(), "UTF-8"); + Map map = BDPJettyServerHelper.gson().fromJson(responseStr, Map.class); + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java new file mode 100644 index 0000000000..5a099e7b65 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java @@ -0,0 +1,68 @@ +/* + * 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.monitor.scan.app.monitor.until; + +import org.apache.linkis.common.utils.Utils; +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.monitor.scan.utils.log.LogUtils; + +import org.springframework.context.ApplicationContext; +import org.springframework.context.event.ApplicationContextEvent; + +import java.util.*; +import java.util.concurrent.*; + +import scala.concurrent.ExecutionContextExecutorService; + +import org.slf4j.Logger; + +public class ThreadUtils extends ApplicationContextEvent { + + private static final Logger logger = LogUtils.stdOutLogger(); + + public static ExecutionContextExecutorService executors = + Utils.newCachedExecutionContext(5, "alert-pool-thread-", false); + + public ThreadUtils(ApplicationContext source) { + super(source); + } + + public static String run(List cmdList, String shellName) { + FutureTask future = new FutureTask(() -> Utils.exec(cmdList.toArray(new String[2]), -1)); + executors.submit(future); + String msg = ""; + try { + msg = future.get(MonitorConfig.SHELL_TIMEOUT.getValue(), TimeUnit.MINUTES).toString(); + } catch (TimeoutException e) { + // 增加告警提示 + logger.info("超时告警 {}", shellName); + HashMap parms = new HashMap<>(); + parms.put("$shellName", shellName); + Map ecmResourceAlerts = + MonitorAlertUtils.getAlerts(Constants.THREAD_TIME_OUT_IM(), parms); + PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12014")); + } catch (ExecutionException | InterruptedException e) { + logger.error("Thread error msg {}", e.getMessage()); + } + return msg; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml new file mode 100644 index 0000000000..b9c1d19612 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml @@ -0,0 +1,67 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + l.`id`, l.`label_key`, l.`label_value`, l.`label_feature`, + l.`label_value_size`, l.`update_time`, l.`create_time` + + + + s.`id`, s.`instance`, s.`name`, s.`update_time`, s.`create_time` + + + + + + + + + \ No newline at end of file diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml new file mode 100644 index 0000000000..c0ab7f2b22 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml @@ -0,0 +1,53 @@ + + + + + + + + + + + + + + + + `id`, `instance`, `name`, `update_time`, + `create_time` + + + + DELETE FROM linkis_ps_instance_info WHERE instance = #{instance} + + + + + + + \ No newline at end of file diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml new file mode 100644 index 0000000000..d92d999d47 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml @@ -0,0 +1,57 @@ + + + + + + + + + + + + + + + + + + `id`, `label_key`, `label_value`, `label_feature`, + `label_value_size`, `update_time`, `create_time` + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml new file mode 100644 index 0000000000..70d1f30158 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml @@ -0,0 +1,172 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + `id`,`job_req_id`,`submit_user`,`execute_user`,`labels`,`params`,`status`,`error_code`,`created_time`, + `updated_time`,`instances`,`observe_info` + + + + + + + + UPDATE linkis_ps_job_history_group_history + + status = #{targetStatus} + + + + #{element} + + + + + + UPDATE linkis_ps_job_history_group_history + + status = #{targetStatus}, error_code=21304, error_desc='Automatically killed because entrance is dead' + + + created_time >= #{startDate} + AND instances = #{instanceName} + AND + + #{element} + + + LIMIT 5000 + + + + + + + + + + diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala new file mode 100644 index 0000000000..7ee159d936 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala @@ -0,0 +1,49 @@ +/* + * 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.monitor.scan.app + +import org.apache.linkis.monitor.scan.app.factory.MapperFactory +import org.apache.linkis.monitor.scan.app.instance.dao.InstanceInfoDao +import org.apache.linkis.monitor.scan.app.jobhistory.dao.JobHistoryMapper + +import org.springframework.beans.factory.annotation.Autowired +import org.springframework.context.annotation.{ComponentScan, Configuration} + +import javax.annotation.PostConstruct + +/** + * Created by shangda on 2021/11/19. + */ + +@Configuration +@ComponentScan(Array("org.apache.linkis.monitor.scan", "org.apache.linkis.mybatis")) +class LinkisJobHistoryScanSpringConfiguration { + + @Autowired + private var jobHistoryMapper: JobHistoryMapper = _ + + @Autowired + private var instanceInfoMapper: InstanceInfoDao = _ + + @PostConstruct + def init(): Unit = { + MapperFactory.setJobHistoryMapper(jobHistoryMapper) + MapperFactory.setInstanceInfoMapper(instanceInfoMapper) + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala new file mode 100644 index 0000000000..337592bf72 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.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.monitor.scan.app.factory + +import org.apache.linkis.monitor.scan.app.instance.dao.{ + InsLabelRelationDao, + InstanceInfoDao, + InstanceLabelDao +} +import org.apache.linkis.monitor.scan.app.instance.dao.InstanceInfoDao +import org.apache.linkis.monitor.scan.app.jobhistory.dao.JobHistoryMapper + + +object MapperFactory { + // val bmlVersionCleanScanOper = new BmlVersionCleanScanOper + + private var jobHistoryMapper: JobHistoryMapper = _ + + private var instanceInfoMapper: InstanceInfoDao = _ + + private var instanceLabelMapper: InstanceLabelDao = _ + + private var instanceLabelRelationMapper: InsLabelRelationDao = _ + + def getJobHistoryMapper() = jobHistoryMapper + + def setJobHistoryMapper(jobHistoryMapper: JobHistoryMapper) = { + MapperFactory.jobHistoryMapper = jobHistoryMapper + } + + def getInstanceInfoMapper() = instanceInfoMapper + + def setInstanceInfoMapper(instanceInfoMapper: InstanceInfoDao) = { + MapperFactory.instanceInfoMapper = instanceInfoMapper + } + + def getInstanceLabelMapper() = instanceLabelMapper + + def setInstanceLabelMapper(instanceLabelMapper: InstanceLabelDao) = { + MapperFactory.instanceLabelMapper = instanceLabelMapper + } + + def getInsLabelRelationMapper() = instanceLabelRelationMapper + + def setInsLabelRelationMapper(instanceLabelRelationMapper: InsLabelRelationDao) = { + MapperFactory.instanceLabelRelationMapper = instanceLabelRelationMapper + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala new file mode 100644 index 0000000000..b8eff63ec8 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala @@ -0,0 +1,111 @@ +/* + * 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.monitor.scan.app.jobhistory + +import java.util +import java.util.Date + +import org.apache.commons.lang3.StringUtils +import org.apache.linkis.common.utils.{Logging, Utils} +import org.apache.linkis.monitor.scan.app.jobhistory.dao.JobHistoryMapper +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.pac.AbstractDataFetcher + + +class JobHistoryDataFetcher(args: Array[Any], mapper: JobHistoryMapper) + extends AbstractDataFetcher + with Logging { + + /** + * retrieve JobHistory Data starts from startTimeMs and ends at startTimeMs + intervalsMs + * + * @return + */ + /** + * get arguments for querying data + * + * @return + */ + override def getArgs(): Array[Any] = args + + /** + * 1. get Data given some arguments + */ + override def getData(): util.List[scala.Any] = { + if (!args.isInstanceOf[Array[String]]) { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobHistoryDataFetcher. DataType: " + args.getClass.getCanonicalName + ) + } + if (args != null && args.length == 2) { + val start = Utils.tryCatch(args(0).asInstanceOf[String].toLong) { t => + { + logger.error("Failed to get data from DB: Illegal arguments.", t) + throw t + } + } + val end = Utils.tryCatch(args(1).asInstanceOf[String].toLong) { t => + { + logger.error("Failed to get data from DB: Illegal arguments.", t) + throw t + } + } + mapper + .search(null, null, null, new Date(start), new Date(end), null) + .asInstanceOf[util.List[scala.Any]] + } else if (args != null && args.length == 4) { + val start = Utils.tryCatch(args(0).asInstanceOf[String].toLong) { t => + { + logger.error("Failed to get data from DB: Illegal arguments.", t) + throw t + } + } + val end = Utils.tryCatch(args(1).asInstanceOf[String].toLong) { t => + { + logger.error("Failed to get data from DB: Illegal arguments.", t) + throw t + } + } + val id = Utils.tryCatch(args(2).asInstanceOf[String].toLong) { t => + { + logger.error("Failed to get data from DB: Illegal arguments.", t) + throw t + } + } + if (StringUtils.isNotBlank(args(3).asInstanceOf[String]) && args(3).asInstanceOf[String].equals("updated_time")) { + val list = new util.ArrayList[String]() + Constants.DIRTY_DATA_FINISHED_JOB_STATUS_ARRAY.foreach(list.add(_)) + mapper + .searchByCacheAndUpdateTime(id, null, list, new Date(start), new Date(end), null) + .asInstanceOf[util.List[scala.Any]] + } else { + mapper + .searchByCache(id, null, null, new Date(start), new Date(end), null) + .asInstanceOf[util.List[scala.Any]] + } + } else { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobHistoryDataFetcher. Data: " + args + ) + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala new file mode 100644 index 0000000000..f899952c71 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala @@ -0,0 +1,23 @@ +/* + * 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.monitor.scan.app.jobhistory.errorcode + +import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent + + +class JobHistoryErrCodeHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala new file mode 100644 index 0000000000..c613b3d306 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala @@ -0,0 +1,80 @@ +/* + * 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.monitor.scan.app.jobhistory.errorcode + +import java.util + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils +import org.apache.linkis.monitor.scan.core.ob.Observer +import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} +import scala.collection.JavaConverters._ + + +/** + * 针对执行任务返回的错误码进行监控,执行脚本任务时,会记录执行的错误码在数据库中, + * 服务会根据数据库中记录的错误码,来进行告警,如果错误码中包含(11001,11002)即可触发告警 + */ +class JobHistoryErrCodeRule(errorCodes: util.Set[String], hitObserver: Observer) + extends AbstractScanRule(event = new JobHistoryErrCodeHitEvent, observer = hitObserver) + with Logging { + private val scanRuleList = CacheUtils.cacheBuilder + + /** + * if data match the pattern, return true and trigger observer should call isMatched() + * + * @param data + * @return + */ + override def triggerIfMatched(data: util.List[ScannedData]): Boolean = { + + if (!getHitEvent().isRegistered || null == data) { + logger.error("ScanRule is not bind with an observer. Will not be triggered") + return false + } + + val alertData: util.List[JobHistory] = new util.ArrayList[JobHistory]() + for (sd <- data.asScala) { + if (sd != null && sd.getData() != null) { + for (d <- sd.getData().asScala) { + d match { + case history: JobHistory => + if (errorCodes.contains(String.valueOf(history.getErrorCode))) { + alertData.add(history) + } + scanRuleList.put("jobHistoryId", history.getId) + case _ => + logger.warn("Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName) + } + } + } else { + logger.warn("Ignored null scanned data") + } + + } + logger.info("hit " + alertData.size() + " data in one iteration") + if (alertData.size() > 0) { + getHitEvent().notifyObserver(getHitEvent(), alertData) + true + } else { + false + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala new file mode 100644 index 0000000000..a037e0f017 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala @@ -0,0 +1,97 @@ +/* + * 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.monitor.scan.app.jobhistory.errorcode + +import java.util + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.linkis.monitor.scan.utils.alert.ims.{ImsAlertDesc, PooledImsAlertUtils} +import scala.collection.JavaConverters._ + + + +class JobHistoryErrorCodeAlertSender(alerts: util.Map[String, AlertDesc]) + extends Observer + with Logging { + + override def update(e: Event, jobHistoryList: scala.Any): Unit = { + if (!e.isInstanceOf[JobHistoryErrCodeHitEvent]) { + throw new AnomalyScannerException( + 21304, + "Wrong event that triggers JobHistoryErrorCodeAlertSender. Input DataType: " + e.getClass.getCanonicalName + ) + } + if (null == jobHistoryList || !jobHistoryList.isInstanceOf[util.List[_]]) { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobHistoryErrorCodeAlertSender. Input DataType: " + jobHistoryList.getClass.getCanonicalName + ) + } + val toSend = new util.HashMap[String, ImsAlertDesc] + for (a <- jobHistoryList.asInstanceOf[util.List[_]].asScala) { + if (a == null) { + logger.warn("Ignore null input data") + } else if (!a.isInstanceOf[JobHistory]) { + logger.warn("Ignore wrong input data Type : " + a.getClass.getCanonicalName) + } else { + val jobHistory = a.asInstanceOf[JobHistory] + val errorCode = String.valueOf(jobHistory.getErrorCode) + if (alerts.containsKey(errorCode) && alerts.get(errorCode).isInstanceOf[ImsAlertDesc]) { + val alert = if (!toSend.containsKey(errorCode)) { + alerts.get(errorCode).asInstanceOf[ImsAlertDesc] + } else { + toSend.get(errorCode) + } + + var newInfo = if (!toSend.containsKey(errorCode)) { + alert.alertInfo + "\n" + + "[error_code] " + jobHistory.getErrorCode + ", " + jobHistory.getErrorDesc + "\n" + } else { + alert.alertInfo + } + newInfo = newInfo + + "[job-info] " + + "submit-user: " + jobHistory.getSubmitUser + ", " + + "execute-user: " + jobHistory.getExecuteUser + ", " + + "engine_type: " + jobHistory.getEngineType + ", " + + "create_time: " + jobHistory.getCreatedTime + ", " + + "instance: " + jobHistory.getInstances + ". \n" + val newNumHit = alert.numHit + 1 + toSend.put(errorCode, alert.copy(alertInfo = newInfo, numHit = newNumHit)) + } else if (!alerts.containsKey(errorCode)) { + logger.warn("Ignored unregistered error code: " + errorCode) + } else if (!alerts.get(errorCode).isInstanceOf[ImsAlertDesc]) { + logger.warn( + "Ignored invalid alertDesc. DataType: " + alerts + .get(errorCode) + .getClass + .getCanonicalName + ) + } + } + } + for ((_, alert) <- toSend.asScala) { + PooledImsAlertUtils.addAlert(alert) + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala new file mode 100644 index 0000000000..aa564ab335 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala @@ -0,0 +1,112 @@ +/* + * 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.monitor.scan.app.jobhistory.jobtime + +import java.text.MessageFormat +import java.util + +import org.apache.linkis.common.utils.{Logging, Utils} +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.linkis.monitor.scan.utils.alert.ims.{ImsAlertDesc, PooledImsAlertUtils} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + + +class JobTimeExceedAlertSender(alerts: util.Map[String, AlertDesc]) + extends Observer + with Logging { + + private val orderedThresholds: Array[Long] = { + val ret = new ArrayBuffer[Long]() + if (alerts != null) { + for (k <- alerts.keySet().asScala) { + Utils.tryCatch(ret.append(k.toLong)) { t => + logger.warn("Ignored illegal threshold: " + k, t) + false + } + } + } + ret.toArray + } + + override def update(e: Event, jobHistoryList: scala.Any): Unit = { + if (!e.isInstanceOf[JobTimeExceedHitEvent]) { + throw new AnomalyScannerException( + 21304, + "Wrong event that triggers JobTimeExceedAlertSender. Input DataType: " + e.getClass.getCanonicalName + ) + } + if (null == jobHistoryList || !jobHistoryList.isInstanceOf[util.List[_]]) { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobTimeExceedAlertSender. Input DataType: " + jobHistoryList.getClass.getCanonicalName + ) + } + if (orderedThresholds.length == 0) { + logger.warn("Found none legal threshold, will not send any alert: " + this) + return + } + val toSend = new util.HashMap[String, ImsAlertDesc] + for (a <- jobHistoryList.asInstanceOf[util.List[_]].asScala) { + if (a == null) { + logger.warn("Ignore null input data") + } else if (!a.isInstanceOf[JobHistory]) { + logger.warn("Ignore wrong input data Type : " + a.getClass.getCanonicalName) + } else { + val jobHistory = a.asInstanceOf[JobHistory] + val elapse = System.currentTimeMillis() - jobHistory.getCreatedTime.getTime + var ts = 0L + for (t <- orderedThresholds) { // search max threshold that is smaller than elapse + if (elapse >= t) { + ts = t + } else { + + } + } + val name = ts.toString + val alert = if (!toSend.containsKey(name)) { + alerts + .get(name) + .asInstanceOf[ + ImsAlertDesc + ] + } else { + toSend.get(name) + } + + val newInfo = MessageFormat.format("[Linkis任务信息]您好,您在Linkis/DSS提交的任务(任务ID:{0}),已经运行超过{1}h," + + "请关注是否任务正常,如果不正常您可以到Linkis/DSS管理台进行任务的kill,集群信息为BDAP({2})。详细解决方案见Q47:{3} " + , jobHistory.getId, (elapse / 1000 / 60 / 60).toString, jobHistory.getInstances, MonitorConfig.SOLUTION_URL.getValue) + + val newNumHit = alert.numHit + 1 + val receiver = new util.HashSet[String]() + receiver.add(jobHistory.getSubmitUser) + receiver.add(jobHistory.getExecuteUser) + receiver.addAll(alert.alertReceivers) + val ImsAlertDesc = alert.copy(alertInfo = newInfo, alertReceivers = receiver, numHit = newNumHit) + PooledImsAlertUtils.addAlert(ImsAlertDesc) + + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala new file mode 100644 index 0000000000..b7b883e09b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala @@ -0,0 +1,23 @@ +/* + * 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.monitor.scan.app.jobhistory.jobtime + +import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent + + +class JobTimeExceedHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala new file mode 100644 index 0000000000..b91d605d4e --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala @@ -0,0 +1,103 @@ +/* + * 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.monitor.scan.app.jobhistory.jobtime + +import java.util +import java.util.Locale + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.Observer +import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} + +import scala.collection.JavaConverters._ + +/** + * 针对执行任务状态进行监控,扫描12小时之外,24小时之内的的数据, + * 如果规则范围内,有数据状态是(Inited,WaitForRetry,Scheduled,Running)其中之一,则触发告警 + */ +class JobTimeExceedRule(thresholds: util.Set[String], hitObserver: Observer) + extends AbstractScanRule(event = new JobTimeExceedHitEvent, observer = hitObserver) + with Logging { + + private val threshold: Long = { + if (thresholds == null) { + throw new AnomalyScannerException(21304, "thresholds should not be null") + } + var t = Long.MaxValue + for (k <- thresholds.asScala) { + if (k != null) { + if (t > k.toLong) { + t = k.toLong + } + } else { + logger.warn("ignored null input") + } + } + t + } + + private val scanRuleList = CacheUtils.cacheBuilder + + /** + * if data match the pattern, return true and trigger observer should call isMatched() + * + * @param data + * @return + */ + override def triggerIfMatched(data: util.List[ScannedData]): Boolean = { + if (!getHitEvent.isRegistered || data == null) { + logger.error("ScanRule is not bind with an observer. Will not be triggered") + return false + } + val alertData: util.List[JobHistory] = new util.ArrayList[JobHistory]() + for (sd <- data.asScala) { + if (sd != null && sd.getData() != null) { + for (d <- sd.getData().asScala) { + if (d.isInstanceOf[JobHistory]) { + val jobHistory = d.asInstanceOf[JobHistory] + val status = jobHistory.getStatus.toUpperCase(Locale.getDefault) + if (Constants.DIRTY_DATA_UNFINISHED_JOB_STATUS.contains(status)) { + val elapse = System.currentTimeMillis() - jobHistory.getCreatedTime.getTime + if (elapse / 1000 >= threshold) { + alertData.add(d.asInstanceOf[JobHistory]) + } + } + scanRuleList.put("jobhistoryScan", jobHistory.getId) + } else { + logger.warn("Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName) + } + } + } else { + logger.warn("Ignored null scanned data") + } + + } + logger.info("hit " + alertData.size() + " data in one iteration") + if (alertData.size() > 0) { + getHitEvent.notifyObserver(getHitEvent, alertData) + true + } else { + false + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala new file mode 100644 index 0000000000..77bc29ffe2 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala @@ -0,0 +1,69 @@ +/* + * 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.monitor.scan.app.jobhistory.labels + +import java.util + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.linkis.monitor.scan.utils.alert.ims.{PooledImsAlertUtils, UserLabelAlertUtils} +import org.apache.linkis.server.BDPJettyServerHelper + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +class JobHistoryLabelsAlertSender() extends Observer with Logging { + + override def update(e: Event, jobHistoryList: scala.Any): Unit = { + if (!e.isInstanceOf[JobHistoryLabelsHitEvent]) { + throw new AnomalyScannerException( + 21304, + "Wrong event that triggers JobHistoryLabelsAlertSender. Input DataType: " + e.getClass.getCanonicalName + ) + } + if (null == jobHistoryList || !jobHistoryList.isInstanceOf[util.List[_]]) { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobHistoryLabelsAlertSender. Input DataType: " + jobHistoryList.getClass.getCanonicalName + ) + } + val toSend = new ArrayBuffer[String] + for (a <- jobHistoryList.asInstanceOf[util.List[_]].asScala) { + if (a == null) { + logger.warn("Ignore null input data") + } else if (!a.isInstanceOf[JobHistory]) { + logger.warn("Ignore wrong input data Type : " + a.getClass.getCanonicalName) + } else { + val jobHistory = a.asInstanceOf[JobHistory] + toSend.append(jobHistory.getLabels) + } + } + for (str <- toSend.distinct) { + val labelsMap: util.Map[String, String] = + BDPJettyServerHelper.gson.fromJson(str, classOf[java.util.Map[String, String]]) + val alerts: util.Map[String, AlertDesc] = + UserLabelAlertUtils.getAlerts(Constants.USER_LABEL_MONITOR, labelsMap.get("userCreator")) + PooledImsAlertUtils.addAlert(alerts.get("12010")); + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsHitEvent.scala new file mode 100644 index 0000000000..6f47136acd --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsHitEvent.scala @@ -0,0 +1,22 @@ +/* + * 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.monitor.scan.app.jobhistory.labels + +import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent + +class JobHistoryLabelsHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala new file mode 100644 index 0000000000..6308ac28b0 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala @@ -0,0 +1,112 @@ +/* + * 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.monitor.scan.app.jobhistory.labels + +import java.util + +import com.google.common.collect.HashBiMap +import org.apache.commons.lang3.StringUtils +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.Observer +import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} +import org.apache.linkis.server.BDPJettyServerHelper + +import scala.collection.JavaConverters._ + +/** + * 对前20分钟内的执行数据进行扫描,对数据的labels字段进行判断, + * 判断依据monitor配置(linkis.monitor.jobhistory.userLabel.tenant) + */ +class JobHistoryLabelsRule(hitObserver: Observer) + extends AbstractScanRule(event = new JobHistoryLabelsHitEvent, observer = hitObserver) + with Logging { + + private val scanRuleList = CacheUtils.cacheBuilder + + /** + * if data match the pattern, return true and trigger observer should call isMatched() + * + * @param data + * @return + */ + override def triggerIfMatched(data: util.List[ScannedData]): Boolean = { + if (!getHitEvent.isRegistered || null == data) { + logger.error("ScanRule is not bind with an observer. Will not be triggered") + return false + } + val alertData: util.List[JobHistory] = new util.ArrayList[JobHistory]() + for (sd <- data.asScala) { + if (sd != null && sd.getData() != null) { + for (d <- sd.getData().asScala) { + if (d.isInstanceOf[JobHistory]) { + logger.info(" start jobhistory user label rule data : {}", d) + val jobHistory = d.asInstanceOf[JobHistory] + val labels = jobHistory.getLabels + val labelsMap: util.Map[String, String] = + BDPJettyServerHelper.gson.fromJson(labels, classOf[java.util.Map[String, String]]) + val userCreator = labelsMap.get("userCreator"); + val tenant = labelsMap.get("tenant"); + if (StringUtils.isNotBlank(userCreator)) { + val configMap = BDPJettyServerHelper.gson.fromJson( + Constants.USER_LABEL_TENANT.getValue, + classOf[java.util.Map[String, String]] + ) + // 当任务的creator是qualitis(或dops)时,tenant不是qualitis发出告警 + val listIterator = configMap.keySet.iterator + while ({ + listIterator.hasNext + }) { + val next = listIterator.next + if (userCreator.contains(next)) { + val value = configMap.get(next) + if (!value.equals(tenant)) { + alertData.add(d.asInstanceOf[JobHistory]) + } + } + } + // 当任务代理tenant:Qualitis标签,但是creator不是qualitis标签也进行告警 + if (configMap.values().contains(tenant)) { + val bimap: HashBiMap[String, String] = HashBiMap.create(configMap) + val key = bimap.inverse().get(tenant) + if (!key.contains(userCreator)) { + alertData.add(d.asInstanceOf[JobHistory]) + } + } + } + scanRuleList.put("jobHistoryId", jobHistory.getId) + } else { + logger.warn("Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName) + } + } + } else { + logger.warn("Ignored null scanned data") + } + } + logger.info("hit " + alertData.size() + " data in one iteration") + if (alertData.size() > 0) { + getHitEvent.notifyObserver(getHitEvent, alertData) + true + } else { + false + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala new file mode 100644 index 0000000000..1912acf8b4 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala @@ -0,0 +1,79 @@ +/* + * 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.monitor.scan.app.jobhistory.runtime + +import org.apache.commons.lang3.StringUtils +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.Observer +import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} + +import java.util +import scala.collection.JavaConverters._ + +/** + * 对前20分钟内的执行数据进行扫描, + * 1.数据的ObserveInfo字段进行判断是否为空, + * 2.任务状态已经完成(Succeed,Failed,Cancelled,Timeout,ALL) + * 满足条件即可触发告警 + */ +class CommonJobRunTimeRule(hitObserver: Observer) + extends AbstractScanRule(event = new JobHistoryRunTimeHitEvent, observer = hitObserver) + with Logging { + + /** + * if data match the pattern, return true and trigger observer should call isMatched() + * + * @param data + * @return + */ + override def triggerIfMatched(data: util.List[ScannedData]): Boolean = { + if (!getHitEvent.isRegistered || null == data) { + logger.error("ScanRule is not bind with an observer. Will not be triggered") + return false + } + val alertData: util.List[JobHistory] = new util.ArrayList[JobHistory]() + for (sd <- data.asScala) { + if (sd != null && sd.getData() != null) { + for (d <- sd.getData().asScala) { + d match { + case jobHistory: JobHistory => + if (Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains(jobHistory.getStatus.toUpperCase()) + &&StringUtils.isNotBlank(jobHistory.getObserveInfo)) { + alertData.add(jobHistory) + } else { + logger.warn("jobHistory is not completely , taskid :" + d) + } + case _ => + } + } + } else { + logger.warn("Ignored null scanned data") + } + } + logger.info("hit " + alertData.size() + " data in one iteration") + if (alertData.size() > 0) { + getHitEvent.notifyObserver(getHitEvent, alertData) + true + } else { + false + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala new file mode 100644 index 0000000000..6aca4c38ff --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala @@ -0,0 +1,88 @@ +/* + * 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.monitor.scan.app.jobhistory.runtime + +import java.net.InetAddress +import java.text.SimpleDateFormat +import java.util +import java.util.Date + +import org.apache.commons.collections.MapUtils +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.utils.alert.ims.{MonitorAlertUtils, PooledImsAlertUtils} +import org.apache.linkis.server.BDPJettyServerHelper + +import scala.collection.JavaConverters._ + + +class CommonRunTimeAlertSender() + extends Observer + with Logging { + private val dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + + override def update(e: Event, jobHistoryList: scala.Any): Unit = { + if (!e.isInstanceOf[JobHistoryRunTimeHitEvent]) { + throw new AnomalyScannerException( + 21304, + "Wrong event that triggers JobHistoryErrorCodeAlertSender. Input DataType: " + e.getClass.getCanonicalName + ) + } + if (!jobHistoryList.isInstanceOf[util.List[_]] || null == jobHistoryList) { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobHistoryErrorCodeAlertSender. Input DataType: " + jobHistoryList.getClass.getCanonicalName + ) + } + for (a <- jobHistoryList.asInstanceOf[util.List[_]].asScala) { + if (a == null) { + logger.warn("Ignore null input data") + } else if (!a.isInstanceOf[JobHistory]) { + logger.warn("Ignore wrong input data Type : " + a.getClass.getCanonicalName) + } else { + val jobHistory = a.asInstanceOf[JobHistory] + val observeInfoMap = BDPJettyServerHelper.gson.fromJson(jobHistory.getObserveInfo, classOf[java.util.Map[String, String]]) + val extraMap = MapUtils.getMap(observeInfoMap, "extra") + observeInfoMap.put("title", extraMap.get("title").toString + ",任务id:" + jobHistory.getId + ",执行结果 :" + jobHistory.getStatus) + observeInfoMap.put("$detail", extraMap.get("detail").toString + ",执行结果 :" + jobHistory.getStatus) + observeInfoMap.put("$submitUser", jobHistory.getSubmitUser) + observeInfoMap.put("$status", jobHistory.getStatus) + observeInfoMap.put("$id", jobHistory.getId.toString) + observeInfoMap.put("$date", dateFormat.format(new Date())) + var alterSysInfo = "" + if (null != extraMap.get("alterSysInfo")) { + alterSysInfo = extraMap.get("alterSysInfo").toString + } + observeInfoMap.put("$sysid", alterSysInfo) + var alterObject = "" + if (null != extraMap.get("alterObject")) { + alterObject = extraMap.get("alterObject").toString + } + observeInfoMap.put("$object", alterObject) + observeInfoMap.put("$ip", InetAddress.getLocalHost.getHostAddress) + observeInfoMap.remove("taskId") + observeInfoMap.remove("extra") + val alters = MonitorAlertUtils.getAlertsByDss(Constants.JOB_RESULT_IM, observeInfoMap) + PooledImsAlertUtils.addAlert(alters.get("12016")) + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeHitEvent.scala new file mode 100644 index 0000000000..641ccc2a7d --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeHitEvent.scala @@ -0,0 +1,22 @@ +/* + * 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.monitor.scan.app.jobhistory.runtime + +import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent + +class CommonRunTimeHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala new file mode 100644 index 0000000000..b9f35ce7c7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala @@ -0,0 +1,73 @@ +/* + * 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.monitor.scan.app.jobhistory.runtime + +import java.util + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.utils.alert.ims.{MonitorAlertUtils, PooledImsAlertUtils} + +import scala.collection.JavaConverters._ + +/** + * 对前20分钟内的执行数据进行扫描,对已结束的任务进行判断, + * 1.jobhistory中的parm字段中包含(task.notification.conditions) + * 2.执行任务的结果是(Succeed,Failed,Cancelled,Timeout,ALL)其中任意一个,则触发告警 + * 3.job的结果是已经结束 + * 同时满足上述三个条件即可触发告警 + */ +class JobHistoryRunTimeAlertSender() + extends Observer + with Logging { + + override def update(e: Event, jobHistroyList: scala.Any): Unit = { + if (!e.isInstanceOf[JobHistoryRunTimeHitEvent]) { + throw new AnomalyScannerException( + 21304, + "Wrong event that triggers JobHistoryErrorCodeAlertSender. Input DataType: " + e.getClass.getCanonicalName + ) + } + if (null == jobHistroyList || !jobHistroyList.isInstanceOf[util.List[_]]) { + throw new AnomalyScannerException( + 21304, + "Wrong input for JobHistoryErrorCodeAlertSender. Input DataType: " + jobHistroyList.getClass.getCanonicalName + ) + } + for (a <- jobHistroyList.asInstanceOf[util.List[_]].asScala) { + if (a == null) { + logger.warn("Ignore null input data") + } else if (!a.isInstanceOf[JobHistory]) { + logger.warn("Ignore wrong input data Type : " + a.getClass.getCanonicalName) + } else { + // 您的任务ID 1234 执行完成,最终状态为:成功、失败、取消 + val jobHistory = a.asInstanceOf[JobHistory] + val status = jobHistory.getStatus + val replaceParm: util.HashMap[String, String] = new util.HashMap[String, String] + replaceParm.put("$id", String.valueOf(jobHistory.getId)) + replaceParm.put("$status", status) + replaceParm.put("$alteruser", jobHistory.getSubmitUser) + val alters = MonitorAlertUtils.getAlerts(Constants.JOB_RESULT_IM, replaceParm) + PooledImsAlertUtils.addAlert(alters.get("12015")) + } + } + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeHitEvent.scala new file mode 100644 index 0000000000..40e837d66c --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeHitEvent.scala @@ -0,0 +1,22 @@ +/* + * 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.monitor.scan.app.jobhistory.runtime + +import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent + +class JobHistoryRunTimeHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala new file mode 100644 index 0000000000..e136ea46d0 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala @@ -0,0 +1,82 @@ +/* + * 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.monitor.scan.app.jobhistory.runtime + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory +import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.core.ob.Observer +import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} +import org.apache.linkis.protocol.utils.TaskUtils +import org.apache.linkis.server.BDPJettyServerHelper + +import java.util +import scala.collection.JavaConverters._ + + +class JobHistoryRunTimeRule(hitObserver: Observer) + extends AbstractScanRule(event = new JobHistoryRunTimeHitEvent, observer = hitObserver) + with Logging { + private val scanRuleList = CacheUtils.cacheBuilder + + /** + * if data match the pattern, return true and trigger observer should call isMatched() + * + * @param data + * @return + */ + override def triggerIfMatched(data: util.List[ScannedData]): Boolean = { + if (null == data || !getHitEvent.isRegistered) { + logger.error("ScanRule is not bind with an observer. Will not be triggered") + return false + } + val alertData: util.List[JobHistory] = new util.ArrayList[JobHistory]() + for (sd <- data.asScala) { + if (sd != null && sd.getData() != null) { + for (d <- sd.getData().asScala) { + d match { + case jobHistory: JobHistory => + if (Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains(jobHistory.getStatus.toUpperCase())) { + val parmsMap: util.Map[String, scala.AnyRef] = BDPJettyServerHelper.gson.fromJson(jobHistory.getParams, classOf[util.Map[String, scala.AnyRef]]) + val runtimeMap = TaskUtils.getRuntimeMap(parmsMap) + if (runtimeMap.containsKey("task.notification.conditions") && + Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains(String.valueOf(runtimeMap.get("task.notification.conditions")).toUpperCase())) { + alertData.add(jobHistory) + } + } else { + logger.warn("Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName) + } + scanRuleList.put("jobHistoryId", jobHistory.getId) + case _ => + } + } + } else { + logger.warn("Ignored null scanned data") + } + } + logger.info("hit " + alertData.size() + " data in one iteration") + if (alertData.size() > 0) { + getHitEvent.notifyObserver(getHitEvent, alertData) + true + } else { + false + } + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala new file mode 100644 index 0000000000..8813aba210 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala @@ -0,0 +1,117 @@ +/* + * 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.monitor.scan.client + +import org.apache.linkis.httpclient.authentication.AuthenticationStrategy +import org.apache.linkis.httpclient.dws.authentication.StaticAuthenticationStrategy +import org.apache.linkis.httpclient.dws.config.{DWSClientConfig, DWSClientConfigBuilder} +import org.apache.linkis.httpclient.response.Result +import org.apache.linkis.monitor.scan.request.{EmsListAction, EntranceTaskAction, MonitorResourceAction} +import org.apache.linkis.monitor.scan.response.EntranceTaskResult +import org.apache.linkis.ujes.client.response.EmsListResult + +import java.io.Closeable +import java.util.concurrent.TimeUnit + +abstract class MonitorHTTPClient extends Closeable { + + protected[client] def executeJob(ujesJobAction: MonitorResourceAction): Result + + def list(emsListAction: EmsListAction): EmsListResult = { + executeJob(emsListAction).asInstanceOf[EmsListResult] + } + + def entranList(entranceTaskAction: EntranceTaskAction): EntranceTaskResult = { + executeJob(entranceTaskAction).asInstanceOf[EntranceTaskResult] + } + +} + +object MonitorHTTPClient { + + def apply(clientConfig: DWSClientConfig): MonitorHTTPClient = new MonitorHTTPClientClientImpl( + clientConfig + ) + + def apply(serverUrl: String): MonitorHTTPClient = apply(serverUrl, 30000, 10) + + def apply(serverUrl: String, readTimeout: Int, maxConnection: Int): MonitorHTTPClient = + apply(serverUrl, readTimeout, maxConnection, new StaticAuthenticationStrategy, "v1") + + def apply( + serverUrl: String, + readTimeout: Int, + maxConnection: Int, + authenticationStrategy: AuthenticationStrategy, + dwsVersion: String + ): MonitorHTTPClient = { + val clientConfig = DWSClientConfigBuilder + .newBuilder() + .addServerUrl(serverUrl) + .connectionTimeout(30000) + .discoveryEnabled(false) + .loadbalancerEnabled(false) + .maxConnectionSize(maxConnection) + .retryEnabled(false) + .readTimeout(readTimeout) + .setAuthenticationStrategy(authenticationStrategy) + .setDWSVersion(dwsVersion) + .build() + apply(clientConfig) + } + + def getDiscoveryClient(serverUrl: String): MonitorHTTPClient = + getDiscoveryClient(serverUrl, 30000, 10) + + def getDiscoveryClient( + serverUrl: String, + readTimeout: Int, + maxConnection: Int + ): MonitorHTTPClient = + getDiscoveryClient( + serverUrl, + readTimeout, + maxConnection, + new StaticAuthenticationStrategy, + "v1" + ) + + def getDiscoveryClient( + serverUrl: String, + readTimeout: Int, + maxConnection: Int, + authenticationStrategy: AuthenticationStrategy, + dwsVersion: String + ): MonitorHTTPClient = { + val clientConfig = DWSClientConfigBuilder + .newBuilder() + .addServerUrl(serverUrl) + .connectionTimeout(30000) + .discoveryEnabled(true) + .discoveryFrequency(1, TimeUnit.MINUTES) + .loadbalancerEnabled(true) + .maxConnectionSize(maxConnection) + .retryEnabled(false) + .readTimeout(readTimeout) + .setAuthenticationStrategy(authenticationStrategy) + .setDWSVersion(dwsVersion) + .build() + apply(clientConfig) + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClientClientImpl.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClientClientImpl.scala new file mode 100644 index 0000000000..b2b43d7758 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClientClientImpl.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.monitor.scan.client + +import org.apache.linkis.httpclient.dws.DWSHttpClient +import org.apache.linkis.httpclient.dws.config.DWSClientConfig +import org.apache.linkis.httpclient.request.Action +import org.apache.linkis.httpclient.response.Result +import org.apache.linkis.monitor.scan.request.MonitorResourceAction + +class MonitorHTTPClientClientImpl(clientConfig: DWSClientConfig) extends MonitorHTTPClient { + + private val dwsHttpClient = + new DWSHttpClient(clientConfig, "Linkis-MonitorResource-Execution-Thread") + + override protected[client] def executeJob(ujesJobAction: MonitorResourceAction): Result = + ujesJobAction match { + + case action: Action => dwsHttpClient.execute(action) + + } + + override def close(): Unit = dwsHttpClient.close() +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClient.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClient.scala new file mode 100644 index 0000000000..bf605de28b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClient.scala @@ -0,0 +1,112 @@ +/* + * 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.monitor.scan.client + +import org.apache.linkis.httpclient.authentication.AuthenticationStrategy +import org.apache.linkis.httpclient.dws.authentication.StaticAuthenticationStrategy +import org.apache.linkis.httpclient.dws.config.{DWSClientConfig, DWSClientConfigBuilder} +import org.apache.linkis.httpclient.response.Result +import org.apache.linkis.monitor.scan.request.{EmsListAction, MonitorResourceAction} +import org.apache.linkis.ujes.client.response.EmsListResult + +import java.io.Closeable +import java.util.concurrent.TimeUnit + +abstract class MonitorResourceClient extends Closeable { + + protected[client] def executeJob(ujesJobAction: MonitorResourceAction): Result + + def list(jobListAction: EmsListAction): EmsListResult = { + executeJob(jobListAction).asInstanceOf[EmsListResult] + } + +} + +object MonitorResourceClient { + + def apply(clientConfig: DWSClientConfig): MonitorResourceClient = new MonitorResourceClientImpl( + clientConfig + ) + + def apply(serverUrl: String): MonitorResourceClient = apply(serverUrl, 30000, 10) + + def apply(serverUrl: String, readTimeout: Int, maxConnection: Int): MonitorResourceClient = + apply(serverUrl, readTimeout, maxConnection, new StaticAuthenticationStrategy, "v1") + + def apply( + serverUrl: String, + readTimeout: Int, + maxConnection: Int, + authenticationStrategy: AuthenticationStrategy, + dwsVersion: String + ): MonitorResourceClient = { + val clientConfig = DWSClientConfigBuilder + .newBuilder() + .addServerUrl(serverUrl) + .connectionTimeout(30000) + .discoveryEnabled(false) + .loadbalancerEnabled(false) + .maxConnectionSize(maxConnection) + .retryEnabled(false) + .readTimeout(readTimeout) + .setAuthenticationStrategy(authenticationStrategy) + .setDWSVersion(dwsVersion) + .build() + apply(clientConfig) + } + + def getDiscoveryClient(serverUrl: String): MonitorResourceClient = + getDiscoveryClient(serverUrl, 30000, 10) + + def getDiscoveryClient( + serverUrl: String, + readTimeout: Int, + maxConnection: Int + ): MonitorResourceClient = + getDiscoveryClient( + serverUrl, + readTimeout, + maxConnection, + new StaticAuthenticationStrategy, + "v1" + ) + + def getDiscoveryClient( + serverUrl: String, + readTimeout: Int, + maxConnection: Int, + authenticationStrategy: AuthenticationStrategy, + dwsVersion: String + ): MonitorResourceClient = { + val clientConfig = DWSClientConfigBuilder + .newBuilder() + .addServerUrl(serverUrl) + .connectionTimeout(30000) + .discoveryEnabled(true) + .discoveryFrequency(1, TimeUnit.MINUTES) + .loadbalancerEnabled(true) + .maxConnectionSize(maxConnection) + .retryEnabled(false) + .readTimeout(readTimeout) + .setAuthenticationStrategy(authenticationStrategy) + .setDWSVersion(dwsVersion) + .build() + apply(clientConfig) + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClientImpl.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClientImpl.scala new file mode 100644 index 0000000000..8e740a94ea --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorResourceClientImpl.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.monitor.scan.client + +import org.apache.linkis.httpclient.dws.DWSHttpClient +import org.apache.linkis.httpclient.dws.config.DWSClientConfig +import org.apache.linkis.httpclient.request.Action +import org.apache.linkis.httpclient.response.Result +import org.apache.linkis.monitor.scan.request.MonitorResourceAction + +class MonitorResourceClientImpl(clientConfig: DWSClientConfig) extends MonitorResourceClient { + + private val dwsHttpClient = + new DWSHttpClient(clientConfig, "Linkis-MonitorResource-Execution-Thread") + + override protected[client] def executeJob(ujesJobAction: MonitorResourceAction): Result = + ujesJobAction match { + + case action: Action => dwsHttpClient.execute(action) + + } + + override def close(): Unit = dwsHttpClient.close() +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala new file mode 100644 index 0000000000..f3e6d74c48 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala @@ -0,0 +1,113 @@ +/* + * 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.monitor.scan.constants + +import org.apache.linkis.common.conf.CommonVars + + +object Constants { + + val ALERT_IMS_URL = CommonVars.properties.getProperty( + "wds.linkis.alert.url", + "http://127.0.0.1:10812/ims_data_access/send_alarm.do" + ) + + val ALERT_PROPS_FILE_PATH = CommonVars.properties.getProperty( + "wds.linkis.alert.ims.file.path", + "linkis-et-monitor-ims.properties" + ) + + val ALERT_IMS_MAX_LINES = CommonVars[Int]("wds.linkis.alert.ims.max.lines", 8).getValue + + val SCAN_INTERVALS_SECONDS = + CommonVars[Long]("wds.linkis.errorcode.scanner.interval.seconds", 1 * 60 * 60).getValue + + val MAX_INTERVALS_SECONDS = + CommonVars[Long]("wds.linkis.errorcode.scanner.max.interval.seconds", 1 * 60 * 60).getValue + + val ALERT_SUB_SYSTEM_ID = + CommonVars.properties.getProperty("wds.linkis.alert.ims.sub_system_id", "5435") + + val ALERT_DEFAULT_RECEIVERS = CommonVars.properties + .getProperty("wds.linkis.alert.receiver.default", "") + .split(",") + .toSet[String] + + val SCAN_PREFIX_ERRORCODE = "jobhistory.errorcode." + val SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC = "jobhistory.unfinished.time.exceed.sec." + + val SCAN_RULE_UNFINISHED_JOB_STATUS = + "Inited,WaitForRetry,Scheduled,Running".split(",").map(s => s.toUpperCase()) + + val DIRTY_DATA_EUREKA_DELETE_INSTANCE_URL = + CommonVars.apply("wds.linkis.eureka.defaultZone", "http://localhost:20303").getValue + + val DIRTY_DATA_EUREKA_DELETE_PATH = CommonVars + .apply("wds.linkis.dirty.data.eureka.delete.path", "/apps/{springName}/{instance}") + .getValue + + val DIRTY_DATA_UNFINISHED_JOB_STATUS = + "Inited,WaitForRetry,Scheduled,Running".split(",").map(s => s.toUpperCase()) + + val DIRTY_DATA_JOB_TARGET_STATUS = "Cancelled" + + val DIRTY_DATA_ENTRANCE_APPLICATIONNAME = + CommonVars("wds.linkis.entrance.spring.name", "linkis-cg-entrance").getValue + + val MODIFY_DB_DATA_DAYS = CommonVars("wds.linkis.dirty.data.modify.db.days", 1).getValue + val ALERT_RESOURCE_MONITOR = "ecm.resource.monitor.im." + + val LINKIS_API_VERSION: CommonVars[String] = + CommonVars[String]("wds.linkis.bml.api.version", "v1") + + val AUTH_TOKEN_KEY: CommonVars[String] = + CommonVars[String]("wds.linkis.bml.auth.token.key", "Validation-Code") + + val AUTH_TOKEN_VALUE: CommonVars[String] = + CommonVars[String]("wds.linkis.bml.auth.token.value", "BML-AUTH") + + val CONNECTION_MAX_SIZE: CommonVars[Int] = + CommonVars[Int]("wds.linkis.bml.connection.max.size", 10) + + val CONNECTION_TIMEOUT: CommonVars[Int] = + CommonVars[Int]("wds.linkis.bml.connection.timeout", 5 * 60 * 1000) + + val CONNECTION_READ_TIMEOUT: CommonVars[Int] = + CommonVars[Int]("wds.linkis.bml.connection.read.timeout", 10 * 60 * 1000) + + val AUTH_TOKEN_KEY_SHORT_NAME = "tokenKey" + val AUTH_TOKEN_VALUE_SHORT_NAME = "tokenValue" + val CONNECTION_MAX_SIZE_SHORT_NAME = "maxConnection" + val CONNECTION_TIMEOUT_SHORT_NAME = "connectTimeout" + val CONNECTION_READ_TIMEOUT_SHORT_NAME = "readTimeout" + val CLIENT_NAME_SHORT_NAME = "clientName" + val USER_LABEL_MONITOR = "jobhistory.label.monitor.im." + + val USER_LABEL_TENANT: CommonVars[String] = + CommonVars[String]("linkis.monitor.jobhistory.userLabel.tenant", "{}") + + val USER_RESOURCE_MONITOR = "user.mode.monitor.im." + val BML_CLEAR_IM = "bml.clear.monitor.im." + val THREAD_TIME_OUT_IM = "thread.monitor.timeout.im." + val JOB_RESULT_IM = "jobhistory.result.monitor.im." + + val DIRTY_DATA_FINISHED_JOB_STATUS = + "Succeed,Failed,Cancelled,Timeout,ALL".split(",").map(s => s.toUpperCase()) + val DIRTY_DATA_FINISHED_JOB_STATUS_ARRAY = "Succeed,Failed,Cancelled,Timeout".split(",") + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala new file mode 100644 index 0000000000..8ff3755747 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala @@ -0,0 +1,24 @@ +/* + * 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.monitor.scan.constants + + +object ScanOperatorEnum extends Enumeration { + type ScanOperatorEnum = Value + val BML_VERSION, JOB_HISTORY = Value +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala new file mode 100644 index 0000000000..60322c5814 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala @@ -0,0 +1,29 @@ +/* + * 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.monitor.scan.core.ob + + +trait Event { + def isRegistered: Boolean + + def register(observer: Observer): Unit + + def unRegister(observer: Observer): Unit + + def notifyObserver(event: Event, message: Any): Unit +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala new file mode 100644 index 0000000000..58849dd06c --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala @@ -0,0 +1,27 @@ +/* + * 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.monitor.scan.core.ob + + +trait Observer { + + /** + * Observer Pattern + */ + def update(event: Event, msg: Any): Unit +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/SingleObserverEvent.java b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/SingleObserverEvent.java new file mode 100644 index 0000000000..2e7a6127bc --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/SingleObserverEvent.java @@ -0,0 +1,43 @@ +/* + * 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.monitor.scan.core.ob; + + +public class SingleObserverEvent implements Event { + private Observer observer; + + @Override + public boolean isRegistered() { + return observer != null; + } + + @Override + public void register(Observer observer) { + this.observer = observer; + } + + @Override + public void unRegister(Observer observer) { + this.observer = null; + } + + @Override + public void notifyObserver(Event event, Object message) { + observer.update(event, message); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala new file mode 100644 index 0000000000..02ad320c55 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala @@ -0,0 +1,30 @@ +/* + * 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.monitor.scan.core.pac + + +abstract class AbstractDataFetcher(customName: String = "") extends DataFetcher { + + private val name: String = if (!customName.isEmpty) { + customName + } else { + this.getClass.getName + "@" + Integer.toHexString(this.hashCode) + } + + def getName(): String = this.name +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala new file mode 100644 index 0000000000..8d6762c562 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala @@ -0,0 +1,52 @@ +/* + * 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.monitor.scan.core.pac + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.core.ob.Observer + + +abstract class AbstractScanRule(customName: String = "", event: Event, observer: Observer) + extends ScanRule + with Logging { + event.register(observer) + + private val name: String = if (!customName.isEmpty) { + customName + } else { + this.getClass.getName + "@" + Integer.toHexString(this.hashCode) + } + + def getName(): String = this.name + + /** + * register an observer to trigger if this rule is matched + * + * @param observer + */ + override def addObserver(observer: Observer): Unit = event.register(observer) + + /** + * return registered event + * + * @return + */ + override def getHitEvent(): Event = event + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala new file mode 100644 index 0000000000..02f32a3fc5 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala @@ -0,0 +1,27 @@ +/* + * 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.monitor.scan.core.pac + +import java.util + + +class BaseScannedData(owner: String, data: util.List[scala.Any]) extends ScannedData { + override def getOwner(): String = this.owner + + override def getData(): util.List[scala.Any] = this.data +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/DataFetcher.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/DataFetcher.scala new file mode 100644 index 0000000000..8ac2467684 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/DataFetcher.scala @@ -0,0 +1,40 @@ +/* + * 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.monitor.scan.core.pac + +import java.util + +/** + * ScanOperator should encapsulate lower-level client for accessing data from an arbitrary + * datasource. e.g. if we want to scan a DB table. Then operator should encapsulate a DAO + */ +trait DataFetcher { + def getName(): String + + /** + * get arguments for querying data + * + * @return + */ + def getArgs(): Array[scala.Any] + + /** + * make a query to mysql/hive etc. given args + */ + def getData(): util.List[scala.Any] +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala new file mode 100644 index 0000000000..d38c02dea7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala @@ -0,0 +1,43 @@ +/* + * 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.monitor.scan.core.pac + +import java.util +import java.util.concurrent.LinkedBlockingDeque + + +class ScanBuffer { + val buffer: LinkedBlockingDeque[ScannedData] = new LinkedBlockingDeque[ScannedData] + + def write(data: ScannedData): Unit = buffer.add(data) + + def write(data: util.List[ScannedData]): Unit = buffer.addAll(data) + + def drain(maxSize: Int = -1): util.List[ScannedData] = { + val ret = new util.LinkedList[ScannedData] + val realSize = if (maxSize < 0) { + buffer.size + } else { + maxSize + } + buffer.drainTo(ret, realSize) + return ret + } + + def size(): Int = buffer.size() +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanRule.scala new file mode 100644 index 0000000000..8f8c3442bb --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanRule.scala @@ -0,0 +1,50 @@ +/* + * 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.monitor.scan.core.pac + +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.core.ob.Observer + +import java.util + +trait ScanRule { + + def getName(): String + + /** + * register an observer to trigger if this rule is matched + * + * @param observer + */ + def addObserver(observer: Observer): Unit + + /** + * return registered event + * + * @return + */ + def getHitEvent(): Event + + /** + * if data match the pattern, return true and trigger observer should call isMatched() + * + * @param data + * @return + */ + def triggerIfMatched(data: util.List[ScannedData]): Boolean +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala new file mode 100644 index 0000000000..a6914c3e60 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala @@ -0,0 +1,27 @@ +/* + * 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.monitor.scan.core.pac + +import java.util + + +trait ScannedData { + def getOwner(): String + + def getData(): util.List[scala.Any] +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala new file mode 100644 index 0000000000..e6939df36e --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala @@ -0,0 +1,168 @@ +/* + * 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.monitor.scan.core.scanner + +import java.util +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.linkis.common.utils.Logging +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} +import org.apache.linkis.monitor.scan.core.pac._ + + +abstract class AbstractScanner extends AnomalyScanner with Logging { + private val buffer: ScanBuffer = new ScanBuffer + + private val dataFetcherIdx: AtomicInteger = + new AtomicInteger(0) // mark next fetcher for sequentially produce data + + private val dataFetcherList: CopyOnWriteArrayList[DataFetcher] = + new CopyOnWriteArrayList[DataFetcher] + + private val scanRuleList: CopyOnWriteArrayList[ScanRule] = new CopyOnWriteArrayList[ScanRule] + + /** + * Producer + */ + override def addDataFetcher(fetcher: DataFetcher): Unit = { + if (fetcher != null) { + dataFetcherList.add(fetcher) + } else { + warn("ignore null DataFetcher") + } + } + + override def addDataFetchers(fetchers: util.List[DataFetcher]): Unit = { + if (fetchers != null && fetchers.size != 0) { + dataFetcherList.addAll(fetchers) + } else { + warn("ignore null or empty DataFetcher") + } + } + + override def getDataFetchers: util.List[DataFetcher] = dataFetcherList + + /** + * directly feed data to buffer + */ + override def feedData(data: util.List[ScannedData]): Unit = { + if (data != null && data.size != 0) { + buffer.write(data) + } else { + warn("Fed with null or empty data") + } + } + + /** + * Returns a buffer that allows read/write simultaneously buffer is allowed to be written by + * other thread + */ + override def getBuffer(): ScanBuffer = buffer + + /** + * add rules to scanner + */ + override def addScanRule(rule: ScanRule): Unit = { + if (rule != null) { + scanRuleList.add(rule) + } else { + warn("ignore null ScanRule") + } + } + + override def addScanRules(rules: util.List[ScanRule]): Unit = { + if (rules != null && rules.size != 0) { + scanRuleList.addAll(rules) + } else { + warn("ignore null or empty ScanRule") + } + } + + override def getScanRules(): util.List[ScanRule] = scanRuleList + + /** + * blocking call, scan and analyze until all dataFetchers are accessed once + */ + override def run(): Unit = { + if (dataFetcherList.size() == 0) { + throw new AnomalyScannerException( + 21304, + "attempting to run scanner with empty dataFetchers" + ) + } + if (buffer == null) { + throw new AnomalyScannerException(21304, "attempting to run scanner with null buffer") + } + if (scanRuleList.size == 0) { + throw new AnomalyScannerException(21304, "attempting to run scanner with empty rules") + } + while (dataFetcherIdx.get() < dataFetcherList.size()) { + scanOneIteration() + analyzeOneIteration() + } + } + + /** + * 1. scan data for 1 iteration 2. should be a blocking call 3. see if [[ScanRule]] is matched + * 4. trigger [[Event]] and inform observer + */ + override def scanOneIteration(): Unit = { + val idx = dataFetcherIdx.getAndIncrement() + val fetcher = dataFetcherList.get(idx) + if (fetcher != null) { + val rawData = fetcher.getData() + info("scanned " + rawData.size + " data. Rule: " + fetcher.getName); + if (rawData != null && rawData.size != 0) { + buffer.write(new BaseScannedData(fetcher.getName, rawData)) + } + } else { + warn("ignored null fetcher!!") + } + } + + /** + * 1. should be a blocking call 2. read from [[ScanBuffer]] 2. see if [[ScanRule]] is matched + * 3. trigger [[Observer]] + */ + override def analyzeOneIteration(): Unit = { + val dataToAnalyze = buffer.drain() + if (dataToAnalyze != null && dataToAnalyze.size() != 0) { + for (scanRule: ScanRule <- scanRuleList) { + if (scanRule != null) { + info("analyzing " + dataToAnalyze.size + " data. Rule: " + scanRule.getName) + scanRule.triggerIfMatched(dataToAnalyze) + } else { + warn("found empty or null ScanRule") + } + } + } else { + info("analyzed 0 data.") + } + } + + /** + * 1. should be non-blocking 2. keeps calling scanOneIteration() and analyzeOneIteration() + * utils stop() is called + */ + override def start(): Unit = { + // TODO + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala new file mode 100644 index 0000000000..854ca78c3f --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala @@ -0,0 +1,95 @@ +/* + * 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.monitor.scan.core.scanner + +import org.apache.linkis.monitor.scan.core.ob.Event +import org.apache.linkis.monitor.scan.core.pac.{DataFetcher, ScanBuffer, ScannedData, ScanRule} +import org.apache.linkis.monitor.scan.core.pac.DataFetcher + +import java.util + +/** + * A Scanner that: + * 1. scan a datasource using [[DataFetcher]], write data into [[ScanBuffer]] 2. read data from + * [[ScanBuffer]] see if [[ScanRule]] is matched 3. trigger [[Event]] in [[ScanRule]] and + * inform observer + */ +trait AnomalyScanner { + + /** + * Producer + */ + def addDataFetcher(dataFetcher: DataFetcher): Unit + + def addDataFetchers(dataFetchers: util.List[DataFetcher]): Unit + + def getDataFetchers: util.List[DataFetcher] + + /** + * directly feed data to buffer + */ + def feedData(data: util.List[ScannedData]): Unit + + /** + * Buffer + */ + + /** + * add rules to scanner + */ + def addScanRule(rule: ScanRule): Unit + + def addScanRules(rules: util.List[ScanRule]): Unit + + /** + * Consumer + */ + + def getScanRules(): util.List[ScanRule] + + /** + * scan and analyze for 1 iteration + */ + def run(): Unit + + /** + * 1. should be non-blocking 2. keeps calling scan() utils stop() is called + */ + def start(): Unit + + def shutdown(): Unit + + /** + * 1. should be a blocking call 2. call [[DataFetcher]] to read data 3. write result to + * [[ScanBuffer]] + */ + protected def scanOneIteration(): Unit + + /** + * Returns a buffer that allows read/write simultaneously buffer is allowed to be written by + * other thread + */ + protected def getBuffer(): ScanBuffer + + /** + * 1. should be a blocking call 2. read from [[ScanBuffer]] 2. see if [[ScanRule]] is matched + * 3. trigger [[[[org.apache.linkis.tools.core.ob.Observer]]]] + */ + protected def analyzeOneIteration(): Unit + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala new file mode 100644 index 0000000000..c56a3f8182 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala @@ -0,0 +1,29 @@ +/* + * 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.monitor.scan.core.scanner + +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils + + +class DefaultScanner extends AbstractScanner { + + override def shutdown(): Unit = { + PooledImsAlertUtils.shutDown(true, -1) + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EmsListAction.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EmsListAction.scala new file mode 100644 index 0000000000..f3906c8634 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EmsListAction.scala @@ -0,0 +1,74 @@ +/* + * 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.monitor.scan.request + +import org.apache.linkis.httpclient.request.GetAction + +import org.apache.commons.lang3.StringUtils + +import scala.collection.mutable.ArrayBuffer + +class EmsListAction extends GetAction with MonitorResourceAction { + + override def suffixURLs: Array[String] = Array("linkisManager", "listAllEMs") + +} + +object EmsListAction { + def newBuilder(): Builder = new Builder + + class Builder private[EmsListAction] () { + private var user: String = _ + private var instance: String = _ + private var nodeHealthy: String = _ + private var owner: String = _ + + def setInstance(instance: String): Builder = { + this.instance = instance + this + } + + def setNodeHealthy(nodeHealthy: String): Builder = { + this.nodeHealthy = nodeHealthy + this + } + + def setOwner(owner: String): Builder = { + this.owner = owner + this + } + + def setUser(user: String): Builder = { + this.user = user + this + } + + def build(): EmsListAction = { + val emsListAction = new EmsListAction + if (StringUtils.isNotBlank(instance)) emsListAction.setParameter("instance", instance) + if (StringUtils.isNotBlank(nodeHealthy)) { + emsListAction.setParameter("nodeHealthy", nodeHealthy) + } + if (StringUtils.isNotBlank(owner)) emsListAction.setParameter("owner", owner) + if (StringUtils.isNotBlank(user)) emsListAction.setUser(user) + emsListAction + } + + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala new file mode 100644 index 0000000000..6126a2b04b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala @@ -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.linkis.monitor.scan.request + +import org.apache.linkis.httpclient.request.GetAction + +import org.apache.commons.lang3.StringUtils + +class EntranceTaskAction extends GetAction with MonitorResourceAction { + override def suffixURLs: Array[String] = Array("entrance/operation/metrics", "taskinfo") +} + +object EntranceTaskAction { + def newBuilder(): Builder = new Builder + + class Builder private[EntranceTaskAction] () { + private var user: String = _ + private var creator: String = _ + private var engineTypeLabel: String = _ + private var instance: String = _ + + def setCreator(creator: String): Builder = { + this.creator = creator + this + } + + def setEngineTypeLabel(engineTypeLabel: String): Builder = { + this.engineTypeLabel = engineTypeLabel + this + } + + def setUser(user: String): Builder = { + this.user = user + this + } + + def setInstance(instance: String): Builder = { + this.instance = instance + this + } + + def build(): EntranceTaskAction = { + val entranceTaskAction = new EntranceTaskAction + if (StringUtils.isNotBlank(creator)) entranceTaskAction.setParameter("creator", creator) + if (StringUtils.isNotBlank(engineTypeLabel)) entranceTaskAction.setParameter("engineTypeLabel", engineTypeLabel) + if (StringUtils.isNotBlank(instance)) entranceTaskAction.setParameter("instance", instance) + if (StringUtils.isNotBlank(user)) { + // hadoop用户应该获取全部用户entrance信息,则无需传user,即可获取全部entrance信息 + if (user.equals("hadoop")) { + entranceTaskAction.setParameter("user", "") + } else { + entranceTaskAction.setParameter("user", user) + } + } + if (StringUtils.isNotBlank(user)) entranceTaskAction.setUser(user) + entranceTaskAction + } + + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/MonitorResourceAction.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/MonitorResourceAction.scala new file mode 100644 index 0000000000..a8661ec4e7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/MonitorResourceAction.scala @@ -0,0 +1,22 @@ +/* + * 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.monitor.scan.request + +import org.apache.linkis.httpclient.dws.request.DWSHttpAction + +trait MonitorResourceAction extends DWSHttpAction with UserAction diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/UserAction.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/UserAction.scala new file mode 100644 index 0000000000..7cae916a74 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/UserAction.scala @@ -0,0 +1,26 @@ +/* + * 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.monitor.scan.request + +trait UserAction extends org.apache.linkis.httpclient.request.UserAction { + private var user: String = _ + + override def setUser(user: String): Unit = this.user = user + + override def getUser: String = user +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/EntranceTaskResult.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/EntranceTaskResult.scala new file mode 100644 index 0000000000..fa6ed8080e --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/EntranceTaskResult.scala @@ -0,0 +1,36 @@ +/* + * 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.monitor.scan.response + +import org.apache.linkis.httpclient.dws.annotation.DWSHttpMessageResult +import org.apache.linkis.httpclient.dws.response.DWSResult + +import java.util + +import scala.beans.BeanProperty + +@DWSHttpMessageResult("/api/rest_j/v\\d+/entrance/operation/metrics/taskinfo") +class EntranceTaskResult extends DWSResult { + + @BeanProperty + var tasks: util.ArrayList[util.Map[String, Object]] = _ + + @BeanProperty + var totalPage: Int = _ + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/MonitorResourceResult.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/MonitorResourceResult.scala new file mode 100644 index 0000000000..3cd9e74d6d --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/response/MonitorResourceResult.scala @@ -0,0 +1,33 @@ +/* + * 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.monitor.scan.response + +import org.apache.linkis.httpclient.dws.response.DWSResult +import org.apache.linkis.httpclient.request.UserAction + +trait MonitorResourceResult extends DWSResult with UserAction { + + private var execID: String = _ + + def getExecID: String = execID + + def setExecID(execID: String): Unit = { + this.execID = execID + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/ScanUtils.java b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/ScanUtils.java new file mode 100644 index 0000000000..064865821b --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/ScanUtils.java @@ -0,0 +1,71 @@ +/* + * 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.monitor.scan.utils; + + +public class ScanUtils { + public static int getNumOfLines(String str) { + if (str == null || str.length() == 0) { + return 0; + } + int lines = 1; + int len = str.length(); + for (int pos = 0; pos < len; pos++) { + char c = str.charAt(pos); + if (c == '\r') { + lines++; + if (pos + 1 < len && str.charAt(pos + 1) == '\n') { + pos++; + } + } else if (c == '\n') { + lines++; + } + } + return lines; + } + + public static int getFirstIndexSkippingLines(String str, Integer lines) { + if (str == null || str.length() == 0 || lines < 0) { + return -1; + } + if (lines == 0) { + return 0; + } + + int curLineIdx = 0; + int len = str.length(); + for (int pos = 0; pos < len; pos++) { + char c = str.charAt(pos); + if (c == '\r') { + curLineIdx++; + if (pos + 1 < len && str.charAt(pos + 1) == '\n') { + pos++; + } + } else if (c == '\n') { + curLineIdx++; + } else { + continue; + } + + if (curLineIdx >= lines) { + return pos + 1; + } + } + return -1; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala new file mode 100644 index 0000000000..b1f29530f8 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala @@ -0,0 +1,26 @@ +/* + * 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.monitor.scan.utils.alert + + +trait AlertDesc { + + /** + * define necessary information for an alert e.g. alert title, alert receiver etc. + */ +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala new file mode 100644 index 0000000000..5ae1b960e5 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala @@ -0,0 +1,30 @@ +/* + * 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.monitor.scan.utils.alert + + +trait AlertSender { + + /** + * traverse all registered alertActions and send alert + * + * @return + * true if it is a success + */ + def doSendAlert(alertAction: AlertDesc): Boolean +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala new file mode 100644 index 0000000000..10eb367d2a --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala @@ -0,0 +1,112 @@ +/* + * 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.monitor.scan.utils.alert + +import org.apache.linkis.common.conf.CommonVars +import org.apache.linkis.common.utils.{Logging, Utils} + +import java.util.concurrent.{Future, LinkedBlockingQueue} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + + +abstract class PooledAlertSender extends AlertSender with Logging { + private val THREAD_POOL_SIZE = CommonVars[Int]("wds.linkis.alert.pool.size", 5).getValue + + private val alertDescQ: LinkedBlockingQueue[AlertDesc] = + new LinkedBlockingQueue[AlertDesc](1000) + + protected implicit val executors = + Utils.newCachedExecutionContext(THREAD_POOL_SIZE, "alert-pool-thread-", false) + + private val stopped: AtomicBoolean = new AtomicBoolean(false) + private val runningNumber: AtomicInteger = new AtomicInteger(0) + private var future: Future[_] = _ + + /** + * add an alertDesc to queue + * + * @param alertDesc + * should encapsulates every information an alert platform needs for sending an alarm + */ + def addAlertToPool(alertDesc: AlertDesc): Unit = { + alertDescQ.add(alertDesc) + } + + /** + * describes actual actions for sending an alert + * + * @return + * true if it is a success + */ + override def doSendAlert(alertDesc: AlertDesc): Boolean + + def start(): Unit = { + future = Utils.defaultScheduler.submit(new Runnable() { + override def run() { + info("Pooled alert thread started!") + while (!stopped.get) { + executors synchronized { + while (!stopped.get && runningNumber.get >= THREAD_POOL_SIZE) { + info("Pooled alert thread is full, start waiting") + executors.wait() + } + } + info("Pooled alert thread continue processing") + + if (stopped.get && alertDescQ.size() == 0) return + val alertDesc = Utils.tryQuietly(alertDescQ.take) + if (alertDesc == null) return + executors.submit(new Runnable { + override def run() { + runningNumber.addAndGet(1) + Utils.tryAndWarn { + info("sending alert , information: " + alertDesc) + val ok = doSendAlert(alertDesc) + if (!ok) { + warn("Failed to send alert: " + alertDesc) + } else { + info("successfully send alert: " + alertDesc) + } + runningNumber.decrementAndGet + executors synchronized executors.notify + } + } + }) + } + } + }) + } + + def shutdown(waitComplete: Boolean = true, timeoutMs: Long = -1): Unit = { + info("stopping the Pooled alert thread...") + if (waitComplete) { + val startTime = System.currentTimeMillis() + while ( + (alertDescQ.size() > 0 || runningNumber + .get() > 0) && (timeoutMs == -1 || System.currentTimeMillis() - startTime > timeoutMs) + ) { + Utils.tryQuietly(Thread.sleep(5 * 1000L)) + } + } + executors.shutdown + stopped.set(true) + future.cancel(true) + info("Pooled alert thread is stopped") + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala new file mode 100644 index 0000000000..6b12da9584 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala @@ -0,0 +1,155 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.utils.ScanUtils +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc + +import org.apache.commons.collections.CollectionUtils +import org.apache.commons.lang3.StringUtils + +import java.util +import java.util.HashSet + +import scala.collection.JavaConverters._ + +import ImsAlertLevel.ImsAlertLevel +import ImsAlertWay.ImsAlertWay + + +case class ImsAlertDesc( + var subSystemId: String, + var alertTitle: String, + var alertObj: String, + var alertInfo: String, + alertLevel: ImsAlertLevel = ImsAlertLevel.INFO, + alertIp: String, + canRecover: Int = 0, // 默认0,为1时,需要有对应的恢复告警 + alertWays: util.Set[ImsAlertWay] = new HashSet[ImsAlertWay], + var alertReceivers: util.Set[String] = new HashSet[String], + var numHit: Int = 0, + var hitIntervalMs: Long = 0L +) extends AlertDesc { + + override val toString: String = { + val sb = new StringBuilder + sb.append("sub_system_id=").append(subSystemId).append("&alert_title=").append(alertTitle) + if (alertLevel != null) sb.append("&alert_level=").append(alertLevel.toString) + if (StringUtils.isNotEmpty(alertObj)) sb.append("&alert_obj=").append(alertObj) + if (StringUtils.isNotEmpty(alertInfo)) { + sb.append("&alert_info=") + .append(alertInfo) + .append( + "[freq_info] hit " + numHit + " time(s) within " + hitIntervalMs / 1000 / 60 + " mins" + ) + } + if (canRecover == 0 || canRecover == 1) sb.append("&can_recover=").append(canRecover) + if (alertWays != null && alertWays.size > 0) { + sb.append("&alert_way=") + sb.append(alertWays.asScala.map(_.toString).mkString(",")) + } + if (alertReceivers != null && alertReceivers.size > 0) { + sb.append("&alert_reciver=") + sb.append(alertReceivers.asScala.mkString(",")) + } + if (alertIp != null) { + sb.append("&alert_ip=").append(alertIp) + + } + sb.toString + } + + val toMap: Map[String, String] = { + val map = scala.collection.mutable.Map[String, String]() + map += "sub_system_id" -> subSystemId + map += "alert_title" -> alertTitle + if (alertLevel != null) map += "alert_level" -> alertLevel.toString + if (StringUtils.isNotEmpty(alertObj)) map += "alert_obj" -> alertObj + if (StringUtils.isNotEmpty(alertInfo)) { + map += "alert_info" + "[freq_info] hit " + numHit + " time(s) within " + hitIntervalMs / 1000 / 60 + " mins" -> alertInfo + } + if (canRecover == 0 || canRecover == 1) map += "can_recover" -> canRecover.toString + if (alertWays != null && alertWays.size > 0) { + map += "alert_way" -> alertWays.asScala.map(_.toString).mkString(",") + } + if (alertReceivers != null && alertReceivers.size > 0) { + map += "alert_reciver" -> alertReceivers.asScala.mkString(",") + } + map.toMap + } + + val toImsRequest: ImsRequest = { + val params = validate() + val alertEntity = AlertEntity( + params(0).asInstanceOf[String], + params(1).asInstanceOf[String], + params( + 3 + ) + "[freq_info] hit " + numHit + " time(s) within " + hitIntervalMs / 1000 / 60 + " mins", + alertWays.asScala.map(_.toString).mkString(","), + params(4).asInstanceOf[util.Set[String]].asScala.mkString(","), + alertLevel.toString, + params(2).asInstanceOf[String], + canRecover.toString + ) + + val alertEntityList = new util.ArrayList[AlertEntity] + alertEntityList.add(alertEntity) + + ImsRequest(alertEntityList) + } + + def validate(): Array[Any] = { + assert(StringUtils.isNumeric(subSystemId) && subSystemId.length == 4) + assert(StringUtils.isNotEmpty(alertTitle)) + val newAlertTitle = if (alertTitle.length > 100) { + alertTitle.substring(0, 96) + "... ..." + } else { + alertTitle + } + val newAlertObj = if (StringUtils.isNotEmpty(alertObj) && alertObj.length >= 50) { + alertObj = alertObj.substring(0, 36) + "... ..." + } else { + alertObj + } + val newAlertInfo = + if ( + StringUtils.isNotEmpty(alertInfo) && ScanUtils.getNumOfLines( + alertInfo + ) > Constants.ALERT_IMS_MAX_LINES + ) { + StringUtils.substring( + alertInfo, + 0, + ScanUtils.getFirstIndexSkippingLines(alertInfo, Constants.ALERT_IMS_MAX_LINES) + ) + "... ...\n" + } else { + alertInfo + } + val newAlertReceivers = + if (CollectionUtils.isNotEmpty(alertReceivers) && alertReceivers.size > 15) { + alertReceivers.asScala.take(15) + } else { + alertReceivers + } + + Array(subSystemId, newAlertTitle, newAlertObj, newAlertInfo, newAlertReceivers) + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala new file mode 100644 index 0000000000..cb304e9e4f --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala @@ -0,0 +1,29 @@ +/* + * 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.monitor.scan.utils.alert.ims + + +object ImsAlertLevel extends Enumeration { + type ImsAlertLevel = Value + val INFO = Value("5") + val WARN = Value("4") + val MINOR = Value("3") + val MAJOR = Value("2") + val CRITICAL = Value("1") + val CLEAR = Value("0") +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala new file mode 100644 index 0000000000..207df613d7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala @@ -0,0 +1,27 @@ +/* + * 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.monitor.scan.utils.alert.ims + + +object ImsAlertWay extends Enumeration { + type ImsAlertWay = Value + val NoAlert = Value("0") + val RTX = Value("1") + val Email = Value("2") + val WeChat = Value("3") +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala new file mode 100644 index 0000000000..64b80d3a95 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala @@ -0,0 +1,36 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import java.util + +import com.fasterxml.jackson.annotation.JsonProperty + + +case class ImsRequest(@JsonProperty("alertList") alertList: util.List[AlertEntity]) + +case class AlertEntity( + @JsonProperty("sub_system_id") subSystemId: String, + @JsonProperty("alert_title") alertTitle: String, + @JsonProperty("alert_info") alertInfo: String, + @JsonProperty("alert_way") alertWays: String, + @JsonProperty("alert_reciver") alertReceivers: String, + @JsonProperty("alert_level") alertLevel: String, + @JsonProperty("alert_obj") alertObj: String, + @JsonProperty("can_recover") canRecover: String +) diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala new file mode 100644 index 0000000000..95b64f3f4e --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala @@ -0,0 +1,132 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.commons.lang3.StringUtils +import org.apache.commons.lang3.exception.ExceptionUtils + +import java.io.{BufferedReader, File, FileInputStream, InputStream, InputStreamReader} +import java.text.SimpleDateFormat +import java.util +import java.util.Properties +import scala.collection.JavaConverters._ +import com.fasterxml.jackson.annotation.JsonProperty +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.commons.io.IOUtils + + +object JobHistoryScanImsAlertPropFileParserUtils extends Logging { + + private val mapper = { + val ret = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) + ret.registerModule(DefaultScalaModule) + ret.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + ret + } + + def getAlerts(prefix: String): util.Map[String, AlertDesc] = { + val ret = new util.HashMap[String, AlertDesc]() + val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) + if (url == null) { + throw new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH + ) + } + logger.info("reading alert properties from: " + url.getFile) + val properties = new Properties() + var inputStream: InputStream = null + var reader: InputStreamReader = null + var buff: BufferedReader = null + + Utils.tryFinally { + Utils.tryCatch { + inputStream = new FileInputStream(new File(url.getFile)) + reader = new InputStreamReader(inputStream, "UTF-8") + buff = new BufferedReader(reader) + properties.load(buff) + } { t => { + throw new AnomalyScannerException(21304, "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t)) + return ret + } + } + } { + IOUtils.closeQuietly(buff) + IOUtils.closeQuietly(reader) + IOUtils.closeQuietly(inputStream) + } + for ((k: String, v: String) <- properties.asScala) { + if (ret.containsKey(k)) { + logger.warn("found duplicate key in alert properties, accept only the first one") + } else if (StringUtils.startsWith(k, prefix)) { + val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) + val receivers = { + val set: util.Set[String] = new util.HashSet[String] + if (StringUtils.isNotBlank(data.alertReceivers)) { + data.alertReceivers.split(",").map(r => set.add(r)) + } + Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { + if (StringUtils.isNotBlank(e)) { + set.add(e) + } + }) + set + } + val alertDesc = Utils.tryAndWarn( + new ImsAlertDesc( + Constants.ALERT_SUB_SYSTEM_ID, + data.alertTitle, + data.alertObj, + data.alertInfo, + ImsAlertLevel.withName(data.alertLevel), + null, // Not used + 0, { + val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] + if (StringUtils.isNotBlank(data.alertWays)) { + data.alertWays + .split(",") + .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) + } + set + }, + receivers + ) + ) + val realK = StringUtils.substringAfter(k, prefix) + ret.put(realK, alertDesc) + } + } + ret + } + +} + +case class ImsAlertPropFileData( + @JsonProperty("alert_title") alertTitle: String, + @JsonProperty("alert_info") alertInfo: String, + @JsonProperty("alert_way") alertWays: String, + @JsonProperty("alert_reciver") alertReceivers: String, + @JsonProperty("alert_level") alertLevel: String, + @JsonProperty("alert_obj") alertObj: String, + @JsonProperty("can_recover") canRecover: String +) diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala new file mode 100644 index 0000000000..021ec3b588 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala @@ -0,0 +1,229 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.commons.lang3.StringUtils +import org.apache.commons.lang3.exception.ExceptionUtils + +import java.io.{BufferedReader, File, FileInputStream, InputStream, InputStreamReader} +import java.text.SimpleDateFormat +import java.util +import java.util.Properties +import scala.collection.JavaConverters._ +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.commons.io.IOUtils + +object MonitorAlertUtils extends Logging { + + + private val mapper = { + val ret = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) + ret.registerModule(DefaultScalaModule) + ret.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + ret + } + + def getAlerts( + prefix: String, + parms: util.HashMap[String, String] + ): util.Map[String, AlertDesc] = { + val ret = new util.HashMap[String, AlertDesc]() + val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) + if (url == null) { + throw new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH + ) + } + logger.info("reading alert properties from: " + url.getFile) + val properties = new Properties() + var inputStream: InputStream = null + + var reader: InputStreamReader = null + var buff: BufferedReader = null + + Utils.tryFinally { + Utils.tryThrow { + inputStream = new FileInputStream(new File(url.getFile)) + reader = new InputStreamReader(inputStream, "UTF-8") + buff = new BufferedReader(reader) + properties.load(buff) + } { + case t: Throwable => + new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t) + ) + } + } { + IOUtils.closeQuietly(buff) + IOUtils.closeQuietly(reader) + IOUtils.closeQuietly(inputStream) + } + for ((k: String, v: String) <- properties.asScala) { + if (ret.containsKey(k)) { + logger.warn("found duplicate key in alert properties, accept only the first one") + } else if (StringUtils.startsWith(k, prefix)) { + val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) + var alertInfo = new String( + new StringBuilder().append(data.alertInfo).toString().getBytes(), + "utf-8" + ).replace("$name", data.alertReceivers) + val interator = parms.keySet.iterator + while (interator.hasNext) { + val key = interator.next + val value = parms.get(key) + alertInfo = alertInfo.replace(key, value) + } + val receivers = { + val set: util.Set[String] = new util.HashSet[String] + if (StringUtils.isNotBlank(data.alertReceivers)) { + data.alertReceivers.split(",").map(r => set.add(r)) + } + Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { + if (StringUtils.isNotBlank(e)) { + set.add(e) + } + }) + set.add(parms.get("$alteruser")) + set + } + val alertDesc = Utils.tryAndWarn( + new ImsAlertDesc( + Constants.ALERT_SUB_SYSTEM_ID, + data.alertTitle, + data.alertObj, + alertInfo, + ImsAlertLevel.withName(data.alertLevel), + null, // Not used + 0, { + val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] + if (StringUtils.isNotBlank(data.alertWays)) { + data.alertWays + .split(",") + .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) + } + set + }, + receivers + ) + ) + val realK = StringUtils.substringAfter(k, prefix) + ret.put(realK, alertDesc) + } + } + ret + } + + + def getAlertsByDss(prefix: String, parms: util.Map[String, String]): util.Map[String, AlertDesc] = { + val ret = new util.HashMap[String, AlertDesc]() + val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) + if (url == null) { + throw new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH + ) + } + logger.info("reading alert properties from: " + url.getFile) + val properties = new Properties() + var inputStream: InputStream = null + + var reader: InputStreamReader = null + var buff: BufferedReader = null + + Utils.tryFinally { + Utils.tryThrow{ + inputStream = new FileInputStream(new File(url.getFile)) + reader = new InputStreamReader(inputStream, "UTF-8") + buff = new BufferedReader(reader) + properties.load(buff) + } { + case t: Throwable => + new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t) + ) + } + } { + IOUtils.closeQuietly(buff) + IOUtils.closeQuietly(reader) + IOUtils.closeQuietly(inputStream) + } + for ((k: String, v: String) <- properties.asScala) { + if (ret.containsKey(k)) { + warn("found duplicate key in alert properties, accept only the first one") + } else if (StringUtils.startsWith(k, prefix)) { + val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) + var alertInfo = data.alertInfo + val interator = parms.keySet.iterator + while (interator.hasNext) { + val key = interator.next + val value = parms.get(key) + alertInfo = alertInfo.replace(key, value) + } +// alertInfo = parms.getOrDefault("detail", "").concat(alertInfo) + val receivers = { + val set: util.Set[String] = new util.HashSet[String] + if (StringUtils.isNotBlank(data.alertReceivers)) { + data.alertReceivers.split(",").map(r => set.add(r)) + } + Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { + if (StringUtils.isNotBlank(e)) { + set.add(e) + } + }) + if (StringUtils.isNotBlank(parms.get("receiver"))) { + parms.get("receiver").split(",").map(r => set.add(r)) + } + set + } + val alertDesc = Utils.tryAndWarn( + ImsAlertDesc( + parms.getOrDefault("subSystemId", Constants.ALERT_SUB_SYSTEM_ID), + parms.getOrDefault("title", ""), + data.alertObj, + alertInfo + , + ImsAlertLevel.withName(parms.getOrDefault("monitorLevel", "4")), + null, // Not used + 0, { + val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] + if (StringUtils.isNotBlank(data.alertWays)) { + data.alertWays + .split(",") + .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) + } + set + }, + receivers + ) + ) + val realK = StringUtils.substringAfter(k, prefix) + ret.put(realK, alertDesc) + } + } + ret + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala new file mode 100644 index 0000000000..d2508b37a3 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala @@ -0,0 +1,110 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} +import org.apache.linkis.monitor.scan.utils.alert.{AlertDesc, PooledAlertSender} +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.linkis.monitor.scan.utils.log.LogUtils + +import org.apache.http.client.config.RequestConfig +import org.apache.http.client.methods.HttpPost +import org.apache.http.entity.{ContentType, StringEntity} +import org.apache.http.impl.client.HttpClients +import org.apache.http.util.EntityUtils + +import java.text.SimpleDateFormat +import java.util + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule + + +class PooledImsAlertSender( + subSystemId: String, + alertUrl: String, + default_Receivers: util.Set[String] +) extends PooledAlertSender + with Logging { + + protected val httpClient = HttpClients.createDefault // TODO: Linkis-httpclient + + private val mapper = + new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) + + /** + * describes actual actions for sending an alert + * + * @return + * true if it is a success + */ + + /** + * describes actual actions for sending an alert + * + * @return + * true if it is a success + */ + override def doSendAlert(alertDesc: AlertDesc): Boolean = { + if (!alertDesc.isInstanceOf[ImsAlertDesc]) { + logger.warn("wrong alertDesc dataType: " + alertDesc.getClass.getCanonicalName) + return false + } + logger.info("sending an alert to IMS, information: " + alertDesc) + val imsRequest = alertDesc.asInstanceOf[ImsAlertDesc].toImsRequest + + mapper.registerModule(DefaultScalaModule) + val paramContent = Utils.tryCatch(mapper.writeValueAsString(imsRequest)) { t => + logger.warn("ignore alert: " + imsRequest, t) + return false + } + if (paramContent.isEmpty) { + logger. warn("alertParams is empty, will not send alarm") + return false + } + + val requestConfig = RequestConfig.DEFAULT + + val entity = new StringEntity( + paramContent, + ContentType.create(ContentType.APPLICATION_JSON.getMimeType, "UTF-8") + ) + entity.setContentEncoding("UTF-8") + + val httpPost = new HttpPost(alertUrl) + + httpPost.setConfig(requestConfig) + httpPost.setEntity(entity) + + val response = Utils.tryAndErrorMsg(httpClient.execute(httpPost))("send alert to IMS failed") + + if (response != null) { + val responseInfo = EntityUtils.toString(response.getEntity, "UTF-8") + logger.info("Alert: " + paramContent + "Response: " + responseInfo) + LogUtils.stdOutLogger.info("Alert: " + paramContent + "Response: " + responseInfo) + if (response.getStatusLine.getStatusCode == 200) return true + } + return false + } + + override def shutdown(waitComplete: Boolean = true, timeoutMs: Long = -1): Unit = { + super.shutdown(waitComplete, timeoutMs) + httpClient.close + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala new file mode 100644 index 0000000000..4a50161438 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala @@ -0,0 +1,115 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import org.apache.linkis.common.utils.{Logging, Utils} +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc + +import org.apache.commons.collections.CollectionUtils +import org.apache.commons.lang3.StringUtils +import org.apache.commons.lang3.exception.ExceptionUtils + +import java.net.InetAddress +import java.util +import java.util.HashSet + +import scala.collection.JavaConverters._ + +import ImsAlertWay.ImsAlertWay + + +object PooledImsAlertUtils extends Logging { + + private val sender: PooledImsAlertSender = { + val ret = new PooledImsAlertSender( + Constants.ALERT_SUB_SYSTEM_ID, + Constants.ALERT_IMS_URL, + Constants.ALERT_DEFAULT_RECEIVERS.asJava + ) + ret.start() + ret + } + + private val localIp = InetAddress.getLocalHost.getHostAddress + + def addAlertAndLogException(message: String): Unit = Utils.tryAndError(addAlert(message)) + + def addAlert(message: String): Unit = addExceptionAlert(message, null, null) + + def addExceptionAlert(message: String, t: Throwable): Unit = + addExceptionAlert(message, t, null) + + def addExceptionAlertAndLogException(message: String, t: Throwable): Unit = + Utils.tryAndError(addExceptionAlert(message, t, null)) + + def addExceptionAlert(message: String, t: Throwable, alertWays: util.Set[ImsAlertWay]): Unit = { + val alertObj = + if (StringUtils.isEmpty(message) && t != null) t.getMessage + else if (StringUtils.isEmpty(message)) { + throw new NullPointerException("both message and exception are null!") + } else { + message + } + val _alertWays = + if (CollectionUtils.isNotEmpty(alertWays)) alertWays else new HashSet[ImsAlertWay]() + val (alertInfo, alertLevel) = if (t != null) { + _alertWays.add(ImsAlertWay.Email) + _alertWays.add(ImsAlertWay.RTX) + _alertWays.add(ImsAlertWay.WeChat) + (ExceptionUtils.getRootCauseMessage(t), ImsAlertLevel.MAJOR) + } else { + _alertWays.add(ImsAlertWay.RTX) + (message, ImsAlertLevel.WARN) + } + val alertDesc = new ImsAlertDesc( + Constants.ALERT_SUB_SYSTEM_ID, + "BDP Alert", + alertObj, + alertInfo, + alertLevel, + localIp, + 0, + _alertWays + ) + addAlert(alertDesc) + } + + def addAlert(alertDesc: AlertDesc): Unit = { + if (!alertDesc.isInstanceOf[ImsAlertDesc]) { + logger.warn("Ignore wrong alertDesc. DataType: " + alertDesc.getClass.getCanonicalName) + } else { + sender.addAlertToPool(alertDesc) + logger.info("successfully added alert") + } + } + + def addAlertAndLogException(alertDesc: ImsAlertDesc): Unit = + Utils.tryAndError(addAlert(alertDesc)) + + def clearAlert(alertDesc: ImsAlertDesc): Unit = { + assert(alertDesc.canRecover == 1) + assert(alertDesc.alertLevel == ImsAlertLevel.CLEAR) + sender.addAlertToPool(alertDesc) + } + + def shutDown(waitComplete: Boolean = true, timeoutMs: Long = -1): Unit = { + sender.shutdown(waitComplete, timeoutMs) + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala new file mode 100644 index 0000000000..1083967783 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala @@ -0,0 +1,126 @@ +/* + * 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.monitor.scan.utils.alert.ims + +import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} +import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException +import org.apache.linkis.monitor.scan.constants.Constants +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc +import org.apache.commons.lang3.StringUtils +import org.apache.commons.lang3.exception.ExceptionUtils + +import java.io.{BufferedReader, File, FileInputStream, InputStream, InputStreamReader} +import java.text.SimpleDateFormat +import java.util +import java.util.Properties +import scala.collection.JavaConverters._ +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.commons.io.IOUtils + +object UserLabelAlertUtils extends Logging { + + private val mapper = { + val ret = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) + ret.registerModule(DefaultScalaModule) + ret.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + ret + } + + def getAlerts(prefix: String, instans: String): util.Map[String, AlertDesc] = { + val ret = new util.HashMap[String, AlertDesc]() + val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) + if (url == null) { + throw new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH + ) + } + logger.info("reading alert properties from: " + url.getFile) + val properties = new Properties() + var inputStream: InputStream = null + + var reader: InputStreamReader = null + var buff: BufferedReader = null + + Utils.tryFinally { + Utils.tryThrow { + inputStream = new FileInputStream(new File(url.getFile)) + reader = new InputStreamReader(inputStream, "UTF-8") + buff = new BufferedReader(reader) + properties.load(buff) + } { + case t: Throwable => + new AnomalyScannerException( + 21304, + "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t) + ) + } + } { + IOUtils.closeQuietly(buff) + IOUtils.closeQuietly(reader) + IOUtils.closeQuietly(inputStream) + } + for ((k: String, v: String) <- properties.asScala) { + if (ret.containsKey(k)) { + logger.warn("found duplicate key in alert properties, accept only the first one") + } else if (StringUtils.startsWith(k, prefix)) { + val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) + val alertInfo = + new String(new StringBuilder().append(data.alertInfo).toString().getBytes(), "utf-8") + .replace("$userCreator", instans) + val receivers = { + val set: util.Set[String] = new util.HashSet[String] + if (StringUtils.isNotBlank(data.alertReceivers)) { + data.alertReceivers.split(",").map(r => set.add(r)) + } + Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { + if (StringUtils.isNotBlank(e)) { + set.add(e) + } + }) + set + } + val alertDesc = Utils.tryAndWarn( + new ImsAlertDesc( + Constants.ALERT_SUB_SYSTEM_ID, + data.alertTitle, + data.alertObj, + alertInfo, + ImsAlertLevel.withName(data.alertLevel), + null, // Not used + 0, { + val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] + if (StringUtils.isNotBlank(data.alertWays)) { + data.alertWays + .split(",") + .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) + } + set + }, + receivers + ) + ) + val realK = StringUtils.substringAfter(k, prefix) + ret.put(realK, alertDesc) + } + } + ret + } + +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala new file mode 100644 index 0000000000..38af7b6104 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala @@ -0,0 +1,25 @@ +/* + * 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.monitor.scan.utils.log + +import org.slf4j.LoggerFactory + + +object LogUtils { + val stdOutLogger = LoggerFactory.getLogger("PlaintTextConsoleLogger") +} diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java new file mode 100644 index 0000000000..600ccafddb --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java @@ -0,0 +1,48 @@ +/* + * 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.monitor.scan.app; + +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; +import org.apache.linkis.server.utils.LinkisMainHelper; + +import java.util.Map; + +import org.junit.Assert; + +public class JobHistoryScanImsAlertPropFileParserUtilsTest { + // @Before + public void before() { + System.getProperties().setProperty(LinkisMainHelper.SERVER_NAME_KEY(), "linkis-et-monitor"); + System.getProperties() + .setProperty("log4j.configurationFile", "src/test/resources/log4j2-console.xml"); + // System.getProperties().setProperty("wds.linkis.server.conf", + // "linkis-et-monitor.properties"); + } + + // @Test + public void getAlerts() throws Exception { + Map alerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + for (Map.Entry kv : alerts.entrySet()) { + System.out.println(kv.getKey() + ": " + kv.getValue().toString()); + } + Assert.assertEquals(alerts.size(), 2); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java new file mode 100644 index 0000000000..716ad5cf43 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java @@ -0,0 +1,38 @@ +/* + * 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.monitor.scan.app; + +import org.apache.linkis.server.utils.LinkisMainHelper; + + +public class LinkisJobHistoryScanApplicationTest { + // @Before + public void before() { + System.getProperties().setProperty(LinkisMainHelper.SERVER_NAME_KEY(), "linkis-et-monitor"); + System.getProperties() + .setProperty("log4j.configurationFile", "src/test/resources/log4j2-console.xml"); + // System.getProperties().setProperty("wds.linkis.server.conf", + // "linkis-et-monitor.properties"); + } + + // @Test + public void main() throws Exception { + LinkisJobHistoryScanApplication.main(new String[] {}); + // LinkisJobHistoryScanApplication.main(new String[]{"2021122919", "2021122921"}); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java new file mode 100644 index 0000000000..8cfad4c989 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.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.linkis.monitor.scan.utils.alert; + +import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertDesc; +import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertLevel; +import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertWay; +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertSender; +import org.apache.linkis.server.utils.LinkisMainHelper; + +import java.util.HashSet; +import java.util.Set; + +public class PooledImsAlertSenderTest { + // @Before + public void before() { + System.getProperties().setProperty(LinkisMainHelper.SERVER_NAME_KEY(), "linkis-et-monitor"); + System.getProperties() + .setProperty("log4j.configurationFile", "src/test/resources/log4j2-console.xml"); + // System.getProperties().setProperty("wds.linkis.server.conf", + // "linkis-et-monitor.properties"); + } + + // @org.junit.Test + public void doSendAlert() throws Exception { + Set ways = new HashSet<>(); + ways.add(ImsAlertWay.WeChat()); + ways.add(ImsAlertWay.Email()); + + Set receivers = new HashSet<>(); + receivers.add("shangda, johnnwang"); + ImsAlertDesc desc = + new ImsAlertDesc( + "5435", + "linkis_alert_test", + "linkis_alert", + "this is a test for linkis", + ImsAlertLevel.MINOR(), + "10.127.0.0.1", + 0, + ways, + receivers, + 3, + 12); + + System.out.println(desc); + String url = "http://172.21.0.130:10812/ims_data_access/send_alarm_by_json.do"; + + PooledImsAlertSender sender = new PooledImsAlertSender("5136", url, receivers); + sender.doSendAlert(desc); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java new file mode 100644 index 0000000000..8ca5ff1a98 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java @@ -0,0 +1,49 @@ +/* + * 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.monitor.scan.utils.alert; + +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.server.utils.LinkisMainHelper; + +import java.util.Map; + +public class PooledImsAlertUtilsTest { + // @Before + public void before() { + System.getProperties().setProperty(LinkisMainHelper.SERVER_NAME_KEY(), "linkis-et-monitor"); + System.getProperties() + .setProperty("log4j.configurationFile", "src/test/resources/log4j2-console.xml"); + // System.getProperties().setProperty("wds.linkis.server.conf", + // "linkis-et-monitor.properties"); + } + + // @Test + public void addAlert() throws Exception { + PooledImsAlertUtils.addAlert("1st test"); + Map alerts = + JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + for (Map.Entry kv : alerts.entrySet()) { + System.out.println(kv.getKey() + ": " + kv.getValue().toString()); + PooledImsAlertUtils.addAlert(kv.getValue()); + } + Thread.sleep(2000l); + PooledImsAlertUtils.shutDown(true, -1); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties b/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties new file mode 100644 index 0000000000..f08d130651 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties @@ -0,0 +1,6 @@ +jobhistory.errorcode.11001={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} +jobhistory.errorcode.11002={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} +jobhistory.errorcode.12011={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} +jobhistory.errorcode.43003={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} +jobhistory.errorcode.42011={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} +jobhistory.unfinished.time.exceed.sec.300={"alert_title":"linkis_alert_test_job_time","alert_info":"this is a test for linkis 5 job time exceed","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} \ No newline at end of file diff --git a/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties b/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties new file mode 100644 index 0000000000..0eca42bd5d --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties @@ -0,0 +1,14 @@ +wds.linkis.server.mybatis.mapperLocations=classpath*:org/apache/linkis/jobhistory/scan/app/jobhistory/dao/impl/*.xml +wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.monitor.scan.app.jobhistory.entity +wds.linkis.server.mybatis.BasePackage=org.apache.linkis.monitor.scan.app.jobhistory.dao +wds.linkis.mysql.is.encrypt=false +#wds.linkis.server.mybatis.datasource.url=jdbc:mysql://10.108.161.105:15304/bdp_easy_ide?characterEncoding=UTF-8 +#wds.linkis.server.mybatis.datasource.username=bdpeasyide +#wds.linkis.server.mybatis.datasource.password=bdpeasyide@bdpsit +wds.linkis.server.mybatis.datasource.url=jdbc:mysql://10.107.108.111:3306/vsbi_gz_bdap_sit_01?characterEncoding=UTF-8 +wds.linkis.server.mybatis.datasource.username=bdp_vsbi +wds.linkis.server.mybatis.datasource.password=bdpVsbi@2019 +wds.linkis.alert.url=http://172.21.0.130:10812/ims_data_access/send_alarm_by_json.do +wds.linkis.alert.receiver.default=johnnwang +wds.linkis.errorcode.scanner.interval.seconds=7200 +wds.linkis.errorcode.scanner.max.interval.seconds=3600 \ No newline at end of file diff --git a/linkis-extensions/linkis-et-monitor/src/test/resources/log4j2-console.xml b/linkis-extensions/linkis-et-monitor/src/test/resources/log4j2-console.xml new file mode 100644 index 0000000000..49eabc51e6 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/test/resources/log4j2-console.xml @@ -0,0 +1,46 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/linkis-extensions/pom.xml b/linkis-extensions/pom.xml index 661b6be0bf..7233141565 100644 --- a/linkis-extensions/pom.xml +++ b/linkis-extensions/pom.xml @@ -28,6 +28,7 @@ pom linkis-io-file-client + linkis-et-monitor From e88d5b5b70bd90384d5a3d60ba0d0090cafa7a2d Mon Sep 17 00:00:00 2001 From: peacewong Date: Tue, 5 Sep 2023 21:25:42 +0800 Subject: [PATCH 55/72] code optimize --- linkis-extensions/linkis-et-monitor/pom.xml | 10 +- .../src/main/assembly/distribution.xml | 6 +- .../app/LinkisDirtyDataCleanApplication.java | 111 ----- .../app/LinkisJobHistoryScanApplication.java | 172 -------- .../app/instance/dao/InstanceLabelDao.java | 1 - .../instance/entity/InsPersistenceLabel.java | 16 + .../app/jobhistory/dao/JobHistoryMapper.java | 1 - .../app/monitor/config/MonitorConfig.java | 33 +- .../app/monitor/entity/ChatbotEntity.java | 114 ++--- .../scan/app/monitor/scheduled/BmlClear.java | 1 - .../app/monitor/scheduled/EcRecordClear.java | 4 +- .../monitor/scheduled/JobHistoryClear.java | 4 +- .../monitor/scheduled/JobHistoryMonitor.java | 395 +++++++++--------- .../monitor/scheduled/ResourceMonitor.java | 73 ++-- .../app/monitor/scheduled/TaskLogClear.java | 4 +- .../app/monitor/scheduled/TaskMonitor.java | 78 ++-- .../monitor/scheduled/UserModeMonitor.java | 212 +++++----- .../app/monitor/scheduled/ValidatorClear.java | 34 +- .../scan/app/monitor/until/HttpsUntils.java | 13 +- ...kisJobHistoryScanSpringConfiguration.scala | 4 - .../scan/app/factory/MapperFactory.scala | 19 +- .../jobhistory/JobHistoryDataFetcher.scala | 14 +- .../errorcode/JobHistoryErrCodeHitEvent.scala | 1 - .../errorcode/JobHistoryErrCodeRule.scala | 20 +- .../JobHistoryErrorCodeAlertSender.scala | 5 +- .../jobtime/JobTimeExceedAlertSender.scala | 35 +- .../jobtime/JobTimeExceedHitEvent.scala | 1 - .../jobtime/JobTimeExceedRule.scala | 15 +- .../labels/JobHistoryLabelsAlertSender.scala | 4 +- .../labels/JobHistoryLabelsRule.scala | 20 +- .../runtime/CommonJobRunTimeRule.scala | 26 +- .../runtime/CommonRunTimeAlertSender.scala | 38 +- .../JobHistoryRunTimeAlertSender.scala | 19 +- .../runtime/JobHistoryRunTimeRule.scala | 33 +- .../scan/client/MonitorHTTPClient.scala | 6 +- .../monitor/scan/constants/Constants.scala | 68 +-- .../scan/constants/ScanOperatorEnum.scala | 1 - .../linkis/monitor/scan/core/ob/Event.scala | 1 - .../monitor/scan/core/ob/Observer.scala | 1 - .../scan/core/pac/AbstractDataFetcher.scala | 1 - .../scan/core/pac/AbstractScanRule.scala | 1 - .../scan/core/pac/BaseScannedData.scala | 1 - .../monitor/scan/core/pac/ScanBuffer.scala | 1 - .../monitor/scan/core/pac/ScannedData.scala | 1 - .../scan/core/scanner/AbstractScanner.scala | 24 +- .../scan/core/scanner/AnomalyScanner.scala | 8 +- .../scan/core/scanner/DefaultScanner.scala | 1 - .../scan/request/EntranceTaskAction.scala | 3 +- .../monitor/scan/utils/alert/AlertDesc.scala | 1 - .../scan/utils/alert/AlertSender.scala | 1 - .../scan/utils/alert/PooledAlertSender.scala | 17 +- .../scan/utils/alert/ims/ImsAlertDesc.scala | 3 +- .../scan/utils/alert/ims/ImsAlertLevel.scala | 1 - .../alert/ims/ImsAlertPropFileData.scala} | 30 +- .../scan/utils/alert/ims/ImsAlertWay.scala | 3 +- .../scan/utils/alert/ims/ImsRequest.scala | 1 - ...storyScanImsAlertPropFileParserUtils.scala | 132 ------ .../utils/alert/ims/MonitorAlertUtils.scala | 171 +++----- .../alert/ims/PooledImsAlertSender.scala | 14 +- .../utils/alert/ims/PooledImsAlertUtils.scala | 11 +- .../utils/alert/ims/UserLabelAlertUtils.scala | 95 +---- .../monitor/scan/utils/log/LogUtils.scala | 1 - ...ryScanImsAlertPropFileParserUtilsTest.java | 48 --- .../utils/alert/PooledImsAlertUtilsTest.java | 4 +- 64 files changed, 814 insertions(+), 1373 deletions(-) delete mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java delete mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java rename linkis-extensions/linkis-et-monitor/src/{test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java => main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertPropFileData.scala} (52%) delete mode 100644 linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala delete mode 100644 linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java diff --git a/linkis-extensions/linkis-et-monitor/pom.xml b/linkis-extensions/linkis-et-monitor/pom.xml index 5e62fa60b5..6972c9e1a4 100644 --- a/linkis-extensions/linkis-et-monitor/pom.xml +++ b/linkis-extensions/linkis-et-monitor/pom.xml @@ -87,11 +87,11 @@ ${project.version} provided - - org.apache.linkis - linkis-rpc - ${project.version} - + + org.apache.linkis + linkis-rpc + ${project.version} + diff --git a/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml b/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml index 8cc9e7ea7b..647b84c833 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml +++ b/linkis-extensions/linkis-et-monitor/src/main/assembly/distribution.xml @@ -16,10 +16,8 @@ ~ limitations under the License. --> - + linkis-et-monitor dir diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java deleted file mode 100644 index 3530ef5139..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisDirtyDataCleanApplication.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.monitor.scan.app; - -import org.apache.linkis.monitor.scan.app.dirtydata.entrance.EntranceDirtyDataHandler; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; -import org.apache.linkis.server.utils.LinkisMainHelper; - -import org.apache.commons.lang3.StringUtils; - -import org.springframework.context.annotation.AnnotationConfigApplicationContext; -import org.springframework.context.support.AbstractApplicationContext; - -import org.slf4j.Logger; - -public class LinkisDirtyDataCleanApplication { - private static final Logger logger = LogUtils.stdOutLogger(); - - /** @param args: args[0]: host args[1] port */ - public static void main(String[] args) throws ReflectiveOperationException { - if (args.length == 0 - || StringUtils.equalsIgnoreCase(args[0], "help") - || StringUtils.equalsIgnoreCase(args[0], "--help")) { - LogUtils.stdOutLogger() - .info( - "[help-message]this app cleans entrance dirty-data. args[0]: command-type (entrance/help/...) args[1]: entrance-hostname(not null), args[2]: entrance-port(can be null)"); - return; - } - String serviceName = System.getProperty(LinkisMainHelper.SERVER_NAME_KEY()); - LinkisMainHelper.formatPropertyFiles(serviceName); - - if (StringUtils.equalsIgnoreCase(args[0], "entrance")) { - AbstractApplicationContext context = - new AnnotationConfigApplicationContext(LinkisJobHistoryScanSpringConfiguration.class); - - String host = ""; - String port = ""; - if (args.length > 1) { - host = args[1]; - } - if (args.length > 2) { - port = args[2]; - } - if (args.length > 3) { - printIllegalInput("wrong number of arguments"); - return; - } - try { - removeDirtyEurekaInstance(host, port); - } catch (Exception e) { - LogUtils.stdOutLogger().error("Failed to remove dirty eureka-instance", e); - } - try { - removeDbDirtyData(host, port); - } catch (Exception e) { - LogUtils.stdOutLogger().error("Failed to remove dirty db-data", e); - } - - context.close(); - } else { - LogUtils.stdOutLogger().error("Upsupported command type: " + args[0]); - } - } - - private static void printIllegalInput(String msg) { - LogUtils.stdOutLogger().error("illegal input: " + msg); - LogUtils.stdOutLogger() - .info( - "[help-message] this app cleans entrance dirty-data. args[0]: entrance-hostname, args[1]: entrance-port"); - return; - } - - private static void removeDirtyEurekaInstance(String host, String port) { - if (StringUtils.isBlank(host)) { - printIllegalInput("host cannot be blank"); - return; - } - if (StringUtils.isBlank(port)) { - EntranceDirtyDataHandler.handleEurekaDirtyData(host); - } else { - EntranceDirtyDataHandler.handleEurekaDirtyData(host, port); - } - } - - private static void removeDbDirtyData(String host, String port) { - if (StringUtils.isBlank(host)) { - printIllegalInput("host cannot be blank"); - return; - } - if (StringUtils.isBlank(port)) { - EntranceDirtyDataHandler.handleDbDirtData(host); - } else { - EntranceDirtyDataHandler.handleDbDirtData(host, port); - } - } -} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java deleted file mode 100644 index 8b4366805e..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplication.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * 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.monitor.scan.app; - -import org.apache.linkis.monitor.scan.app.factory.MapperFactory; -import org.apache.linkis.monitor.scan.app.jobhistory.JobHistoryDataFetcher; -import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrCodeRule; -import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrorCodeAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedRule; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.core.pac.DataFetcher; -import org.apache.linkis.monitor.scan.core.scanner.AnomalyScanner; -import org.apache.linkis.monitor.scan.core.scanner.DefaultScanner; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; -import org.apache.linkis.server.utils.LinkisMainHelper; - -import org.apache.commons.lang3.exception.ExceptionUtils; - -import org.springframework.context.annotation.AnnotationConfigApplicationContext; -import org.springframework.context.support.AbstractApplicationContext; - -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Map; - -import org.slf4j.Logger; - -public class LinkisJobHistoryScanApplication { - private static final Logger logger = LogUtils.stdOutLogger(); - - /** @param args: args[0]: startTime, args[1] endTime */ - public static void main(String[] args) throws ReflectiveOperationException { - - String serviceName = System.getProperty(LinkisMainHelper.SERVER_NAME_KEY()); - LinkisMainHelper.formatPropertyFiles(serviceName); - - long intervalMs = Constants.SCAN_INTERVALS_SECONDS() * 1000; - long maxIntervalMs = Constants.MAX_INTERVALS_SECONDS() * 1000; - long endTime = System.currentTimeMillis(); - long startTime = endTime - intervalMs; - /** parse input into timestamp */ - if (args != null && args.length == 2) { - SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHH"); - Long tmpStart; - Long tmpEnd; - try { - tmpStart = format.parse(args[0]).getTime(); - tmpEnd = format.parse(args[1]).getTime(); - } catch (ParseException e) { - logger.info( - "Failed to parse input into startTime and endTime." + ExceptionUtils.getMessage(e)); - return; - } - if (endTime <= startTime) { - logger.info("[error] startTime larger than endTime"); - return; - } - if (tmpStart != null && tmpEnd != null) { - startTime = tmpStart; - endTime = tmpEnd; - } - } - long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; - - runApp(startTime, endTime, realIntervals, maxIntervalMs); - } - - private static void runApp(long startTime, long endTime, long realIntervals, long maxIntervalMs) { - - AbstractApplicationContext context = - new AnnotationConfigApplicationContext(LinkisJobHistoryScanSpringConfiguration.class); - - AnomalyScanner scanner = new DefaultScanner(); - boolean shouldStart = false; - - List fetchers = generateFetchers(startTime, endTime, maxIntervalMs); - if (fetchers == null) { - logger.warn("generated 0 dataFetchers, plz check input"); - return; - } - - Map errorCodeAlerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); - if (errorCodeAlerts == null || errorCodeAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 errorcode alert from alert-rule properties file."); - } else { - logger.info( - "[INFO] Loaded {} errorcode alert from alert-rules properties file.", - errorCodeAlerts.size()); - shouldStart = true; - addIntervalToImsAlerts(errorCodeAlerts, realIntervals); - JobHistoryErrCodeRule jobHistoryErrCodeRule = - new JobHistoryErrCodeRule( - errorCodeAlerts.keySet(), new JobHistoryErrorCodeAlertSender(errorCodeAlerts)); - scanner.addScanRule(jobHistoryErrCodeRule); - } - - Map jobTimeAlerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts( - Constants.SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC()); - if (jobTimeAlerts == null || jobTimeAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 alerts jobtime alert-rule from alert properties file."); - } else { - logger.info( - "[INFO] Loaded {} alerts jobtime alert-rules from alert properties file.", - jobTimeAlerts.size()); - shouldStart = true; - addIntervalToImsAlerts(jobTimeAlerts, realIntervals); - JobTimeExceedRule jobTimeExceedRule = - new JobTimeExceedRule( - jobTimeAlerts.keySet(), new JobTimeExceedAlertSender(jobTimeAlerts)); - scanner.addScanRule(jobTimeExceedRule); - } - if (shouldStart) { - scanner.addDataFetchers(fetchers); - scanner.run(); - scanner.shutdown(); // wait all alert to be send - } - context.close(); - } - - private static List generateFetchers( - long startTime, long endTime, long maxIntervalMs) { - List ret = new ArrayList<>(); - long pe = endTime; - long ps; - while (pe > startTime) { - ps = pe - maxIntervalMs > startTime ? pe - maxIntervalMs : startTime; - String[] fetcherArgs = new String[] {String.valueOf(ps), String.valueOf(pe)}; - ret.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); - logger.info( - "Generated dataFetcher for startTime: " - + new Date(ps).toString() - + ". EndTime: " - + new Date(pe).toString()); - pe = pe - maxIntervalMs; - } - return ret; - } - - private static void addIntervalToImsAlerts(Map alerts, long realIntervals) { - for (AlertDesc alert : alerts.values()) { - if (!(alert instanceof ImsAlertDesc)) { - logger.info("[warn] ignore wrong alert" + alert); - } else { - ((ImsAlertDesc) alert).hitIntervalMs_$eq(realIntervals); - } - } - } -} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java index 8f73f0349a..5fa3b7a5c0 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java @@ -21,7 +21,6 @@ import java.util.List; - public interface InstanceLabelDao { /** * Remove label diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java index e828328b43..36fb21c13b 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java @@ -79,4 +79,20 @@ public Date getCreateTime() { public void setCreateTime(Date createTime) { this.createTime = createTime; } + + public String getLabelKey() { + return labelKey; + } + + public void setLabelKey(String labelKey) { + this.labelKey = labelKey; + } + + public String getFeature() { + return feature; + } + + public void setFeature(String feature) { + this.feature = feature; + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java index b3774860af..bfdc640652 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java @@ -24,7 +24,6 @@ import java.util.Date; import java.util.List; - public interface JobHistoryMapper { List selectJobHistory(JobHistory jobReq); diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java index fab9a5cbe7..32b47c289c 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java @@ -29,9 +29,9 @@ public class MonitorConfig { public static final CommonVars USER_MODE_TIMEOUT = CommonVars.apply("linkis.monitor.user.timeOut", 300); public static final CommonVars USER_MODE_AUTHTOKEN = - CommonVars.apply("linkis.monitor.user.authToken","VALIDATOR-AUTH"); + CommonVars.apply("linkis.monitor.user.authToken", "VALIDATOR-AUTH"); public static final CommonVars USER_MODE_ENGINE = - CommonVars.apply("linkis.monitor.user.enginelist","[]"); + CommonVars.apply("linkis.monitor.user.enginelist", "[]"); public static final CommonVars ECM_TASK_MAJOR = CommonVars.apply("linkis.monitor.ecmResourceTask.major", 0.03); @@ -43,7 +43,7 @@ public class MonitorConfig { CommonVars.apply("linkis.monitor.metrics.userAuthKey"); public static final CommonVars JOB_HISTORY_TIME_EXCEED = - CommonVars.apply("linkis.monitor.jobhistory.id.timeExceed",0L); + CommonVars.apply("linkis.monitor.jobhistory.id.timeExceed", 0L); public static final CommonVars ENTRANCE_TASK_USERTOTAL = CommonVars.apply("linkis.monitor.entranceTask.userTotalTask", 1000); @@ -52,7 +52,7 @@ public class MonitorConfig { public static final CommonVars ENTRANCE_TASK_TOTAL_MINOR = CommonVars.apply("linkis.monitor.entranceTask.linkisTotalTaskMinor", 10000); public static final CommonVars ENTRANCE_TASK_USERLIST = - CommonVars.apply("linkis.monitor.entranceTask.userlist","[]"); + CommonVars.apply("linkis.monitor.entranceTask.userlist", "[]"); public static final CommonVars SCHEDULED_CONFIG_NUM = CommonVars.apply("linkis.monitor.scheduled.pool.cores.num", 10); @@ -61,11 +61,24 @@ public class MonitorConfig { CommonVars.apply("linkis.monitor.shell.time.out.minute", 30); public static final CommonVars USER_MODE_INTERFACE_TIMEOUT = - CommonVars.apply("linkis.monitor.user.mode.time.out", 30*1000); + CommonVars.apply("linkis.monitor.user.mode.time.out", 30 * 1000); - public static final CommonVars CHATBOT_KEY_ID = CommonVars.apply("linkis.monitor.chatbot.key.id","23e6afad1b78a0c5eed67e4d24de7063"); - public static final CommonVars CHATBOT_TYPE = CommonVars.apply("linkis.monitor.chatbot.type","text"); - public static final CommonVars CHATBOT_SERVICE_NAME= CommonVars.apply("linkis.monitor.chatbot.serviceName","大数据生产助手(BDP_PRD)"); - public static final CommonVars CHATBOT_URL= CommonVars.apply("linkis.monitor.chatbot.url","http://172.21.3.43:1377/pros-chatbot/yuanfang/sendEMsg"); - public static final CommonVars SOLUTION_URL = CommonVars.apply("linkis.monitor.jobhistory.solution.url", "http://kn.dss.weoa.com/linkis/qa"); + public static final CommonVars CHATBOT_KEY_ID = + CommonVars.apply("linkis.monitor.chatbot.key.id", "23e6afad1b78a0c5eed67e4d24de7063"); + public static final CommonVars CHATBOT_TYPE = + CommonVars.apply("linkis.monitor.chatbot.type", "text"); + public static final CommonVars CHATBOT_SERVICE_NAME = + CommonVars.apply("linkis.monitor.chatbot.serviceName", ""); + public static final CommonVars CHATBOT_URL = + CommonVars.apply("linkis.monitor.chatbot.url", ""); + public static final CommonVars SOLUTION_URL = + CommonVars.apply( + "linkis.monitor.jobhistory.solution.url", + "https://linkis.apache.org/docs/latest/tuning-and-troubleshooting/error-guide/error-code"); + + public static final CommonVars TASK_RUNTIME_TIMEOUT_DESC = + CommonVars.apply( + "linkis.monitor.jobhistory.task.timeout.desc", + "[Linkis任务信息]您好,您在Linkis/DSS提交的任务(任务ID:{0}),已经运行超过{1}h," + + "请关注是否任务正常,如果不正常您可以到Linkis/DSS管理台进行任务的kill,集群信息为BDAP({2})。详细解决方案见Q47:{3} "); } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java index 3cf288adbb..4f912b9e82 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java @@ -20,68 +20,78 @@ import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; public class ChatbotEntity { - private String keyid; - private String content; - private String type; - private String userName; - private String serviceName; + private String keyid; + private String content; + private String type; + private String userName; + private String serviceName; - public ChatbotEntity(String content, String userName) { - this.keyid = MonitorConfig.CHATBOT_KEY_ID.getValue(); - this.content = content; - this.type = MonitorConfig.CHATBOT_TYPE.getValue(); - this.userName = userName; - this.serviceName = MonitorConfig.CHATBOT_SERVICE_NAME.getValue(); - } + public ChatbotEntity(String content, String userName) { + this.keyid = MonitorConfig.CHATBOT_KEY_ID.getValue(); + this.content = content; + this.type = MonitorConfig.CHATBOT_TYPE.getValue(); + this.userName = userName; + this.serviceName = MonitorConfig.CHATBOT_SERVICE_NAME.getValue(); + } - public String getKeyid() { - return keyid; - } + public String getKeyid() { + return keyid; + } - public void setKeyid(String keyid) { - this.keyid = keyid; - } + public void setKeyid(String keyid) { + this.keyid = keyid; + } - public String getContent() { - return content; - } + public String getContent() { + return content; + } - public void setContent(String content) { - this.content = content; - } + public void setContent(String content) { + this.content = content; + } - public String getType() { - return type; - } + public String getType() { + return type; + } - public void setType(String type) { - this.type = type; - } + public void setType(String type) { + this.type = type; + } - public String getUserName() { - return userName; - } + public String getUserName() { + return userName; + } - public void setUserName(String userName) { - this.userName = userName; - } + public void setUserName(String userName) { + this.userName = userName; + } - public String getServiceName() { - return serviceName; - } + public String getServiceName() { + return serviceName; + } - public void setServiceName(String serviceNameuserName) { - this.serviceName = serviceNameuserName; - } + public void setServiceName(String serviceNameuserName) { + this.serviceName = serviceNameuserName; + } - @Override - public String toString() { - return "ChatbotEntity{" + - "keyid='" + keyid + '\'' + - ", content='" + content + '\'' + - ", type='" + type + '\'' + - ", userName='" + userName + '\'' + - ", serviceName='" + serviceName + '\'' + - '}'; - } + @Override + public String toString() { + return "ChatbotEntity{" + + "keyid='" + + keyid + + '\'' + + ", content='" + + content + + '\'' + + ", type='" + + type + + '\'' + + ", userName='" + + userName + + '\'' + + ", serviceName='" + + serviceName + + '\'' + + '}'; + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java index 75d415ac0d..11ee06226f 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java @@ -29,7 +29,6 @@ import org.slf4j.Logger; - @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class BmlClear { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java index dced6117b1..ac9cb4ea61 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java @@ -30,9 +30,7 @@ import org.slf4j.Logger; -/*** - * Task: clean up linkis_cg_ec_resource_info_record data - */ +/** * Task: clean up linkis_cg_ec_resource_info_record data */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class EcRecordClear { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java index 4880a600cc..ae14770420 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java @@ -30,9 +30,7 @@ import org.slf4j.Logger; -/*** - * Task: clean up linkis_ps_job_history_group_history data - */ +/** * Task: clean up linkis_ps_job_history_group_history data */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class JobHistoryClear { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java index 01873a988b..fbca945a8d 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java @@ -25,8 +25,8 @@ import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedRule; import org.apache.linkis.monitor.scan.app.jobhistory.labels.JobHistoryLabelsAlertSender; import org.apache.linkis.monitor.scan.app.jobhistory.labels.JobHistoryLabelsRule; -import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonRunTimeAlertSender; import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonJobRunTimeRule; +import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonRunTimeAlertSender; import org.apache.linkis.monitor.scan.app.jobhistory.runtime.JobHistoryRunTimeAlertSender; import org.apache.linkis.monitor.scan.app.jobhistory.runtime.JobHistoryRunTimeRule; import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; @@ -37,7 +37,7 @@ import org.apache.linkis.monitor.scan.core.scanner.DefaultScanner; import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; import org.apache.linkis.monitor.scan.utils.alert.ims.UserLabelAlertUtils; import org.apache.linkis.monitor.scan.utils.log.LogUtils; @@ -49,216 +49,219 @@ import org.slf4j.Logger; -/*** - * jobHistory monitor - * 1.errorCode: Monitor the error code - * 2.userLabel: tenant label monitoring, scan the execution data within the first 20 minutes, and judge the labels field of the data - * 3.jobResultRunTime: Scan the execution data within the first 20 minutes, and judge the completed tasks. If the parm field in the jobhistory contains (task.notification.conditions) and the result of executing the task is (Succeed, Failed, Canceled, Timeout, ALL) any one of them, an alarm is triggered and the result of the job is that it has ended. All three are indispensable - * 4.jobResultRunTimeForDSS: Scan the execution data within the first 20 minutes, scan the tasks that have been marked for notification, if the task has ended, a notification will be initiated - * 5.jobHistoryUnfinishedScan: monitor the status of the execution task, scan the data outside 12 hours and within 24 hours +/** + * * jobHistory monitor 1.errorCode: Monitor the error code 2.userLabel: tenant label monitoring, + * scan the execution data within the first 20 minutes, and judge the labels field of the data + * 3.jobResultRunTime: Scan the execution data within the first 20 minutes, and judge the completed + * tasks. If the parm field in the jobhistory contains (task.notification.conditions) and the result + * of executing the task is (Succeed, Failed, Canceled, Timeout, ALL) any one of them, an alarm is + * triggered and the result of the job is that it has ended. All three are indispensable + * 4.jobResultRunTimeForDSS: Scan the execution data within the first 20 minutes, scan the tasks + * that have been marked for notification, if the task has ended, a notification will be initiated + * 5.jobHistoryUnfinishedScan: monitor the status of the execution task, scan the data outside 12 + * hours and within 24 hours */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class JobHistoryMonitor { - private static final Logger logger = LogUtils.stdOutLogger(); - private static final long backtrackNum = 1000000L; + private static final Logger logger = LogUtils.stdOutLogger(); + private static final long backtrackNum = 1000000L; - /** - * Scan tasks completed within 20 minutes - */ - @Scheduled(cron = "0 0/10 * * * ?") - public void jobHistoryFinishedScan() { - long intervalMs = 20 * 60 * 1000; // 20分钟 - long maxIntervalMs = Constants.MAX_INTERVALS_SECONDS() * 1000; - long endTime = System.currentTimeMillis(); - long startTime = endTime - intervalMs; - long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; - AnomalyScanner scanner = new DefaultScanner(); - boolean shouldStart = false; - long id; - if (null == CacheUtils.cacheBuilder.getIfPresent("jobHistoryId")) { - //20230206:新增获取最大值-100W 作为初始id进行查询,防止慢查询 - long maxId = MapperFactory.getJobHistoryMapper().selectMaxId(); - long beginId = 0L; - if (maxId > backtrackNum) { - beginId = maxId - backtrackNum; - } - id = MapperFactory.getJobHistoryMapper().selectIdByHalfDay(beginId); - CacheUtils.cacheBuilder.put("jobHistoryId", id); - } else { - id = CacheUtils.cacheBuilder.getIfPresent("jobHistoryId"); - } - List fetchers = generateFetchersfortime(startTime, endTime, id, "updated_time"); - if (fetchers == null) { - logger.warn("generated 0 dataFetchers, plz check input"); - return; - } - // errorCode - try { - Map errorCodeAlerts = JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); - - if (errorCodeAlerts == null || errorCodeAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 errorcode alert from alert-rule properties file."); - } else { - logger.info( - "[INFO] Loaded {} errorcode alert from alert-rules properties file.", - errorCodeAlerts.size()); - shouldStart = true; - addIntervalToImsAlerts(errorCodeAlerts, realIntervals); - JobHistoryErrCodeRule jobHistoryErrCodeRule = - new JobHistoryErrCodeRule( - errorCodeAlerts.keySet(), new JobHistoryErrorCodeAlertSender(errorCodeAlerts)); - scanner.addScanRule(jobHistoryErrCodeRule); - } - } catch (Exception e) { - logger.warn("Jobhistory Monitor ErrorCode Faily: "+ e.getMessage()); - } - // userLabel - try { - Map userLabelAlerts = - UserLabelAlertUtils.getAlerts(Constants.USER_LABEL_MONITOR(), ""); - if (userLabelAlerts == null || userLabelAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 alerts userLabel alert-rule from alert properties file."); - } else { - logger.info( - "[INFO] Loaded {} alerts userLabel alert-rules from alert properties file.", - userLabelAlerts.size()); - shouldStart = true; - JobHistoryLabelsRule jobHistoryLabelsRule = - new JobHistoryLabelsRule(new JobHistoryLabelsAlertSender()); - scanner.addScanRule(jobHistoryLabelsRule); - } - } catch (Exception e) { - logger.warn("Jobhistory Monitor UserLabel Faily: "+ e.getMessage()); - } - // jobResultRunTime - try { - Map jobResultAlerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); - if (jobResultAlerts == null || jobResultAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 jobResult alert from alert-rule properties file."); - } else { - logger.info( - "[INFO] Loaded {} alerts jobResult alert-rules from alert properties file.", - jobResultAlerts.size()); - shouldStart = true; - JobHistoryRunTimeRule jobHistoryRunTimeRule = - new JobHistoryRunTimeRule(new JobHistoryRunTimeAlertSender()); - scanner.addScanRule(jobHistoryRunTimeRule); - } - } catch (Exception e) { - logger.warn("Jobhistory Monitor JobResultRunTime Faily: "+ e.getMessage()); - } - // jobResultRunTimeForDSS - try { - Map dssJobResultAlerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); - if (dssJobResultAlerts == null || dssJobResultAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 jobResult alert from alert-rule properties file."); - } else { - logger.info( - "[INFO] Loaded {} alerts jobResult alert-rules from alert properties file.", - dssJobResultAlerts.size()); - shouldStart = true; + /** Scan tasks completed within 20 minutes */ + @Scheduled(cron = "0 0/10 * * * ?") + public void jobHistoryFinishedScan() { + long intervalMs = 20 * 60 * 1000; // 20分钟 + long maxIntervalMs = Constants.ERRORCODE_MAX_INTERVALS_SECONDS() * 1000; + long endTime = System.currentTimeMillis(); + long startTime = endTime - intervalMs; + long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + AnomalyScanner scanner = new DefaultScanner(); + boolean shouldStart = false; + long id; + if (null == CacheUtils.cacheBuilder.getIfPresent("jobHistoryId")) { + long maxId = MapperFactory.getJobHistoryMapper().selectMaxId(); + long beginId = 0L; + if (maxId > backtrackNum) { + beginId = maxId - backtrackNum; + } + id = MapperFactory.getJobHistoryMapper().selectIdByHalfDay(beginId); + CacheUtils.cacheBuilder.put("jobHistoryId", id); + } else { + id = CacheUtils.cacheBuilder.getIfPresent("jobHistoryId"); + } + List fetchers = generateFetchersfortime(startTime, endTime, id, "updated_time"); + if (fetchers == null) { + logger.warn("generated 0 dataFetchers, plz check input"); + return; + } + // errorCode + try { + Map errorCodeAlerts = + MonitorAlertUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE(), null); - CommonJobRunTimeRule commonJobRunTimeRule = - new CommonJobRunTimeRule(new CommonRunTimeAlertSender()); - scanner.addScanRule(commonJobRunTimeRule); - } - } catch (Exception e) { - logger.warn("Jobhistory JobResultRunTimeForDSS ErrorCode Faily: "+ e.getMessage()); - } - run(scanner, fetchers, shouldStart); + if (errorCodeAlerts == null || errorCodeAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 errorcode alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} errorcode alert from alert-rules properties file.", + errorCodeAlerts.size()); + shouldStart = true; + addIntervalToImsAlerts(errorCodeAlerts, realIntervals); + JobHistoryErrCodeRule jobHistoryErrCodeRule = + new JobHistoryErrCodeRule( + errorCodeAlerts.keySet(), new JobHistoryErrorCodeAlertSender(errorCodeAlerts)); + scanner.addScanRule(jobHistoryErrCodeRule); + } + } catch (Exception e) { + logger.warn("Jobhistory Monitor ErrorCode Faily: " + e.getMessage()); + } + // userLabel + try { + Map userLabelAlerts = + UserLabelAlertUtils.getAlerts(Constants.USER_LABEL_MONITOR(), ""); + if (userLabelAlerts == null || userLabelAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 alerts userLabel alert-rule from alert properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts userLabel alert-rules from alert properties file.", + userLabelAlerts.size()); + shouldStart = true; + JobHistoryLabelsRule jobHistoryLabelsRule = + new JobHistoryLabelsRule(new JobHistoryLabelsAlertSender()); + scanner.addScanRule(jobHistoryLabelsRule); + } + } catch (Exception e) { + logger.warn("Jobhistory Monitor UserLabel Faily: " + e.getMessage()); + } + // jobResultRunTime + try { + Map jobResultAlerts = + MonitorAlertUtils.getAlerts((Constants.SCAN_PREFIX_ERRORCODE()), null); + if (jobResultAlerts == null || jobResultAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 jobResult alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobResult alert-rules from alert properties file.", + jobResultAlerts.size()); + shouldStart = true; + JobHistoryRunTimeRule jobHistoryRunTimeRule = + new JobHistoryRunTimeRule(new JobHistoryRunTimeAlertSender()); + scanner.addScanRule(jobHistoryRunTimeRule); + } + } catch (Exception e) { + logger.warn("Jobhistory Monitor JobResultRunTime Faily: " + e.getMessage()); } + // jobResultRunTimeForDSS + try { + Map dssJobResultAlerts = + MonitorAlertUtils.getAlerts((Constants.SCAN_PREFIX_ERRORCODE()), null); + if (dssJobResultAlerts == null || dssJobResultAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 jobResult alert from alert-rule properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobResult alert-rules from alert properties file.", + dssJobResultAlerts.size()); + shouldStart = true; - /*** - * Whether scanning data within 12 hours has timed out - */ - @Scheduled(cron = "${linkis.monitor.jobHistory.ScanTask.cron}") - public void jobHistoryUnfinishedScan() { - long id = - Optional.ofNullable(CacheUtils.cacheBuilder.getIfPresent("jobhistoryScan")) - .orElse(MonitorConfig.JOB_HISTORY_TIME_EXCEED.getValue()); - long intervalMs = Constants.SCAN_INTERVALS_SECONDS() * 1000; - long maxIntervalMs = Constants.MAX_INTERVALS_SECONDS() * 1000; - long endTime = System.currentTimeMillis(); - long startTime = endTime - intervalMs; - long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; - AnomalyScanner scanner = new DefaultScanner(); - boolean shouldStart = false; - List fetchers = generateFetchers(startTime, endTime, maxIntervalMs, id, "created_time"); - if (fetchers == null) { - logger.warn("generated 0 dataFetchers, plz check input"); - return; - } - Map jobTimeAlerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts( - Constants.SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC()); - if (jobTimeAlerts == null || jobTimeAlerts.size() == 0) { - logger.info("[INFO] Loaded 0 alerts jobtime alert-rule from alert properties file."); - } else { - logger.info( - "[INFO] Loaded {} alerts jobtime alert-rules from alert properties file.", - jobTimeAlerts.size()); - shouldStart = true; - addIntervalToImsAlerts(jobTimeAlerts, realIntervals); - JobTimeExceedRule jobTimeExceedRule = - new JobTimeExceedRule( - jobTimeAlerts.keySet(), new JobTimeExceedAlertSender(jobTimeAlerts)); - scanner.addScanRule(jobTimeExceedRule); - } - run(scanner, fetchers, shouldStart); + CommonJobRunTimeRule commonJobRunTimeRule = + new CommonJobRunTimeRule(new CommonRunTimeAlertSender()); + scanner.addScanRule(commonJobRunTimeRule); + } + } catch (Exception e) { + logger.warn("Jobhistory JobResultRunTimeForDSS ErrorCode Faily: " + e.getMessage()); } + run(scanner, fetchers, shouldStart); + } - public static void run(AnomalyScanner scanner, List fetchers, Boolean shouldStart) { - if (shouldStart) { - scanner.addDataFetchers(fetchers); - scanner.run(); - // scanner.shutdown(); // wait all alert to be send - } + /** * Whether scanning data within 12 hours has timed out */ + @Scheduled(cron = "${linkis.monitor.jobHistory.ScanTask.cron}") + public void jobHistoryUnfinishedScan() { + long id = + Optional.ofNullable(CacheUtils.cacheBuilder.getIfPresent("jobhistoryScan")) + .orElse(MonitorConfig.JOB_HISTORY_TIME_EXCEED.getValue()); + long intervalMs = Constants.ERRORCODE_SCAN_INTERVALS_SECONDS() * 1000; + long maxIntervalMs = Constants.ERRORCODE_MAX_INTERVALS_SECONDS() * 1000; + long endTime = System.currentTimeMillis(); + long startTime = endTime - intervalMs; + long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + AnomalyScanner scanner = new DefaultScanner(); + boolean shouldStart = false; + List fetchers = + generateFetchers(startTime, endTime, maxIntervalMs, id, "created_time"); + if (fetchers == null) { + logger.warn("generated 0 dataFetchers, plz check input"); + return; } + Map jobTimeAlerts = + MonitorAlertUtils.getAlerts((Constants.SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC()), null); + if (jobTimeAlerts == null || jobTimeAlerts.size() == 0) { + logger.info("[INFO] Loaded 0 alerts jobtime alert-rule from alert properties file."); + } else { + logger.info( + "[INFO] Loaded {} alerts jobtime alert-rules from alert properties file.", + jobTimeAlerts.size()); + shouldStart = true; + addIntervalToImsAlerts(jobTimeAlerts, realIntervals); + JobTimeExceedRule jobTimeExceedRule = + new JobTimeExceedRule( + jobTimeAlerts.keySet(), new JobTimeExceedAlertSender(jobTimeAlerts)); + scanner.addScanRule(jobTimeExceedRule); + } + run(scanner, fetchers, shouldStart); + } - private static List generateFetchers( - long startTime, long endTime, long maxIntervalMs, long id, String timeType) { - List ret = new ArrayList<>(); - long pe = endTime; - long ps; - while (pe > startTime) { - ps = pe - maxIntervalMs > startTime ? pe - maxIntervalMs : startTime; - String[] fetcherArgs = - new String[]{String.valueOf(ps), String.valueOf(pe), String.valueOf(id), timeType}; - ret.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); - logger.info( - "Generated dataFetcher for startTime: " - + new Date(ps).toString() - + ". EndTime: " - + new Date(pe).toString()); - pe = pe - maxIntervalMs; - } - return ret; + public static void run(AnomalyScanner scanner, List fetchers, Boolean shouldStart) { + if (shouldStart) { + scanner.addDataFetchers(fetchers); + scanner.run(); + // scanner.shutdown(); // wait all alert to be send } + } - private static List generateFetchersfortime(long startTime, long endTime, long id, String timeType) { - List fetchers = new ArrayList<>(); - String[] fetcherArgs = - new String[]{String.valueOf(startTime), String.valueOf(endTime), String.valueOf(id), timeType}; - fetchers.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); - logger.info( - "Generated dataFetcher for startTime: " - + new Date(startTime).toString() - + ". EndTime: " - + new Date(endTime).toString()); - return fetchers; + private static List generateFetchers( + long startTime, long endTime, long maxIntervalMs, long id, String timeType) { + List ret = new ArrayList<>(); + long pe = endTime; + long ps; + while (pe > startTime) { + ps = pe - maxIntervalMs > startTime ? pe - maxIntervalMs : startTime; + String[] fetcherArgs = + new String[] {String.valueOf(ps), String.valueOf(pe), String.valueOf(id), timeType}; + ret.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); + logger.info( + "Generated dataFetcher for startTime: " + + new Date(ps).toString() + + ". EndTime: " + + new Date(pe).toString()); + pe = pe - maxIntervalMs; } + return ret; + } + + private static List generateFetchersfortime( + long startTime, long endTime, long id, String timeType) { + List fetchers = new ArrayList<>(); + String[] fetcherArgs = + new String[] { + String.valueOf(startTime), String.valueOf(endTime), String.valueOf(id), timeType + }; + fetchers.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); + logger.info( + "Generated dataFetcher for startTime: " + + new Date(startTime).toString() + + ". EndTime: " + + new Date(endTime).toString()); + return fetchers; + } - private static void addIntervalToImsAlerts(Map alerts, long realIntervals) { - for (AlertDesc alert : alerts.values()) { - if (!(alert instanceof ImsAlertDesc)) { - logger.info("[warn] ignore wrong alert" + alert); - } else { - ((ImsAlertDesc) alert).hitIntervalMs_$eq(realIntervals); - } - } + private static void addIntervalToImsAlerts(Map alerts, long realIntervals) { + for (AlertDesc alert : alerts.values()) { + if (!(alert instanceof ImsAlertDesc)) { + logger.info("[warn] ignore wrong alert" + alert); + } else { + ((ImsAlertDesc) alert).hitIntervalMs_$eq(realIntervals); + } } + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java index b7066ba420..459aaf70ff 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java @@ -41,9 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/*** - * Monitor the usage of ECM resources for monitoring and metrics reporting - */ +/** * Monitor the usage of ECM resources for monitoring and metrics reporting */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class ResourceMonitor { @@ -68,7 +66,7 @@ public void ecmResourceTask() { List> emNodeVoList = data.getOrDefault("EMs", new ArrayList<>()); StringJoiner minor = new StringJoiner(","); StringJoiner major = new StringJoiner(","); - // deal ecm resource + // deal ecm resource emNodeVoList.forEach( emNodeVo -> { Map leftResource = MapUtils.getMap(emNodeVo, "leftResource"); @@ -78,8 +76,10 @@ public void ecmResourceTask() { labels.stream() .filter(labelmap -> labelmap.containsKey("tenant")) .forEach(map -> tenant.set("租户标签:" + map.get("stringValue").toString())); - String leftmemory = ByteTimeUtils.bytesToString((long) leftResource.getOrDefault("memory",0)); - String maxmemory = ByteTimeUtils.bytesToString((long) maxResource.getOrDefault("memory",0)); + String leftmemory = + ByteTimeUtils.bytesToString((long) leftResource.getOrDefault("memory", 0)); + String maxmemory = + ByteTimeUtils.bytesToString((long) maxResource.getOrDefault("memory", 0)); String leftmemoryStr = leftmemory.split(" ")[0]; String maxmemoryStr = maxmemory.split(" ")[0]; @@ -94,9 +94,12 @@ public void ecmResourceTask() { BigDecimal maxMemory = new BigDecimal(maxmemoryStr); BigDecimal maxCores = new BigDecimal((int) maxResource.get("cores")); BigDecimal maxInstance = new BigDecimal((int) maxResource.get("instance")); - double memorydouble = leftMemory.divide(maxMemory,2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); - double coresdouble = leftCores.divide(maxCores,2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); - double instancedouble = leftInstance.divide(maxInstance,2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); + double memorydouble = + leftMemory.divide(maxMemory, 2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); + double coresdouble = + leftCores.divide(maxCores, 2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); + double instancedouble = + leftInstance.divide(maxInstance, 2, BigDecimal.ROUND_HALF_DOWN).doubleValue(); Double majorValue = MonitorConfig.ECM_TASK_MAJOR.getValue(); Double minorValue = MonitorConfig.ECM_TASK_MINOR.getValue(); if (((memorydouble) <= majorValue) @@ -126,26 +129,38 @@ public void ecmResourceTask() { MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), replaceParm); PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12003")); } - // ECM资源占比上报 - resourceSendToIms(coresdouble, memorydouble, instancedouble, HttpsUntils.localHost,"USED"); - }); - //ECM 剩余资源总数上报 - resourceSendToIms(totalCores.get(), totalMemory.get(), totalInstance.get(), HttpsUntils.localHost,"TOTAL"); - } + // ECM资源占比上报 + resourceSendToIms( + coresdouble, memorydouble, instancedouble, HttpsUntils.localHost, "USED"); + }); + // ECM 剩余资源总数上报 + resourceSendToIms( + totalCores.get(), totalMemory.get(), totalInstance.get(), HttpsUntils.localHost, "TOTAL"); + } - private void resourceSendToIms(Double coresdouble, Double memorydouble, Double instancedouble, String loaclhost, String name) { - List list = new ArrayList<>(); - logger.info("ResourceMonitor send index "); - String core ="ECM_CPU_"; - String memory ="ECM_MEMORY_"; - String instance ="ECM_INSTANCE_"; - list.add(new IndexEntity(core.concat(name), "CPU", "INDEX", loaclhost, String.valueOf(coresdouble))); - list.add(new IndexEntity(memory.concat(name), "MEMORY", "INDEX", loaclhost, String.valueOf(memorydouble))); - list.add(new IndexEntity(instance.concat(name), "INSTANCE", "INDEX", loaclhost, String.valueOf(instancedouble))); - try { - HttpsUntils.sendIndex(list); - } catch (IOException e) { - logger.warn("failed to send EcmResource index"); - } + private void resourceSendToIms( + Double coresdouble, + Double memorydouble, + Double instancedouble, + String loaclhost, + String name) { + List list = new ArrayList<>(); + logger.info("ResourceMonitor send index "); + String core = "ECM_CPU_"; + String memory = "ECM_MEMORY_"; + String instance = "ECM_INSTANCE_"; + list.add( + new IndexEntity(core.concat(name), "CPU", "INDEX", loaclhost, String.valueOf(coresdouble))); + list.add( + new IndexEntity( + memory.concat(name), "MEMORY", "INDEX", loaclhost, String.valueOf(memorydouble))); + list.add( + new IndexEntity( + instance.concat(name), "INSTANCE", "INDEX", loaclhost, String.valueOf(instancedouble))); + try { + HttpsUntils.sendIndex(list); + } catch (IOException e) { + logger.warn("failed to send EcmResource index"); } + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java index bae3dc53f7..ae7e1bc21d 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java @@ -30,9 +30,7 @@ import org.slf4j.Logger; -/*** - * Task: clean up logs, file data of ec materials - */ +/** * Task: clean up logs, file data of ec materials */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class TaskLogClear { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java index 87bc6d6e22..5f6728ef62 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java @@ -43,9 +43,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/*** - * Entrance monitors the number of tasks for specified users and systems. - * If the configured threshold is exceeded, an alarm will be triggered. +/** + * * Entrance monitors the number of tasks for specified users and systems. If the configured + * threshold is exceeded, an alarm will be triggered. */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") @@ -56,7 +56,6 @@ public class TaskMonitor { private static final String ENTRANCE_RUNNING_TASK = "entrance_running_task"; private static final String ENTRANCE_QUEUED_TASK = "entrance_queued_task"; - @Scheduled(cron = "${linkis.monitor.entranceTask.cron}") public void entranceTask() throws IOException { List> userlist = new ArrayList<>(); @@ -71,7 +70,8 @@ public void entranceTask() throws IOException { try { data = MapUtils.getMap( - HttpsUntils.getEntranceTask(null, entranceEntity.get("username"),null), "data"); + HttpsUntils.getEntranceTask(null, entranceEntity.get("username"), null), + "data"); logger.info("TaskMonitor userlist response {}:", data); } catch (IOException e) { logger.warn("failed to get EntranceTask data"); @@ -119,7 +119,7 @@ public void entranceTask() throws IOException { }); Map likisData = null; try { - likisData = MapUtils.getMap(HttpsUntils.getEntranceTask(null, "hadoop",null), "data"); + likisData = MapUtils.getMap(HttpsUntils.getEntranceTask(null, "hadoop", null), "data"); logger.info("TaskMonitor hadoop response {}:", likisData); } catch (IOException e) { logger.warn("failed to get EntranceTask data"); @@ -149,35 +149,49 @@ public void entranceTask() throws IOException { MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12008")); } - //指标上报 + // 指标上报 resourceSendToIms(); } - public static void resourceSendToIms() { - //获取所有的entrance实例,逐个上送IMS - ServiceInstance[] instances = Sender.getInstances(Constants.DIRTY_DATA_ENTRANCE_APPLICATIONNAME()); - if (null != instances) { - for (ServiceInstance instance : instances) { - String serviceInstance = instance.getInstance(); - try { - Map instanceData = MapUtils.getMap(HttpsUntils.getEntranceTask(null, "hadoop", serviceInstance), "data"); - int runningNumber = 0; - int queuedNumber = 0; - if (instanceData.containsKey("runningNumber")) { - runningNumber = (int) instanceData.get("runningNumber"); - } - if (instanceData.containsKey("queuedNumber")) { - queuedNumber = (int) instanceData.get("queuedNumber"); - } - logger.info("ResourceMonitor send index "); - List list = new ArrayList<>(); - list.add(new IndexEntity(serviceInstance, "entrance", ENTRANCE_RUNNING_TASK, HttpsUntils.localHost, String.valueOf(runningNumber))); - list.add(new IndexEntity(serviceInstance, "entrance", ENTRANCE_QUEUED_TASK, HttpsUntils.localHost, String.valueOf(queuedNumber))); - HttpsUntils.sendIndex(list); - } catch (IOException e) { - logger.warn("failed to send EcmResource index :" + e); - } - } + public static void resourceSendToIms() { + // 获取所有的entrance实例,逐个上送IMS + ServiceInstance[] instances = + Sender.getInstances(Constants.DIRTY_DATA_ENTRANCE_APPLICATIONNAME()); + if (null != instances) { + for (ServiceInstance instance : instances) { + String serviceInstance = instance.getInstance(); + try { + Map instanceData = + MapUtils.getMap(HttpsUntils.getEntranceTask(null, "hadoop", serviceInstance), "data"); + int runningNumber = 0; + int queuedNumber = 0; + if (instanceData.containsKey("runningNumber")) { + runningNumber = (int) instanceData.get("runningNumber"); + } + if (instanceData.containsKey("queuedNumber")) { + queuedNumber = (int) instanceData.get("queuedNumber"); + } + logger.info("ResourceMonitor send index "); + List list = new ArrayList<>(); + list.add( + new IndexEntity( + serviceInstance, + "entrance", + ENTRANCE_RUNNING_TASK, + HttpsUntils.localHost, + String.valueOf(runningNumber))); + list.add( + new IndexEntity( + serviceInstance, + "entrance", + ENTRANCE_QUEUED_TASK, + HttpsUntils.localHost, + String.valueOf(queuedNumber))); + HttpsUntils.sendIndex(list); + } catch (IOException e) { + logger.warn("failed to send EcmResource index :" + e); } + } } + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java index 03db536344..c642a24762 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java @@ -35,8 +35,10 @@ import org.apache.linkis.ujes.client.response.GetTableStatisticInfoResult; import org.apache.linkis.ujes.client.response.JobExecuteResult; import org.apache.linkis.ujes.client.response.JobInfoResult; + import org.springframework.scheduling.annotation.Scheduled; import org.springframework.stereotype.Component; + import java.net.SocketTimeoutException; import java.util.*; import java.util.concurrent.TimeUnit; @@ -45,112 +47,130 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/*** - * User mode monitoring: regularly trigger scripts to monitor whether the engine status is running normally +/** + * * User mode monitoring: regularly trigger scripts to monitor whether the engine status is running + * normally */ @Component public class UserModeMonitor { - private static final Logger logger = LoggerFactory.getLogger(UserModeMonitor.class); + private static final Logger logger = LoggerFactory.getLogger(UserModeMonitor.class); - private static final DWSClientConfig clientConfig = HttpsUntils.dwsClientConfig; + private static final DWSClientConfig clientConfig = HttpsUntils.dwsClientConfig; - private static final UJESClient client = new UJESClientImpl(clientConfig); + private static final UJESClient client = new UJESClientImpl(clientConfig); - @Scheduled(cron = "${linkis.monitor.user.cron}") - public void job() { - Optional.ofNullable(MonitorConfig.USER_MODE_ENGINE.getValue()).ifPresent(configStr -> { - ArrayList> userModeStr = - BDPJettyServerHelper.gson().fromJson(configStr, ArrayList.class); - userModeStr.forEach(engine -> { - // 3. build job and execute - JobExecuteResult jobExecuteResult = toSubmit(engine); + @Scheduled(cron = "${linkis.monitor.user.cron}") + public void job() { + Optional.ofNullable(MonitorConfig.USER_MODE_ENGINE.getValue()) + .ifPresent( + configStr -> { + ArrayList> userModeStr = + BDPJettyServerHelper.gson().fromJson(configStr, ArrayList.class); + userModeStr.forEach( + engine -> { + // 3. build job and execute + JobExecuteResult jobExecuteResult = toSubmit(engine); + logger.info( + "start run engineType: {},job id : {}", + engine.get("engineType"), + jobExecuteResult.taskID()); + HashMap parms = new HashMap<>(); + parms.put("$engineType", engine.get("engineType")); + parms.put("$url", MonitorConfig.GATEWAY_URL.getValue()); + parms.put("$jobId", jobExecuteResult.taskID()); + Utils.sleepQuietly(MonitorConfig.USER_MODE_TIMEOUT.getValue() * 1000); + JobInfoResult jobInfo = client.getJobInfo(jobExecuteResult); + if (jobInfo.isCompleted()) { + if (jobInfo.getJobStatus().equals("Failed")) { logger.info( - "start run engineType: {},job id : {}", - engine.get("engineType"), - jobExecuteResult.taskID()); - HashMap parms = new HashMap<>(); - parms.put("$engineType", engine.get("engineType")); - parms.put("$url", MonitorConfig.GATEWAY_URL.getValue()); - parms.put("$jobId", jobExecuteResult.taskID()); - Utils.sleepQuietly(MonitorConfig.USER_MODE_TIMEOUT.getValue() * 1000); - JobInfoResult jobInfo = client.getJobInfo(jobExecuteResult); - if (jobInfo.isCompleted()) { - if (jobInfo.getJobStatus().equals("Failed")) { - logger.info("run fail engineType: {},job id : {}", engine.get("engineType"), jobExecuteResult.taskID()); - RequestPersistTask requestPersistTask = jobInfo.getRequestPersistTask(); - parms.put("$errorCode", String.valueOf(requestPersistTask.getErrCode())); - parms.put("$errorMsg", requestPersistTask.getErrDesc()); - Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); - PooledImsAlertUtils.addAlert(failedAlerts.get("12012")); - } - } else { - logger.info("run timeout engineType: {},job id : {}", engine.get("engineType"), jobExecuteResult.taskID()); - Map alerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); - PooledImsAlertUtils.addAlert(alerts.get("12011")); - } + "run fail engineType: {},job id : {}", + engine.get("engineType"), + jobExecuteResult.taskID()); + RequestPersistTask requestPersistTask = jobInfo.getRequestPersistTask(); + parms.put("$errorCode", String.valueOf(requestPersistTask.getErrCode())); + parms.put("$errorMsg", requestPersistTask.getErrDesc()); + Map failedAlerts = + MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(failedAlerts.get("12012")); + } + } else { + logger.info( + "run timeout engineType: {},job id : {}", + engine.get("engineType"), + jobExecuteResult.taskID()); + Map alerts = + MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + PooledImsAlertUtils.addAlert(alerts.get("12011")); } - ); - }); - } + }); + }); + } - private static JobExecuteResult toSubmit(LinkedTreeMap engine) { - // 1. build params - // set label map :EngineTypeLabel/UserCreatorLabel/EngineRunTypeLabel/Tenant - Map labels = new HashMap(); - labels.put(LabelKeyConstant.ENGINE_TYPE_KEY, engine.get("engineType")); // required engineType Label - labels.put(LabelKeyConstant.USER_CREATOR_TYPE_KEY, engine.get("executeUser") + "-IDE");// required execute user and creator eg:hadoop-IDE - labels.put(LabelKeyConstant.CODE_TYPE_KEY, engine.get("runType")); // required codeType - Map startupMap = new HashMap(16); - // setting linkis params - //startupMap.put("wds.linkis.rm.yarnqueue", "dws"); - // 2. build jobSubmitAction - JobSubmitAction jobSubmitAction = JobSubmitAction.builder() - .addExecuteCode(engine.get("code")) - .setStartupParams(startupMap) - .setUser(engine.get("executeUser")) //submit user - .addExecuteUser(engine.get("executeUser")) // execute user - .setLabels(labels) - .build(); - // 3. to execute - return client.submit(jobSubmitAction); - } + private static JobExecuteResult toSubmit(LinkedTreeMap engine) { + // 1. build params + // set label map :EngineTypeLabel/UserCreatorLabel/EngineRunTypeLabel/Tenant + Map labels = new HashMap(); + labels.put( + LabelKeyConstant.ENGINE_TYPE_KEY, engine.get("engineType")); // required engineType Label + labels.put( + LabelKeyConstant.USER_CREATOR_TYPE_KEY, + engine.get("executeUser") + "-IDE"); // required execute user and creator eg:hadoop-IDE + labels.put(LabelKeyConstant.CODE_TYPE_KEY, engine.get("runType")); // required codeType + Map startupMap = new HashMap(16); + // setting linkis params + // startupMap.put("wds.linkis.rm.yarnqueue", "dws"); + // 2. build jobSubmitAction + JobSubmitAction jobSubmitAction = + JobSubmitAction.builder() + .addExecuteCode(engine.get("code")) + .setStartupParams(startupMap) + .setUser(engine.get("executeUser")) // submit user + .addExecuteUser(engine.get("executeUser")) // execute user + .setLabels(labels) + .build(); + // 3. to execute + return client.submit(jobSubmitAction); + } - @Scheduled(cron = "${linkis.monitor.user.db.cron:0 0/10 * * * ?}") - public void dbJob() { - Map properties= new HashMap<>(); - properties.put("readTimeout",MonitorConfig.USER_MODE_INTERFACE_TIMEOUT.getValue()); - DWSClientConfig clientConfig = HttpsUntils.createClientConfig(null, properties); - UJESClientImpl ujesClient = new UJESClientImpl(clientConfig); - GetTableStatisticInfoAction builder = GetTableStatisticInfoAction - .builder() - .setUser("hadoop") - .setDatabase("default") - .setTable("dual") - .builder(); - HashMap parms = new HashMap<>(); - try { - GetTableStatisticInfoResult tableStatisticInfo = ujesClient.getTableStatisticInfo(builder); - if (tableStatisticInfo.getStatus() != 0) { - logger.info("元数据查询服务用户态,执行失败,异常信息:"+tableStatisticInfo.getMessage()); -// parms.put("$msg", tableStatisticInfo.getMessage()); -// Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); -// PooledImsAlertUtils.addAlert(failedAlerts.get("12017")); - } - } catch (Exception e) { - if(e instanceof SocketTimeoutException){ - Integer timeoutValue = MonitorConfig.USER_MODE_INTERFACE_TIMEOUT.getValue(); - long timeout = TimeUnit.MILLISECONDS.toSeconds(timeoutValue); - logger.info("元数据查询服务用户态,执行超时:"+timeout+"秒"); -// parms.put("$timeout", String.valueOf(timeout)); -// Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); -// PooledImsAlertUtils.addAlert(failedAlerts.get("12018")); - } else { - logger.error("元数据查询服务用户态,执行异常:"+ e); -// parms.put("$msg", e.getMessage()); -// Map failedAlerts = MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); -// PooledImsAlertUtils.addAlert(failedAlerts.get("12017")); - } - } + @Scheduled(cron = "${linkis.monitor.user.db.cron:0 0/10 * * * ?}") + public void dbJob() { + Map properties = new HashMap<>(); + properties.put("readTimeout", MonitorConfig.USER_MODE_INTERFACE_TIMEOUT.getValue()); + DWSClientConfig clientConfig = HttpsUntils.createClientConfig(null, properties); + UJESClientImpl ujesClient = new UJESClientImpl(clientConfig); + GetTableStatisticInfoAction builder = + GetTableStatisticInfoAction.builder() + .setUser("hadoop") + .setDatabase("default") + .setTable("dual") + .builder(); + HashMap parms = new HashMap<>(); + try { + GetTableStatisticInfoResult tableStatisticInfo = ujesClient.getTableStatisticInfo(builder); + if (tableStatisticInfo.getStatus() != 0) { + logger.info("元数据查询服务用户态,执行失败,异常信息:" + tableStatisticInfo.getMessage()); + // parms.put("$msg", tableStatisticInfo.getMessage()); + // Map failedAlerts = + // MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + // PooledImsAlertUtils.addAlert(failedAlerts.get("12017")); + } + } catch (Exception e) { + if (e instanceof SocketTimeoutException) { + Integer timeoutValue = MonitorConfig.USER_MODE_INTERFACE_TIMEOUT.getValue(); + long timeout = TimeUnit.MILLISECONDS.toSeconds(timeoutValue); + logger.info("元数据查询服务用户态,执行超时:" + timeout + "秒"); + // parms.put("$timeout", String.valueOf(timeout)); + // Map failedAlerts = + // MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + // PooledImsAlertUtils.addAlert(failedAlerts.get("12018")); + } else { + logger.error("元数据查询服务用户态,执行异常:" + e); + // parms.put("$msg", e.getMessage()); + // Map failedAlerts = + // MonitorAlertUtils.getAlerts(Constants.USER_RESOURCE_MONITOR(), parms); + // PooledImsAlertUtils.addAlert(failedAlerts.get("12017")); + } } + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java index 440b7a6bc4..4df828e36c 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java @@ -20,32 +20,32 @@ import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; import org.apache.linkis.monitor.scan.utils.log.LogUtils; -import org.slf4j.Logger; + import org.springframework.context.annotation.PropertySource; import org.springframework.scheduling.annotation.Scheduled; import org.springframework.stereotype.Component; import java.util.ArrayList; import java.util.List; -/*** - * Task: clean up linkis_et_validator_checkinfo data - */ +import org.slf4j.Logger; + +/** * Task: clean up linkis_et_validator_checkinfo data */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") public class ValidatorClear { - private static final Logger logger = LogUtils.stdOutLogger(); - - @Scheduled(cron = "${linkis.monitor.clear.validator.cron}") - public void ValidatorClear() { - logger.info("Start to clear_validator_record shell"); - List cmdlist = new ArrayList<>(); - cmdlist.add("sh"); - cmdlist.add(MonitorConfig.shellPath + "clear_validator_record.sh"); - logger.info("clear_validator_record shell command {}", cmdlist); - String exec = ThreadUtils.run(cmdlist, "clear_validator_record.sh"); - logger.info("shell log {}", exec); - logger.info("End to clear_validator_record shell "); - } + private static final Logger logger = LogUtils.stdOutLogger(); + + @Scheduled(cron = "${linkis.monitor.clear.validator.cron}") + public void ValidatorClear() { + logger.info("Start to clear_validator_record shell"); + List cmdlist = new ArrayList<>(); + cmdlist.add("sh"); + cmdlist.add(MonitorConfig.shellPath + "clear_validator_record.sh"); + logger.info("clear_validator_record shell command {}", cmdlist); + String exec = ThreadUtils.run(cmdlist, "clear_validator_record.sh"); + logger.info("shell log {}", exec); + logger.info("End to clear_validator_record shell "); + } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java index 1208a6ef64..34e12d175d 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java @@ -32,6 +32,7 @@ import org.apache.linkis.monitor.scan.request.EntranceTaskAction; import org.apache.linkis.monitor.scan.response.EntranceTaskResult; import org.apache.linkis.server.BDPJettyServerHelper; +import org.apache.linkis.ujes.client.response.EmsListResult; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -44,7 +45,6 @@ import org.apache.http.impl.client.HttpClients; import org.apache.http.util.EntityUtils; -import org.apache.linkis.ujes.client.response.EmsListResult; import org.springframework.util.Assert; import java.io.IOException; @@ -132,14 +132,16 @@ public static DWSClientConfig createClientConfig(String url, Map return clientConfig; } - public static Map getEntranceTask(String url, String user,String Instance) throws IOException { + public static Map getEntranceTask(String url, String user, String Instance) + throws IOException { if (null == dwsClientConfig) { dwsClientConfig = createClientConfig(null, null); } if (null == client) { client = new MonitorHTTPClientClientImpl(dwsClientConfig); } - EntranceTaskAction build = EntranceTaskAction.newBuilder().setUser(user).setInstance(Instance).build(); + EntranceTaskAction build = + EntranceTaskAction.newBuilder().setUser(user).setInstance(Instance).build(); EntranceTaskResult result = client.entranList(build); return result.getResultMap(); } @@ -170,7 +172,9 @@ public static void sendIndex(List list) throws IOException { public static void sendChatbot(ChatbotEntity chatbotEntity) throws IOException { String json = BDPJettyServerHelper.gson().toJson(chatbotEntity); - StringEntity entity = new StringEntity(json, ContentType.create(ContentType.APPLICATION_JSON.getMimeType(), "UTF-8")); + StringEntity entity = + new StringEntity( + json, ContentType.create(ContentType.APPLICATION_JSON.getMimeType(), "UTF-8")); entity.setContentEncoding("UTF-8"); HttpPost httpPost = new HttpPost(MonitorConfig.CHATBOT_URL.getValue()); httpPost.setConfig(RequestConfig.DEFAULT); @@ -179,5 +183,4 @@ public static void sendChatbot(ChatbotEntity chatbotEntity) throws IOException { String responseStr = EntityUtils.toString(execute.getEntity(), "UTF-8"); Map map = BDPJettyServerHelper.gson().fromJson(responseStr, Map.class); } - } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala index 7ee159d936..520c0a78c3 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanSpringConfiguration.scala @@ -26,10 +26,6 @@ import org.springframework.context.annotation.{ComponentScan, Configuration} import javax.annotation.PostConstruct -/** - * Created by shangda on 2021/11/19. - */ - @Configuration @ComponentScan(Array("org.apache.linkis.monitor.scan", "org.apache.linkis.mybatis")) class LinkisJobHistoryScanSpringConfiguration { diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala index 337592bf72..c7ebf10e88 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/factory/MapperFactory.scala @@ -22,12 +22,9 @@ import org.apache.linkis.monitor.scan.app.instance.dao.{ InstanceInfoDao, InstanceLabelDao } -import org.apache.linkis.monitor.scan.app.instance.dao.InstanceInfoDao import org.apache.linkis.monitor.scan.app.jobhistory.dao.JobHistoryMapper - object MapperFactory { - // val bmlVersionCleanScanOper = new BmlVersionCleanScanOper private var jobHistoryMapper: JobHistoryMapper = _ @@ -37,27 +34,27 @@ object MapperFactory { private var instanceLabelRelationMapper: InsLabelRelationDao = _ - def getJobHistoryMapper() = jobHistoryMapper + def getJobHistoryMapper(): JobHistoryMapper = jobHistoryMapper - def setJobHistoryMapper(jobHistoryMapper: JobHistoryMapper) = { + def setJobHistoryMapper(jobHistoryMapper: JobHistoryMapper): Unit = { MapperFactory.jobHistoryMapper = jobHistoryMapper } - def getInstanceInfoMapper() = instanceInfoMapper + def getInstanceInfoMapper(): InstanceInfoDao = instanceInfoMapper - def setInstanceInfoMapper(instanceInfoMapper: InstanceInfoDao) = { + def setInstanceInfoMapper(instanceInfoMapper: InstanceInfoDao): Unit = { MapperFactory.instanceInfoMapper = instanceInfoMapper } - def getInstanceLabelMapper() = instanceLabelMapper + def getInstanceLabelMapper(): InstanceLabelDao = instanceLabelMapper - def setInstanceLabelMapper(instanceLabelMapper: InstanceLabelDao) = { + def setInstanceLabelMapper(instanceLabelMapper: InstanceLabelDao): Unit = { MapperFactory.instanceLabelMapper = instanceLabelMapper } - def getInsLabelRelationMapper() = instanceLabelRelationMapper + def getInsLabelRelationMapper(): InsLabelRelationDao = instanceLabelRelationMapper - def setInsLabelRelationMapper(instanceLabelRelationMapper: InsLabelRelationDao) = { + def setInsLabelRelationMapper(instanceLabelRelationMapper: InsLabelRelationDao): Unit = { MapperFactory.instanceLabelRelationMapper = instanceLabelRelationMapper } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala index b8eff63ec8..c7fd3f40e8 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/JobHistoryDataFetcher.scala @@ -17,16 +17,16 @@ package org.apache.linkis.monitor.scan.app.jobhistory -import java.util -import java.util.Date - -import org.apache.commons.lang3.StringUtils import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.monitor.scan.app.jobhistory.dao.JobHistoryMapper import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException import org.apache.linkis.monitor.scan.constants.Constants import org.apache.linkis.monitor.scan.core.pac.AbstractDataFetcher +import org.apache.commons.lang3.StringUtils + +import java.util +import java.util.Date class JobHistoryDataFetcher(args: Array[Any], mapper: JobHistoryMapper) extends AbstractDataFetcher @@ -89,7 +89,11 @@ class JobHistoryDataFetcher(args: Array[Any], mapper: JobHistoryMapper) throw t } } - if (StringUtils.isNotBlank(args(3).asInstanceOf[String]) && args(3).asInstanceOf[String].equals("updated_time")) { + if ( + StringUtils.isNotBlank(args(3).asInstanceOf[String]) && args(3) + .asInstanceOf[String] + .equals("updated_time") + ) { val list = new util.ArrayList[String]() Constants.DIRTY_DATA_FINISHED_JOB_STATUS_ARRAY.foreach(list.add(_)) mapper diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala index f899952c71..f89c680624 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala @@ -19,5 +19,4 @@ package org.apache.linkis.monitor.scan.app.jobhistory.errorcode import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent - class JobHistoryErrCodeHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala index c613b3d306..32a7cbe474 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrCodeRule.scala @@ -17,19 +17,21 @@ package org.apache.linkis.monitor.scan.app.jobhistory.errorcode -import java.util - import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils import org.apache.linkis.monitor.scan.core.ob.Observer import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} -import scala.collection.JavaConverters._ +import java.util + +import scala.collection.JavaConverters._ /** - * 针对执行任务返回的错误码进行监控,执行脚本任务时,会记录执行的错误码在数据库中, - * 服务会根据数据库中记录的错误码,来进行告警,如果错误码中包含(11001,11002)即可触发告警 + * Monitor the error codes returned by executing tasks. When executing script tasks, the executed + * error codes will be recorded in the database. The service will generate an alarm based on the + * error code recorded in the database. If the error code contains (11001, 11002), the alarm will be + * triggered. */ class JobHistoryErrCodeRule(errorCodes: util.Set[String], hitObserver: Observer) extends AbstractScanRule(event = new JobHistoryErrCodeHitEvent, observer = hitObserver) @@ -60,7 +62,9 @@ class JobHistoryErrCodeRule(errorCodes: util.Set[String], hitObserver: Observer) } scanRuleList.put("jobHistoryId", history.getId) case _ => - logger.warn("Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName) + logger.warn( + "Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName + ) } } } else { @@ -71,9 +75,9 @@ class JobHistoryErrCodeRule(errorCodes: util.Set[String], hitObserver: Observer) logger.info("hit " + alertData.size() + " data in one iteration") if (alertData.size() > 0) { getHitEvent().notifyObserver(getHitEvent(), alertData) - true + true } else { - false + false } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala index a037e0f017..8683dad15e 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala @@ -17,17 +17,16 @@ package org.apache.linkis.monitor.scan.app.jobhistory.errorcode -import java.util - import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} import org.apache.linkis.monitor.scan.utils.alert.AlertDesc import org.apache.linkis.monitor.scan.utils.alert.ims.{ImsAlertDesc, PooledImsAlertUtils} -import scala.collection.JavaConverters._ +import java.util +import scala.collection.JavaConverters._ class JobHistoryErrorCodeAlertSender(alerts: util.Map[String, AlertDesc]) extends Observer diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala index aa564ab335..fd311e7fcc 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedAlertSender.scala @@ -17,9 +17,6 @@ package org.apache.linkis.monitor.scan.app.jobhistory.jobtime -import java.text.MessageFormat -import java.util - import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException @@ -28,13 +25,13 @@ import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} import org.apache.linkis.monitor.scan.utils.alert.AlertDesc import org.apache.linkis.monitor.scan.utils.alert.ims.{ImsAlertDesc, PooledImsAlertUtils} +import java.text.MessageFormat +import java.util + import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer - -class JobTimeExceedAlertSender(alerts: util.Map[String, AlertDesc]) - extends Observer - with Logging { +class JobTimeExceedAlertSender(alerts: util.Map[String, AlertDesc]) extends Observer with Logging { private val orderedThresholds: Array[Long] = { val ret = new ArrayBuffer[Long]() @@ -79,34 +76,36 @@ class JobTimeExceedAlertSender(alerts: util.Map[String, AlertDesc]) for (t <- orderedThresholds) { // search max threshold that is smaller than elapse if (elapse >= t) { ts = t - } else { - - } + } else {} } val name = ts.toString val alert = if (!toSend.containsKey(name)) { alerts .get(name) - .asInstanceOf[ - ImsAlertDesc - ] + .asInstanceOf[ImsAlertDesc] } else { toSend.get(name) } - - val newInfo = MessageFormat.format("[Linkis任务信息]您好,您在Linkis/DSS提交的任务(任务ID:{0}),已经运行超过{1}h," + - "请关注是否任务正常,如果不正常您可以到Linkis/DSS管理台进行任务的kill,集群信息为BDAP({2})。详细解决方案见Q47:{3} " - , jobHistory.getId, (elapse / 1000 / 60 / 60).toString, jobHistory.getInstances, MonitorConfig.SOLUTION_URL.getValue) + + val newInfo = MessageFormat.format( + MonitorConfig.TASK_RUNTIME_TIMEOUT_DESC.getValue, + jobHistory.getId, + (elapse / 1000 / 60 / 60).toString, + jobHistory.getInstances, + MonitorConfig.SOLUTION_URL.getValue + ) val newNumHit = alert.numHit + 1 val receiver = new util.HashSet[String]() receiver.add(jobHistory.getSubmitUser) receiver.add(jobHistory.getExecuteUser) receiver.addAll(alert.alertReceivers) - val ImsAlertDesc = alert.copy(alertInfo = newInfo, alertReceivers = receiver, numHit = newNumHit) + val ImsAlertDesc = + alert.copy(alertInfo = newInfo, alertReceivers = receiver, numHit = newNumHit) PooledImsAlertUtils.addAlert(ImsAlertDesc) } } } + } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala index b7b883e09b..a83cd0ee01 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedHitEvent.scala @@ -19,5 +19,4 @@ package org.apache.linkis.monitor.scan.app.jobhistory.jobtime import org.apache.linkis.monitor.scan.core.ob.SingleObserverEvent - class JobTimeExceedHitEvent extends SingleObserverEvent diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala index b91d605d4e..821a368d65 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/jobtime/JobTimeExceedRule.scala @@ -17,9 +17,6 @@ package org.apache.linkis.monitor.scan.app.jobhistory.jobtime -import java.util -import java.util.Locale - import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException @@ -28,11 +25,15 @@ import org.apache.linkis.monitor.scan.constants.Constants import org.apache.linkis.monitor.scan.core.ob.Observer import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} +import java.util +import java.util.Locale + import scala.collection.JavaConverters._ /** - * 针对执行任务状态进行监控,扫描12小时之外,24小时之内的的数据, - * 如果规则范围内,有数据状态是(Inited,WaitForRetry,Scheduled,Running)其中之一,则触发告警 + * Monitor the execution status of tasks, scan data outside 12 hours and within 24 hours, If within + * the scope of the rule, there is data whose status is one of (Inited, WaitForRetry, Scheduled, + * Running), an alarm will be triggered. */ class JobTimeExceedRule(thresholds: util.Set[String], hitObserver: Observer) extends AbstractScanRule(event = new JobTimeExceedHitEvent, observer = hitObserver) @@ -94,9 +95,9 @@ class JobTimeExceedRule(thresholds: util.Set[String], hitObserver: Observer) logger.info("hit " + alertData.size() + " data in one iteration") if (alertData.size() > 0) { getHitEvent.notifyObserver(getHitEvent, alertData) - true + true } else { - false + false } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala index 77bc29ffe2..e7ad384e56 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsAlertSender.scala @@ -17,8 +17,6 @@ package org.apache.linkis.monitor.scan.app.jobhistory.labels -import java.util - import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException @@ -28,6 +26,8 @@ import org.apache.linkis.monitor.scan.utils.alert.AlertDesc import org.apache.linkis.monitor.scan.utils.alert.ims.{PooledImsAlertUtils, UserLabelAlertUtils} import org.apache.linkis.server.BDPJettyServerHelper +import java.util + import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala index 6308ac28b0..f4c65f7ba4 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/labels/JobHistoryLabelsRule.scala @@ -17,10 +17,6 @@ package org.apache.linkis.monitor.scan.app.jobhistory.labels -import java.util - -import com.google.common.collect.HashBiMap -import org.apache.commons.lang3.StringUtils import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils @@ -29,11 +25,17 @@ import org.apache.linkis.monitor.scan.core.ob.Observer import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} import org.apache.linkis.server.BDPJettyServerHelper +import org.apache.commons.lang3.StringUtils + +import java.util + import scala.collection.JavaConverters._ +import com.google.common.collect.HashBiMap + /** - * 对前20分钟内的执行数据进行扫描,对数据的labels字段进行判断, - * 判断依据monitor配置(linkis.monitor.jobhistory.userLabel.tenant) + * Scan the execution data within the previous 20 minutes and judge the labels field of the data. + * Judgment based on monitor configuration (linkis.monitor.jobhistory.userLabel.tenant) */ class JobHistoryLabelsRule(hitObserver: Observer) extends AbstractScanRule(event = new JobHistoryLabelsHitEvent, observer = hitObserver) @@ -69,7 +71,6 @@ class JobHistoryLabelsRule(hitObserver: Observer) Constants.USER_LABEL_TENANT.getValue, classOf[java.util.Map[String, String]] ) - // 当任务的creator是qualitis(或dops)时,tenant不是qualitis发出告警 val listIterator = configMap.keySet.iterator while ({ listIterator.hasNext @@ -82,7 +83,6 @@ class JobHistoryLabelsRule(hitObserver: Observer) } } } - // 当任务代理tenant:Qualitis标签,但是creator不是qualitis标签也进行告警 if (configMap.values().contains(tenant)) { val bimap: HashBiMap[String, String] = HashBiMap.create(configMap) val key = bimap.inverse().get(tenant) @@ -103,9 +103,9 @@ class JobHistoryLabelsRule(hitObserver: Observer) logger.info("hit " + alertData.size() + " data in one iteration") if (alertData.size() > 0) { getHitEvent.notifyObserver(getHitEvent, alertData) - true + true } else { - false + false } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala index 1912acf8b4..3c42600432 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonJobRunTimeRule.scala @@ -17,24 +17,26 @@ package org.apache.linkis.monitor.scan.app.jobhistory.runtime -import org.apache.commons.lang3.StringUtils import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.constants.Constants import org.apache.linkis.monitor.scan.core.ob.Observer import org.apache.linkis.monitor.scan.core.pac.{AbstractScanRule, ScannedData} +import org.apache.commons.lang3.StringUtils + import java.util + import scala.collection.JavaConverters._ /** - * 对前20分钟内的执行数据进行扫描, - * 1.数据的ObserveInfo字段进行判断是否为空, - * 2.任务状态已经完成(Succeed,Failed,Cancelled,Timeout,ALL) - * 满足条件即可触发告警 + * Scan the execution data within the first 20 minutes, + * 1. The ObserveInfo field of the data is judged whether it is empty, 2. The task status has been + * completed (Succeed, Failed, Cancelled, Timeout, ALL) Alarms can be triggered when conditions + * are met */ class CommonJobRunTimeRule(hitObserver: Observer) - extends AbstractScanRule(event = new JobHistoryRunTimeHitEvent, observer = hitObserver) + extends AbstractScanRule(event = new JobHistoryRunTimeHitEvent, observer = hitObserver) with Logging { /** @@ -54,8 +56,12 @@ class CommonJobRunTimeRule(hitObserver: Observer) for (d <- sd.getData().asScala) { d match { case jobHistory: JobHistory => - if (Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains(jobHistory.getStatus.toUpperCase()) - &&StringUtils.isNotBlank(jobHistory.getObserveInfo)) { + if ( + Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains( + jobHistory.getStatus.toUpperCase() + ) + && StringUtils.isNotBlank(jobHistory.getObserveInfo) + ) { alertData.add(jobHistory) } else { logger.warn("jobHistory is not completely , taskid :" + d) @@ -70,9 +76,9 @@ class CommonJobRunTimeRule(hitObserver: Observer) logger.info("hit " + alertData.size() + " data in one iteration") if (alertData.size() > 0) { getHitEvent.notifyObserver(getHitEvent, alertData) - true + true } else { - false + false } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala index 6aca4c38ff..841c430504 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/CommonRunTimeAlertSender.scala @@ -17,12 +17,6 @@ package org.apache.linkis.monitor.scan.app.jobhistory.runtime -import java.net.InetAddress -import java.text.SimpleDateFormat -import java.util -import java.util.Date - -import org.apache.commons.collections.MapUtils import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException @@ -31,12 +25,16 @@ import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} import org.apache.linkis.monitor.scan.utils.alert.ims.{MonitorAlertUtils, PooledImsAlertUtils} import org.apache.linkis.server.BDPJettyServerHelper -import scala.collection.JavaConverters._ +import org.apache.commons.collections.MapUtils +import java.net.InetAddress +import java.text.SimpleDateFormat +import java.util +import java.util.Date -class CommonRunTimeAlertSender() - extends Observer - with Logging { +import scala.collection.JavaConverters._ + +class CommonRunTimeAlertSender() extends Observer with Logging { private val dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") override def update(e: Event, jobHistoryList: scala.Any): Unit = { @@ -59,10 +57,21 @@ class CommonRunTimeAlertSender() logger.warn("Ignore wrong input data Type : " + a.getClass.getCanonicalName) } else { val jobHistory = a.asInstanceOf[JobHistory] - val observeInfoMap = BDPJettyServerHelper.gson.fromJson(jobHistory.getObserveInfo, classOf[java.util.Map[String, String]]) + val observeInfoMap = BDPJettyServerHelper.gson.fromJson( + jobHistory.getObserveInfo, + classOf[java.util.Map[String, String]] + ) val extraMap = MapUtils.getMap(observeInfoMap, "extra") - observeInfoMap.put("title", extraMap.get("title").toString + ",任务id:" + jobHistory.getId + ",执行结果 :" + jobHistory.getStatus) - observeInfoMap.put("$detail", extraMap.get("detail").toString + ",执行结果 :" + jobHistory.getStatus) + observeInfoMap.put( + "title", + extraMap + .get("title") + .toString + ",任务id:" + jobHistory.getId + ",执行结果 :" + jobHistory.getStatus + ) + observeInfoMap.put( + "$detail", + extraMap.get("detail").toString + ",执行结果 :" + jobHistory.getStatus + ) observeInfoMap.put("$submitUser", jobHistory.getSubmitUser) observeInfoMap.put("$status", jobHistory.getStatus) observeInfoMap.put("$id", jobHistory.getId.toString) @@ -80,9 +89,10 @@ class CommonRunTimeAlertSender() observeInfoMap.put("$ip", InetAddress.getLocalHost.getHostAddress) observeInfoMap.remove("taskId") observeInfoMap.remove("extra") - val alters = MonitorAlertUtils.getAlertsByDss(Constants.JOB_RESULT_IM, observeInfoMap) + val alters = MonitorAlertUtils.getAlerts(Constants.JOB_RESULT_IM, observeInfoMap) PooledImsAlertUtils.addAlert(alters.get("12016")) } } } + } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala index b9f35ce7c7..621aacb2d9 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala @@ -17,8 +17,6 @@ package org.apache.linkis.monitor.scan.app.jobhistory.runtime -import java.util - import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException @@ -26,18 +24,18 @@ import org.apache.linkis.monitor.scan.constants.Constants import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} import org.apache.linkis.monitor.scan.utils.alert.ims.{MonitorAlertUtils, PooledImsAlertUtils} +import java.util + import scala.collection.JavaConverters._ /** - * 对前20分钟内的执行数据进行扫描,对已结束的任务进行判断, - * 1.jobhistory中的parm字段中包含(task.notification.conditions) - * 2.执行任务的结果是(Succeed,Failed,Cancelled,Timeout,ALL)其中任意一个,则触发告警 - * 3.job的结果是已经结束 - * 同时满足上述三个条件即可触发告警 + * Scan the execution data within the first 20 minutes, judge the completed tasks, + * 1. The parm field in jobhistory contains (task.notification.conditions) 2. If the result of + * executing the task is any one of (Succeed, Failed, Canceled, Timeout, ALL), an alarm will be + * triggered 3.The result of the job is that it has ended The alarm can be triggered if the + * above three conditions are met at the same time */ -class JobHistoryRunTimeAlertSender() - extends Observer - with Logging { +class JobHistoryRunTimeAlertSender() extends Observer with Logging { override def update(e: Event, jobHistroyList: scala.Any): Unit = { if (!e.isInstanceOf[JobHistoryRunTimeHitEvent]) { @@ -70,4 +68,5 @@ class JobHistoryRunTimeAlertSender() } } } + } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala index e136ea46d0..4f91be337a 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/app/jobhistory/runtime/JobHistoryRunTimeRule.scala @@ -27,11 +27,11 @@ import org.apache.linkis.protocol.utils.TaskUtils import org.apache.linkis.server.BDPJettyServerHelper import java.util -import scala.collection.JavaConverters._ +import scala.collection.JavaConverters._ class JobHistoryRunTimeRule(hitObserver: Observer) - extends AbstractScanRule(event = new JobHistoryRunTimeHitEvent, observer = hitObserver) + extends AbstractScanRule(event = new JobHistoryRunTimeHitEvent, observer = hitObserver) with Logging { private val scanRuleList = CacheUtils.cacheBuilder @@ -52,15 +52,28 @@ class JobHistoryRunTimeRule(hitObserver: Observer) for (d <- sd.getData().asScala) { d match { case jobHistory: JobHistory => - if (Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains(jobHistory.getStatus.toUpperCase())) { - val parmsMap: util.Map[String, scala.AnyRef] = BDPJettyServerHelper.gson.fromJson(jobHistory.getParams, classOf[util.Map[String, scala.AnyRef]]) + if ( + Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains( + jobHistory.getStatus.toUpperCase() + ) + ) { + val parmsMap: util.Map[String, scala.AnyRef] = BDPJettyServerHelper.gson.fromJson( + jobHistory.getParams, + classOf[util.Map[String, scala.AnyRef]] + ) val runtimeMap = TaskUtils.getRuntimeMap(parmsMap) - if (runtimeMap.containsKey("task.notification.conditions") && - Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains(String.valueOf(runtimeMap.get("task.notification.conditions")).toUpperCase())) { - alertData.add(jobHistory) + if ( + runtimeMap.containsKey("task.notification.conditions") && + Constants.DIRTY_DATA_FINISHED_JOB_STATUS.contains( + String.valueOf(runtimeMap.get("task.notification.conditions")).toUpperCase() + ) + ) { + alertData.add(jobHistory) } } else { - logger.warn("Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName) + logger.warn( + "Ignored wrong input data Type : " + d + ", " + d.getClass.getCanonicalName + ) } scanRuleList.put("jobHistoryId", jobHistory.getId) case _ => @@ -73,9 +86,9 @@ class JobHistoryRunTimeRule(hitObserver: Observer) logger.info("hit " + alertData.size() + " data in one iteration") if (alertData.size() > 0) { getHitEvent.notifyObserver(getHitEvent, alertData) - true + true } else { - false + false } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala index 8813aba210..97cd429887 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/client/MonitorHTTPClient.scala @@ -21,7 +21,11 @@ import org.apache.linkis.httpclient.authentication.AuthenticationStrategy import org.apache.linkis.httpclient.dws.authentication.StaticAuthenticationStrategy import org.apache.linkis.httpclient.dws.config.{DWSClientConfig, DWSClientConfigBuilder} import org.apache.linkis.httpclient.response.Result -import org.apache.linkis.monitor.scan.request.{EmsListAction, EntranceTaskAction, MonitorResourceAction} +import org.apache.linkis.monitor.scan.request.{ + EmsListAction, + EntranceTaskAction, + MonitorResourceAction +} import org.apache.linkis.monitor.scan.response.EntranceTaskResult import org.apache.linkis.ujes.client.response.EmsListResult diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala index f3e6d74c48..89ab330b4d 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala @@ -19,76 +19,58 @@ package org.apache.linkis.monitor.scan.constants import org.apache.linkis.common.conf.CommonVars - object Constants { - val ALERT_IMS_URL = CommonVars.properties.getProperty( - "wds.linkis.alert.url", - "http://127.0.0.1:10812/ims_data_access/send_alarm.do" - ) + val SCAN_PREFIX_ERRORCODE = "jobhistory.errorcode." + val SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC = "jobhistory.unfinished.time.exceed.sec." + val ALERT_RESOURCE_MONITOR = "ecm.resource.monitor.im." val ALERT_PROPS_FILE_PATH = CommonVars.properties.getProperty( - "wds.linkis.alert.ims.file.path", - "linkis-et-monitor-ims.properties" + "linkis.alert.conf.file.path", + "linkis-et-monitor-file.properties" ) - val ALERT_IMS_MAX_LINES = CommonVars[Int]("wds.linkis.alert.ims.max.lines", 8).getValue - - val SCAN_INTERVALS_SECONDS = - CommonVars[Long]("wds.linkis.errorcode.scanner.interval.seconds", 1 * 60 * 60).getValue - - val MAX_INTERVALS_SECONDS = - CommonVars[Long]("wds.linkis.errorcode.scanner.max.interval.seconds", 1 * 60 * 60).getValue + val ALERT_IMS_URL = CommonVars.properties.getProperty( + "linkis.alert.url", + "http://127.0.0.1:10812/ims_data_access/send_alarm.do" + ) val ALERT_SUB_SYSTEM_ID = - CommonVars.properties.getProperty("wds.linkis.alert.ims.sub_system_id", "5435") + CommonVars.properties.getProperty("linkis.alert.sub_system_id", "10001") val ALERT_DEFAULT_RECEIVERS = CommonVars.properties - .getProperty("wds.linkis.alert.receiver.default", "") + .getProperty("linkis.alert.receiver.default", "") .split(",") .toSet[String] - val SCAN_PREFIX_ERRORCODE = "jobhistory.errorcode." - val SCAN_PREFIX_UNFINISHED_JOBTIME_EXCEED_SEC = "jobhistory.unfinished.time.exceed.sec." - - val SCAN_RULE_UNFINISHED_JOB_STATUS = - "Inited,WaitForRetry,Scheduled,Running".split(",").map(s => s.toUpperCase()) + val ALERT_IMS_MAX_LINES = CommonVars[Int]("linkis.alert.content.max.lines", 8).getValue - val DIRTY_DATA_EUREKA_DELETE_INSTANCE_URL = - CommonVars.apply("wds.linkis.eureka.defaultZone", "http://localhost:20303").getValue + val ERRORCODE_SCAN_INTERVALS_SECONDS = + CommonVars[Long]("linkis.errorcode.scanner.interval.seconds", 1 * 60 * 60).getValue - val DIRTY_DATA_EUREKA_DELETE_PATH = CommonVars - .apply("wds.linkis.dirty.data.eureka.delete.path", "/apps/{springName}/{instance}") - .getValue + val ERRORCODE_MAX_INTERVALS_SECONDS = + CommonVars[Long]("linkis.errorcode.scanner.max.interval.seconds", 1 * 60 * 60).getValue - val DIRTY_DATA_UNFINISHED_JOB_STATUS = + val SCAN_RULE_UNFINISHED_JOB_STATUS = "Inited,WaitForRetry,Scheduled,Running".split(",").map(s => s.toUpperCase()) - val DIRTY_DATA_JOB_TARGET_STATUS = "Cancelled" - - val DIRTY_DATA_ENTRANCE_APPLICATIONNAME = - CommonVars("wds.linkis.entrance.spring.name", "linkis-cg-entrance").getValue - - val MODIFY_DB_DATA_DAYS = CommonVars("wds.linkis.dirty.data.modify.db.days", 1).getValue - val ALERT_RESOURCE_MONITOR = "ecm.resource.monitor.im." - val LINKIS_API_VERSION: CommonVars[String] = - CommonVars[String]("wds.linkis.bml.api.version", "v1") + CommonVars[String]("linkis.bml.api.version", "v1") val AUTH_TOKEN_KEY: CommonVars[String] = - CommonVars[String]("wds.linkis.bml.auth.token.key", "Validation-Code") + CommonVars[String]("linkis.bml.auth.token.key", "Validation-Code") val AUTH_TOKEN_VALUE: CommonVars[String] = - CommonVars[String]("wds.linkis.bml.auth.token.value", "BML-AUTH") + CommonVars[String]("linkis.bml.auth.token.value", "BML-AUTH") val CONNECTION_MAX_SIZE: CommonVars[Int] = - CommonVars[Int]("wds.linkis.bml.connection.max.size", 10) + CommonVars[Int]("linkis.bml.connection.max.size", 10) val CONNECTION_TIMEOUT: CommonVars[Int] = - CommonVars[Int]("wds.linkis.bml.connection.timeout", 5 * 60 * 1000) + CommonVars[Int]("linkis.bml.connection.timeout", 5 * 60 * 1000) val CONNECTION_READ_TIMEOUT: CommonVars[Int] = - CommonVars[Int]("wds.linkis.bml.connection.read.timeout", 10 * 60 * 1000) + CommonVars[Int]("linkis.bml.connection.read.timeout", 10 * 60 * 1000) val AUTH_TOKEN_KEY_SHORT_NAME = "tokenKey" val AUTH_TOKEN_VALUE_SHORT_NAME = "tokenValue" @@ -106,8 +88,4 @@ object Constants { val THREAD_TIME_OUT_IM = "thread.monitor.timeout.im." val JOB_RESULT_IM = "jobhistory.result.monitor.im." - val DIRTY_DATA_FINISHED_JOB_STATUS = - "Succeed,Failed,Cancelled,Timeout,ALL".split(",").map(s => s.toUpperCase()) - val DIRTY_DATA_FINISHED_JOB_STATUS_ARRAY = "Succeed,Failed,Cancelled,Timeout".split(",") - } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala index 8ff3755747..05b244a458 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/ScanOperatorEnum.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.constants - object ScanOperatorEnum extends Enumeration { type ScanOperatorEnum = Value val BML_VERSION, JOB_HISTORY = Value diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala index 60322c5814..888a3aa20b 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Event.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.core.ob - trait Event { def isRegistered: Boolean diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala index 58849dd06c..04359e309f 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/ob/Observer.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.core.ob - trait Observer { /** diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala index 02ad320c55..2f7f9a67e6 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractDataFetcher.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.core.pac - abstract class AbstractDataFetcher(customName: String = "") extends DataFetcher { private val name: String = if (!customName.isEmpty) { diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala index 8d6762c562..f572bd90bd 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/AbstractScanRule.scala @@ -21,7 +21,6 @@ import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} import org.apache.linkis.monitor.scan.core.ob.Observer - abstract class AbstractScanRule(customName: String = "", event: Event, observer: Observer) extends ScanRule with Logging { diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala index 02f32a3fc5..efc573132a 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/BaseScannedData.scala @@ -19,7 +19,6 @@ package org.apache.linkis.monitor.scan.core.pac import java.util - class BaseScannedData(owner: String, data: util.List[scala.Any]) extends ScannedData { override def getOwner(): String = this.owner diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala index d38c02dea7..8075792161 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScanBuffer.scala @@ -20,7 +20,6 @@ package org.apache.linkis.monitor.scan.core.pac import java.util import java.util.concurrent.LinkedBlockingDeque - class ScanBuffer { val buffer: LinkedBlockingDeque[ScannedData] = new LinkedBlockingDeque[ScannedData] diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala index a6914c3e60..2f5cc68208 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/pac/ScannedData.scala @@ -19,7 +19,6 @@ package org.apache.linkis.monitor.scan.core.pac import java.util - trait ScannedData { def getOwner(): String diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala index e6939df36e..17fae63fb0 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AbstractScanner.scala @@ -17,15 +17,14 @@ package org.apache.linkis.monitor.scan.core.scanner -import java.util -import java.util.concurrent.CopyOnWriteArrayList -import java.util.concurrent.atomic.AtomicInteger - import org.apache.linkis.common.utils.Logging import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException import org.apache.linkis.monitor.scan.core.ob.{Event, Observer} import org.apache.linkis.monitor.scan.core.pac._ +import java.util +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.atomic.AtomicInteger abstract class AbstractScanner extends AnomalyScanner with Logging { private val buffer: ScanBuffer = new ScanBuffer @@ -71,8 +70,8 @@ abstract class AbstractScanner extends AnomalyScanner with Logging { } /** - * Returns a buffer that allows read/write simultaneously buffer is allowed to be written by - * other thread + * Returns a buffer that allows read/write simultaneously buffer is allowed to be written by other + * thread */ override def getBuffer(): ScanBuffer = buffer @@ -102,10 +101,7 @@ abstract class AbstractScanner extends AnomalyScanner with Logging { */ override def run(): Unit = { if (dataFetcherList.size() == 0) { - throw new AnomalyScannerException( - 21304, - "attempting to run scanner with empty dataFetchers" - ) + throw new AnomalyScannerException(21304, "attempting to run scanner with empty dataFetchers") } if (buffer == null) { throw new AnomalyScannerException(21304, "attempting to run scanner with null buffer") @@ -138,8 +134,8 @@ abstract class AbstractScanner extends AnomalyScanner with Logging { } /** - * 1. should be a blocking call 2. read from [[ScanBuffer]] 2. see if [[ScanRule]] is matched - * 3. trigger [[Observer]] + * 1. should be a blocking call 2. read from [[ScanBuffer]] 2. see if [[ScanRule]] is matched 3. + * trigger [[Observer]] */ override def analyzeOneIteration(): Unit = { val dataToAnalyze = buffer.drain() @@ -158,8 +154,8 @@ abstract class AbstractScanner extends AnomalyScanner with Logging { } /** - * 1. should be non-blocking 2. keeps calling scanOneIteration() and analyzeOneIteration() - * utils stop() is called + * 1. should be non-blocking 2. keeps calling scanOneIteration() and analyzeOneIteration() utils + * stop() is called */ override def start(): Unit = { // TODO diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala index 854ca78c3f..d887e7bf86 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/AnomalyScanner.scala @@ -81,14 +81,14 @@ trait AnomalyScanner { protected def scanOneIteration(): Unit /** - * Returns a buffer that allows read/write simultaneously buffer is allowed to be written by - * other thread + * Returns a buffer that allows read/write simultaneously buffer is allowed to be written by other + * thread */ protected def getBuffer(): ScanBuffer /** - * 1. should be a blocking call 2. read from [[ScanBuffer]] 2. see if [[ScanRule]] is matched - * 3. trigger [[[[org.apache.linkis.tools.core.ob.Observer]]]] + * 1. should be a blocking call 2. read from [[ScanBuffer]] 2. see if [[ScanRule]] is matched 3. + * trigger [[[[org.apache.linkis.tools.core.ob.Observer]]]] */ protected def analyzeOneIteration(): Unit diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala index c56a3f8182..2463439873 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/core/scanner/DefaultScanner.scala @@ -19,7 +19,6 @@ package org.apache.linkis.monitor.scan.core.scanner import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils - class DefaultScanner extends AbstractScanner { override def shutdown(): Unit = { diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala index 6126a2b04b..94636a41ca 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/request/EntranceTaskAction.scala @@ -57,7 +57,8 @@ object EntranceTaskAction { def build(): EntranceTaskAction = { val entranceTaskAction = new EntranceTaskAction if (StringUtils.isNotBlank(creator)) entranceTaskAction.setParameter("creator", creator) - if (StringUtils.isNotBlank(engineTypeLabel)) entranceTaskAction.setParameter("engineTypeLabel", engineTypeLabel) + if (StringUtils.isNotBlank(engineTypeLabel)) + entranceTaskAction.setParameter("engineTypeLabel", engineTypeLabel) if (StringUtils.isNotBlank(instance)) entranceTaskAction.setParameter("instance", instance) if (StringUtils.isNotBlank(user)) { // hadoop用户应该获取全部用户entrance信息,则无需传user,即可获取全部entrance信息 diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala index b1f29530f8..068746667c 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertDesc.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.utils.alert - trait AlertDesc { /** diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala index 5ae1b960e5..fb2cff458d 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/AlertSender.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.utils.alert - trait AlertSender { /** diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala index 10eb367d2a..5ff274438a 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/PooledAlertSender.scala @@ -23,9 +23,8 @@ import org.apache.linkis.common.utils.{Logging, Utils} import java.util.concurrent.{Future, LinkedBlockingQueue} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} - abstract class PooledAlertSender extends AlertSender with Logging { - private val THREAD_POOL_SIZE = CommonVars[Int]("wds.linkis.alert.pool.size", 5).getValue + private val THREAD_POOL_SIZE = CommonVars[Int]("linkis.alert.pool.size", 5).getValue private val alertDescQ: LinkedBlockingQueue[AlertDesc] = new LinkedBlockingQueue[AlertDesc](1000) @@ -58,15 +57,15 @@ abstract class PooledAlertSender extends AlertSender with Logging { def start(): Unit = { future = Utils.defaultScheduler.submit(new Runnable() { override def run() { - info("Pooled alert thread started!") + logger.info("Pooled alert thread started!") while (!stopped.get) { executors synchronized { while (!stopped.get && runningNumber.get >= THREAD_POOL_SIZE) { - info("Pooled alert thread is full, start waiting") + logger.info("Pooled alert thread is full, start waiting") executors.wait() } } - info("Pooled alert thread continue processing") + logger.info("Pooled alert thread continue processing") if (stopped.get && alertDescQ.size() == 0) return val alertDesc = Utils.tryQuietly(alertDescQ.take) @@ -75,12 +74,12 @@ abstract class PooledAlertSender extends AlertSender with Logging { override def run() { runningNumber.addAndGet(1) Utils.tryAndWarn { - info("sending alert , information: " + alertDesc) + logger.info("sending alert , information: " + alertDesc) val ok = doSendAlert(alertDesc) if (!ok) { warn("Failed to send alert: " + alertDesc) } else { - info("successfully send alert: " + alertDesc) + logger.info("successfully send alert: " + alertDesc) } runningNumber.decrementAndGet executors synchronized executors.notify @@ -93,7 +92,7 @@ abstract class PooledAlertSender extends AlertSender with Logging { } def shutdown(waitComplete: Boolean = true, timeoutMs: Long = -1): Unit = { - info("stopping the Pooled alert thread...") + logger.info("stopping the Pooled alert thread...") if (waitComplete) { val startTime = System.currentTimeMillis() while ( @@ -106,7 +105,7 @@ abstract class PooledAlertSender extends AlertSender with Logging { executors.shutdown stopped.set(true) future.cancel(true) - info("Pooled alert thread is stopped") + logger.info("Pooled alert thread is stopped") } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala index 6b12da9584..6aa19d5092 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertDesc.scala @@ -32,7 +32,6 @@ import scala.collection.JavaConverters._ import ImsAlertLevel.ImsAlertLevel import ImsAlertWay.ImsAlertWay - case class ImsAlertDesc( var subSystemId: String, var alertTitle: String, @@ -149,7 +148,7 @@ case class ImsAlertDesc( alertReceivers } - Array(subSystemId, newAlertTitle, newAlertObj, newAlertInfo, newAlertReceivers) + Array(subSystemId, newAlertTitle, newAlertObj, newAlertInfo, newAlertReceivers) } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala index cb304e9e4f..13948eac82 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertLevel.scala @@ -17,7 +17,6 @@ package org.apache.linkis.monitor.scan.utils.alert.ims - object ImsAlertLevel extends Enumeration { type ImsAlertLevel = Value val INFO = Value("5") diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertPropFileData.scala similarity index 52% rename from linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java rename to linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertPropFileData.scala index 716ad5cf43..d5fb95e910 100644 --- a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/LinkisJobHistoryScanApplicationTest.java +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertPropFileData.scala @@ -15,24 +15,16 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app; +package org.apache.linkis.monitor.scan.utils.alert.ims -import org.apache.linkis.server.utils.LinkisMainHelper; +import com.fasterxml.jackson.annotation.JsonProperty - -public class LinkisJobHistoryScanApplicationTest { - // @Before - public void before() { - System.getProperties().setProperty(LinkisMainHelper.SERVER_NAME_KEY(), "linkis-et-monitor"); - System.getProperties() - .setProperty("log4j.configurationFile", "src/test/resources/log4j2-console.xml"); - // System.getProperties().setProperty("wds.linkis.server.conf", - // "linkis-et-monitor.properties"); - } - - // @Test - public void main() throws Exception { - LinkisJobHistoryScanApplication.main(new String[] {}); - // LinkisJobHistoryScanApplication.main(new String[]{"2021122919", "2021122921"}); - } -} +case class ImsAlertPropFileData( + @JsonProperty("alert_title") alertTitle: String, + @JsonProperty("alert_info") alertInfo: String, + @JsonProperty("alert_way") alertWays: String, + @JsonProperty("alert_reciver") alertReceivers: String, + @JsonProperty("alert_level") alertLevel: String, + @JsonProperty("alert_obj") alertObj: String, + @JsonProperty("can_recover") canRecover: String +) diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala index 207df613d7..244e995775 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsAlertWay.scala @@ -17,11 +17,10 @@ package org.apache.linkis.monitor.scan.utils.alert.ims - object ImsAlertWay extends Enumeration { type ImsAlertWay = Value val NoAlert = Value("0") - val RTX = Value("1") + val WXWork = Value("1") val Email = Value("2") val WeChat = Value("3") } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala index 64b80d3a95..4e93f53a05 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/ImsRequest.scala @@ -21,7 +21,6 @@ import java.util import com.fasterxml.jackson.annotation.JsonProperty - case class ImsRequest(@JsonProperty("alertList") alertList: util.List[AlertEntity]) case class AlertEntity( diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala deleted file mode 100644 index 95b64f3f4e..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/JobHistoryScanImsAlertPropFileParserUtils.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.monitor.scan.utils.alert.ims - -import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} -import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException -import org.apache.linkis.monitor.scan.constants.Constants -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc -import org.apache.commons.lang3.StringUtils -import org.apache.commons.lang3.exception.ExceptionUtils - -import java.io.{BufferedReader, File, FileInputStream, InputStream, InputStreamReader} -import java.text.SimpleDateFormat -import java.util -import java.util.Properties -import scala.collection.JavaConverters._ -import com.fasterxml.jackson.annotation.JsonProperty -import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import org.apache.commons.io.IOUtils - - -object JobHistoryScanImsAlertPropFileParserUtils extends Logging { - - private val mapper = { - val ret = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) - ret.registerModule(DefaultScalaModule) - ret.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) - ret - } - - def getAlerts(prefix: String): util.Map[String, AlertDesc] = { - val ret = new util.HashMap[String, AlertDesc]() - val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) - if (url == null) { - throw new AnomalyScannerException( - 21304, - "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH - ) - } - logger.info("reading alert properties from: " + url.getFile) - val properties = new Properties() - var inputStream: InputStream = null - var reader: InputStreamReader = null - var buff: BufferedReader = null - - Utils.tryFinally { - Utils.tryCatch { - inputStream = new FileInputStream(new File(url.getFile)) - reader = new InputStreamReader(inputStream, "UTF-8") - buff = new BufferedReader(reader) - properties.load(buff) - } { t => { - throw new AnomalyScannerException(21304, "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t)) - return ret - } - } - } { - IOUtils.closeQuietly(buff) - IOUtils.closeQuietly(reader) - IOUtils.closeQuietly(inputStream) - } - for ((k: String, v: String) <- properties.asScala) { - if (ret.containsKey(k)) { - logger.warn("found duplicate key in alert properties, accept only the first one") - } else if (StringUtils.startsWith(k, prefix)) { - val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) - val receivers = { - val set: util.Set[String] = new util.HashSet[String] - if (StringUtils.isNotBlank(data.alertReceivers)) { - data.alertReceivers.split(",").map(r => set.add(r)) - } - Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { - if (StringUtils.isNotBlank(e)) { - set.add(e) - } - }) - set - } - val alertDesc = Utils.tryAndWarn( - new ImsAlertDesc( - Constants.ALERT_SUB_SYSTEM_ID, - data.alertTitle, - data.alertObj, - data.alertInfo, - ImsAlertLevel.withName(data.alertLevel), - null, // Not used - 0, { - val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] - if (StringUtils.isNotBlank(data.alertWays)) { - data.alertWays - .split(",") - .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) - } - set - }, - receivers - ) - ) - val realK = StringUtils.substringAfter(k, prefix) - ret.put(realK, alertDesc) - } - } - ret - } - -} - -case class ImsAlertPropFileData( - @JsonProperty("alert_title") alertTitle: String, - @JsonProperty("alert_info") alertInfo: String, - @JsonProperty("alert_way") alertWays: String, - @JsonProperty("alert_reciver") alertReceivers: String, - @JsonProperty("alert_level") alertLevel: String, - @JsonProperty("alert_obj") alertObj: String, - @JsonProperty("can_recover") canRecover: String -) diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala index 021ec3b588..285f5075dd 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/MonitorAlertUtils.scala @@ -21,6 +21,8 @@ import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException import org.apache.linkis.monitor.scan.constants.Constants import org.apache.linkis.monitor.scan.utils.alert.AlertDesc + +import org.apache.commons.io.IOUtils import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils @@ -28,14 +30,14 @@ import java.io.{BufferedReader, File, FileInputStream, InputStream, InputStreamR import java.text.SimpleDateFormat import java.util import java.util.Properties + import scala.collection.JavaConverters._ + import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.DefaultScalaModule -import org.apache.commons.io.IOUtils object MonitorAlertUtils extends Logging { - private val mapper = { val ret = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) ret.registerModule(DefaultScalaModule) @@ -43,11 +45,7 @@ object MonitorAlertUtils extends Logging { ret } - def getAlerts( - prefix: String, - parms: util.HashMap[String, String] - ): util.Map[String, AlertDesc] = { - val ret = new util.HashMap[String, AlertDesc]() + val properties = { val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) if (url == null) { throw new AnomalyScannerException( @@ -58,29 +56,37 @@ object MonitorAlertUtils extends Logging { logger.info("reading alert properties from: " + url.getFile) val properties = new Properties() var inputStream: InputStream = null - var reader: InputStreamReader = null var buff: BufferedReader = null - Utils.tryFinally { - Utils.tryThrow { - inputStream = new FileInputStream(new File(url.getFile)) - reader = new InputStreamReader(inputStream, "UTF-8") - buff = new BufferedReader(reader) - properties.load(buff) - } { - case t: Throwable => - new AnomalyScannerException( + Utils.tryCatch { + inputStream = new FileInputStream(new File(url.getFile)) + reader = new InputStreamReader(inputStream, "UTF-8") + buff = new BufferedReader(reader) + properties.load(buff) + } { t => + { + throw new AnomalyScannerException( 21304, "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t) ) } + } } { IOUtils.closeQuietly(buff) IOUtils.closeQuietly(reader) IOUtils.closeQuietly(inputStream) } - for ((k: String, v: String) <- properties.asScala) { + properties.asScala + } + + def getAlerts( + prefix: String, + params: util.HashMap[String, String] + ): util.Map[String, AlertDesc] = { + val ret = new util.HashMap[String, AlertDesc]() + + for ((k: String, v: String) <- properties) { if (ret.containsKey(k)) { logger.warn("found duplicate key in alert properties, accept only the first one") } else if (StringUtils.startsWith(k, prefix)) { @@ -89,10 +95,10 @@ object MonitorAlertUtils extends Logging { new StringBuilder().append(data.alertInfo).toString().getBytes(), "utf-8" ).replace("$name", data.alertReceivers) - val interator = parms.keySet.iterator + val interator = params.keySet.iterator while (interator.hasNext) { val key = interator.next - val value = parms.get(key) + val value = params.get(key) alertInfo = alertInfo.replace(key, value) } val receivers = { @@ -100,113 +106,38 @@ object MonitorAlertUtils extends Logging { if (StringUtils.isNotBlank(data.alertReceivers)) { data.alertReceivers.split(",").map(r => set.add(r)) } - Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { - if (StringUtils.isNotBlank(e)) { - set.add(e) - } - }) - set.add(parms.get("$alteruser")) - set - } - val alertDesc = Utils.tryAndWarn( - new ImsAlertDesc( - Constants.ALERT_SUB_SYSTEM_ID, - data.alertTitle, - data.alertObj, - alertInfo, - ImsAlertLevel.withName(data.alertLevel), - null, // Not used - 0, { - val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] - if (StringUtils.isNotBlank(data.alertWays)) { - data.alertWays - .split(",") - .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) + if (!params.containsKey("$alteruser")) { + Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { + if (StringUtils.isNotBlank(e)) { + set.add(e) } - set - }, - receivers - ) - ) - val realK = StringUtils.substringAfter(k, prefix) - ret.put(realK, alertDesc) - } - } - ret - } - - - def getAlertsByDss(prefix: String, parms: util.Map[String, String]): util.Map[String, AlertDesc] = { - val ret = new util.HashMap[String, AlertDesc]() - val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) - if (url == null) { - throw new AnomalyScannerException( - 21304, - "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH - ) - } - logger.info("reading alert properties from: " + url.getFile) - val properties = new Properties() - var inputStream: InputStream = null - - var reader: InputStreamReader = null - var buff: BufferedReader = null - - Utils.tryFinally { - Utils.tryThrow{ - inputStream = new FileInputStream(new File(url.getFile)) - reader = new InputStreamReader(inputStream, "UTF-8") - buff = new BufferedReader(reader) - properties.load(buff) - } { - case t: Throwable => - new AnomalyScannerException( - 21304, - "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t) - ) - } - } { - IOUtils.closeQuietly(buff) - IOUtils.closeQuietly(reader) - IOUtils.closeQuietly(inputStream) - } - for ((k: String, v: String) <- properties.asScala) { - if (ret.containsKey(k)) { - warn("found duplicate key in alert properties, accept only the first one") - } else if (StringUtils.startsWith(k, prefix)) { - val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) - var alertInfo = data.alertInfo - val interator = parms.keySet.iterator - while (interator.hasNext) { - val key = interator.next - val value = parms.get(key) - alertInfo = alertInfo.replace(key, value) - } -// alertInfo = parms.getOrDefault("detail", "").concat(alertInfo) - val receivers = { - val set: util.Set[String] = new util.HashSet[String] - if (StringUtils.isNotBlank(data.alertReceivers)) { - data.alertReceivers.split(",").map(r => set.add(r)) + }) + } else { + set.add(params.get("$alteruser")) } - Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { - if (StringUtils.isNotBlank(e)) { - set.add(e) - } - }) - if (StringUtils.isNotBlank(parms.get("receiver"))) { - parms.get("receiver").split(",").map(r => set.add(r)) + if (StringUtils.isNotBlank(params.get("receiver"))) { + params.get("receiver").split(",").map(r => set.add(r)) } set } + + val subSystemId = params.getOrDefault("subSystemId", Constants.ALERT_SUB_SYSTEM_ID) + val alertTitle = params.getOrDefault("title", data.alertTitle) + val alertLevel = + if (StringUtils.isNotBlank(data.alertLevel)) { + ImsAlertLevel.withName(params.getOrDefault("monitorLevel", data.alertLevel)) + } else { + ImsAlertLevel.withName(params.getOrDefault("monitorLevel", ImsAlertLevel.WARN.toString)) + } + val alertDesc = Utils.tryAndWarn( - ImsAlertDesc( - parms.getOrDefault("subSystemId", Constants.ALERT_SUB_SYSTEM_ID), - parms.getOrDefault("title", ""), + ImsAlertDesc( + subSystemId, + alertTitle, data.alertObj, - alertInfo - , - ImsAlertLevel.withName(parms.getOrDefault("monitorLevel", "4")), - null, // Not used + alertInfo, + alertLevel, + null, 0, { val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] if (StringUtils.isNotBlank(data.alertWays)) { diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala index d2508b37a3..a553cbba89 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertSender.scala @@ -34,15 +34,9 @@ import java.util import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule +class PooledImsAlertSender(alertUrl: String) extends PooledAlertSender with Logging { -class PooledImsAlertSender( - subSystemId: String, - alertUrl: String, - default_Receivers: util.Set[String] -) extends PooledAlertSender - with Logging { - - protected val httpClient = HttpClients.createDefault // TODO: Linkis-httpclient + protected val httpClient = HttpClients.createDefault private val mapper = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) @@ -74,7 +68,7 @@ class PooledImsAlertSender( return false } if (paramContent.isEmpty) { - logger. warn("alertParams is empty, will not send alarm") + logger.warn("alertParams is empty, will not send alarm") return false } @@ -99,7 +93,7 @@ class PooledImsAlertSender( LogUtils.stdOutLogger.info("Alert: " + paramContent + "Response: " + responseInfo) if (response.getStatusLine.getStatusCode == 200) return true } - return false + false } override def shutdown(waitComplete: Boolean = true, timeoutMs: Long = -1): Unit = { diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala index 4a50161438..37fd35724b 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/PooledImsAlertUtils.scala @@ -33,15 +33,10 @@ import scala.collection.JavaConverters._ import ImsAlertWay.ImsAlertWay - object PooledImsAlertUtils extends Logging { private val sender: PooledImsAlertSender = { - val ret = new PooledImsAlertSender( - Constants.ALERT_SUB_SYSTEM_ID, - Constants.ALERT_IMS_URL, - Constants.ALERT_DEFAULT_RECEIVERS.asJava - ) + val ret = new PooledImsAlertSender(Constants.ALERT_IMS_URL) ret.start() ret } @@ -70,11 +65,11 @@ object PooledImsAlertUtils extends Logging { if (CollectionUtils.isNotEmpty(alertWays)) alertWays else new HashSet[ImsAlertWay]() val (alertInfo, alertLevel) = if (t != null) { _alertWays.add(ImsAlertWay.Email) - _alertWays.add(ImsAlertWay.RTX) + _alertWays.add(ImsAlertWay.WXWork) _alertWays.add(ImsAlertWay.WeChat) (ExceptionUtils.getRootCauseMessage(t), ImsAlertLevel.MAJOR) } else { - _alertWays.add(ImsAlertWay.RTX) + _alertWays.add(ImsAlertWay.WXWork) (message, ImsAlertLevel.WARN) } val alertDesc = new ImsAlertDesc( diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala index 1083967783..d5fb4a9b07 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/alert/ims/UserLabelAlertUtils.scala @@ -21,6 +21,8 @@ import org.apache.linkis.common.utils.{JsonUtils, Logging, Utils} import org.apache.linkis.monitor.scan.app.jobhistory.exception.AnomalyScannerException import org.apache.linkis.monitor.scan.constants.Constants import org.apache.linkis.monitor.scan.utils.alert.AlertDesc + +import org.apache.commons.io.IOUtils import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.exception.ExceptionUtils @@ -28,99 +30,18 @@ import java.io.{BufferedReader, File, FileInputStream, InputStream, InputStreamR import java.text.SimpleDateFormat import java.util import java.util.Properties + import scala.collection.JavaConverters._ + import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.DefaultScalaModule -import org.apache.commons.io.IOUtils object UserLabelAlertUtils extends Logging { - private val mapper = { - val ret = new ObjectMapper().setDateFormat(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ")) - ret.registerModule(DefaultScalaModule) - ret.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) - ret - } - - def getAlerts(prefix: String, instans: String): util.Map[String, AlertDesc] = { - val ret = new util.HashMap[String, AlertDesc]() - val url = getClass.getClassLoader.getResource(Constants.ALERT_PROPS_FILE_PATH) - if (url == null) { - throw new AnomalyScannerException( - 21304, - "Failed to load alerts from alert properties. Alert properties file does not exist: " + Constants.ALERT_PROPS_FILE_PATH - ) - } - logger.info("reading alert properties from: " + url.getFile) - val properties = new Properties() - var inputStream: InputStream = null - - var reader: InputStreamReader = null - var buff: BufferedReader = null - - Utils.tryFinally { - Utils.tryThrow { - inputStream = new FileInputStream(new File(url.getFile)) - reader = new InputStreamReader(inputStream, "UTF-8") - buff = new BufferedReader(reader) - properties.load(buff) - } { - case t: Throwable => - new AnomalyScannerException( - 21304, - "Failed to load alerts from alert properties. Cause: " + ExceptionUtils.getMessage(t) - ) - } - } { - IOUtils.closeQuietly(buff) - IOUtils.closeQuietly(reader) - IOUtils.closeQuietly(inputStream) - } - for ((k: String, v: String) <- properties.asScala) { - if (ret.containsKey(k)) { - logger.warn("found duplicate key in alert properties, accept only the first one") - } else if (StringUtils.startsWith(k, prefix)) { - val data = mapper.readValue(v, classOf[ImsAlertPropFileData]) - val alertInfo = - new String(new StringBuilder().append(data.alertInfo).toString().getBytes(), "utf-8") - .replace("$userCreator", instans) - val receivers = { - val set: util.Set[String] = new util.HashSet[String] - if (StringUtils.isNotBlank(data.alertReceivers)) { - data.alertReceivers.split(",").map(r => set.add(r)) - } - Constants.ALERT_DEFAULT_RECEIVERS.foreach(e => { - if (StringUtils.isNotBlank(e)) { - set.add(e) - } - }) - set - } - val alertDesc = Utils.tryAndWarn( - new ImsAlertDesc( - Constants.ALERT_SUB_SYSTEM_ID, - data.alertTitle, - data.alertObj, - alertInfo, - ImsAlertLevel.withName(data.alertLevel), - null, // Not used - 0, { - val set: util.Set[ImsAlertWay.Value] = new util.HashSet[ImsAlertWay.Value] - if (StringUtils.isNotBlank(data.alertWays)) { - data.alertWays - .split(",") - .map(alertWayStr => set.add(ImsAlertWay.withName(alertWayStr))) - } - set - }, - receivers - ) - ) - val realK = StringUtils.substringAfter(k, prefix) - ret.put(realK, alertDesc) - } - } - ret + def getAlerts(prefix: String, userCreator: String): util.Map[String, AlertDesc] = { + val replaceParams: util.HashMap[String, String] = new util.HashMap[String, String] + replaceParams.put("$userCreator", userCreator) + MonitorAlertUtils.getAlerts(prefix, replaceParams) } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala index 38af7b6104..73daf86721 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/utils/log/LogUtils.scala @@ -19,7 +19,6 @@ package org.apache.linkis.monitor.scan.utils.log import org.slf4j.LoggerFactory - object LogUtils { val stdOutLogger = LoggerFactory.getLogger("PlaintTextConsoleLogger") } diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java deleted file mode 100644 index 600ccafddb..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/app/JobHistoryScanImsAlertPropFileParserUtilsTest.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.monitor.scan.app; - -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; -import org.apache.linkis.server.utils.LinkisMainHelper; - -import java.util.Map; - -import org.junit.Assert; - -public class JobHistoryScanImsAlertPropFileParserUtilsTest { - // @Before - public void before() { - System.getProperties().setProperty(LinkisMainHelper.SERVER_NAME_KEY(), "linkis-et-monitor"); - System.getProperties() - .setProperty("log4j.configurationFile", "src/test/resources/log4j2-console.xml"); - // System.getProperties().setProperty("wds.linkis.server.conf", - // "linkis-et-monitor.properties"); - } - - // @Test - public void getAlerts() throws Exception { - Map alerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); - for (Map.Entry kv : alerts.entrySet()) { - System.out.println(kv.getKey() + ": " + kv.getValue().toString()); - } - Assert.assertEquals(alerts.size(), 2); - } -} diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java index 8ca5ff1a98..2014058561 100644 --- a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java +++ b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertUtilsTest.java @@ -18,7 +18,7 @@ package org.apache.linkis.monitor.scan.utils.alert; import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.utils.alert.ims.JobHistoryScanImsAlertPropFileParserUtils; +import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; import org.apache.linkis.server.utils.LinkisMainHelper; @@ -38,7 +38,7 @@ public void before() { public void addAlert() throws Exception { PooledImsAlertUtils.addAlert("1st test"); Map alerts = - JobHistoryScanImsAlertPropFileParserUtils.getAlerts(Constants.SCAN_PREFIX_ERRORCODE()); + MonitorAlertUtils.getAlerts((Constants.SCAN_PREFIX_ERRORCODE()), null); for (Map.Entry kv : alerts.entrySet()) { System.out.println(kv.getKey() + ": " + kv.getValue().toString()); PooledImsAlertUtils.addAlert(kv.getValue()); From 5e52f909e4c06aedb4c5e9ab7fa2c8bae7f23884 Mon Sep 17 00:00:00 2001 From: peacewong Date: Fri, 8 Sep 2023 18:57:48 +0800 Subject: [PATCH 56/72] add bml --- linkis-extensions/linkis-et-monitor/pom.xml | 24 +- .../scan/app/bml/cleaner/dao/VersionDao.java | 61 +++++ .../entity/CleanedResourceVersion.java | 209 ++++++++++++++++++ .../bml/cleaner/entity/ResourceVersion.java | 206 +++++++++++++++++ .../bml/cleaner/service/CleanerService.java | 23 ++ .../bml/cleaner/service/VersionService.java | 37 ++++ .../service/impl/CleanerServiceImpl.java | 170 ++++++++++++++ .../service/impl/VersionServiceImpl.java | 63 ++++++ .../app/bml/cleaner/vo/CleanResourceVo.java | 48 ++++ .../scan/app/monitor/scheduled/BmlClear.java | 2 +- .../monitor/scan/constants/Constants.scala | 19 ++ .../utils/alert/PooledImsAlertSenderTest.java | 2 +- .../linkis-et-jobhistory-scan-ims.properties | 6 - .../linkis-et-jobhistory-scan.properties | 14 -- 14 files changed, 842 insertions(+), 42 deletions(-) create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java create mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java delete mode 100644 linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties delete mode 100644 linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties diff --git a/linkis-extensions/linkis-et-monitor/pom.xml b/linkis-extensions/linkis-et-monitor/pom.xml index 6972c9e1a4..0e5f44c076 100644 --- a/linkis-extensions/linkis-et-monitor/pom.xml +++ b/linkis-extensions/linkis-et-monitor/pom.xml @@ -36,13 +36,13 @@ org.apache.linkis - linkis-wedatasphere-common + linkis-storage ${project.version} + provided - org.apache.linkis - linkis-module + linkis-rpc ${project.version} provided @@ -68,7 +68,6 @@ junit junit - 4.12 test @@ -76,22 +75,7 @@ linkis-computation-client ${project.version} - - org.apache.linkis - linkis-bml-cleaner - ${project.version} - - - org.apache.linkis - linkis-storage - ${project.version} - provided - - - org.apache.linkis - linkis-rpc - ${project.version} - + diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java new file mode 100644 index 0000000000..6b6a101b45 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.linkis.monitor.scan.app.bml.cleaner.dao; + +import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.ResourceVersion; +import org.apache.linkis.monitor.scan.app.bml.cleaner.vo.CleanResourceVo; + +import org.apache.ibatis.annotations.*; + +import java.util.Date; +import java.util.List; + +public interface VersionDao { + + @Select( + "select resource_id, count(resource_id) as version_count, max(version) as max_version from " + + "linkis_ps_bml_resources_version lpbrv where start_time < #{startTime} GROUP BY resource_id HAVING count(resource_id) > #{maxVersionNum} limit #{limitNum}") + List getAllNeedCleanResource( + @Param("maxVersionNum") Integer maxVersionNum, + @Param("startTime") Date startTime, + @Param("limitNum") int num); + + @Select( + "select * from linkis_ps_bml_resources_version where resource_id = #{resourceId} and version < #{minKeepVersion} and version <> 'v000001'") + List getCleanVersionsByResourceId( + @Param("resourceId") String resourceId, @Param("minKeepVersion") String minKeepVersion); + + @Insert({ + "insert into linkis_ps_bml_cleaned_resources_version(`resource_id`,`file_md5`,`version`,`size`,`start_byte`, `end_byte`,`resource`,`description`," + + "`start_time`,`end_time`,`client_ip`,`updator`,`enable_flag`,`old_resource`) values(#{resourceId},#{fileMd5},#{version},#{size},#{startByte},#{endByte}" + + ",#{resource},#{description},#{startTime},#{endTime},#{clientIp},#{updator},#{enableFlag},#{oldResource})" + }) + @Options(useGeneratedKeys = true, keyProperty = "id") + void insertCleanResourceVersion(CleanedResourceVersion cleanedResourceVersion); + + @Delete("delete from linkis_ps_bml_resources_version where id=#{id}") + void deleteResourceVersionById(@Param("id") long id); + + @Select( + "select version from linkis_ps_bml_resources_version where resource_id =#{resourceId} and version <= #{maxVersion} order by version desc limit #{keepNum},1") + String getMinKeepVersion( + @Param("resourceId") String resourceId, + @Param("maxVersion") String maxVersion, + @Param("keepNum") int keepNum); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java new file mode 100644 index 0000000000..3c6f103183 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java @@ -0,0 +1,209 @@ +/* + * 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.monitor.scan.app.bml.cleaner.entity; + +import java.util.Date; + +public class CleanedResourceVersion { + + private long id; + + private String resourceId; + + private String fileMd5; + + private String version; + + private long size; + + private String resource; + + private String oldResource; + + private String description; + + private String clientIp; + + private boolean enableFlag; + + private String user; + + private String system; + + private Date startTime; + + private Date endTime; + + private long startByte; + + private long endByte; + + private String updator; + + public String getResourceId() { + return resourceId; + } + + public void setResourceId(String resourceId) { + this.resourceId = resourceId; + } + + public String getUser() { + return user; + } + + public void setUser(String user) { + this.user = user; + } + + public String getSystem() { + return system; + } + + public void setSystem(String system) { + this.system = system; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + + public String getResource() { + return resource; + } + + public void setResource(String resource) { + this.resource = resource; + } + + public String getOldResource() { + return oldResource; + } + + public void setOldResource(String oldResource) { + this.oldResource = oldResource; + } + + public long getId() { + return id; + } + + public void setId(long id) { + this.id = id; + } + + public String getFileMd5() { + return fileMd5; + } + + public void setFileMd5(String fileMd5) { + this.fileMd5 = fileMd5; + } + + public long getSize() { + return size; + } + + public void setSize(long size) { + this.size = size; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public String getClientIp() { + return clientIp; + } + + public void setClientIp(String clientIp) { + this.clientIp = clientIp; + } + + public boolean isEnableFlag() { + return enableFlag; + } + + public void setEnableFlag(boolean enableFlag) { + this.enableFlag = enableFlag; + } + + public long getStartByte() { + return startByte; + } + + public void setStartByte(long startByte) { + this.startByte = startByte; + } + + public long getEndByte() { + return endByte; + } + + public void setEndByte(long endByte) { + this.endByte = endByte; + } + + public Date getStartTime() { + return startTime; + } + + public void setStartTime(Date startTime) { + this.startTime = startTime; + } + + public Date getEndTime() { + return endTime; + } + + public void setEndTime(Date endTime) { + this.endTime = endTime; + } + + public String getUpdator() { + return updator; + } + + public void setUpdator(String updator) { + this.updator = updator; + } + + public static CleanedResourceVersion copyFromResourceVersion(ResourceVersion resourceVersion) { + CleanedResourceVersion cleanedResourceVersion = new CleanedResourceVersion(); + cleanedResourceVersion.setResourceId(resourceVersion.getResourceId()); + cleanedResourceVersion.setOldResource(resourceVersion.getResource()); + cleanedResourceVersion.setFileMd5(resourceVersion.getFileMd5()); + cleanedResourceVersion.setClientIp(resourceVersion.getClientIp()); + cleanedResourceVersion.setSize(resourceVersion.getSize()); + cleanedResourceVersion.setEnableFlag(resourceVersion.getEnableFlag()); + cleanedResourceVersion.setVersion(resourceVersion.getVersion()); + cleanedResourceVersion.setStartByte(resourceVersion.getStartByte()); + cleanedResourceVersion.setEndByte(resourceVersion.getEndByte()); + cleanedResourceVersion.setStartTime(resourceVersion.getStartTime()); + cleanedResourceVersion.setEndTime(resourceVersion.getEndTime()); + return cleanedResourceVersion; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java new file mode 100644 index 0000000000..604ab3662a --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java @@ -0,0 +1,206 @@ +/* + * 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.monitor.scan.app.bml.cleaner.entity; + +import java.util.Date; + +public class ResourceVersion { + + private long id; + + private String resourceId; + + private String fileMd5; + + private String version; + + private long size; + + private String resource; + + private String description; + + private String clientIp; + + private boolean enableFlag; + + private String user; + + private String system; + + private Date startTime; + + private Date endTime; + + private long startByte; + + private long endByte; + + private String updator; + + public String getResourceId() { + return resourceId; + } + + public void setResourceId(String resourceId) { + this.resourceId = resourceId; + } + + public String getUser() { + return user; + } + + public void setUser(String user) { + this.user = user; + } + + public String getSystem() { + return system; + } + + public void setSystem(String system) { + this.system = system; + } + + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + + public String getResource() { + return resource; + } + + public void setResource(String resource) { + this.resource = resource; + } + + public long getId() { + return id; + } + + public void setId(long id) { + this.id = id; + } + + public String getFileMd5() { + return fileMd5; + } + + public void setFileMd5(String fileMd5) { + this.fileMd5 = fileMd5; + } + + public long getSize() { + return size; + } + + public void setSize(long size) { + this.size = size; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public String getClientIp() { + return clientIp; + } + + public void setClientIp(String clientIp) { + this.clientIp = clientIp; + } + + public boolean getEnableFlag() { + return enableFlag; + } + + public void setEnableFlag(boolean enableFlag) { + this.enableFlag = enableFlag; + } + + public long getStartByte() { + return startByte; + } + + public void setStartByte(long startByte) { + this.startByte = startByte; + } + + public long getEndByte() { + return endByte; + } + + public void setEndByte(long endByte) { + this.endByte = endByte; + } + + public Date getStartTime() { + return startTime; + } + + public void setStartTime(Date startTime) { + this.startTime = startTime; + } + + public Date getEndTime() { + return endTime; + } + + public void setEndTime(Date endTime) { + this.endTime = endTime; + } + + public String getUpdator() { + return updator; + } + + public void setUpdator(String updator) { + this.updator = updator; + } + + public static ResourceVersion createNewResourceVersion( + String resourceId, + String resourcePath, + String fileMd5, + String clientIp, + long size, + String version, + long startByte) { + ResourceVersion resourceVersion = new ResourceVersion(); + resourceVersion.setResourceId(resourceId); + resourceVersion.setResource(resourcePath); + resourceVersion.setFileMd5(fileMd5); + resourceVersion.setClientIp(clientIp); + resourceVersion.setSize(size); + resourceVersion.setEnableFlag(true); + resourceVersion.setVersion(version); + resourceVersion.setStartByte(startByte); + resourceVersion.setEndByte(startByte + size - 1); + resourceVersion.setStartTime(new Date(System.currentTimeMillis())); + resourceVersion.setEndTime(new Date(System.currentTimeMillis())); + return resourceVersion; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java new file mode 100644 index 0000000000..ccbea7d8c7 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java @@ -0,0 +1,23 @@ +/* + * 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.monitor.scan.app.bml.cleaner.service; + +public interface CleanerService { + + public void run(); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java new file mode 100644 index 0000000000..3162c096e9 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java @@ -0,0 +1,37 @@ +/* + * 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.monitor.scan.app.bml.cleaner.service; + +import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.common.io.FsPath; +import org.apache.linkis.storage.fs.FileSystem; + +import java.io.IOException; + +public interface VersionService { + + void doMove( + FileSystem fs, + FsPath srcPath, + FsPath destPath, + CleanedResourceVersion insertVersion, + long delVersionId) + throws IOException; + + void moveOnDb(CleanedResourceVersion insertVersion, long delVersionId); +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java new file mode 100644 index 0000000000..e17d55b7dd --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java @@ -0,0 +1,170 @@ +/* + * 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.monitor.scan.app.bml.cleaner.service.impl; + +import org.apache.linkis.monitor.scan.app.bml.cleaner.dao.VersionDao; +import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.ResourceVersion; +import org.apache.linkis.monitor.scan.app.bml.cleaner.service.CleanerService; +import org.apache.linkis.monitor.scan.app.bml.cleaner.service.VersionService; +import org.apache.linkis.monitor.scan.app.bml.cleaner.vo.CleanResourceVo; +import org.apache.linkis.common.io.FsPath; +import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.storage.FSFactory; +import org.apache.linkis.storage.fs.FileSystem; +import org.apache.linkis.storage.utils.StorageConfiguration; +import org.apache.linkis.storage.utils.StorageUtils; + +import org.apache.commons.io.IOUtils; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Service; + +import java.io.File; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Service +public class CleanerServiceImpl implements CleanerService { + + private final Logger logger = LoggerFactory.getLogger("CleanerServiceImpl"); + + private final SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMdd"); + + public static final String VERSION_FORMAT = "%06d"; + public static final String VERSION_PREFIX = "v"; + public static final String TRASH_DIR = "/trash"; + + private FileSystem fs = null; + + @Autowired private VersionDao versionDao; + + public void setVersionDao(VersionDao versionDao) { + this.versionDao = versionDao; + } + + private Set cleanedResourceIds = new HashSet(); + + Date previous; + + @Autowired VersionService versionService; + + public void clean() { + previous = + new Date( + System.currentTimeMillis() + - (Long) Constants.BML_PREVIOUS_INTERVAL_TIME_DAYS().getValue() + * 86400 + * 1000); + + if ((Integer) Constants.BML_VERSION_MAX_NUM().getValue() + - (Integer) Constants.BML_VERSION_KEEP_NUM().getValue() + <= 1) { + logger.error("conf error need to keep version num > 1"); + return; + } + List needCleanResources = getCleanResources(); + while (needCleanResources != null && needCleanResources.size() > 0) { + logger.info("need cleaned resource count:{}", needCleanResources.size()); + fs = + (FileSystem) + FSFactory.getFs( + StorageUtils.HDFS, StorageConfiguration.HDFS_ROOT_USER.getValue()); + for (CleanResourceVo resourceVo : needCleanResources) { + String minVersion = + versionDao.getMinKeepVersion( + resourceVo.getResourceId(), + resourceVo.getMaxVersion(), + (Integer) Constants.BML_VERSION_KEEP_NUM().getValue() - 1); + List cleanVersionList = + versionDao.getCleanVersionsByResourceId(resourceVo.getResourceId(), minVersion); + // move on hdfs + for (ResourceVersion version : cleanVersionList) { + FsPath srcPath = new FsPath(version.getResource()); + // fs放到外层 + try { + fs.init(null); + if (!fs.exists(srcPath)) { + logger.error("try to move but bml source file:{} not exists!", version.getResource()); + CleanedResourceVersion cleanedResourceVersion = + CleanedResourceVersion.copyFromResourceVersion(version); + cleanedResourceVersion.setResource(""); + versionService.moveOnDb(cleanedResourceVersion, version.getId()); + continue; + } + String destPrefix = + version.getResource().substring(0, version.getResource().indexOf("/bml/") + 4); + String destPath = + destPrefix + + TRASH_DIR + + File.separator + + sdf.format(new Date()) + + File.separator + + version.getResourceId() + + "_" + + version.getVersion(); + FsPath dest = new FsPath(destPath); + if (!fs.exists(dest.getParent())) { + fs.mkdirs(dest.getParent()); + } + logger.info("begin to mv bml resource:{} to dest:{}", version.getResource(), destPath); + CleanedResourceVersion cleanedResourceVersion = + CleanedResourceVersion.copyFromResourceVersion(version); + cleanedResourceVersion.setResource(destPath); + versionService.doMove(fs, srcPath, dest, cleanedResourceVersion, version.getId()); + } catch (Exception e) { + logger.error("failed to mv bml resource:{}", e.getMessage(), e); + } + } + + cleanedResourceIds.add(resourceVo.getResourceId()); + } + needCleanResources = getCleanResources(); + } + } + + public void run() { + logger.info("start to clean."); + clean(); + logger.info("start to shutdown."); + shutdown(); + } + + void shutdown() { + IOUtils.closeQuietly(fs); + } + + List getCleanResources() { + List cleanResourceVoList = + versionDao.getAllNeedCleanResource( + (Integer) Constants.BML_VERSION_MAX_NUM().getValue(), + previous, + (Integer) Constants.BML_CLEAN_ONCE_RESOURCE_LIMIT_NUM().getValue()); + + return cleanResourceVoList.stream() + .filter(cleanResourceVo -> !cleanedResourceIds.contains(cleanResourceVo.getResourceId())) + .collect(Collectors.toList()); + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java new file mode 100644 index 0000000000..8234d524af --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java @@ -0,0 +1,63 @@ +/* + * 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.monitor.scan.app.bml.cleaner.service.impl; + +import org.apache.linkis.monitor.scan.app.bml.cleaner.dao.VersionDao; +import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.scan.app.bml.cleaner.service.VersionService; +import org.apache.linkis.common.io.FsPath; +import org.apache.linkis.storage.fs.FileSystem; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Service; +import org.springframework.transaction.annotation.Transactional; + +import java.io.IOException; + +@Service +public class VersionServiceImpl implements VersionService { + + @Autowired VersionDao versionDao; + + public void setVersionDao(VersionDao versionDao) { + this.versionDao = versionDao; + } + + @Transactional(rollbackFor = Throwable.class) + public void doMove( + FileSystem fs, + FsPath srcPath, + FsPath destPath, + CleanedResourceVersion insertVersion, + long delVersionId) + throws IOException { + versionDao.insertCleanResourceVersion(insertVersion); + versionDao.deleteResourceVersionById(delVersionId); + fs.renameTo(srcPath, destPath); + } + + @Transactional + public void moveOnDb(CleanedResourceVersion insertVersion, long delVersionId) { + versionDao.insertCleanResourceVersion(insertVersion); + versionDao.deleteResourceVersionById(delVersionId); + } + + public String test() { + return "this a test string"; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java new file mode 100644 index 0000000000..cfcf6e5ea6 --- /dev/null +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java @@ -0,0 +1,48 @@ +/* + * 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.monitor.scan.app.bml.cleaner.vo; + +public class CleanResourceVo { + private String resourceId; + private int versionCount; + private String maxVersion; + + public String getResourceId() { + return resourceId; + } + + public void setResourceId(String resourceId) { + this.resourceId = resourceId; + } + + public int getVersionCount() { + return versionCount; + } + + public void setVersionCount(int versionCount) { + this.versionCount = versionCount; + } + + public String getMaxVersion() { + return maxVersion; + } + + public void setMaxVersion(String maxVersion) { + this.maxVersion = maxVersion; + } +} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java index 11ee06226f..362af45abe 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java @@ -17,7 +17,7 @@ package org.apache.linkis.monitor.scan.app.monitor.scheduled; -import org.apache.linkis.bml.cleaner.service.CleanerService; +import org.apache.linkis.monitor.scan.app.bml.cleaner.service.CleanerService; import org.apache.linkis.monitor.scan.utils.log.LogUtils; import org.springframework.beans.factory.annotation.Autowired; diff --git a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala index 89ab330b4d..11ea350af3 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala +++ b/linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/scan/constants/Constants.scala @@ -88,4 +88,23 @@ object Constants { val THREAD_TIME_OUT_IM = "thread.monitor.timeout.im." val JOB_RESULT_IM = "jobhistory.result.monitor.im." + + val BML_VERSION_MAX_NUM: CommonVars[Int] = + CommonVars[Int]("linkis.bml.cleaner.version.max.num", 50) + + val BML_VERSION_KEEP_NUM: CommonVars[Int] = + CommonVars[Int]("linkis.bml.cleaner.version.keep.num", 20) + + val BML_PREVIOUS_INTERVAL_TIME_DAYS: CommonVars[Long] = + CommonVars[Long]("linkis.bml.cleaner.previous.interval.days", 30) + + val BML_CLEAN_ONCE_RESOURCE_LIMIT_NUM: CommonVars[Int] = + CommonVars[Int]("linkis.bml.cleaner.once.limit.num", 100) + + val BML_TRASH_PATH_PREFIX: CommonVars[String] = CommonVars[String]( + "linkis.bml.trash.prefix.path", + "hdfs:///tmp/linkis/trash/bml_trash" + ) + + } diff --git a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java index 8cfad4c989..bd6b0a927e 100644 --- a/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java +++ b/linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/scan/utils/alert/PooledImsAlertSenderTest.java @@ -61,7 +61,7 @@ public void doSendAlert() throws Exception { System.out.println(desc); String url = "http://172.21.0.130:10812/ims_data_access/send_alarm_by_json.do"; - PooledImsAlertSender sender = new PooledImsAlertSender("5136", url, receivers); + PooledImsAlertSender sender = new PooledImsAlertSender(url); sender.doSendAlert(desc); } } diff --git a/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties b/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties deleted file mode 100644 index f08d130651..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan-ims.properties +++ /dev/null @@ -1,6 +0,0 @@ -jobhistory.errorcode.11001={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} -jobhistory.errorcode.11002={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} -jobhistory.errorcode.12011={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} -jobhistory.errorcode.43003={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} -jobhistory.errorcode.42011={"alert_title":"linkis_alert_test","alert_info":"this is a test for linkis errorcode","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} -jobhistory.unfinished.time.exceed.sec.300={"alert_title":"linkis_alert_test_job_time","alert_info":"this is a test for linkis 5 job time exceed","alert_way":"3,2","alert_reciver":"shangda","alert_level":"2","alert_obj":"linkis_alert","can_recover":"0"} \ No newline at end of file diff --git a/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties b/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties deleted file mode 100644 index 0eca42bd5d..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/test/resources/linkis-et-jobhistory-scan.properties +++ /dev/null @@ -1,14 +0,0 @@ -wds.linkis.server.mybatis.mapperLocations=classpath*:org/apache/linkis/jobhistory/scan/app/jobhistory/dao/impl/*.xml -wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.monitor.scan.app.jobhistory.entity -wds.linkis.server.mybatis.BasePackage=org.apache.linkis.monitor.scan.app.jobhistory.dao -wds.linkis.mysql.is.encrypt=false -#wds.linkis.server.mybatis.datasource.url=jdbc:mysql://10.108.161.105:15304/bdp_easy_ide?characterEncoding=UTF-8 -#wds.linkis.server.mybatis.datasource.username=bdpeasyide -#wds.linkis.server.mybatis.datasource.password=bdpeasyide@bdpsit -wds.linkis.server.mybatis.datasource.url=jdbc:mysql://10.107.108.111:3306/vsbi_gz_bdap_sit_01?characterEncoding=UTF-8 -wds.linkis.server.mybatis.datasource.username=bdp_vsbi -wds.linkis.server.mybatis.datasource.password=bdpVsbi@2019 -wds.linkis.alert.url=http://172.21.0.130:10812/ims_data_access/send_alarm_by_json.do -wds.linkis.alert.receiver.default=johnnwang -wds.linkis.errorcode.scanner.interval.seconds=7200 -wds.linkis.errorcode.scanner.max.interval.seconds=3600 \ No newline at end of file From 703eb501e781a509ad5ec615eaa462598bbe82d1 Mon Sep 17 00:00:00 2001 From: peacewong Date: Mon, 11 Sep 2023 22:30:05 +0800 Subject: [PATCH 57/72] code optimize --- .../conf/linkis-et-monitor-ims.properties | 143 ++++++++++++++++++ .../package/conf/linkis-et-monitor.properties | 86 +++++++++++ .../LinksMonitorResourceApplication.java | 2 +- .../app => }/bml/cleaner/dao/VersionDao.java | 9 +- .../entity/CleanedResourceVersion.java | 2 +- .../bml/cleaner/entity/ResourceVersion.java | 2 +- .../bml/cleaner/service/CleanerService.java | 2 +- .../bml/cleaner/service/VersionService.java | 5 +- .../service/impl/CleanerServiceImpl.java | 18 +-- .../service/impl/VersionServiceImpl.java | 13 +- .../bml/cleaner/vo/CleanResourceVo.java | 2 +- .../monitor => }/config/ListenerConfig.java | 8 +- .../monitor => }/config/MonitorConfig.java | 12 +- .../monitor => }/config/ScheduledConfig.java | 2 +- .../app/monitor => }/entity/EngineEntity.java | 2 +- .../monitor => }/entity/EntranceEntity.java | 2 +- .../app/monitor => }/entity/IndexEntity.java | 4 +- .../instance/dao/InsLabelRelationDao.java | 5 +- .../instance/dao/InstanceInfoDao.java | 5 +- .../instance/dao/InstanceLabelDao.java | 5 +- .../instance/entity/InsPersistenceLabel.java | 2 +- .../entity/InsPersistenceLabelValue.java | 2 +- .../instance/entity/InstanceInfo.java | 2 +- .../{scan/app => }/jobhistory/QueryUtils.java | 2 +- .../jobhistory/dao/JobHistoryMapper.java | 5 +- .../jobhistory/entity/JobHistory.java | 5 +- .../exception/AnomalyScannerException.java | 2 +- .../exception/DirtyDataCleanException.java | 2 +- .../app/monitor/entity/ChatbotEntity.java | 97 ------------ .../app/monitor/scheduled/ValidatorClear.java | 51 ------- .../app/monitor => }/scheduled/BmlClear.java | 10 +- .../monitor => }/scheduled/EcRecordClear.java | 8 +- .../EntranceTaskMonitor.java} | 31 ++-- .../scheduled/JobHistoryClear.java | 10 +- .../scheduled/JobHistoryMonitor.java | 83 +++++----- .../scheduled/ResourceMonitor.java | 22 ++- .../monitor => }/scheduled/TaskLogClear.java | 8 +- .../scheduled/UserModeMonitor.java | 16 +- .../app/monitor => }/until/CacheUtils.java | 2 +- .../app/monitor => }/until/HttpsUntils.java | 33 ++-- .../app/monitor => }/until/ThreadUtils.java | 17 +-- .../mapper/common/InsLabelRelationMapper.xml | 10 +- .../mapper/common/InstanceInfoMapper.xml | 4 +- .../mapper/common/InstanceLabelMapper.xml | 4 +- .../mapper/common/JobHistoryMapper.xml | 6 +- ...kisJobHistoryScanSpringConfiguration.scala | 9 +- .../{scan => }/client/MonitorHTTPClient.scala | 6 +- .../client/MonitorHTTPClientClientImpl.scala | 4 +- .../client/MonitorResourceClient.scala | 4 +- .../client/MonitorResourceClientImpl.scala | 4 +- .../{scan => }/constants/Constants.scala | 39 +++-- .../constants/ScanOperatorEnum.scala | 2 +- .../monitor/{scan => }/core/ob/Event.scala | 2 +- .../monitor/{scan => }/core/ob/Observer.scala | 2 +- .../core/ob/SingleObserverEvent.java | 2 +- .../core/pac/AbstractDataFetcher.scala | 2 +- .../core/pac/AbstractScanRule.scala | 6 +- .../{scan => }/core/pac/BaseScannedData.scala | 2 +- .../{scan => }/core/pac/DataFetcher.scala | 2 +- .../{scan => }/core/pac/ScanBuffer.scala | 2 +- .../{scan => }/core/pac/ScanRule.scala | 6 +- .../{scan => }/core/pac/ScannedData.scala | 2 +- .../core/scanner/AbstractScanner.scala | 34 +++-- .../core/scanner/AnomalyScanner.scala | 8 +- .../core/scanner/DefaultScanner.scala | 4 +- .../app => }/factory/MapperFactory.scala | 11 +- .../jobhistory/JobHistoryDataFetcher.scala | 17 ++- .../errorcode/JobHistoryErrCodeHitEvent.scala | 4 +- .../errorcode/JobHistoryErrCodeRule.scala | 11 +- .../JobHistoryErrorCodeAlertSender.scala | 13 +- .../jobtime/JobTimeExceedAlertSender.scala | 15 +- .../jobtime/JobTimeExceedHitEvent.scala | 4 +- .../jobtime/JobTimeExceedRule.scala | 17 +-- .../labels/JobHistoryLabelsAlertSender.scala | 15 +- .../labels/JobHistoryLabelsHitEvent.scala | 4 +- .../labels/JobHistoryLabelsRule.scala | 15 +- .../runtime/CommonJobRunTimeRule.scala | 14 +- .../runtime/CommonRunTimeAlertSender.scala | 14 +- .../runtime/CommonRunTimeHitEvent.scala | 4 +- .../JobHistoryRunTimeAlertSender.scala | 13 +- .../runtime/JobHistoryRunTimeHitEvent.scala | 4 +- .../runtime/JobHistoryRunTimeRule.scala | 17 +-- .../{scan => }/request/EmsListAction.scala | 2 +- .../request/EntranceTaskAction.scala | 2 +- .../request/MonitorResourceAction.scala | 2 +- .../{scan => }/request/UserAction.scala | 2 +- .../response/EntranceTaskResult.scala | 2 +- .../response/MonitorResourceResult.scala | 2 +- .../monitor/{scan => }/utils/ScanUtils.java | 2 +- .../{scan => }/utils/alert/AlertDesc.scala | 2 +- .../{scan => }/utils/alert/AlertSender.scala | 2 +- .../utils/alert/PooledAlertSender.scala | 2 +- .../utils/alert/ims/ImsAlertDesc.scala | 11 +- .../utils/alert/ims/ImsAlertLevel.scala | 2 +- .../alert/ims/ImsAlertPropFileData.scala | 2 +- .../utils/alert/ims/ImsAlertWay.scala | 2 +- .../utils/alert/ims/ImsRequest.scala | 2 +- .../utils/alert/ims/MonitorAlertUtils.scala | 13 +- .../alert/ims/PooledImsAlertSender.scala | 8 +- .../utils/alert/ims/PooledImsAlertUtils.scala | 6 +- .../utils/alert/ims/UserLabelAlertUtils.scala | 11 +- .../{scan => }/utils/log/LogUtils.scala | 2 +- .../utils/alert/PooledImsAlertSenderTest.java | 10 +- .../utils/alert/PooledImsAlertUtilsTest.java | 8 +- 104 files changed, 620 insertions(+), 574 deletions(-) create mode 100644 linkis-dist/package/conf/linkis-et-monitor-ims.properties create mode 100644 linkis-dist/package/conf/linkis-et-monitor.properties rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/LinksMonitorResourceApplication.java (96%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/dao/VersionDao.java (89%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/entity/CleanedResourceVersion.java (98%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/entity/ResourceVersion.java (98%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/service/CleanerService.java (93%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/service/VersionService.java (85%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/service/impl/CleanerServiceImpl.java (91%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/service/impl/VersionServiceImpl.java (81%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/bml/cleaner/vo/CleanResourceVo.java (95%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/config/ListenerConfig.java (84%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/config/MonitorConfig.java (85%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/config/ScheduledConfig.java (96%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/entity/EngineEntity.java (95%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/entity/EntranceEntity.java (96%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/entity/IndexEntity.java (94%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/instance/dao/InsLabelRelationDao.java (85%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/instance/dao/InstanceInfoDao.java (88%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/instance/dao/InstanceLabelDao.java (85%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/instance/entity/InsPersistenceLabel.java (97%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/instance/entity/InsPersistenceLabelValue.java (96%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/instance/entity/InstanceInfo.java (97%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/jobhistory/QueryUtils.java (95%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/jobhistory/dao/JobHistoryMapper.java (92%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/jobhistory/entity/JobHistory.java (96%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/jobhistory/exception/AnomalyScannerException.java (95%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app => }/jobhistory/exception/DirtyDataCleanException.java (95%) delete mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java delete mode 100644 linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/BmlClear.java (83%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/EcRecordClear.java (87%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor/scheduled/TaskMonitor.java => scheduled/EntranceTaskMonitor.java} (89%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/JobHistoryClear.java (84%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/JobHistoryMonitor.java (77%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/ResourceMonitor.java (90%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/TaskLogClear.java (87%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/scheduled/UserModeMonitor.java (94%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/until/CacheUtils.java (95%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/until/HttpsUntils.java (82%) rename linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/{scan/app/monitor => }/until/ThreadUtils.java (80%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/LinkisJobHistoryScanSpringConfiguration.scala (85%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/client/MonitorHTTPClient.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/client/MonitorHTTPClientClientImpl.scala (92%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/client/MonitorResourceClient.scala (96%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/client/MonitorResourceClientImpl.scala (92%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/constants/Constants.scala (70%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/constants/ScanOperatorEnum.scala (94%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/ob/Event.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/ob/Observer.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/ob/SingleObserverEvent.java (96%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/AbstractDataFetcher.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/AbstractScanRule.scala (89%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/BaseScannedData.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/DataFetcher.scala (96%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/ScanBuffer.scala (96%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/ScanRule.scala (88%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/pac/ScannedData.scala (94%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/scanner/AbstractScanner.scala (83%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/scanner/AnomalyScanner.scala (90%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/core/scanner/DefaultScanner.scala (88%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/factory/MapperFactory.scala (85%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/JobHistoryDataFetcher.scala (86%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/errorcode/JobHistoryErrCodeHitEvent.scala (86%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/errorcode/JobHistoryErrCodeRule.scala (89%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/errorcode/JobHistoryErrorCodeAlertSender.scala (89%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/jobtime/JobTimeExceedAlertSender.scala (87%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/jobtime/JobTimeExceedHitEvent.scala (87%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/jobtime/JobTimeExceedRule.scala (85%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/labels/JobHistoryLabelsAlertSender.scala (83%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/labels/JobHistoryLabelsHitEvent.scala (87%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/labels/JobHistoryLabelsRule.scala (91%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/runtime/CommonJobRunTimeRule.scala (87%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/runtime/CommonRunTimeAlertSender.scala (89%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/runtime/CommonRunTimeHitEvent.scala (87%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/runtime/JobHistoryRunTimeAlertSender.scala (86%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/runtime/JobHistoryRunTimeHitEvent.scala (87%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan/app => }/jobhistory/runtime/JobHistoryRunTimeRule.scala (85%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/request/EmsListAction.scala (97%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/request/EntranceTaskAction.scala (98%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/request/MonitorResourceAction.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/request/UserAction.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/response/EntranceTaskResult.scala (96%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/response/MonitorResourceResult.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/ScanUtils.java (97%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/AlertDesc.scala (94%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/AlertSender.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/PooledAlertSender.scala (98%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/ImsAlertDesc.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/ImsAlertLevel.scala (94%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/ImsAlertPropFileData.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/ImsAlertWay.scala (94%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/ImsRequest.scala (96%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/MonitorAlertUtils.scala (94%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/PooledImsAlertSender.scala (93%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/PooledImsAlertUtils.scala (95%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/alert/ims/UserLabelAlertUtils.scala (86%) rename linkis-extensions/linkis-et-monitor/src/main/scala/org/apache/linkis/monitor/{scan => }/utils/log/LogUtils.scala (94%) rename linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/{scan => }/utils/alert/PooledImsAlertSenderTest.java (86%) rename linkis-extensions/linkis-et-monitor/src/test/java/org/apache/linkis/monitor/{scan => }/utils/alert/PooledImsAlertUtilsTest.java (87%) diff --git a/linkis-dist/package/conf/linkis-et-monitor-ims.properties b/linkis-dist/package/conf/linkis-et-monitor-ims.properties new file mode 100644 index 0000000000..22a45841d8 --- /dev/null +++ b/linkis-dist/package/conf/linkis-et-monitor-ims.properties @@ -0,0 +1,143 @@ +# +# 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. +# + +jobhistory.errorcode.01002={"alert_title":"Linkis Service load is too high, please contact Linkis owner","alert_info":"Linkis Linkis Service load is too high, please contact Linkis owner","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.errorcode.01003={"alert_title":"Linkis Service load is too high, please contact Linkis owner","alert_info":"Linkis Service load is too high, please contact Linkis owner","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.errorcode.01101={"alert_title":"Linkis Service load is too high, please contact Linkis owner","alert_info":" Service load is too high, please contact Linkis owner","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.errorcode.01102={"alert_title":"Linkis ECM memory Service load is too high, please contact Linkis owner","alert_info":" Service load is too high, please contact Linkis owner","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.errorcode.01103={"alert_title":"Linkis ECM CPU Service load is too high, please contact Linkis owner","alert_info":" Service load is too high, please contact Linkis owner","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.errorcode.01104={"alert_title":"Linkis ECM instances Service load is too high, please contact Linkis owner","alert_info":" Service load is too high, please contact Linkis owner","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.errorcode.01105={"alert_title":"Linkis Memory Service load is too high, please contact Linkis owner","alert_info":"The machine has insufficient memory. Please contact the administrator to expand the memory.","alert_way":"1,2,3","alert_reciver":"hadoop","alert_level":"2","alert_obj":"linkis-alert-obj","can_recover":"0"} +jobhistory.unfinished.time.exceed.sec.43200={"alert_title":"Linkis task execute timeout","alert_info":"Linkis task execute timeout 12h","alert_way":"1,2,3","alert_reciver":"","alert_level":"4","alert_obj":"Linkis","can_recover":"0"} + +ecm.resource.monitor.im.12003=\ + {"alert_title":"Linkis Monitor Resource Alert",\ + "alert_info":" $instance ECM Memory/cpu insufficient resource < 0.2 please contact Linkis owner: $name",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"3",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +ecm.resource.monitor.im.12004=\ + {"alert_title":"Linkis Monitor Resource Alert",\ + "alert_info":" $instance ECM Memory/cpu insufficient resource < 0.2 please contact Linkis owner: $name",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"2",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +ecm.resource.monitor.im.12005=\ + {"alert_title":"Linkis entrance user running task monitor",\ + "alert_info":"User $username runninng task at linkis ( $url ) > $runningtask ,please attention",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"3",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +ecm.resource.monitor.im.12006=\ + {"alert_title":"Linkis entrance user queued task monitor",\ + "alert_info":"User $username queued task at linkis ( $url ) > $queuedtask ,please attention",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"3",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +ecm.resource.monitor.im.12007=\ + {"alert_title":"Linkis entrance user total task monitor",\ + "alert_info":"User $username queued task at linkis ( $url ) > $tasktotal ,please attention",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"3",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +ecm.resource.monitor.im.12008=\ + {"alert_title":"Linkis entrance all running task monitor",\ + "alert_info":"linkis ( $url ) running task > $taskminor,please attention",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"3",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +ecm.resource.monitor.im.12009=\ + {"alert_title":"Linkis entrance all running task monitor",\ + "alert_info":"linkis ( $url ) all task > $taskmajor,please attention",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"2",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} + + + +user.mode.monitor.im.12011=\ + {"alert_title":"User mode execution timeout alarm",\ + "alert_info":"User mode execution timeout alarm Linkis url: $url engineType:$engineType Task ID: $jobId please attention $name",\ + "alert_way":"1,2,3",\ + "alert_reciver":"alexyang,hadoop",\ + "alert_level":"2",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +user.mode.monitor.im.12012=\ + {"alert_title":"User mode execution failure alarm",\ + "alert_info":"User mode execution failure alarm Linkis url: $url Engine: $engineType TaskID: $jobId ER=rrorCode?$errorCode errorMsg: $errorMsg please attention $name",\ + "alert_way":"1,2,3",\ + "alert_reciver":"alexyang,hadoop",\ + "alert_level":"2",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} + +user.mode.monitor.im.12017=\ + {"alert_title":"@alert_title",\ + "alert_info":"task execute failed, reason $msg",\ + "alert_way":"1,2,3",\ + "alert_reciver":"@alert_reciver",\ + "alert_level":"@alert_level",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +user.mode.monitor.im.12018=\ + {"alert_title":"@alert_title",\ + "alert_info":"task execute time out $timeout",\ + "alert_way":"1,2,3",\ + "alert_reciver":"@alert_reciver",\ + "alert_level":"@alert_level",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} + +thread.monitor.timeout.im.12014=\ + {"alert_title":" Linkis Shell Timeout Alert ",\ + "alert_info":"Monitor Shell execute time out $shellName",\ + "alert_way":"1,2,3",\ + "alert_reciver":"hadoop",\ + "alert_level":"3",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +jobhistory.result.monitor.im.12015=\ + {"alert_title":"Linkis \u4EFB\u52A1\u72B6\u6001\u901A\u77E5",\ + "alert_info":"Task ID:$id final status: $status",\ + "alert_way":"1,2,3",\ + "alert_reciver":"",\ + "alert_level":"4",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} +jobhistory.result.monitor.im.12016=\ + {"alert_title":"@alert_title",\ + "alert_info":"[Alarm time]$date\n[Subsystem]$sysid\n[Alarm IP]$ip\n[Alarm object]$object\n[Alarm information]$detail",\ + "alert_way":"1,2,3",\ + "alert_reciver":"@alert_reciver",\ + "alert_level":"@alert_level",\ + "alert_obj":"linkis_alert",\ + "can_recover":"0"} + + diff --git a/linkis-dist/package/conf/linkis-et-monitor.properties b/linkis-dist/package/conf/linkis-et-monitor.properties new file mode 100644 index 0000000000..92fcc188af --- /dev/null +++ b/linkis-dist/package/conf/linkis-et-monitor.properties @@ -0,0 +1,86 @@ +# +# 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. +# + +wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.monitor.scan.app.instance.entity,org.apache.linkis.monitor.scan.app.jobhistory.entity,org.apache.linkis.bml.cleaner.dao +wds.linkis.server.mybatis.BasePackage=org.apache.linkis.monitor.scan.app.instance.dao,org.apache.linkis.monitor.scan.app.jobhistory.dao,org.apache.linkis.bml.cleaner.dao + + +# alert server url +linkis.alert.url=http://127.0.0.1/ + +# alert receiver +linkis.alert.receiver.default=hadoop + +# monitor ecm resource +linkis.monitor.ecm.resource.cron=0 0/10 * * * ? + +# Resource remaining ratio +linkis.monitor.ecmResourceTask.major=0.005 +linkis.monitor.ecmResourceTask.minor=0.01 + +# entrance task metrics cron +linkis.monitor.entrance.task.cron=0 0/10 * * * ? + +# Timeout task cron +linkis.monitor.jobHistory.timeout.cron=0 0/20 * * * ? + +# time out interval 24h +linkis.monitor.scanner.timeout.interval.seconds=86400 + + +# Finished task cron +linkis.monitor.jobHistory.finished.cron=0 0/20 * * * ? + +# linkis user mode cron +linkis.monitor.user.mode.cron=0 0/20 * * * ? + +# user mode for engine +linkis.monitor.user.enginelist=[{"engineType":"hive-2.3.3","code":"show tables","runType":"hql","executeUser":"hadoop"},\ + {"engineType":"spark-2.4.3","code":"show tables","runType":"sql","executeUser":"hadoop"},\ + {"engineType":"shell-1","code":"pwd","runType":"sh","executeUser":"hadoop"}] + +linkis.monitor.user.authToken=BML-AUTH + +linkis.monitor.user.mode.timeOut=300 + + +# bml clear cron +linkis.monitor.bml.clear.history.version.cron=0 0 12 * * ? +# bml max version +linkis.bml.cleaner.version.max.num=50 +# keep verssion +linkis.bml.cleaner.version.keep.num=20 +# clear max interval +linkis.bml.cleaner.previous.interval.days=30 +# once scan limit +linkis.bml.cleaner.once.limit.num=100 + +# clear db ec record cron +linkis.monitor.clear.ecRecord.cron=0 10 12 * * ? + +# clear task log cron +linkis.monitor.clear.taskLog.cron=0 10 12 * * ? + +# clear db task history cron +linkis.monitor.clear.history.task.cron=0 0 13 * * ? + +# poll size +linkis.monitor.scheduled.pool.cores.num=20 + +# shell time out conf minute +linkis.monitor.shell.time.out.minute=60 + +##Spring +spring.server.port=9119 diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/LinksMonitorResourceApplication.java similarity index 96% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/LinksMonitorResourceApplication.java index 405d94037b..941c722f77 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/LinksMonitorResourceApplication.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/LinksMonitorResourceApplication.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app; +package org.apache.linkis.monitor; import org.apache.linkis.LinkisBaseServerApp; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/dao/VersionDao.java similarity index 89% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/dao/VersionDao.java index 6b6a101b45..5f498500eb 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/dao/VersionDao.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/dao/VersionDao.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.dao; +package org.apache.linkis.monitor.bml.cleaner.dao; -import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; -import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.ResourceVersion; -import org.apache.linkis.monitor.scan.app.bml.cleaner.vo.CleanResourceVo; +import org.apache.linkis.monitor.bml.cleaner.vo.CleanResourceVo; +import org.apache.linkis.monitor.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.entity.ResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.vo.CleanResourceVo; import org.apache.ibatis.annotations.*; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/entity/CleanedResourceVersion.java similarity index 98% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/entity/CleanedResourceVersion.java index 3c6f103183..5a0bfcc487 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/CleanedResourceVersion.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/entity/CleanedResourceVersion.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.entity; +package org.apache.linkis.monitor.bml.cleaner.entity; import java.util.Date; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/entity/ResourceVersion.java similarity index 98% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/entity/ResourceVersion.java index 604ab3662a..5d297cf180 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/entity/ResourceVersion.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/entity/ResourceVersion.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.entity; +package org.apache.linkis.monitor.bml.cleaner.entity; import java.util.Date; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/CleanerService.java similarity index 93% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/CleanerService.java index ccbea7d8c7..0dfa15f396 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/CleanerService.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/CleanerService.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.service; +package org.apache.linkis.monitor.bml.cleaner.service; public interface CleanerService { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/VersionService.java similarity index 85% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/VersionService.java index 3162c096e9..5cc2ee217e 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/VersionService.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/VersionService.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.service; +package org.apache.linkis.monitor.bml.cleaner.service; -import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.entity.CleanedResourceVersion; import org.apache.linkis.common.io.FsPath; import org.apache.linkis.storage.fs.FileSystem; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/impl/CleanerServiceImpl.java similarity index 91% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/impl/CleanerServiceImpl.java index e17d55b7dd..0005cc08fe 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/CleanerServiceImpl.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/impl/CleanerServiceImpl.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.service.impl; - -import org.apache.linkis.monitor.scan.app.bml.cleaner.dao.VersionDao; -import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; -import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.ResourceVersion; -import org.apache.linkis.monitor.scan.app.bml.cleaner.service.CleanerService; -import org.apache.linkis.monitor.scan.app.bml.cleaner.service.VersionService; -import org.apache.linkis.monitor.scan.app.bml.cleaner.vo.CleanResourceVo; +package org.apache.linkis.monitor.bml.cleaner.service.impl; + +import org.apache.linkis.monitor.bml.cleaner.dao.VersionDao; +import org.apache.linkis.monitor.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.entity.ResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.service.CleanerService; +import org.apache.linkis.monitor.bml.cleaner.vo.CleanResourceVo; +import org.apache.linkis.monitor.bml.cleaner.service.VersionService; import org.apache.linkis.common.io.FsPath; -import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.constants.Constants; import org.apache.linkis.storage.FSFactory; import org.apache.linkis.storage.fs.FileSystem; import org.apache.linkis.storage.utils.StorageConfiguration; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/impl/VersionServiceImpl.java similarity index 81% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/impl/VersionServiceImpl.java index 8234d524af..8e78baa31b 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/service/impl/VersionServiceImpl.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/service/impl/VersionServiceImpl.java @@ -15,11 +15,13 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.service.impl; +package org.apache.linkis.monitor.bml.cleaner.service.impl; -import org.apache.linkis.monitor.scan.app.bml.cleaner.dao.VersionDao; -import org.apache.linkis.monitor.scan.app.bml.cleaner.entity.CleanedResourceVersion; -import org.apache.linkis.monitor.scan.app.bml.cleaner.service.VersionService; +import org.apache.linkis.monitor.bml.cleaner.dao.VersionDao; +import org.apache.linkis.monitor.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.dao.VersionDao; +import org.apache.linkis.monitor.bml.cleaner.entity.CleanedResourceVersion; +import org.apache.linkis.monitor.bml.cleaner.service.VersionService; import org.apache.linkis.common.io.FsPath; import org.apache.linkis.storage.fs.FileSystem; @@ -32,7 +34,8 @@ @Service public class VersionServiceImpl implements VersionService { - @Autowired VersionDao versionDao; + @Autowired + VersionDao versionDao; public void setVersionDao(VersionDao versionDao) { this.versionDao = versionDao; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/vo/CleanResourceVo.java similarity index 95% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/vo/CleanResourceVo.java index cfcf6e5ea6..4ef205effd 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/bml/cleaner/vo/CleanResourceVo.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/bml/cleaner/vo/CleanResourceVo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.bml.cleaner.vo; +package org.apache.linkis.monitor.bml.cleaner.vo; public class CleanResourceVo { private String resourceId; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/ListenerConfig.java similarity index 84% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/ListenerConfig.java index fae16b63a3..eb5c11af87 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ListenerConfig.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/ListenerConfig.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.config; +package org.apache.linkis.monitor.config; -import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; -import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.context.annotation.Configuration; import org.springframework.context.event.ContextClosedEvent; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/MonitorConfig.java similarity index 85% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/MonitorConfig.java index 32b47c289c..50516e140c 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/MonitorConfig.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/MonitorConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.config; +package org.apache.linkis.monitor.config; import org.apache.linkis.common.conf.CommonVars; import org.apache.linkis.common.conf.Configuration; @@ -29,7 +29,7 @@ public class MonitorConfig { public static final CommonVars USER_MODE_TIMEOUT = CommonVars.apply("linkis.monitor.user.timeOut", 300); public static final CommonVars USER_MODE_AUTHTOKEN = - CommonVars.apply("linkis.monitor.user.authToken", "VALIDATOR-AUTH"); + CommonVars.apply("linkis.monitor.user.authToken", "BML-AUTH"); public static final CommonVars USER_MODE_ENGINE = CommonVars.apply("linkis.monitor.user.enginelist", "[]"); @@ -63,14 +63,6 @@ public class MonitorConfig { public static final CommonVars USER_MODE_INTERFACE_TIMEOUT = CommonVars.apply("linkis.monitor.user.mode.time.out", 30 * 1000); - public static final CommonVars CHATBOT_KEY_ID = - CommonVars.apply("linkis.monitor.chatbot.key.id", "23e6afad1b78a0c5eed67e4d24de7063"); - public static final CommonVars CHATBOT_TYPE = - CommonVars.apply("linkis.monitor.chatbot.type", "text"); - public static final CommonVars CHATBOT_SERVICE_NAME = - CommonVars.apply("linkis.monitor.chatbot.serviceName", ""); - public static final CommonVars CHATBOT_URL = - CommonVars.apply("linkis.monitor.chatbot.url", ""); public static final CommonVars SOLUTION_URL = CommonVars.apply( "linkis.monitor.jobhistory.solution.url", diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/ScheduledConfig.java similarity index 96% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/ScheduledConfig.java index 30495a87e1..e954122671 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/config/ScheduledConfig.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/config/ScheduledConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.config; +package org.apache.linkis.monitor.config; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/EngineEntity.java similarity index 95% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/EngineEntity.java index 650fa081cc..760c06ba1b 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EngineEntity.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/EngineEntity.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.entity; +package org.apache.linkis.monitor.entity; import java.io.Serializable; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/EntranceEntity.java similarity index 96% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/EntranceEntity.java index 6fa9441474..241439b499 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/EntranceEntity.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/EntranceEntity.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.entity; +package org.apache.linkis.monitor.entity; import java.io.Serializable; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/IndexEntity.java similarity index 94% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/IndexEntity.java index fbea8886e8..2905f8aced 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/IndexEntity.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/entity/IndexEntity.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.entity; +package org.apache.linkis.monitor.entity; -import org.apache.linkis.monitor.scan.constants.Constants; +import org.apache.linkis.monitor.constants.Constants; public class IndexEntity { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InsLabelRelationDao.java similarity index 85% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InsLabelRelationDao.java index 3705288e66..9e573f42d4 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InsLabelRelationDao.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InsLabelRelationDao.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.instance.dao; +package org.apache.linkis.monitor.instance.dao; -import org.apache.linkis.monitor.scan.app.instance.entity.InsPersistenceLabel; +import org.apache.linkis.monitor.instance.entity.InsPersistenceLabel; +import org.apache.linkis.monitor.instance.entity.InsPersistenceLabel; import java.util.List; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InstanceInfoDao.java similarity index 88% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InstanceInfoDao.java index 73a7166a5f..136df3a252 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceInfoDao.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InstanceInfoDao.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.instance.dao; +package org.apache.linkis.monitor.instance.dao; import org.apache.linkis.common.ServiceInstance; -import org.apache.linkis.monitor.scan.app.instance.entity.InstanceInfo; +import org.apache.linkis.monitor.instance.entity.InstanceInfo; +import org.apache.linkis.monitor.instance.entity.InstanceInfo; import org.apache.ibatis.annotations.Param; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InstanceLabelDao.java similarity index 85% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InstanceLabelDao.java index 5fa3b7a5c0..4476444460 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/dao/InstanceLabelDao.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/dao/InstanceLabelDao.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.instance.dao; +package org.apache.linkis.monitor.instance.dao; -import org.apache.linkis.monitor.scan.app.instance.entity.InsPersistenceLabel; +import org.apache.linkis.monitor.instance.entity.InsPersistenceLabel; +import org.apache.linkis.monitor.instance.entity.InsPersistenceLabel; import java.util.List; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InsPersistenceLabel.java similarity index 97% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InsPersistenceLabel.java index 36fb21c13b..0959c27988 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabel.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InsPersistenceLabel.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.instance.entity; +package org.apache.linkis.monitor.instance.entity; import java.util.Date; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InsPersistenceLabelValue.java similarity index 96% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InsPersistenceLabelValue.java index 53ab4bd806..029f552608 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InsPersistenceLabelValue.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InsPersistenceLabelValue.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.instance.entity; +package org.apache.linkis.monitor.instance.entity; public class InsPersistenceLabelValue { diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InstanceInfo.java similarity index 97% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InstanceInfo.java index a74cfaabb0..5360ffd323 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/instance/entity/InstanceInfo.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/instance/entity/InstanceInfo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.instance.entity; +package org.apache.linkis.monitor.instance.entity; import java.util.ArrayList; import java.util.Date; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/QueryUtils.java similarity index 95% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/QueryUtils.java index 799eae89a6..aa73471c49 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/QueryUtils.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/QueryUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.jobhistory; +package org.apache.linkis.monitor.jobhistory; import java.text.DateFormat; import java.text.SimpleDateFormat; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/dao/JobHistoryMapper.java similarity index 92% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/dao/JobHistoryMapper.java index bfdc640652..a3cf6fa823 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/dao/JobHistoryMapper.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/dao/JobHistoryMapper.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.jobhistory.dao; +package org.apache.linkis.monitor.jobhistory.dao; -import org.apache.linkis.monitor.scan.app.jobhistory.entity.JobHistory; +import org.apache.linkis.monitor.jobhistory.entity.JobHistory; +import org.apache.linkis.monitor.jobhistory.entity.JobHistory; import org.apache.ibatis.annotations.Param; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/entity/JobHistory.java similarity index 96% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/entity/JobHistory.java index 937eff410b..ae378ec202 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/entity/JobHistory.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/entity/JobHistory.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.jobhistory.entity; +package org.apache.linkis.monitor.jobhistory.entity; -import org.apache.linkis.monitor.scan.app.jobhistory.QueryUtils; +import org.apache.linkis.monitor.jobhistory.QueryUtils; +import org.apache.linkis.monitor.jobhistory.QueryUtils; import java.util.Date; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/exception/AnomalyScannerException.java similarity index 95% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/exception/AnomalyScannerException.java index 0e3dd10d6b..b2c83be013 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/AnomalyScannerException.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/exception/AnomalyScannerException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.jobhistory.exception; +package org.apache.linkis.monitor.jobhistory.exception; import org.apache.linkis.common.exception.ErrorException; import org.apache.linkis.common.exception.ExceptionLevel; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/exception/DirtyDataCleanException.java similarity index 95% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/exception/DirtyDataCleanException.java index 93b6065f75..b1c5e64743 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/jobhistory/exception/DirtyDataCleanException.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/jobhistory/exception/DirtyDataCleanException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.jobhistory.exception; +package org.apache.linkis.monitor.jobhistory.exception; import org.apache.linkis.common.exception.ErrorException; import org.apache.linkis.common.exception.ExceptionLevel; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java deleted file mode 100644 index 4f912b9e82..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/entity/ChatbotEntity.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.monitor.scan.app.monitor.entity; - -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; - -public class ChatbotEntity { - private String keyid; - private String content; - private String type; - private String userName; - private String serviceName; - - public ChatbotEntity(String content, String userName) { - this.keyid = MonitorConfig.CHATBOT_KEY_ID.getValue(); - this.content = content; - this.type = MonitorConfig.CHATBOT_TYPE.getValue(); - this.userName = userName; - this.serviceName = MonitorConfig.CHATBOT_SERVICE_NAME.getValue(); - } - - public String getKeyid() { - return keyid; - } - - public void setKeyid(String keyid) { - this.keyid = keyid; - } - - public String getContent() { - return content; - } - - public void setContent(String content) { - this.content = content; - } - - public String getType() { - return type; - } - - public void setType(String type) { - this.type = type; - } - - public String getUserName() { - return userName; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public String getServiceName() { - return serviceName; - } - - public void setServiceName(String serviceNameuserName) { - this.serviceName = serviceNameuserName; - } - - @Override - public String toString() { - return "ChatbotEntity{" - + "keyid='" - + keyid - + '\'' - + ", content='" - + content - + '\'' - + ", type='" - + type - + '\'' - + ", userName='" - + userName - + '\'' - + ", serviceName='" - + serviceName - + '\'' - + '}'; - } -} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java deleted file mode 100644 index 4df828e36c..0000000000 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ValidatorClear.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.monitor.scan.app.monitor.scheduled; - -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; - -import org.springframework.context.annotation.PropertySource; -import org.springframework.scheduling.annotation.Scheduled; -import org.springframework.stereotype.Component; - -import java.util.ArrayList; -import java.util.List; - -import org.slf4j.Logger; - -/** * Task: clean up linkis_et_validator_checkinfo data */ -@Component -@PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") -public class ValidatorClear { - - private static final Logger logger = LogUtils.stdOutLogger(); - - @Scheduled(cron = "${linkis.monitor.clear.validator.cron}") - public void ValidatorClear() { - logger.info("Start to clear_validator_record shell"); - List cmdlist = new ArrayList<>(); - cmdlist.add("sh"); - cmdlist.add(MonitorConfig.shellPath + "clear_validator_record.sh"); - logger.info("clear_validator_record shell command {}", cmdlist); - String exec = ThreadUtils.run(cmdlist, "clear_validator_record.sh"); - logger.info("shell log {}", exec); - logger.info("End to clear_validator_record shell "); - } -} diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/BmlClear.java similarity index 83% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/BmlClear.java index 362af45abe..cb7d37197f 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/BmlClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/BmlClear.java @@ -15,18 +15,16 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; -import org.apache.linkis.monitor.scan.app.bml.cleaner.service.CleanerService; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.bml.cleaner.service.CleanerService; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.context.annotation.PropertySource; import org.springframework.scheduling.annotation.Scheduled; import org.springframework.stereotype.Component; -import java.util.*; - import org.slf4j.Logger; @Component @@ -37,7 +35,7 @@ public class BmlClear { @Autowired private CleanerService cleanerServices; - @Scheduled(cron = "${linkis.monitor.bml.clear.historyVersion.cron}") + @Scheduled(cron = "${linkis.monitor.bml.clear.history.version.cron}") public void jobHistoryScanTask() { logger.info("start to clear bml history version"); cleanerServices.run(); diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/EcRecordClear.java similarity index 87% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/EcRecordClear.java index ac9cb4ea61..ddb3d3e8c8 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/EcRecordClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/EcRecordClear.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.context.annotation.PropertySource; import org.springframework.scheduling.annotation.Scheduled; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/EntranceTaskMonitor.java similarity index 89% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/EntranceTaskMonitor.java index 5f6728ef62..d28f6a73f8 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/EntranceTaskMonitor.java @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; import org.apache.linkis.common.ServiceInstance; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.entity.IndexEntity; -import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; -import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.governance.common.conf.GovernanceCommonConf; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.entity.IndexEntity; +import org.apache.linkis.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.constants.Constants; +import org.apache.linkis.monitor.utils.alert.AlertDesc; +import org.apache.linkis.monitor.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.utils.alert.ims.PooledImsAlertUtils; import org.apache.linkis.rpc.Sender; import org.apache.linkis.server.BDPJettyServerHelper; @@ -49,21 +50,21 @@ */ @Component @PropertySource(value = "classpath:linkis-et-monitor.properties", encoding = "UTF-8") -public class TaskMonitor { +public class EntranceTaskMonitor { - private static final Logger logger = LoggerFactory.getLogger(TaskMonitor.class); + private static final Logger logger = LoggerFactory.getLogger(EntranceTaskMonitor.class); private static final String ENTRANCE_RUNNING_TASK = "entrance_running_task"; private static final String ENTRANCE_QUEUED_TASK = "entrance_queued_task"; - @Scheduled(cron = "${linkis.monitor.entranceTask.cron}") + @Scheduled(cron = "${linkis.monitor.entrance.task.cron}") public void entranceTask() throws IOException { List> userlist = new ArrayList<>(); String value = MonitorConfig.ENTRANCE_TASK_USERLIST.getValue(); if (StringUtils.isNotBlank(value)) { userlist = BDPJettyServerHelper.gson().fromJson(value, ArrayList.class); } - // 用户监控 + userlist.forEach( entranceEntity -> { Map data = new HashMap<>(); @@ -110,7 +111,6 @@ public void entranceTask() throws IOException { } int usertotalTask = MonitorConfig.ENTRANCE_TASK_USERTOTAL.getValue(); if (total.intValue() > usertotalTask) { - // 触发告警 用户任务总数满 parms.put("$tasktotal", String.valueOf(usertotalTask)); Map ecmResourceAlerts = MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); @@ -142,21 +142,18 @@ public void entranceTask() throws IOException { PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12009")); } else if (total.intValue() >= linkisTotalMinor) { - // 触发告警Minor parms.put("$taskminor", String.valueOf(linkisTotalMinor)); logger.info("TaskMonitor parms {}:", parms); Map ecmResourceAlerts = MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), parms); PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12008")); } - // 指标上报 resourceSendToIms(); } public static void resourceSendToIms() { - // 获取所有的entrance实例,逐个上送IMS ServiceInstance[] instances = - Sender.getInstances(Constants.DIRTY_DATA_ENTRANCE_APPLICATIONNAME()); + Sender.getInstances(GovernanceCommonConf.ENTRANCE_SERVICE_NAME().getValue()); if (null != instances) { for (ServiceInstance instance : instances) { String serviceInstance = instance.getInstance(); diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/JobHistoryClear.java similarity index 84% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/JobHistoryClear.java index ae14770420..346ca9cb06 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/JobHistoryClear.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.context.annotation.PropertySource; import org.springframework.scheduling.annotation.Scheduled; @@ -37,7 +37,7 @@ public class JobHistoryClear { private static final Logger logger = LogUtils.stdOutLogger(); - @Scheduled(cron = "${linkis.monitor.clear.historyTask.cron}") + @Scheduled(cron = "${linkis.monitor.clear.history.task.cron}") public void historyTaskClear() { logger.info("Start to clear_history_task shell"); List cmdlist = new ArrayList<>(); diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/JobHistoryMonitor.java similarity index 77% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/JobHistoryMonitor.java index fbca945a8d..93dd028c3c 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/JobHistoryMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/JobHistoryMonitor.java @@ -15,31 +15,31 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; +import org.apache.linkis.monitor.jobhistory.JobHistoryDataFetcher; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.until.CacheUtils; import org.apache.linkis.monitor.scan.app.factory.MapperFactory; -import org.apache.linkis.monitor.scan.app.jobhistory.JobHistoryDataFetcher; -import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrCodeRule; -import org.apache.linkis.monitor.scan.app.jobhistory.errorcode.JobHistoryErrorCodeAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.jobtime.JobTimeExceedRule; -import org.apache.linkis.monitor.scan.app.jobhistory.labels.JobHistoryLabelsAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.labels.JobHistoryLabelsRule; -import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonJobRunTimeRule; -import org.apache.linkis.monitor.scan.app.jobhistory.runtime.CommonRunTimeAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.runtime.JobHistoryRunTimeAlertSender; -import org.apache.linkis.monitor.scan.app.jobhistory.runtime.JobHistoryRunTimeRule; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.until.CacheUtils; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.core.pac.DataFetcher; -import org.apache.linkis.monitor.scan.core.scanner.AnomalyScanner; -import org.apache.linkis.monitor.scan.core.scanner.DefaultScanner; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.ImsAlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; -import org.apache.linkis.monitor.scan.utils.alert.ims.UserLabelAlertUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.jobhistory.errorcode.JobHistoryErrCodeRule; +import org.apache.linkis.monitor.jobhistory.errorcode.JobHistoryErrorCodeAlertSender; +import org.apache.linkis.monitor.jobhistory.jobtime.JobTimeExceedAlertSender; +import org.apache.linkis.monitor.jobhistory.jobtime.JobTimeExceedRule; +import org.apache.linkis.monitor.jobhistory.labels.JobHistoryLabelsAlertSender; +import org.apache.linkis.monitor.jobhistory.labels.JobHistoryLabelsRule; +import org.apache.linkis.monitor.jobhistory.runtime.CommonJobRunTimeRule; +import org.apache.linkis.monitor.jobhistory.runtime.CommonRunTimeAlertSender; +import org.apache.linkis.monitor.jobhistory.runtime.JobHistoryRunTimeAlertSender; +import org.apache.linkis.monitor.jobhistory.runtime.JobHistoryRunTimeRule; +import org.apache.linkis.monitor.constants.Constants; +import org.apache.linkis.monitor.core.pac.DataFetcher; +import org.apache.linkis.monitor.core.scanner.AnomalyScanner; +import org.apache.linkis.monitor.core.scanner.DefaultScanner; +import org.apache.linkis.monitor.utils.alert.AlertDesc; +import org.apache.linkis.monitor.utils.alert.ims.ImsAlertDesc; +import org.apache.linkis.monitor.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.utils.alert.ims.UserLabelAlertUtils; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.context.annotation.PropertySource; import org.springframework.scheduling.annotation.Scheduled; @@ -50,14 +50,18 @@ import org.slf4j.Logger; /** - * * jobHistory monitor 1.errorCode: Monitor the error code 2.userLabel: tenant label monitoring, - * scan the execution data within the first 20 minutes, and judge the labels field of the data + * * jobHistory monitor + * 1.errorCode: Monitor the error code + * 2.userLabel: tenant label monitoring,scan the execution data within the first 20 minutes, and judge the labels field of the data + * * 3.jobResultRunTime: Scan the execution data within the first 20 minutes, and judge the completed * tasks. If the parm field in the jobhistory contains (task.notification.conditions) and the result * of executing the task is (Succeed, Failed, Canceled, Timeout, ALL) any one of them, an alarm is * triggered and the result of the job is that it has ended. All three are indispensable + * * 4.jobResultRunTimeForDSS: Scan the execution data within the first 20 minutes, scan the tasks * that have been marked for notification, if the task has ended, a notification will be initiated + * * 5.jobHistoryUnfinishedScan: monitor the status of the execution task, scan the data outside 12 * hours and within 24 hours */ @@ -68,14 +72,13 @@ public class JobHistoryMonitor { private static final Logger logger = LogUtils.stdOutLogger(); private static final long backtrackNum = 1000000L; - /** Scan tasks completed within 20 minutes */ - @Scheduled(cron = "0 0/10 * * * ?") + @Scheduled(cron = "${linkis.monitor.jobHistory.finished.cron}") public void jobHistoryFinishedScan() { - long intervalMs = 20 * 60 * 1000; // 20分钟 + long intervalMs = 20 * 60 * 1000; long maxIntervalMs = Constants.ERRORCODE_MAX_INTERVALS_SECONDS() * 1000; long endTime = System.currentTimeMillis(); long startTime = endTime - intervalMs; - long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + long realIntervals = Math.min(endTime - startTime, maxIntervalMs); AnomalyScanner scanner = new DefaultScanner(); boolean shouldStart = false; long id; @@ -91,7 +94,7 @@ public void jobHistoryFinishedScan() { id = CacheUtils.cacheBuilder.getIfPresent("jobHistoryId"); } List fetchers = generateFetchersfortime(startTime, endTime, id, "updated_time"); - if (fetchers == null) { + if (fetchers.isEmpty()) { logger.warn("generated 0 dataFetchers, plz check input"); return; } @@ -174,22 +177,21 @@ public void jobHistoryFinishedScan() { run(scanner, fetchers, shouldStart); } - /** * Whether scanning data within 12 hours has timed out */ - @Scheduled(cron = "${linkis.monitor.jobHistory.ScanTask.cron}") + @Scheduled(cron = "${linkis.monitor.jobHistory.timeout.cron}") public void jobHistoryUnfinishedScan() { long id = Optional.ofNullable(CacheUtils.cacheBuilder.getIfPresent("jobhistoryScan")) .orElse(MonitorConfig.JOB_HISTORY_TIME_EXCEED.getValue()); - long intervalMs = Constants.ERRORCODE_SCAN_INTERVALS_SECONDS() * 1000; + long intervalMs = Constants.TIMEOUT_INTERVALS_SECONDS() * 1000; long maxIntervalMs = Constants.ERRORCODE_MAX_INTERVALS_SECONDS() * 1000; long endTime = System.currentTimeMillis(); long startTime = endTime - intervalMs; - long realIntervals = endTime - startTime < maxIntervalMs ? endTime - startTime : maxIntervalMs; + long realIntervals = Math.min(endTime - startTime, maxIntervalMs); AnomalyScanner scanner = new DefaultScanner(); boolean shouldStart = false; List fetchers = generateFetchers(startTime, endTime, maxIntervalMs, id, "created_time"); - if (fetchers == null) { + if (fetchers.isEmpty()) { logger.warn("generated 0 dataFetchers, plz check input"); return; } @@ -215,7 +217,6 @@ public static void run(AnomalyScanner scanner, List fetchers, Boole if (shouldStart) { scanner.addDataFetchers(fetchers); scanner.run(); - // scanner.shutdown(); // wait all alert to be send } } @@ -225,15 +226,15 @@ private static List generateFetchers( long pe = endTime; long ps; while (pe > startTime) { - ps = pe - maxIntervalMs > startTime ? pe - maxIntervalMs : startTime; + ps = Math.max(pe - maxIntervalMs, startTime); String[] fetcherArgs = new String[] {String.valueOf(ps), String.valueOf(pe), String.valueOf(id), timeType}; ret.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); logger.info( "Generated dataFetcher for startTime: " - + new Date(ps).toString() + + new Date(ps) + ". EndTime: " - + new Date(pe).toString()); + + new Date(pe)); pe = pe - maxIntervalMs; } return ret; @@ -249,9 +250,9 @@ private static List generateFetchersfortime( fetchers.add(new JobHistoryDataFetcher(fetcherArgs, MapperFactory.getJobHistoryMapper())); logger.info( "Generated dataFetcher for startTime: " - + new Date(startTime).toString() + + new Date(startTime) + ". EndTime: " - + new Date(endTime).toString()); + + new Date(endTime)); return fetchers; } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/ResourceMonitor.java similarity index 90% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/ResourceMonitor.java index 459aaf70ff..0bc8fe7024 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/ResourceMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/ResourceMonitor.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; import org.apache.linkis.common.utils.ByteTimeUtils; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.entity.IndexEntity; -import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; -import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.entity.IndexEntity; +import org.apache.linkis.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.constants.Constants; +import org.apache.linkis.monitor.utils.alert.AlertDesc; +import org.apache.linkis.monitor.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.utils.alert.ims.PooledImsAlertUtils; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -48,7 +48,7 @@ public class ResourceMonitor { private static final Logger logger = LoggerFactory.getLogger(ResourceMonitor.class); - @Scheduled(cron = "${linkis.monitor.ecmResourceTask.cron}") + @Scheduled(cron = "${linkis.monitor.ecm.resource.cron}") public void ecmResourceTask() { Map resultmap = null; AtomicReference tenant = new AtomicReference<>("租户标签:公共资源"); @@ -105,12 +105,10 @@ public void ecmResourceTask() { if (((memorydouble) <= majorValue) || ((coresdouble) <= majorValue) || ((instancedouble) <= majorValue)) { - // major告警 major.add(emNodeVo.get("instance").toString()); } else if (((memorydouble) < minorValue) || ((coresdouble) < minorValue) || ((instancedouble) < minorValue)) { - // minor告警 minor.add(emNodeVo.get("instance").toString()); } HashMap replaceParm = new HashMap<>(); @@ -129,11 +127,9 @@ public void ecmResourceTask() { MonitorAlertUtils.getAlerts(Constants.ALERT_RESOURCE_MONITOR(), replaceParm); PooledImsAlertUtils.addAlert(ecmResourceAlerts.get("12003")); } - // ECM资源占比上报 resourceSendToIms( coresdouble, memorydouble, instancedouble, HttpsUntils.localHost, "USED"); }); - // ECM 剩余资源总数上报 resourceSendToIms( totalCores.get(), totalMemory.get(), totalInstance.get(), HttpsUntils.localHost, "TOTAL"); } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/TaskLogClear.java similarity index 87% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/TaskLogClear.java index ae7e1bc21d..6def756982 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/TaskLogClear.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/TaskLogClear.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.until.ThreadUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.until.ThreadUtils; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.context.annotation.PropertySource; import org.springframework.scheduling.annotation.Scheduled; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/UserModeMonitor.java similarity index 94% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/UserModeMonitor.java index c642a24762..e0a7f9f0af 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/scheduled/UserModeMonitor.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scheduled/UserModeMonitor.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.scheduled; +package org.apache.linkis.monitor.scheduled; import org.apache.linkis.common.utils.Utils; import org.apache.linkis.governance.common.entity.task.RequestPersistTask; import org.apache.linkis.httpclient.dws.config.DWSClientConfig; import org.apache.linkis.manager.label.constant.LabelKeyConstant; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.until.HttpsUntils; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; -import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.until.HttpsUntils; +import org.apache.linkis.monitor.constants.Constants; +import org.apache.linkis.monitor.utils.alert.AlertDesc; +import org.apache.linkis.monitor.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.utils.alert.ims.PooledImsAlertUtils; import org.apache.linkis.server.BDPJettyServerHelper; import org.apache.linkis.ujes.client.UJESClient; import org.apache.linkis.ujes.client.UJESClientImpl; @@ -60,7 +60,7 @@ public class UserModeMonitor { private static final UJESClient client = new UJESClientImpl(clientConfig); - @Scheduled(cron = "${linkis.monitor.user.cron}") + @Scheduled(cron = "${linkis.monitor.user.mode.cron}") public void job() { Optional.ofNullable(MonitorConfig.USER_MODE_ENGINE.getValue()) .ifPresent( diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/CacheUtils.java similarity index 95% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/CacheUtils.java index 258d81183c..a768fde555 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/CacheUtils.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/CacheUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.until; +package org.apache.linkis.monitor.until; import java.util.concurrent.TimeUnit; diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/HttpsUntils.java similarity index 82% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/HttpsUntils.java index 34e12d175d..fd3f584b34 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/HttpsUntils.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/HttpsUntils.java @@ -15,22 +15,21 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.until; +package org.apache.linkis.monitor.until; import org.apache.linkis.common.conf.Configuration; import org.apache.linkis.common.utils.Utils; import org.apache.linkis.httpclient.dws.authentication.TokenAuthenticationStrategy; import org.apache.linkis.httpclient.dws.config.DWSClientConfig; import org.apache.linkis.httpclient.dws.config.DWSClientConfigBuilder; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.app.monitor.entity.ChatbotEntity; -import org.apache.linkis.monitor.scan.app.monitor.entity.IndexEntity; -import org.apache.linkis.monitor.scan.client.MonitorHTTPClient; -import org.apache.linkis.monitor.scan.client.MonitorHTTPClientClientImpl; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.request.EmsListAction; -import org.apache.linkis.monitor.scan.request.EntranceTaskAction; -import org.apache.linkis.monitor.scan.response.EntranceTaskResult; +import org.apache.linkis.monitor.entity.IndexEntity; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.client.MonitorHTTPClient; +import org.apache.linkis.monitor.client.MonitorHTTPClientClientImpl; +import org.apache.linkis.monitor.constants.Constants; +import org.apache.linkis.monitor.request.EmsListAction; +import org.apache.linkis.monitor.request.EntranceTaskAction; +import org.apache.linkis.monitor.response.EntranceTaskResult; import org.apache.linkis.server.BDPJettyServerHelper; import org.apache.linkis.ujes.client.response.EmsListResult; @@ -169,18 +168,4 @@ public static void sendIndex(List list) throws IOException { logger.info("send index response :{}", map); Assert.isTrue(!"0".equals(map.get("resultCode")), map.get("resultMsg")); } - - public static void sendChatbot(ChatbotEntity chatbotEntity) throws IOException { - String json = BDPJettyServerHelper.gson().toJson(chatbotEntity); - StringEntity entity = - new StringEntity( - json, ContentType.create(ContentType.APPLICATION_JSON.getMimeType(), "UTF-8")); - entity.setContentEncoding("UTF-8"); - HttpPost httpPost = new HttpPost(MonitorConfig.CHATBOT_URL.getValue()); - httpPost.setConfig(RequestConfig.DEFAULT); - httpPost.setEntity(entity); - CloseableHttpResponse execute = HttpClients.createDefault().execute(httpPost); - String responseStr = EntityUtils.toString(execute.getEntity(), "UTF-8"); - Map map = BDPJettyServerHelper.gson().fromJson(responseStr, Map.class); - } } diff --git a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/ThreadUtils.java similarity index 80% rename from linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java rename to linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/ThreadUtils.java index 5a099e7b65..15a2626379 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/scan/app/monitor/until/ThreadUtils.java +++ b/linkis-extensions/linkis-et-monitor/src/main/java/org/apache/linkis/monitor/until/ThreadUtils.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.linkis.monitor.scan.app.monitor.until; +package org.apache.linkis.monitor.until; import org.apache.linkis.common.utils.Utils; -import org.apache.linkis.monitor.scan.app.monitor.config.MonitorConfig; -import org.apache.linkis.monitor.scan.constants.Constants; -import org.apache.linkis.monitor.scan.utils.alert.AlertDesc; -import org.apache.linkis.monitor.scan.utils.alert.ims.MonitorAlertUtils; -import org.apache.linkis.monitor.scan.utils.alert.ims.PooledImsAlertUtils; -import org.apache.linkis.monitor.scan.utils.log.LogUtils; +import org.apache.linkis.monitor.config.MonitorConfig; +import org.apache.linkis.monitor.constants.Constants; +import org.apache.linkis.monitor.utils.alert.AlertDesc; +import org.apache.linkis.monitor.utils.alert.ims.MonitorAlertUtils; +import org.apache.linkis.monitor.utils.alert.ims.PooledImsAlertUtils; +import org.apache.linkis.monitor.utils.log.LogUtils; import org.springframework.context.ApplicationContext; import org.springframework.context.event.ApplicationContextEvent; @@ -53,8 +53,7 @@ public static String run(List cmdList, String shellName) { try { msg = future.get(MonitorConfig.SHELL_TIMEOUT.getValue(), TimeUnit.MINUTES).toString(); } catch (TimeoutException e) { - // 增加告警提示 - logger.info("超时告警 {}", shellName); + logger.info("execute shell time out {}", shellName); HashMap parms = new HashMap<>(); parms.put("$shellName", shellName); Map ecmResourceAlerts = diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml index b9c1d19612..6c51f6d0aa 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InsLabelRelationMapper.xml @@ -16,24 +16,24 @@ --> - - + + - + - + - + diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml index c0ab7f2b22..d5309a5f04 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceInfoMapper.xml @@ -16,9 +16,9 @@ --> - + - + diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml index d92d999d47..e7c7558017 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/InstanceLabelMapper.xml @@ -16,9 +16,9 @@ --> - + - + diff --git a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml index 70d1f30158..730e8a43da 100644 --- a/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml +++ b/linkis-extensions/linkis-et-monitor/src/main/resources/mapper/common/JobHistoryMapper.xml @@ -18,9 +18,9 @@ - + - @@ -49,7 +49,7 @@ SELECT a.* FROM linkis_ps_job_history_group_history a - where ( + WHERE ( a.instances = '' - or a.instances is null - or a.instances not in #{key} - or EXISTS ( - select 1 from + OR a.instances IS NULL + OR a.instances NOT IN #{key} + OR EXISTS ( + SELECT 1 FROM ( - select #{key} as instances, #{val} as registryTime + SELECT #{key} AS instances, #{val} AS registryTime ) b - where a.instances = b.instances and a.created_time FROM_UNIXTIME(b.registryTime/1000) + WHERE a.instances = b.instances AND a.created_time FROM_UNIXTIME(b.registryTime/1000) ) ) - and - status in #{status} - and a.created_time >= FROM_UNIXTIME(#{startTimestamp}/1000) - limit #{limit} + AND + status IN #{status} + AND a.created_time >= FROM_UNIXTIME(#{startTimestamp}/1000) + LIMIT #{limit} From 3db8b4afceb0fc10cc713914e5da0a2d1b17892b Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Fri, 27 Oct 2023 17:59:18 +0800 Subject: [PATCH 61/72] Keyword uppercase --- .../mapper/postgresql/JobHistoryMapper.xml | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml index e194a2e4cd..f7e75dea0e 100644 --- a/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml +++ b/linkis-public-enhancements/linkis-jobhistory/src/main/resources/mapper/postgresql/JobHistoryMapper.xml @@ -231,24 +231,24 @@ From eb26d59bd76ab317c52d38b3944e17e6d8882b09 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Mon, 30 Oct 2023 16:03:23 +0800 Subject: [PATCH 62/72] Remove redundant configuration --- linkis-dist/package/conf/linkis-cg-entrance.properties | 1 - linkis-dist/package/conf/linkis-mg-gateway.properties | 1 - 2 files changed, 2 deletions(-) diff --git a/linkis-dist/package/conf/linkis-cg-entrance.properties b/linkis-dist/package/conf/linkis-cg-entrance.properties index 62b1de5d5e..64b023ceca 100644 --- a/linkis-dist/package/conf/linkis-cg-entrance.properties +++ b/linkis-dist/package/conf/linkis-cg-entrance.properties @@ -38,6 +38,5 @@ spring.eureka.instance.metadata-map.linkis.conf.version=v1 ## clean dirty data when the entrance start linkis.entrance.auto.clean.dirty.data.enable=true -wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml wds.linkis.server.mybatis.BasePackage=org.apache.linkis.publicservice.common.lock.dao wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.publicservice.common.lock.entity \ No newline at end of file diff --git a/linkis-dist/package/conf/linkis-mg-gateway.properties b/linkis-dist/package/conf/linkis-mg-gateway.properties index 27656f7f31..1f1d2416b4 100644 --- a/linkis-dist/package/conf/linkis-mg-gateway.properties +++ b/linkis-dist/package/conf/linkis-mg-gateway.properties @@ -20,7 +20,6 @@ wds.linkis.gateway.conf.enable.proxy.user=false wds.linkis.gateway.conf.url.pass.auth=/dss/ wds.linkis.gateway.conf.enable.token.auth=true wds.linkis.is.gateway=true -wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity,org.apache.linkis.jobhistory.entity wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao,org.apache.linkis.jobhistory.dao wds.linkis.label.entity.packages=org.apache.linkis.gateway.ujes.route.label From 43531aa275c2c085b5e2f58b867659d85596b6e9 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Thu, 2 Nov 2023 16:23:33 +0800 Subject: [PATCH 63/72] Remove `linkis.entrance.auto.clean.dirty.data.enable` configuration --- linkis-dist/package/conf/linkis-cg-entrance.properties | 3 --- 1 file changed, 3 deletions(-) diff --git a/linkis-dist/package/conf/linkis-cg-entrance.properties b/linkis-dist/package/conf/linkis-cg-entrance.properties index 64b023ceca..4e2741e13e 100644 --- a/linkis-dist/package/conf/linkis-cg-entrance.properties +++ b/linkis-dist/package/conf/linkis-cg-entrance.properties @@ -35,8 +35,5 @@ wds.linkis.entrance.user.creator.ip.interceptor.switch=false ## you may set service version if you want to distinguish different configuration version spring.eureka.instance.metadata-map.linkis.conf.version=v1 -## clean dirty data when the entrance start -linkis.entrance.auto.clean.dirty.data.enable=true - wds.linkis.server.mybatis.BasePackage=org.apache.linkis.publicservice.common.lock.dao wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.publicservice.common.lock.entity \ No newline at end of file From 8da0f80e35a493d02b10d5b1f66f4f9d5d02adec Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Nov 2023 16:50:38 +0800 Subject: [PATCH 64/72] add config --- .../charts/linkis/templates/configmap-linkis-config.yaml | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/linkis-dist/helm/charts/linkis/templates/configmap-linkis-config.yaml b/linkis-dist/helm/charts/linkis/templates/configmap-linkis-config.yaml index e6f419744c..6437188298 100644 --- a/linkis-dist/helm/charts/linkis/templates/configmap-linkis-config.yaml +++ b/linkis-dist/helm/charts/linkis/templates/configmap-linkis-config.yaml @@ -241,8 +241,8 @@ data: wds.linkis.gateway.conf.enable.token.auth=true wds.linkis.is.gateway=true wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml - wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity - wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao + wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.instance.label.entity,org.apache.linkis.jobhistory.entity + wds.linkis.server.mybatis.BasePackage=org.apache.linkis.instance.label.dao,org.apache.linkis.gateway.authentication.dao,org.apache.linkis.jobhistory.dao wds.linkis.label.entity.packages=org.apache.linkis.gateway.ujes.route.label wds.linkis.login_encrypt.enable=false ##LDAP:q @@ -317,6 +317,11 @@ data: wds.linkis.resultSet.store.path=hdfs://{{ .Values.linkis.locations.runtimeDir }} {{- end }} + ##mybatis + wds.linkis.server.mybatis.mapperLocations=classpath*:mapper/common/*.xml,classpath*:mapper/mysql/*.xml + wds.linkis.server.mybatis.BasePackage=org.apache.linkis.publicservice.common.lock.dao + wds.linkis.server.mybatis.typeAliasesPackage=org.apache.linkis.publicservice.common.lock.entity + ##Spring spring.server.port={{ .Values.cgEntrance.port }} From c97eacbec5b29cdaad65f9f4e1799cf0fd5142bf Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Wed, 8 Nov 2023 21:39:45 +0800 Subject: [PATCH 65/72] bugfix `head of empty list` --- .../linkis/common/utils/CodeAndRunTypeUtils.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala index 3870fe6e58..4f32dd14d0 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala @@ -21,6 +21,8 @@ import org.apache.linkis.common.conf.CommonVars import org.apache.commons.lang3.StringUtils +import scala.collection.mutable + object CodeAndRunTypeUtils { private val CONF_LOCK = new Object() @@ -101,7 +103,14 @@ object CodeAndRunTypeUtils { def getLanguageTypeAndCodeTypeRelationMap: Map[String, String] = { val codeTypeAndRunTypeRelationMap = getCodeTypeAndLanguageTypeRelationMap if (codeTypeAndRunTypeRelationMap.isEmpty) Map() - else codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1))) + else { + // codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1))) + val map = mutable.Map[String, String]() + codeTypeAndRunTypeRelationMap.foreach(kv => { + kv._2.foreach(v => map.put(v, kv._1)) + }) + map.toMap + } } def getLanguageTypeByCodeType(codeType: String, defaultLanguageType: String = ""): String = { From fe9c0b2e07ced62db2de59c84055f9e098999934 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Thu, 9 Nov 2023 17:49:27 +0800 Subject: [PATCH 66/72] rollback bugfix `head of empty list` --- .../linkis/common/utils/CodeAndRunTypeUtils.scala | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala index 4f32dd14d0..3870fe6e58 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala @@ -21,8 +21,6 @@ import org.apache.linkis.common.conf.CommonVars import org.apache.commons.lang3.StringUtils -import scala.collection.mutable - object CodeAndRunTypeUtils { private val CONF_LOCK = new Object() @@ -103,14 +101,7 @@ object CodeAndRunTypeUtils { def getLanguageTypeAndCodeTypeRelationMap: Map[String, String] = { val codeTypeAndRunTypeRelationMap = getCodeTypeAndLanguageTypeRelationMap if (codeTypeAndRunTypeRelationMap.isEmpty) Map() - else { - // codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1))) - val map = mutable.Map[String, String]() - codeTypeAndRunTypeRelationMap.foreach(kv => { - kv._2.foreach(v => map.put(v, kv._1)) - }) - map.toMap - } + else codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1))) } def getLanguageTypeByCodeType(codeType: String, defaultLanguageType: String = ""): String = { From 21df2de8a86339fd46d25e526262e6df1b1f393c Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Sun, 19 Nov 2023 12:33:53 +0800 Subject: [PATCH 67/72] bugfix `head of empty list` again --- .../linkis/common/utils/CodeAndRunTypeUtils.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala index 3870fe6e58..cf40f9a349 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala @@ -18,9 +18,10 @@ package org.apache.linkis.common.utils import org.apache.linkis.common.conf.CommonVars - import org.apache.commons.lang3.StringUtils +import scala.collection.mutable + object CodeAndRunTypeUtils { private val CONF_LOCK = new Object() @@ -101,7 +102,14 @@ object CodeAndRunTypeUtils { def getLanguageTypeAndCodeTypeRelationMap: Map[String, String] = { val codeTypeAndRunTypeRelationMap = getCodeTypeAndLanguageTypeRelationMap if (codeTypeAndRunTypeRelationMap.isEmpty) Map() - else codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1))) + else { +// codeTypeAndRunTypeRelationMap.flatMap(x => x._2.map(y => (y, x._1))) + val map = mutable.Map[String, String]() + codeTypeAndRunTypeRelationMap.foreach(kv => { + kv._2.foreach(v => map.put(v, kv._1)) + }) + map.toMap + } } def getLanguageTypeByCodeType(codeType: String, defaultLanguageType: String = ""): String = { From 5a5e5b37b12b00bf24b12c929c16517d6c75b155 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Sun, 19 Nov 2023 13:38:06 +0800 Subject: [PATCH 68/72] bugfix NPE --- .../entrance/interceptor/impl/UserCreatorIPCheckUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/UserCreatorIPCheckUtils.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/UserCreatorIPCheckUtils.scala index 573c134493..653e9ad78b 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/UserCreatorIPCheckUtils.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/UserCreatorIPCheckUtils.scala @@ -67,7 +67,7 @@ object UserCreatorIPCheckUtils extends Logging { def checkUserIp(jobRequest: JobRequest, logAppender: lang.StringBuilder): JobRequest = { // Get IP address - val jobIp = jobRequest.getSource.get(TaskConstant.REQUEST_IP) + val jobIp = jobRequest.getSource.getOrDefault(TaskConstant.REQUEST_IP, "") logger.debug(s"start to checkTenantLabel $jobIp") if (StringUtils.isNotBlank(jobIp)) { jobRequest match { From 9f4a9c881d17ccf8d125155803eb79e7f1299349 Mon Sep 17 00:00:00 2001 From: guoshupei <15764973965@163.com> Date: Sun, 19 Nov 2023 15:48:22 +0800 Subject: [PATCH 69/72] bugfix `refresh maxAllowRunningJobs` --- .../org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala | 1 + .../entrance/scheduler/EntranceParallelConsumerManager.scala | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala index cf40f9a349..917ac53261 100644 --- a/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala +++ b/linkis-commons/linkis-common/src/main/scala/org/apache/linkis/common/utils/CodeAndRunTypeUtils.scala @@ -18,6 +18,7 @@ package org.apache.linkis.common.utils import org.apache.linkis.common.conf.CommonVars + import org.apache.commons.lang3.StringUtils import scala.collection.mutable diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala index 0f86c2a335..2cdee97cc7 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/EntranceParallelConsumerManager.scala @@ -77,7 +77,6 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S } def refreshAllGroupMaxAllowRunningJobs(validInsCount: Int): Unit = { - if (validInsCount <= 1) return listConsumers() .foreach(item => { item.getGroup match { From b6af5f0c3eac21ba73d4a0bb08c16a6fe6b460eb Mon Sep 17 00:00:00 2001 From: ZHANG HUA JIN Date: Sun, 19 Nov 2023 21:42:23 +0800 Subject: [PATCH 70/72] [WIP]fix issues #4972 #4973 (#4972) --- .../scala/org/apache/linkis/server/ticket/RedisClient.scala | 2 +- .../computation/executor/execute/ComputationExecutor.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/ticket/RedisClient.scala b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/ticket/RedisClient.scala index 8f09139e0e..73699f38ef 100644 --- a/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/ticket/RedisClient.scala +++ b/linkis-commons/linkis-module/src/main/scala/org/apache/linkis/server/ticket/RedisClient.scala @@ -70,7 +70,7 @@ object RedisClient { SessionHAConfiguration.RedisHost, SessionHAConfiguration.RedisPort, redisTimeout, - SessionHAConfiguration.RedisSentinalServer + SessionHAConfiguration.RedisPassword ) } diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/execute/ComputationExecutor.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/execute/ComputationExecutor.scala index 940973be61..ea80b625bd 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/execute/ComputationExecutor.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/execute/ComputationExecutor.scala @@ -271,6 +271,7 @@ abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) TaskResponseErrorEvent(engineConnTask.getTaskId, errorExecuteResponse.message) ) transformTaskStatus(engineConnTask, ExecutionNodeStatus.Failed) + case _ => logger.warn(s"task get response is $executeResponse") } executeResponse } From 1afc61295f093834035e41c9eab9dd415cd2c956 Mon Sep 17 00:00:00 2001 From: weixiao Date: Wed, 22 Nov 2023 00:51:09 -0600 Subject: [PATCH 71/72] Optimize HBase engine dependencies and move java exception class from scala package to java package. (#4976) Co-authored-by: Longping Jie --- .../hbase/hbase-core/pom.xml | 192 ++++++++++++++++++ .../exception/ExecutorInitException.java | 0 .../hbase/hbase-shims-1.2.0/pom.xml | 36 +--- .../hbase/hbase-shims-1.4.3/pom.xml | 44 +--- .../hbase/hbase-shims-2.2.6/pom.xml | 55 ++--- .../hbase/hbase-shims-2.5.3/pom.xml | 40 +--- 6 files changed, 221 insertions(+), 146 deletions(-) rename linkis-engineconn-plugins/hbase/hbase-core/src/main/{scala => java}/org/apache/linkis/manager/engineplugin/hbase/exception/ExecutorInitException.java (100%) diff --git a/linkis-engineconn-plugins/hbase/hbase-core/pom.xml b/linkis-engineconn-plugins/hbase/hbase-core/pom.xml index cabc4cde1e..28e79f0a79 100644 --- a/linkis-engineconn-plugins/hbase/hbase-core/pom.xml +++ b/linkis-engineconn-plugins/hbase/hbase-core/pom.xml @@ -31,6 +31,198 @@ org.apache.linkis hbase-shims-${hbase.version} ${project.version} + + + com.google.inject + guice + + + + commons-codec + commons-codec + + + commons-io + commons-io + + + org.apache.commons + commons-lang3 + + + org.apache.commons + commons-math3 + + + commons-lang + commons-lang + + + commons-cli + commons-cli + + + commons-collections + commons-collections + + + commons-httpclient + commons-httpclient + + + org.apache.commons + commons-math + + + commons-logging + commons-logging + + + commons-lang + commons-lang + + + commons-beanutils + commons-beanutils-core + + + commons-configuration + commons-configuration + + + commons-fileupload + commons-fileupload + + + org.apache.commons + commons-compress + + + org.apache.commons + commons-crypto + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + javax.ws.rs + jsr311-api + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + javax.servlet + javax.servlet-api + + + org.glassfish.web + javax.servlet.jsp + + + javax.servlet.jsp + javax.servlet.jsp-api + + + + org.jruby.jcodings + jcodings + + + org.jruby.joni + joni + + + org.mortbay.jetty + jsp-api-2.1 + + + org.mortbay.jetty + servlet-api-2.5 + + + xml-apis + xml-apis + + + xml-apis + xml-apis-ext + + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-server + + + org.glassfish.jersey.containers + jersey-container-servlet-core + + + + org.mortbay.jetty + jsp-2.1 + + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + com.fasterxml.jackson.module + jackson-module-jaxb-annotations + + + com.fasterxml.jackson.core + jackson-annotations + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-jaxrs + + + + log4j + log4j + + + org.slf4j + slf4j-api + + diff --git a/linkis-engineconn-plugins/hbase/hbase-core/src/main/scala/org/apache/linkis/manager/engineplugin/hbase/exception/ExecutorInitException.java b/linkis-engineconn-plugins/hbase/hbase-core/src/main/java/org/apache/linkis/manager/engineplugin/hbase/exception/ExecutorInitException.java similarity index 100% rename from linkis-engineconn-plugins/hbase/hbase-core/src/main/scala/org/apache/linkis/manager/engineplugin/hbase/exception/ExecutorInitException.java rename to linkis-engineconn-plugins/hbase/hbase-core/src/main/java/org/apache/linkis/manager/engineplugin/hbase/exception/ExecutorInitException.java diff --git a/linkis-engineconn-plugins/hbase/hbase-shims-1.2.0/pom.xml b/linkis-engineconn-plugins/hbase/hbase-shims-1.2.0/pom.xml index e05665786f..46bcdd279f 100644 --- a/linkis-engineconn-plugins/hbase/hbase-shims-1.2.0/pom.xml +++ b/linkis-engineconn-plugins/hbase/hbase-shims-1.2.0/pom.xml @@ -43,22 +43,6 @@ hbase-server ${hbase.version} - - commons-codec - commons-codec - - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - - - org.apache.commons - commons-math3 - org.apache.hadoop hadoop-common @@ -72,29 +56,13 @@ hadoop-mapreduce-client-core - javax.ws.rs - jsr311-api + log4j + log4j org.mortbay.jetty jetty - - org.mortbay.jetty - jetty-util - - - javax.servlet - javax.servlet-api - - - org.glassfish.web - javax.servlet.jsp - - - javax.servlet.jsp - javax.servlet.jsp-api - org.jruby.jcodings jcodings diff --git a/linkis-engineconn-plugins/hbase/hbase-shims-1.4.3/pom.xml b/linkis-engineconn-plugins/hbase/hbase-shims-1.4.3/pom.xml index a687caeecd..218b77d813 100644 --- a/linkis-engineconn-plugins/hbase/hbase-shims-1.4.3/pom.xml +++ b/linkis-engineconn-plugins/hbase/hbase-shims-1.4.3/pom.xml @@ -43,22 +43,6 @@ hbase-server ${hbase.version} - - commons-codec - commons-codec - - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - - - org.apache.commons - commons-math3 - org.apache.hadoop hadoop-common @@ -71,30 +55,6 @@ org.apache.hadoop hadoop-mapreduce-client-core - - javax.ws.rs - jsr311-api - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - javax.servlet - javax.servlet-api - - - org.glassfish.web - javax.servlet.jsp - - - javax.servlet.jsp - javax.servlet.jsp-api - org.jruby.jcodings jcodings @@ -103,6 +63,10 @@ org.jruby.joni joni + + org.mortbay.jetty + jetty + diff --git a/linkis-engineconn-plugins/hbase/hbase-shims-2.2.6/pom.xml b/linkis-engineconn-plugins/hbase/hbase-shims-2.2.6/pom.xml index 8e14b438d6..c2a3cf6a7b 100644 --- a/linkis-engineconn-plugins/hbase/hbase-shims-2.2.6/pom.xml +++ b/linkis-engineconn-plugins/hbase/hbase-shims-2.2.6/pom.xml @@ -43,22 +43,6 @@ hbase-server ${hbase.version} - - commons-codec - commons-codec - - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - - - org.apache.commons - commons-math3 - org.apache.hadoop hadoop-common @@ -72,40 +56,39 @@ hadoop-mapreduce-client-core - javax.ws.rs - jsr311-api + org.apache.zookeeper + zookeeper - org.mortbay.jetty - jetty + org.jruby.jcodings + jcodings - org.mortbay.jetty - jetty-util + org.jruby.joni + joni - javax.servlet - javax.servlet-api + org.glassfish + javax.el org.glassfish.web javax.servlet.jsp - - javax.servlet.jsp - javax.servlet.jsp-api - - - org.jruby.jcodings - jcodings - - - org.jruby.joni - joni - + + org.apache.zookeeper + zookeeper + + + + org.apache.zookeeper + zookeeper-jute + ${zookeeper.version} + + org.apache.hbase hbase-testing-util diff --git a/linkis-engineconn-plugins/hbase/hbase-shims-2.5.3/pom.xml b/linkis-engineconn-plugins/hbase/hbase-shims-2.5.3/pom.xml index e1509b16b5..48c8356084 100644 --- a/linkis-engineconn-plugins/hbase/hbase-shims-2.5.3/pom.xml +++ b/linkis-engineconn-plugins/hbase/hbase-shims-2.5.3/pom.xml @@ -42,22 +42,6 @@ org.apache.hbase hbase-server - - commons-codec - commons-codec - - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - - - org.apache.commons - commons-math3 - org.apache.hadoop hadoop-common @@ -71,33 +55,17 @@ hadoop-mapreduce-client-core - javax.ws.rs - jsr311-api - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util + org.apache.zookeeper + zookeeper - javax.servlet - javax.servlet-api + org.glassfish + javax.el org.glassfish.web javax.servlet.jsp - - javax.servlet.jsp - javax.servlet.jsp-api - - - org.apache.zookeeper - zookeeper - org.jruby.jcodings jcodings From 87661dd01b9fa324b43eaee217c871fecdbdf7e5 Mon Sep 17 00:00:00 2001 From: Yonghao Mei <73584269+mayinrain@users.noreply.github.com> Date: Thu, 23 Nov 2023 22:13:14 +0800 Subject: [PATCH 72/72] update linkis-web for 1.5.0 (#4979) * chore: add new pages to linkis-web and fix bugs of web * upd: update version --- linkis-web/package.json | 2 +- .../licenses/LICENSE-hint.css.txt | 21 + .../apps/linkis/assets/styles/console.scss | 21 + .../apps/linkis/assets/styles/hint.min.css | 21 + .../linkis/components/variable/index.scss | 5 + .../apps/linkis/components/variable/index.vue | 10 +- .../src/apps/linkis/i18n/common/en.json | 121 +++- .../src/apps/linkis/i18n/common/zh.json | 85 ++- .../src/apps/linkis/module/ECM/engineConn.vue | 15 +- .../src/apps/linkis/module/ECM/index.vue | 10 +- linkis-web/src/apps/linkis/module/ECM/log.vue | 67 +- .../module/EnginePluginManagement/index.scss | 18 +- .../module/EnginePluginManagement/index.vue | 66 +- .../linkis/module/acrossClusterRule/index.js | 23 + .../module/acrossClusterRule/index.scss | 59 ++ .../linkis/module/acrossClusterRule/index.vue | 586 ++++++++++++++++++ .../apps/linkis/module/codeQuery/index.scss | 5 + .../apps/linkis/module/codeQuery/index.vue | 34 +- .../linkis/module/configManagement/index.js | 23 + .../linkis/module/configManagement/index.scss | 63 ++ .../linkis/module/configManagement/index.vue | 549 ++++++++++++++++ .../apps/linkis/module/datasource/index.vue | 2 +- .../linkis/module/datasourceTypeKey/index.vue | 2 +- .../gatewayAuthToken/EditForm/index.vue | 1 - .../module/globalHistoryManagement/index.scss | 35 +- .../module/globalHistoryManagement/index.vue | 237 +++---- .../globalHistoryManagement/viewHistory.vue | 57 +- .../src/apps/linkis/module/header/index.vue | 4 +- .../linkis/module/ipListManagement/index.vue | 11 +- .../module/resourceManagement/engineConn.vue | 29 +- .../module/resourceManagement/index.vue | 53 +- .../linkis/module/resourceManagement/log.vue | 52 +- .../module/resourceManagement/search.vue | 12 +- .../EditForm/index.vue | 11 +- .../apps/linkis/module/setting/setting.vue | 60 +- .../module/tenantTagManagement/index.vue | 11 +- .../src/apps/linkis/module/udfTree/index.scss | 1 - .../apps/linkis/module/userConfig/index.js | 23 + .../apps/linkis/module/userConfig/index.scss | 43 ++ .../apps/linkis/module/userConfig/index.vue | 524 ++++++++++++++++ .../src/apps/linkis/view/linkis/index.vue | 71 ++- linkis-web/src/common/i18n/en.json | 2 +- linkis-web/src/common/i18n/zh.json | 2 +- .../components/consoleComponent/result.vue | 7 +- .../consoleComponent/resultSetList.vue | 4 +- .../components/consoleComponent/toolbar.vue | 79 ++- linkis-web/src/components/table/table.css | 19 +- linkis-web/src/components/table/table.vue | 76 ++- .../historyTable/historyTable.vue | 7 +- linkis-web/src/dss/module/footer/index.vue | 2 +- linkis-web/src/dss/module/header/index.vue | 4 +- .../src/dss/module/resourceSimple/engine.vue | 245 ++++---- .../src/dss/module/resourceSimple/index.scss | 199 +++--- .../src/dss/module/resourceSimple/job.vue | 2 +- .../src/dss/module/resourceSimple/queue.vue | 21 +- 55 files changed, 3122 insertions(+), 590 deletions(-) create mode 100644 linkis-web/release-docs/licenses/LICENSE-hint.css.txt create mode 100644 linkis-web/src/apps/linkis/assets/styles/hint.min.css create mode 100644 linkis-web/src/apps/linkis/module/acrossClusterRule/index.js create mode 100644 linkis-web/src/apps/linkis/module/acrossClusterRule/index.scss create mode 100644 linkis-web/src/apps/linkis/module/acrossClusterRule/index.vue create mode 100644 linkis-web/src/apps/linkis/module/configManagement/index.js create mode 100644 linkis-web/src/apps/linkis/module/configManagement/index.scss create mode 100644 linkis-web/src/apps/linkis/module/configManagement/index.vue create mode 100644 linkis-web/src/apps/linkis/module/userConfig/index.js create mode 100644 linkis-web/src/apps/linkis/module/userConfig/index.scss create mode 100644 linkis-web/src/apps/linkis/module/userConfig/index.vue diff --git a/linkis-web/package.json b/linkis-web/package.json index b04dbcd537..b43a05d183 100644 --- a/linkis-web/package.json +++ b/linkis-web/package.json @@ -1,6 +1,6 @@ { "name": "linkis", - "version": "1.4.0", + "version": "1.5.0", "private": true, "scripts": { "serve": "vue-cli-service serve", diff --git a/linkis-web/release-docs/licenses/LICENSE-hint.css.txt b/linkis-web/release-docs/licenses/LICENSE-hint.css.txt new file mode 100644 index 0000000000..9961a4fb36 --- /dev/null +++ b/linkis-web/release-docs/licenses/LICENSE-hint.css.txt @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2021 Kushagra Gour + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/linkis-web/src/apps/linkis/assets/styles/console.scss b/linkis-web/src/apps/linkis/assets/styles/console.scss index 45a7a4a6d7..008a4b7686 100644 --- a/linkis-web/src/apps/linkis/assets/styles/console.scss +++ b/linkis-web/src/apps/linkis/assets/styles/console.scss @@ -17,8 +17,29 @@ @charset "UTF-8"; @import '@/common/style/variables.scss'; + @import './hint.min.css'; +// * { +// font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", "PingFang SC", "Hiragino Sans GB", "Microsoft YaHei", "Helvetica Neue", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "JinbiaoSong", "JinbiaoSongExt"; +// } +body { + font-family: "Helvetica Neue",Helvetica,"PingFang SC","Hiragino Sans GB","Microsoft YaHei","微软雅黑",Arial,sans-serif, "JinbiaoSong", "JinbiaoSongExt"; +} .console-page{ + .ivu-input { + font-family: Arial, -apple-system, BlinkMacSystemFont, "Segoe UI", "PingFang SC", "Hiragino Sans GB", "Microsoft YaHei", "Helvetica Neue", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "JinbiaoSong", "JinbiaoSongExt"; + } + font-family: -apple-system, BlinkMacSystemFont, "Segoe UI", "PingFang SC", "Hiragino Sans GB", "Microsoft YaHei", "Helvetica Neue", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "JinbiaoSong", "JinbiaoSongExt" !important; + .monaco-editor { + --monaco-monospace-font: "SF Mono", Monaco, Menlo, Consolas, "Ubuntu Mono", "Liberation Mono", "DejaVu Sans Mono", "Courier New", monospace, "JinbiaoSong", "JinbiaoSongExt"; + } + .monaco-mouse-cursor-text { + font-family: Consolas, "Courier New", monospace, "JinbiaoSong", "JinbiaoSongExt" !important; + } + .ivu-tooltip-inner { + overflow-wrap: break-word; + white-space: normal; + } position: $relative; width: $percent-all; height: $percent-all; diff --git a/linkis-web/src/apps/linkis/assets/styles/hint.min.css b/linkis-web/src/apps/linkis/assets/styles/hint.min.css new file mode 100644 index 0000000000..0c0d22d322 --- /dev/null +++ b/linkis-web/src/apps/linkis/assets/styles/hint.min.css @@ -0,0 +1,21 @@ +/* + * 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. + */ +/*! Hint.css - v2.7.0 - 2021-10-01 +* https://kushagra.dev/lab/hint/ +* Copyright (c) 2021 Kushagra Gour */ + +[class*=hint--]{position:relative ; width: 80;}[class*=hint--]:after,[class*=hint--]:before{position:absolute;-webkit-transform:translate3d(0,0,0);-moz-transform:translate3d(0,0,0);transform:translate3d(0,0,0);visibility:hidden;opacity:0;z-index:1000000;pointer-events:none;-webkit-transition:.3s ease;-moz-transition:.3s ease;transition:.3s ease;-webkit-transition-delay:0s;-moz-transition-delay:0s;transition-delay:0s}[class*=hint--]:hover:after,[class*=hint--]:hover:before{visibility:visible;opacity:1;-webkit-transition-delay:.1s;-moz-transition-delay:.1s;transition-delay:.1s}[class*=hint--]:before{content:'';position:absolute;background:0 0;border:6px solid transparent;z-index:1000001}[class*=hint--]:after{background:#383838;color:#fff;padding:8px 10px;font-size:12px;font-family:-apple-system, BlinkMacSystemFont, "Segoe UI", "PingFang SC", "Hiragino Sans GB", "Microsoft YaHei", "Helvetica Neue", Helvetica, Arial, sans-serif, "Apple Color Emoji", "Segoe UI Emoji", "Segoe UI Symbol", "JinbiaoSong", "JinbiaoSongExt";line-height:12px;white-space:pre-line;word-break: normal;text-align:left;text-shadow:0 -1px 0 #000;box-shadow:4px 4px 8px rgba(0,0,0,.3)}[class*=hint--][aria-label]:after{content:attr(aria-label)}[class*=hint--][data-hint]:after{content:attr(data-hint)}[aria-label='']:after,[aria-label='']:before,[data-hint='']:after,[data-hint='']:before{display:none!important}.hint--top-left:before,.hint--top-right:before,.hint--top:before{border-top-color:#383838}.hint--bottom-left:before,.hint--bottom-right:before,.hint--bottom:before{border-bottom-color:#383838}.hint--top:after,.hint--top:before{bottom:100%;left:50%}.hint--top:before{margin-bottom:-11px;left:calc(50% - 6px)}.hint--top:after{-webkit-transform:translateX(-50%);-moz-transform:translateX(-50%);transform:translateX(-50%)}.hint--top:hover:before{-webkit-transform:translateY(-8px);-moz-transform:translateY(-8px);transform:translateY(-8px)}.hint--top:hover:after{-webkit-transform:translateX(-50%) translateY(-8px);-moz-transform:translateX(-50%) translateY(-8px);transform:translateX(-50%) translateY(-8px)}.hint--bottom:after,.hint--bottom:before{top:100%;left:50%}.hint--bottom:before{margin-top:-11px;left:calc(50% - 6px)}.hint--bottom:after{-webkit-transform:translateX(-50%);-moz-transform:translateX(-50%);transform:translateX(-50%)}.hint--bottom:hover:before{-webkit-transform:translateY(8px);-moz-transform:translateY(8px);transform:translateY(8px)}.hint--bottom:hover:after{-webkit-transform:translateX(-50%) translateY(8px);-moz-transform:translateX(-50%) translateY(8px);transform:translateX(-50%) translateY(8px)}.hint--right:before{border-right-color:#383838;margin-left:-11px;margin-bottom:-6px}.hint--right:after{margin-bottom:-14px}.hint--right:after,.hint--right:before{left:100%;bottom:50%}.hint--right:hover:after,.hint--right:hover:before{-webkit-transform:translateX(8px);-moz-transform:translateX(8px);transform:translateX(8px)}.hint--left:before{border-left-color:#383838;margin-right:-11px;margin-bottom:-6px}.hint--left:after{margin-bottom:-14px}.hint--left:after,.hint--left:before{right:100%;bottom:50%}.hint--left:hover:after,.hint--left:hover:before{-webkit-transform:translateX(-8px);-moz-transform:translateX(-8px);transform:translateX(-8px)}.hint--top-left:after,.hint--top-left:before{bottom:100%;left:50%}.hint--top-left:before{margin-bottom:-11px;left:calc(50% - 6px)}.hint--top-left:after{-webkit-transform:translateX(-100%);-moz-transform:translateX(-100%);transform:translateX(-100%);margin-left:12px}.hint--top-left:hover:before{-webkit-transform:translateY(-8px);-moz-transform:translateY(-8px);transform:translateY(-8px)}.hint--top-left:hover:after{-webkit-transform:translateX(-100%) translateY(-8px);-moz-transform:translateX(-100%) translateY(-8px);transform:translateX(-100%) translateY(-8px)}.hint--top-right:after,.hint--top-right:before{bottom:100%;left:50%}.hint--top-right:before{margin-bottom:-11px;left:calc(50% - 6px)}.hint--top-right:after{-webkit-transform:translateX(0);-moz-transform:translateX(0);transform:translateX(0);margin-left:-12px}.hint--top-right:hover:after,.hint--top-right:hover:before{-webkit-transform:translateY(-8px);-moz-transform:translateY(-8px);transform:translateY(-8px)}.hint--bottom-left:after,.hint--bottom-left:before{top:100%;left:50%}.hint--bottom-left:before{margin-top:-11px;left:calc(50% - 6px)}.hint--bottom-left:after{-webkit-transform:translateX(-100%);-moz-transform:translateX(-100%);transform:translateX(-100%);margin-left:12px}.hint--bottom-left:hover:before{-webkit-transform:translateY(8px);-moz-transform:translateY(8px);transform:translateY(8px)}.hint--bottom-left:hover:after{-webkit-transform:translateX(-100%) translateY(8px);-moz-transform:translateX(-100%) translateY(8px);transform:translateX(-100%) translateY(8px)}.hint--bottom-right:after,.hint--bottom-right:before{top:100%;left:50%}.hint--bottom-right:before{margin-top:-11px;left:calc(50% - 6px)}.hint--bottom-right:after{-webkit-transform:translateX(0);-moz-transform:translateX(0);transform:translateX(0);margin-left:-12px}.hint--bottom-right:hover:after,.hint--bottom-right:hover:before{-webkit-transform:translateY(8px);-moz-transform:translateY(8px);transform:translateY(8px)}.hint--large:after,.hint--medium:after,.hint--small:after{white-space:normal;line-height:1.4em;word-wrap:break-word}.hint--small:after{width:80px}.hint--medium:after{width:150px}.hint--large:after{width:300px}.hint--error:after{background-color:#b34e4d;text-shadow:0 -1px 0 #592726}.hint--error.hint--top-left:before,.hint--error.hint--top-right:before,.hint--error.hint--top:before{border-top-color:#b34e4d}.hint--error.hint--bottom-left:before,.hint--error.hint--bottom-right:before,.hint--error.hint--bottom:before{border-bottom-color:#b34e4d}.hint--error.hint--left:before{border-left-color:#b34e4d}.hint--error.hint--right:before{border-right-color:#b34e4d}.hint--warning:after{background-color:#c09854;text-shadow:0 -1px 0 #6c5328}.hint--warning.hint--top-left:before,.hint--warning.hint--top-right:before,.hint--warning.hint--top:before{border-top-color:#c09854}.hint--warning.hint--bottom-left:before,.hint--warning.hint--bottom-right:before,.hint--warning.hint--bottom:before{border-bottom-color:#c09854}.hint--warning.hint--left:before{border-left-color:#c09854}.hint--warning.hint--right:before{border-right-color:#c09854}.hint--info:after{background-color:#3986ac;text-shadow:0 -1px 0 #1a3c4d}.hint--info.hint--top-left:before,.hint--info.hint--top-right:before,.hint--info.hint--top:before{border-top-color:#3986ac}.hint--info.hint--bottom-left:before,.hint--info.hint--bottom-right:before,.hint--info.hint--bottom:before{border-bottom-color:#3986ac}.hint--info.hint--left:before{border-left-color:#3986ac}.hint--info.hint--right:before{border-right-color:#3986ac}.hint--success:after{background-color:#458746;text-shadow:0 -1px 0 #1a321a}.hint--success.hint--top-left:before,.hint--success.hint--top-right:before,.hint--success.hint--top:before{border-top-color:#458746}.hint--success.hint--bottom-left:before,.hint--success.hint--bottom-right:before,.hint--success.hint--bottom:before{border-bottom-color:#458746}.hint--success.hint--left:before{border-left-color:#458746}.hint--success.hint--right:before{border-right-color:#458746}.hint--always:after,.hint--always:before{opacity:1;visibility:visible}.hint--always.hint--top:before{-webkit-transform:translateY(-8px);-moz-transform:translateY(-8px);transform:translateY(-8px)}.hint--always.hint--top:after{-webkit-transform:translateX(-50%) translateY(-8px);-moz-transform:translateX(-50%) translateY(-8px);transform:translateX(-50%) translateY(-8px)}.hint--always.hint--top-left:before{-webkit-transform:translateY(-8px);-moz-transform:translateY(-8px);transform:translateY(-8px)}.hint--always.hint--top-left:after{-webkit-transform:translateX(-100%) translateY(-8px);-moz-transform:translateX(-100%) translateY(-8px);transform:translateX(-100%) translateY(-8px)}.hint--always.hint--top-right:after,.hint--always.hint--top-right:before{-webkit-transform:translateY(-8px);-moz-transform:translateY(-8px);transform:translateY(-8px)}.hint--always.hint--bottom:before{-webkit-transform:translateY(8px);-moz-transform:translateY(8px);transform:translateY(8px)}.hint--always.hint--bottom:after{-webkit-transform:translateX(-50%) translateY(8px);-moz-transform:translateX(-50%) translateY(8px);transform:translateX(-50%) translateY(8px)}.hint--always.hint--bottom-left:before{-webkit-transform:translateY(8px);-moz-transform:translateY(8px);transform:translateY(8px)}.hint--always.hint--bottom-left:after{-webkit-transform:translateX(-100%) translateY(8px);-moz-transform:translateX(-100%) translateY(8px);transform:translateX(-100%) translateY(8px)}.hint--always.hint--bottom-right:after,.hint--always.hint--bottom-right:before{-webkit-transform:translateY(8px);-moz-transform:translateY(8px);transform:translateY(8px)}.hint--always.hint--left:after,.hint--always.hint--left:before{-webkit-transform:translateX(-8px);-moz-transform:translateX(-8px);transform:translateX(-8px)}.hint--always.hint--right:after,.hint--always.hint--right:before{-webkit-transform:translateX(8px);-moz-transform:translateX(8px);transform:translateX(8px)}.hint--rounded:after{border-radius:4px}.hint--no-animate:after,.hint--no-animate:before{-webkit-transition-duration:0s;-moz-transition-duration:0s;transition-duration:0s}.hint--bounce:after,.hint--bounce:before{-webkit-transition:opacity .3s ease,visibility .3s ease,-webkit-transform .3s cubic-bezier(.71,1.7,.77,1.24);-moz-transition:opacity .3s ease,visibility .3s ease,-moz-transform .3s cubic-bezier(.71,1.7,.77,1.24);transition:opacity .3s ease,visibility .3s ease,transform .3s cubic-bezier(.71,1.7,.77,1.24)}.hint--no-shadow:after,.hint--no-shadow:before{text-shadow:initial;box-shadow:initial}.hint--no-arrow:before{display:none} diff --git a/linkis-web/src/apps/linkis/components/variable/index.scss b/linkis-web/src/apps/linkis/components/variable/index.scss index 9140e16a35..fd603d45fb 100644 --- a/linkis-web/src/apps/linkis/components/variable/index.scss +++ b/linkis-web/src/apps/linkis/components/variable/index.scss @@ -89,6 +89,11 @@ border: 1px solid red; } } + .two-lines { + height: 60px; + resize: none; + overflow-y: scroll; + } .we-variable-content-input[disabled] { background-color: #f3f3f3; opacity: 1; diff --git a/linkis-web/src/apps/linkis/components/variable/index.vue b/linkis-web/src/apps/linkis/components/variable/index.vue index 662a3fef27..0d0ba9aedf 100644 --- a/linkis-web/src/apps/linkis/components/variable/index.vue +++ b/linkis-web/src/apps/linkis/components/variable/index.vue @@ -36,7 +36,7 @@