diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java index 692ce619b2..07bc0510bc 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageUtils.java @@ -272,4 +272,8 @@ public static byte[] mergeByteArrays(byte[] arr1, byte[] arr2) { System.arraycopy(arr2, 0, mergedArray, arr1.length, arr2.length); return mergedArray; } + + public static boolean isHDFSPath(FsPath fsPath) { + return HDFS.equalsIgnoreCase(fsPath.getFsType()); + } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/CacheOutputExecuteResponse.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/constant/CodeConstants.java similarity index 75% rename from linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/CacheOutputExecuteResponse.scala rename to linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/constant/CodeConstants.java index 47a6ce9e9e..4c914bc3f4 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/CacheOutputExecuteResponse.scala +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/java/org/apache/linkis/governance/common/constant/CodeConstants.java @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.linkis.entrance.scheduler.cache +package org.apache.linkis.governance.common.constant; -import org.apache.linkis.scheduler.executer.OutputExecuteResponse - -case class CacheOutputExecuteResponse(alias: String, output: String) extends OutputExecuteResponse { - override def getOutput: String = output +public class CodeConstants { + // will auto append at end of scala code; make sure the last line is not a comment + public static String SCALA_CODE_AUTO_APPEND_CODE = "val linkisVar=123"; } diff --git a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala index 64ece62fd7..d5669ad428 100644 --- a/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala +++ b/linkis-computation-governance/linkis-computation-governance-common/src/main/scala/org/apache/linkis/governance/common/paser/CodeParser.scala @@ -19,6 +19,7 @@ package org.apache.linkis.governance.common.paser import org.apache.linkis.common.utils.{CodeAndRunTypeUtils, Logging, Utils} import org.apache.linkis.governance.common.conf.GovernanceCommonConf +import org.apache.linkis.governance.common.constant.CodeConstants import org.apache.linkis.governance.common.paser.CodeType.CodeType import org.apache.commons.lang3.StringUtils @@ -116,7 +117,7 @@ class ScalaCodeParser extends SingleCodeParser with Logging { if (statementBuffer.nonEmpty) codeBuffer.append(statementBuffer.mkString("\n")) // Make sure the last line is not a comment codeBuffer.append("\n") - codeBuffer.append("val linkisVar=123") + codeBuffer.append(CodeConstants.SCALA_CODE_AUTO_APPEND_CODE) codeBuffer.toArray } diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/bml/BmlEnginePreExecuteHook.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/bml/BmlEnginePreExecuteHook.scala index 3959eb942b..cb04f364fb 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/bml/BmlEnginePreExecuteHook.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/bml/BmlEnginePreExecuteHook.scala @@ -62,7 +62,6 @@ class BmlEnginePreExecuteHook extends ComputationExecutorHook with Logging { ): String = { val props = engineExecutionContext.getProperties if (null != props && props.containsKey(GovernanceConstant.TASK_RESOURCES_STR)) { - val workDir = ComputationEngineUtils.getCurrentWorkDir val jobId = engineExecutionContext.getJobId props.get(GovernanceConstant.TASK_RESOURCES_STR) match { case resources: util.List[Object] => @@ -71,9 +70,7 @@ class BmlEnginePreExecuteHook extends ComputationExecutorHook with Logging { val fileName = resource.get(GovernanceConstant.TASK_RESOURCE_FILE_NAME_STR).toString val resourceId = resource.get(GovernanceConstant.TASK_RESOURCE_ID_STR).toString val version = resource.get(GovernanceConstant.TASK_RESOURCE_VERSION_STR).toString - val fullPath = - if (workDir.endsWith(seperator)) pathType + workDir + fileName - else pathType + workDir + seperator + fileName + val fullPath = fileName val response = Utils.tryCatch { bmlClient.downloadShareResource(processUser, resourceId, version, fullPath, true) } { diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/conf/ComputationExecutorConf.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/conf/ComputationExecutorConf.scala index c072c32794..bcd423fd21 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/conf/ComputationExecutorConf.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/conf/ComputationExecutorConf.scala @@ -33,6 +33,12 @@ object ComputationExecutorConf { "Maximum number of tasks executed by the synchronization EC" ) + val PRINT_TASK_PARAMS_SKIP_KEYS = CommonVars( + "linkis.engineconn.print.task.params.skip.keys", + "jobId", + "skip to print params key at job logs" + ) + val ENGINE_PROGRESS_FETCH_INTERVAL = CommonVars( "wds.linkis.engineconn.progresss.fetch.interval-in-seconds", diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/cs/CSResourceParser.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/cs/CSResourceParser.scala index f59adaadef..fe98e3328e 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/cs/CSResourceParser.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/cs/CSResourceParser.scala @@ -17,7 +17,10 @@ package org.apache.linkis.engineconn.computation.executor.cs +import org.apache.linkis.common.utils.Logging import org.apache.linkis.cs.client.service.CSResourceService +import org.apache.linkis.engineconn.common.conf.EngineConnConf +import org.apache.linkis.governance.common.utils.GovernanceConstant import org.apache.commons.lang3.StringUtils @@ -27,7 +30,7 @@ import java.util.regex.Pattern import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -class CSResourceParser { +class CSResourceParser extends Logging { private val pb = Pattern.compile("cs://[^\\s\"]+[$\\s]{0,1}", Pattern.CASE_INSENSITIVE) @@ -47,7 +50,6 @@ class CSResourceParser { nodeNameStr: String ): String = { - // TODO getBMLResource val bmlResourceList = CSResourceService.getInstance().getUpstreamBMLResource(contextIDValueStr, nodeNameStr) @@ -56,23 +58,25 @@ class CSResourceParser { val preFixNames = new ArrayBuffer[String]() val parsedNames = new ArrayBuffer[String]() + val prefixName = System.currentTimeMillis().toString + "_" preFixResourceNames.foreach { preFixResourceName => val resourceName = preFixResourceName.replace(PREFIX, "").trim val bmlResourceOption = bmlResourceList.asScala.find(_.getDownloadedFileName.equals(resourceName)) if (bmlResourceOption.isDefined) { + val replacementName = EngineConnConf.getEngineTmpDir + prefixName + resourceName val bmlResource = bmlResourceOption.get val map = new util.HashMap[String, Object]() - map.put("resourceId", bmlResource.getResourceId) - map.put("version", bmlResource.getVersion) - map.put("fileName", resourceName) + map.put(GovernanceConstant.TASK_RESOURCE_ID_STR, bmlResource.getResourceId) + map.put(GovernanceConstant.TASK_RESOURCE_VERSION_STR, bmlResource.getVersion) + map.put(GovernanceConstant.TASK_RESOURCE_FILE_NAME_STR, replacementName) parsedResources.add(map) preFixNames.append(preFixResourceName) - parsedNames.append(resourceName) + parsedNames.append(replacementName) + logger.warn(s"Replace cs file from {$preFixResourceName} to {$replacementName}") } - } - props.put("resources", parsedResources) + props.put(GovernanceConstant.TASK_RESOURCES_STR, parsedResources) StringUtils.replaceEach(code, preFixNames.toArray, parsedNames.toArray) } 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 ea80b625bd..bd6d44e4a6 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 @@ -22,6 +22,7 @@ import org.apache.linkis.common.log.LogUtils import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.engineconn.acessible.executor.entity.AccessibleExecutor import org.apache.linkis.engineconn.acessible.executor.listener.event.{ + TaskLogUpdateEvent, TaskResponseErrorEvent, TaskStatusChangedEvent } @@ -40,7 +41,14 @@ import org.apache.linkis.governance.common.paser.CodeParser import org.apache.linkis.governance.common.protocol.task.{EngineConcurrentInfo, RequestTask} import org.apache.linkis.governance.common.utils.{JobUtils, LoggerUtils} import org.apache.linkis.manager.common.entity.enumeration.NodeStatus -import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel +import org.apache.linkis.manager.label.entity.engine.{ + CodeLanguageLabel, + EngineType, + EngineTypeLabel, + RunType, + UserCreatorLabel +} +import org.apache.linkis.manager.label.utils.LabelUtil import org.apache.linkis.protocol.engine.JobProgressInfo import org.apache.linkis.scheduler.executer._ @@ -50,6 +58,8 @@ import org.apache.commons.lang3.exception.ExceptionUtils import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConverters._ + import com.google.common.cache.{Cache, CacheBuilder} abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) @@ -132,6 +142,12 @@ abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) override def close(): Unit = { if (null != lastTask) CLOSE_LOCKER.synchronized { + listenerBusContext.getEngineConnSyncListenerBus.postToAll( + TaskLogUpdateEvent( + lastTask.getTaskId, + LogUtils.generateERROR("EC exits unexpectedly and actively kills the task") + ) + ) killTask(lastTask.getTaskId) } else { @@ -169,9 +185,11 @@ abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) Utils.tryFinally { transformTaskStatus(engineConnTask, ExecutionNodeStatus.Running) val engineExecutionContext = createEngineExecutionContext(engineConnTask) + + val engineCreationContext = EngineConnObject.getEngineCreationContext + var hookedCode = engineConnTask.getCode Utils.tryCatch { - val engineCreationContext = EngineConnObject.getEngineCreationContext ComputationExecutorHook.getComputationExecutorHooks.foreach(hook => { hookedCode = hook.beforeExecutorExecute(engineExecutionContext, engineCreationContext, hookedCode) @@ -182,12 +200,24 @@ abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) } else { logger.info(s"hooked after code: $hookedCode ") } + + // task params log + // spark engine: at org.apache.linkis.engineplugin.spark.executor.SparkEngineConnExecutor.executeLine log special conf + Utils.tryAndWarn { + val engineType = LabelUtil.getEngineType(engineCreationContext.getLabels()) + EngineType.mapStringToEngineType(engineType) match { + case EngineType.HIVE | EngineType.TRINO => printTaskParamsLog(engineExecutionContext) + case _ => + } + } + val localPath = EngineConnConf.getLogDir engineExecutionContext.appendStdout( LogUtils.generateInfo( s"EngineConn local log path: ${DataWorkCloudApplication.getServiceInstance.toString} $localPath" ) ) + var response: ExecuteResponse = null val incomplete = new StringBuilder val codes = @@ -244,6 +274,11 @@ abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) case s: SuccessExecuteResponse => succeedTasks.increase() s + case incompleteExecuteResponse: IncompleteExecuteResponse => + ErrorExecuteResponse( + s"The task cannot be an incomplete response ${incompleteExecuteResponse.message}", + null + ) case _ => response } response @@ -336,6 +371,30 @@ abstract class ComputationExecutor(val outputPrintLimit: Int = 1000) } } + /** + * job task log print task params info + * + * @param engineExecutorContext + * @return + * Unit + */ + + def printTaskParamsLog(engineExecutorContext: EngineExecutionContext): Unit = { + val sb = new StringBuilder + + EngineConnObject.getEngineCreationContext.getOptions.asScala.foreach({ case (key, value) => + // skip log jobId because it corresponding jobid when the ec created + if (!ComputationExecutorConf.PRINT_TASK_PARAMS_SKIP_KEYS.getValue.contains(key)) { + sb.append(s"${key}=${value.toString}\n") + } + }) + + sb.append("\n") + engineExecutorContext.appendStdout( + LogUtils.generateInfo(s"Your job exec with configs:\n${sb.toString()}\n") + ) + } + def transformTaskStatus(task: EngineConnTask, newStatus: ExecutionNodeStatus): Unit = { val oriStatus = task.getStatus logger.info(s"task ${task.getTaskId} from status $oriStatus to new status $newStatus") diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/metrics/ComputationEngineConnMetrics.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/metrics/ComputationEngineConnMetrics.scala index f96896f557..4446bdc672 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/metrics/ComputationEngineConnMetrics.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/metrics/ComputationEngineConnMetrics.scala @@ -91,4 +91,12 @@ object ComputationEngineConnMetrics { getTotalBusyTimeMills(nodeStatus) + getTotalIdleTimeMills(nodeStatus) def getUnlockToShutdownDurationMills(): Long = unlockToShutdownDurationMills.get() + + def getLastUnlockTimestamp(nodeStatus: NodeStatus): Long = { + nodeStatus match { + case NodeStatus.Unlock => lastUnlockTimeMills + case _ => 0 + } + } + } diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/DefaultNodeHeartbeatMsgManager.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/DefaultNodeHeartbeatMsgManager.scala index 010ced97fd..e5d74282de 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/DefaultNodeHeartbeatMsgManager.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/service/DefaultNodeHeartbeatMsgManager.scala @@ -66,6 +66,10 @@ class DefaultNodeHeartbeatMsgManager extends NodeHeartbeatMsgManager with Loggin ECConstants.EC_TOTAL_LOCK_TIME_MILLS_KEY, ComputationEngineConnMetrics.getTotalLockTimeMills(status).asInstanceOf[Object] ) + msgMap.put( + ECConstants.EC_LAST_UNLOCK_TIMESTAMP, + ComputationEngineConnMetrics.getLastUnlockTimestamp(status).asInstanceOf[Object] + ) case _ => } val engineParams = EngineConnObject.getEngineCreationContext.getOptions 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 ca1887e746..23d6ff2586 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 @@ -422,31 +422,51 @@ class TaskExecutionServiceImpl ): Future[_] = { val sleepInterval = ComputationExecutorConf.ENGINE_PROGRESS_FETCH_INTERVAL.getValue scheduler.submit(new Runnable { - override def run(): Unit = Utils.tryAndWarn { + override def run(): Unit = { Utils.tryQuietly(Thread.sleep(TimeUnit.MILLISECONDS.convert(1, TimeUnit.SECONDS))) while (null != taskFuture && !taskFuture.isDone) { - if ( - ExecutionNodeStatus.isCompleted(task.getStatus) || ExecutionNodeStatus - .isRunning(task.getStatus) - ) { - val progressResponse = taskProgress(task.getTaskId) - val resourceResponse = buildResourceMap(task) - val extraInfoMap = buildExtraInfoMap(task) - // todo add other info - val resourceMap = - if (null != resourceResponse) resourceResponse.getResourceMap else null - - val respRunningInfo: ResponseTaskRunningInfo = new ResponseTaskRunningInfo( - progressResponse.execId, - progressResponse.progress, - progressResponse.progressInfo, - resourceMap, - extraInfoMap - ) - - sendToEntrance(task, respRunningInfo) - Thread.sleep(TimeUnit.MILLISECONDS.convert(sleepInterval, TimeUnit.SECONDS)) + if (!ExecutionNodeStatus.isCompleted(task.getStatus)) { + Utils.tryAndWarn { + val progressResponse = Utils.tryCatch(taskProgress(task.getTaskId)) { + case e: Exception => + logger.info("Failed to get progress", e) + null + } + val resourceResponse = Utils.tryCatch(buildResourceMap(task)) { case e: Exception => + logger.info("Failed to get resource", e) + null + } + val extraInfoMap = Utils.tryCatch(buildExtraInfoMap(task)) { case e: Exception => + logger.info("Failed to get extra info ", e) + null + } + val resourceMap = + if (null != resourceResponse) resourceResponse.getResourceMap else null + + /** + * It is guaranteed that there must be progress the progress must be greater than or + * equal to 0.1 + */ + val newProgressResponse = if (null == progressResponse) { + ResponseTaskProgress(task.getTaskId, 0.1f, null) + } else if (progressResponse.progress < 0.1f) { + ResponseTaskProgress(task.getTaskId, 0.1f, progressResponse.progressInfo) + } else { + progressResponse + } + val respRunningInfo: ResponseTaskRunningInfo = new ResponseTaskRunningInfo( + newProgressResponse.execId, + newProgressResponse.progress, + newProgressResponse.progressInfo, + resourceMap, + extraInfoMap + ) + sendToEntrance(task, respRunningInfo) + } } + Utils.tryQuietly( + Thread.sleep(TimeUnit.MILLISECONDS.convert(sleepInterval, TimeUnit.SECONDS)) + ) } } }) @@ -499,7 +519,7 @@ class TaskExecutionServiceImpl } override def taskProgress(taskID: String): ResponseTaskProgress = { - var response = ResponseTaskProgress(taskID, 0, null) + var response = ResponseTaskProgress(taskID, 0.01f, null) if (StringUtils.isBlank(taskID)) return response val executor = taskIdCache.getIfPresent(taskID) if (null != executor) { @@ -514,11 +534,9 @@ class TaskExecutionServiceImpl ResponseTaskProgress(taskID, progress, executor.getProgressInfo(taskID)) ) } - } else { - response = ResponseTaskProgress(taskID, -1, null) } } else { - logger.error(s"Executor of taskId : $taskID is not cached.") + logger.info(s"Executor of taskId : $taskID is not cached.") } response } @@ -606,7 +624,7 @@ class TaskExecutionServiceImpl logger.warn("Unknown event : " + BDPJettyServerHelper.gson.toJson(event)) } - override def onLogUpdate(logUpdateEvent: TaskLogUpdateEvent): Unit = { + override def onLogUpdate(logUpdateEvent: TaskLogUpdateEvent): Unit = Utils.tryAndWarn { if (EngineConnConf.ENGINE_PUSH_LOG_TO_ENTRANCE.getValue) { if (null != logUpdateEvent && StringUtils.isNotBlank(logUpdateEvent.taskId)) { val task = getTaskByTaskId(logUpdateEvent.taskId) @@ -663,32 +681,33 @@ class TaskExecutionServiceImpl } } - override def onProgressUpdate(taskProgressUpdateEvent: TaskProgressUpdateEvent): Unit = { - if (EngineConnConf.ENGINE_PUSH_LOG_TO_ENTRANCE.getValue) { - val task = getTaskByTaskId(taskProgressUpdateEvent.taskId) - if (null != task) { - val resourceResponse = buildResourceMap(task) - val extraInfoMap = buildExtraInfoMap(task) + override def onProgressUpdate(taskProgressUpdateEvent: TaskProgressUpdateEvent): Unit = + Utils.tryAndWarn { + if (EngineConnConf.ENGINE_PUSH_LOG_TO_ENTRANCE.getValue) { + val task = getTaskByTaskId(taskProgressUpdateEvent.taskId) + if (null != task) { + val resourceResponse = buildResourceMap(task) + val extraInfoMap = buildExtraInfoMap(task) - val resourceMap = if (null != resourceResponse) resourceResponse.getResourceMap else null + val resourceMap = if (null != resourceResponse) resourceResponse.getResourceMap else null - val respRunningInfo: ResponseTaskRunningInfo = new ResponseTaskRunningInfo( - taskProgressUpdateEvent.taskId, - taskProgressUpdateEvent.progress, - taskProgressUpdateEvent.progressInfo, - resourceMap, - extraInfoMap - ) + val respRunningInfo: ResponseTaskRunningInfo = new ResponseTaskRunningInfo( + taskProgressUpdateEvent.taskId, + taskProgressUpdateEvent.progress, + taskProgressUpdateEvent.progressInfo, + resourceMap, + extraInfoMap + ) - sendToEntrance(task, respRunningInfo) - } else { - logger.error( - "Task cannot null! taskProgressUpdateEvent : " + ComputationEngineUtils.GSON - .toJson(taskProgressUpdateEvent) - ) + sendToEntrance(task, respRunningInfo) + } else { + logger.error( + "Task cannot null! taskProgressUpdateEvent : " + ComputationEngineUtils.GSON + .toJson(taskProgressUpdateEvent) + ) + } } } - } override def onResultSetCreated(taskResultCreateEvent: TaskResultCreateEvent): Unit = { logger.info(s"start to deal result event ${taskResultCreateEvent.taskId}") diff --git a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/upstream/handler/ECTaskKillHandler.scala b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/upstream/handler/ECTaskKillHandler.scala index d06e8ac077..15e70315e3 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/upstream/handler/ECTaskKillHandler.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-computation-engineconn/src/main/scala/org/apache/linkis/engineconn/computation/executor/upstream/handler/ECTaskKillHandler.scala @@ -39,10 +39,10 @@ class ECTaskKillHandler extends MonitorHandler with Logging { while (elements.hasNext) { val element = elements.next Utils.tryCatch { - doKill(element) logger.error( s"ERROR: entrance : ${element.getUpstreamConnection().getUpstreamServiceInstanceName()} lose connect, will kill job : ${element.getKey()}" ) + doKill(element) } { t => logger.error("Failed to kill job: " + element.getKey, t) } diff --git a/linkis-computation-governance/linkis-engineconn/linkis-engineconn-common/src/main/scala/org/apache/linkis/engineconn/common/conf/EngineConnConf.scala b/linkis-computation-governance/linkis-engineconn/linkis-engineconn-common/src/main/scala/org/apache/linkis/engineconn/common/conf/EngineConnConf.scala index eefc69f5bb..43f7ab4446 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-engineconn-common/src/main/scala/org/apache/linkis/engineconn/common/conf/EngineConnConf.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-engineconn-common/src/main/scala/org/apache/linkis/engineconn/common/conf/EngineConnConf.scala @@ -61,6 +61,8 @@ object EngineConnConf { val ENGINE_CONN_LOCAL_LOG_DIRS_KEY = CommonVars("wds.linkis.engine.logs.dir.key", "LOG_DIRS") + val ENGINE_CONN_LOCAL_TMP_DIR = CommonVars("wds.linkis.engine.tmp.dir", "TEMP_DIRS") + val ENGINE_CONN_CREATION_WAIT_TIME = CommonVars("wds.linkis.engine.connector.init.time", new TimeType("8m")) @@ -84,6 +86,8 @@ object EngineConnConf { def getWorkHome: String = System.getenv(ENGINE_CONN_LOCAL_PATH_PWD_KEY.getValue) + def getEngineTmpDir: String = System.getenv(ENGINE_CONN_LOCAL_TMP_DIR.getValue) + def getLogDir: String = { val logDir = System.getenv(ENGINE_CONN_LOCAL_LOG_DIRS_KEY.getValue) if (StringUtils.isNotEmpty(logDir)) logDir else new File(getWorkHome, "logs").getPath diff --git a/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/conf/AccessibleExecutorConfiguration.scala b/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/conf/AccessibleExecutorConfiguration.scala index 95a01202e8..40cf314853 100644 --- a/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/conf/AccessibleExecutorConfiguration.scala +++ b/linkis-computation-governance/linkis-engineconn/linkis-engineconn-executor/accessible-executor/src/main/scala/org/apache/linkis/engineconn/acessible/executor/conf/AccessibleExecutorConfiguration.scala @@ -37,7 +37,7 @@ object AccessibleExecutorConfiguration { val ENGINECONN_LOG_SEND_SIZE = CommonVars[Int]("wds.linkis.engineconn.log.send.cache.size", 300) val ENGINECONN_MAX_FREE_TIME = - CommonVars("wds.linkis.engineconn.max.free.time", new TimeType("30m")) + CommonVars("wds.linkis.engineconn.max.free.time", new TimeType("10m")) val ENGINECONN_LOCK_CHECK_INTERVAL = CommonVars("wds.linkis.engineconn.lock.free.interval", new TimeType("3m")) 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 cf520c3823..86b1a91f7a 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 @@ -150,7 +150,7 @@ public EntranceInterceptor[] entranceInterceptors() { new ScalaCodeInterceptor(), new SQLLimitEntranceInterceptor(), new CommentInterceptor(), - new SetTenantLabelInterceptor(), + // new SetTenantLabelInterceptor(), new UserCreatorIPCheckInterceptor() }; } 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 b5f90e3070..51a522d3d2 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 @@ -62,6 +62,11 @@ public enum EntranceErrorCodeSummary implements LinkisErrorCode { INVALID_RESULTSETS(20053, "Invalid resultsets, cannot use cache(结果集无效,无法使用 cache)"), SUBMITTING_QUERY_FAILED(30009, "Submitting the query failed(提交查询失败)!"), + + SUBMIT_CODE_ISEMPTY( + 30010, + "Submitting the execution code, after code preprocessing, the real execution code is empty, please check the executed code(提交的执行代码,经过预处理后为空,请检查执行的代码是否为空或则只有注解)!"), + QUERY_STATUS_FAILED(50081, "Query from jobHistory status failed(从 jobHistory 状态查询失败)"), GET_QUERY_RESPONSE(50081, "Get query response incorrectly(获取查询响应结果不正确)"), QUERY_TASKID_ERROR(50081, "Query task of taskId:{0} error(查询任务id:{}的任务出错)"), diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceConsumerRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceConsumerRestfulApi.java new file mode 100644 index 0000000000..424e7ca170 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceConsumerRestfulApi.java @@ -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.entrance.restful; + +import org.apache.linkis.common.conf.Configuration; +import org.apache.linkis.entrance.EntranceServer; +import org.apache.linkis.scheduler.queue.ConsumerManager; +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.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RequestParam; +import org.springframework.web.bind.annotation.RestController; + +import javax.servlet.http.HttpServletRequest; + +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Api(tags = "entrance lable manager") +@RestController +@RequestMapping(path = "/entrance/operation/consumer") +public class EntranceConsumerRestfulApi { + + private EntranceServer entranceServer; + + private static final Logger logger = LoggerFactory.getLogger(EntranceConsumerRestfulApi.class); + + @Autowired + public void setEntranceServer(EntranceServer entranceServer) { + this.entranceServer = entranceServer; + } + + @ApiOperation(value = "kill-consumer", notes = "kill consumer", response = Message.class) + @RequestMapping(path = "/kill", method = RequestMethod.GET) + public Message killConsumer( + HttpServletRequest req, @RequestParam(value = "groupName") String groupName) { + String operationUser = ModuleUserUtils.getOperationUser(req, "kill consumer"); + if (Configuration.isNotAdmin(operationUser)) { + return Message.error("only admin can do this"); + } + logger.info("user {} to kill consumer {}", operationUser, groupName); + ConsumerManager consumerManager = + entranceServer + .getEntranceContext() + .getOrCreateScheduler() + .getSchedulerContext() + .getOrCreateConsumerManager(); + consumerManager.destroyConsumer(groupName); + logger.info("user {} finished to kill consumer {}", operationUser, groupName); + return Message.ok(); + } + + @ApiOperation(value = "consumer-info", notes = "list consumers info", response = Message.class) + @RequestMapping(path = "/info", method = RequestMethod.GET) + public Message countConsumer(HttpServletRequest req) { + String operationUser = ModuleUserUtils.getOperationUser(req, "kill consumer"); + if (Configuration.isNotAdmin(operationUser)) { + return Message.error("only admin can do this"); + } + ConsumerManager consumerManager = + entranceServer + .getEntranceContext() + .getOrCreateScheduler() + .getSchedulerContext() + .getOrCreateConsumerManager(); + return Message.ok().data("consumerNum", consumerManager.listConsumers().length); + } +} 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 841a6a3fb0..0737e25ed8 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 @@ -24,6 +24,7 @@ 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.protocol.label.InsLabelRemoveRequest; import org.apache.linkis.rpc.Sender; import org.apache.linkis.scheduler.SchedulerContext; import org.apache.linkis.server.Message; @@ -57,6 +58,8 @@ public void setEntranceServer(EntranceServer entranceServer) { this.entranceServer = entranceServer; } + private static Boolean offlineFlag = false; + @ApiOperation(value = "update", notes = "update route label", response = Message.class) @ApiOperationSupport(ignoreParameters = {"jsonNode"}) @RequestMapping(path = "/update", method = RequestMethod.POST) @@ -82,12 +85,15 @@ public Message updateRouteLabel(HttpServletRequest req, @RequestBody JsonNode js public Message updateRouteLabel(HttpServletRequest req) { ModuleUserUtils.getOperationUser(req, "markoffline"); Map labels = new HashMap(); - logger.info("Prepare to modify the routelabel of entry to offline"); + logger.info("Prepare to modify the routelabel of entrance to offline"); labels.put(LabelKeyConstant.ROUTE_KEY, LabelValueConstant.OFFLINE_VALUE); InsLabelRefreshRequest insLabelRefreshRequest = new InsLabelRefreshRequest(); insLabelRefreshRequest.setLabels(labels); insLabelRefreshRequest.setServiceInstance(Sender.getThisServiceInstance()); InstanceLabelClient.getInstance().refreshLabelsToInstance(insLabelRefreshRequest); + synchronized (offlineFlag) { + offlineFlag = true; + } logger.info("Finished to modify the routelabel of entry to offline"); logger.info("Prepare to update all not execution task instances to empty string"); @@ -101,4 +107,29 @@ public Message updateRouteLabel(HttpServletRequest req) { return Message.ok(); } + + @ApiOperation( + value = "backonline", + notes = "from offline status to recover", + response = Message.class) + @RequestMapping(path = "/backonline", method = RequestMethod.GET) + public Message backOnline(HttpServletRequest req) { + ModuleUserUtils.getOperationUser(req, "backonline"); + logger.info("Prepare to modify the routelabel of entrance to remove offline"); + InsLabelRemoveRequest insLabelRemoveRequest = new InsLabelRemoveRequest(); + insLabelRemoveRequest.setServiceInstance(Sender.getThisServiceInstance()); + InstanceLabelClient.getInstance().removeLabelsFromInstance(insLabelRemoveRequest); + synchronized (offlineFlag) { + offlineFlag = false; + } + logger.info("Finished to backonline"); + return Message.ok(); + } + + @ApiOperation(value = "isOnline", notes = "entrance isOnline", response = Message.class) + @RequestMapping(path = "/isOnline", method = RequestMethod.GET) + public Message isOnline(HttpServletRequest req) { + logger.info("Whether Entrance is online: {}", !offlineFlag); + return Message.ok().data("isOnline", !offlineFlag); + } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceMetricRestfulApi.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceMetricRestfulApi.java index 7d36df8fec..7b487352d5 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceMetricRestfulApi.java +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/restful/EntranceMetricRestfulApi.java @@ -20,8 +20,7 @@ import org.apache.linkis.common.conf.Configuration; import org.apache.linkis.entrance.EntranceServer; import org.apache.linkis.entrance.execute.EntranceJob; -import org.apache.linkis.manager.label.entity.engine.EngineTypeLabel; -import org.apache.linkis.manager.label.utils.LabelUtil; +import org.apache.linkis.entrance.scheduler.CreatorECTypeDefaultConf; import org.apache.linkis.server.Message; import org.apache.linkis.server.utils.ModuleUserUtils; @@ -67,7 +66,7 @@ public Message taskinfo( HttpServletRequest req, @RequestParam(value = "user", required = false) String user, @RequestParam(value = "creator", required = false) String creator, - @RequestParam(value = "engineTypeLabel", required = false) String engineTypeLabelValue) { + @RequestParam(value = "ecType", required = false) String ecType) { String userName = ModuleUserUtils.getOperationUser(req, "taskinfo"); String queryUser = user; if (Configuration.isNotAdmin(userName)) { @@ -83,23 +82,12 @@ public Message taskinfo( } else if (StringUtils.isBlank(creator)) { filterWords = queryUser; } - EntranceJob[] undoneTasks = entranceServer.getAllUndoneTask(filterWords); - int taskNumber = 0; + EntranceJob[] undoneTasks = entranceServer.getAllUndoneTask(filterWords, ecType); int runningNumber = 0; int queuedNumber = 0; if (null != undoneTasks) { for (EntranceJob task : undoneTasks) { - if (StringUtils.isNotBlank(engineTypeLabelValue)) { - EngineTypeLabel engineTypeLabel = - LabelUtil.getEngineTypeLabel(task.getJobRequest().getLabels()); - // Task types do not match, do not count - if (null == engineTypeLabel - || !engineTypeLabelValue.equalsIgnoreCase(engineTypeLabel.getStringValue())) { - continue; - } - } - taskNumber++; if (task.isRunning()) { runningNumber++; } else { @@ -107,17 +95,25 @@ public Message taskinfo( } } } - return Message.ok("success") - .data("taskNumber", taskNumber) - .data("runningNumber", runningNumber) - .data("queuedNumber", queuedNumber); + Message resp = + Message.ok("success") + .data("taskNumber", undoneTasks.length) + .data("runningNumber", runningNumber) + .data("queuedNumber", queuedNumber); + if (StringUtils.isNoneBlank(creator, ecType)) { + int creatorECTypeMaxRunningJobs = + CreatorECTypeDefaultConf.getCreatorECTypeMaxRunningJobs(creator, ecType); + resp.data("creatorECTypeMaxRunningJobs", creatorECTypeMaxRunningJobs); + resp.data("limitExceeded", runningNumber > creatorECTypeMaxRunningJobs); + } + return resp; } - @ApiOperation(value = "Status", notes = "get running task number ", response = Message.class) + @ApiOperation(value = "runningtask", notes = "get running task number ", response = Message.class) @RequestMapping(path = "/runningtask", method = RequestMethod.GET) - public Message status(HttpServletRequest req) { + public Message runningtask(HttpServletRequest req) { ModuleUserUtils.getOperationUser(req, "runningtask"); - EntranceJob[] undoneTasks = entranceServer.getAllUndoneTask(""); + EntranceJob[] undoneTasks = entranceServer.getAllUndoneTask("", null); Boolean isCompleted = false; if (null == undoneTasks || undoneTasks.length < 1) { isCompleted = true; 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 cf9cd33653..dfab300ab4 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 @@ -114,15 +114,6 @@ public Message execute(HttpServletRequest req, @RequestBody Map JobRequest jobReq = ((EntranceJob) job).getJobRequest(); Long jobReqId = jobReq.getId(); ModuleUserUtils.getOperationUser(req, "execute task,id: " + jobReqId); - pushLog( - LogUtils.generateInfo( - "You have submitted a new job, script code (after variable substitution) is"), - job); - pushLog( - "************************************SCRIPT CODE************************************", job); - pushLog(jobReq.getExecutionCode(), job); - pushLog( - "************************************SCRIPT CODE************************************", job); String execID = ZuulEntranceUtils.generateExecID( job.getId(), @@ -166,15 +157,6 @@ public Message submit(HttpServletRequest req, @RequestBody Map j JobRequest jobRequest = ((EntranceJob) job).getJobRequest(); Long jobReqId = jobRequest.getId(); ModuleUserUtils.getOperationUser(req, "submit jobReqId: " + jobReqId); - pushLog( - LogUtils.generateInfo( - "You have submitted a new job, script code (after variable substitution) is"), - job); - pushLog( - "************************************SCRIPT CODE************************************", job); - pushLog(jobRequest.getExecutionCode(), job); - pushLog( - "************************************SCRIPT CODE************************************", job); pushLog( LogUtils.generateInfo( "Your job is accepted, jobID is " @@ -809,9 +791,9 @@ public Message killJobs( logger.error("kill job {} failed ", job.get().getId(), t); message = Message.error( - "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败)"); + "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败)", + t); message.setMethod("/api/entrance/" + id + "/kill"); - message.setStatus(1); } } messages.add(message); @@ -937,10 +919,11 @@ public Message kill( 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: " - + t.getMessage()); + "An exception occurred while killing the job, kill failed(kill job的时候出现了异常,kill失败) with error:" + + t.getMessage(), + t); message.setMethod("/api/entrance/" + id + "/kill"); + message.setStatus(1); } } return message; diff --git a/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/scheduler/CreatorECTypeDefaultConf.java b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/scheduler/CreatorECTypeDefaultConf.java new file mode 100644 index 0000000000..5a91c71a11 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/java/org/apache/linkis/entrance/scheduler/CreatorECTypeDefaultConf.java @@ -0,0 +1,93 @@ +/* + * 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.conf.Configuration; +import org.apache.linkis.entrance.conf.EntranceConfiguration; +import org.apache.linkis.entrance.utils.EntranceUtils; +import org.apache.linkis.governance.common.protocol.conf.RequestQueryEngineConfig; +import org.apache.linkis.governance.common.protocol.conf.ResponseQueryConfig; +import org.apache.linkis.manager.label.entity.engine.EngineTypeLabel; +import org.apache.linkis.manager.label.entity.engine.UserCreatorLabel; +import org.apache.linkis.rpc.Sender; + +import org.apache.commons.lang3.StringUtils; + +import java.util.concurrent.TimeUnit; + +import scala.Tuple2; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CreatorECTypeDefaultConf { + + private static final Logger logger = LoggerFactory.getLogger(CreatorECTypeDefaultConf.class); + + public static Sender confSender = + Sender.getSender( + Configuration.CLOUD_CONSOLE_CONFIGURATION_SPRING_APPLICATION_NAME().getValue()); + + private static LoadingCache confCache = + CacheBuilder.newBuilder() + .maximumSize(1000) + .expireAfterWrite( + (long) EntranceConfiguration.ENTRANCE_CREATOR_JOB_LIMIT_CONF_CACHE().getValue(), + TimeUnit.MINUTES) + .build( + new CacheLoader() { + @Override + public Integer load(String key) throws Exception { + Tuple2 tuple2 = + EntranceUtils.fromKeyGetLabels(key); + RequestQueryEngineConfig requestQueryEngineConfig = + new RequestQueryEngineConfig(tuple2._1, tuple2._2(), null); + int jobLimit = + (int) EntranceConfiguration.ENTRANCE_CREATOR_JOB_LIMIT().getValue(); + try { + Object response = confSender.ask(requestQueryEngineConfig); + if (response instanceof ResponseQueryConfig) { + jobLimit = + (int) + EntranceConfiguration.ENTRANCE_CREATOR_JOB_LIMIT() + .getValue(((ResponseQueryConfig) response).getKeyAndValue()); + } + } catch (Exception e) { + logger.warn("Failed to get key {} from conf", key, e); + } + return jobLimit; + } + }); + + public static int getCreatorECTypeMaxRunningJobs(String creator, String ecType) { + int jobLimit = (int) EntranceConfiguration.ENTRANCE_CREATOR_JOB_LIMIT().getValue(); + if (StringUtils.isNoneBlank(creator, ecType)) { + try { + String key = EntranceUtils.getDefaultCreatorECTypeKey(creator, ecType); + jobLimit = confCache.get(key); + } catch (Exception e) { + logger.warn("Failed to get key creator {} ecType {} from cache", creator, ecType, e); + } + } + int entranceNumber = EntranceUtils.getRunningEntranceNumber(); + return jobLimit / entranceNumber; + } +} 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 94531cd5fe..7c38d27947 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 @@ -104,7 +104,7 @@ private void shutdownEntrance(ContextClosedEvent event) { } logger.warn("Entrance exit to stop all job"); - EntranceJob[] allUndoneTask = getAllUndoneTask(null); + EntranceJob[] allUndoneTask = getAllUndoneTask(null, null); if (null != allUndoneTask) { for (EntranceJob job : allUndoneTask) { job.onFailure( 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 b023065ee4..a610d524b2 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,7 +19,6 @@ 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 @@ -29,7 +28,7 @@ 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.log.LogReader import org.apache.linkis.entrance.parser.ParserUtils import org.apache.linkis.entrance.timeout.JobTimeoutManager import org.apache.linkis.entrance.utils.JobHistoryHelper @@ -44,16 +43,14 @@ 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 -import org.springframework.beans.BeanUtils - import java.{lang, util} import java.text.{MessageFormat, SimpleDateFormat} import java.util.Date +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ @@ -63,6 +60,8 @@ abstract class EntranceServer extends Logging { private val jobTimeoutManager: JobTimeoutManager = new JobTimeoutManager() + private val timeoutCheck = EntranceConfiguration.ENABLE_JOB_TIMEOUT_CHECK.getValue + def init(): Unit def getName: String @@ -126,6 +125,17 @@ abstract class EntranceServer extends Logging { * this to trigger JobListener.onJobinit() */ Utils.tryAndWarn(job.getJobListener.foreach(_.onJobInited(job))) + if (logger.isDebugEnabled()) { + logger.debug( + s"After code preprocessing, the real execution code is:${jobRequest.getExecutionCode}" + ) + } + if (StringUtils.isBlank(jobRequest.getExecutionCode)) { + throw new SubmitFailedException( + SUBMIT_CODE_ISEMPTY.getErrorCode, + SUBMIT_CODE_ISEMPTY.getErrorDesc + ) + } getEntranceContext.getOrCreateScheduler().submit(job) val msg = LogUtils.generateInfo( s"Job with jobId : ${jobRequest.getId} and execID : ${job.getId()} submitted " @@ -135,7 +145,7 @@ abstract class EntranceServer extends Logging { job match { case entranceJob: EntranceJob => entranceJob.getJobRequest.setReqId(job.getId()) - if (jobTimeoutManager.timeoutCheck && JobTimeoutManager.hasTimeoutLabel(entranceJob)) { + if (timeoutCheck && JobTimeoutManager.hasTimeoutLabel(entranceJob)) { jobTimeoutManager.add(job.getId(), entranceJob) } entranceJob.getLogListener.foreach(_.onLogUpdate(entranceJob, msg)) @@ -492,14 +502,14 @@ abstract class EntranceServer extends Logging { job.setProgressListener(getEntranceContext.getOrCreatePersistenceManager()) job.setJobListener(getEntranceContext.getOrCreatePersistenceManager()) job match { - case entranceJob: EntranceJob => { + case entranceJob: EntranceJob => entranceJob.setEntranceListenerBus(getEntranceContext.getOrCreateEventListenerBus) - } case _ => } Utils.tryCatch { - if (logAppender.length() > 0) + if (logAppender.length() > 0) { job.getLogListener.foreach(_.onLogUpdate(job, logAppender.toString.trim)) + } } { t => logger.error("Failed to write init JobRequest log, reason: ", t) } @@ -510,6 +520,17 @@ abstract class EntranceServer extends Logging { * this to trigger JobListener.onJobinit() */ Utils.tryAndWarn(job.getJobListener.foreach(_.onJobInited(job))) + if (logger.isDebugEnabled()) { + logger.debug( + s"After code preprocessing, the real execution code is:${jobRequest.getExecutionCode}" + ) + } + if (StringUtils.isBlank(jobRequest.getExecutionCode)) { + throw new SubmitFailedException( + SUBMIT_CODE_ISEMPTY.getErrorCode, + SUBMIT_CODE_ISEMPTY.getErrorDesc + ) + } getEntranceContext.getOrCreateScheduler().submit(job) val msg = LogUtils.generateInfo( s"Job with jobId : ${jobRequest.getId} and execID : ${job.getId()} submitted, success to failover" @@ -519,8 +540,9 @@ abstract class EntranceServer extends Logging { job match { case entranceJob: EntranceJob => entranceJob.getJobRequest.setReqId(job.getId()) - if (jobTimeoutManager.timeoutCheck && JobTimeoutManager.hasTimeoutLabel(entranceJob)) + if (timeoutCheck && JobTimeoutManager.hasTimeoutLabel(entranceJob)) { jobTimeoutManager.add(job.getId(), entranceJob) + } entranceJob.getLogListener.foreach(_.onLogUpdate(entranceJob, msg)) case _ => } @@ -621,6 +643,68 @@ abstract class EntranceServer extends Logging { logger.info(s"job ${jobRequest.getId} success to initialize the properties") } + def getAllUndoneTask(filterWords: String, ecType: String = null): Array[EntranceJob] = { + val consumers = getEntranceContext + .getOrCreateScheduler() + .getSchedulerContext + .getOrCreateConsumerManager + .listConsumers() + .toSet + val filterConsumer = if (StringUtils.isNotBlank(filterWords)) { + if (StringUtils.isNotBlank(ecType)) { + consumers.filter(consumer => + consumer.getGroup.getGroupName.contains(filterWords) && consumer.getGroup.getGroupName + .contains(ecType) + ) + } else { + consumers.filter(_.getGroup.getGroupName.contains(filterWords)) + } + } else { + consumers + } + filterConsumer + .flatMap { consumer => + consumer.getRunningEvents ++ consumer.getConsumeQueue.getWaitingEvents + } + .filter(job => job != null && job.isInstanceOf[EntranceJob]) + .map(_.asInstanceOf[EntranceJob]) + .toArray + } + + /** + * to check timeout task,and kill timeout task timeout: default > 48h + */ + def startTimeOutCheck(): Unit = { + Utils.defaultScheduler.scheduleAtFixedRate( + new Runnable() { + override def run(): Unit = { + Utils.tryCatch { + + val timeoutType = EntranceConfiguration.ENTRANCE_TASK_TIMEOUT.getHotValue() + logger.info(s"Start to check timeout Job, timout is ${timeoutType}") + val timeoutTime = System.currentTimeMillis() - timeoutType.toLong + getAllUndoneTask(null, null).filter(job => job.createTime < timeoutTime).foreach { + job => + job.onFailure(s"Job has run for longer than the maximum time $timeoutType", null) + } + logger.info(s"Finished to check timeout Job, timout is ${timeoutType}") + } { case t: Throwable => + logger.warn(s"TimeoutDetective Job failed. ${t.getMessage}", t) + } + } + + }, + EntranceConfiguration.ENTRANCE_TASK_TIMEOUT_SCAN.getValue.toLong, + EntranceConfiguration.ENTRANCE_TASK_TIMEOUT_SCAN.getValue.toLong, + TimeUnit.MILLISECONDS + ) + } + + if (timeoutCheck) { + logger.info("Job time check is enabled") + startTimeOutCheck() + } + } object EntranceServer { diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceWebSocketService.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceWebSocketService.scala index 714b9f0cc2..b5339c9e2e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceWebSocketService.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/EntranceWebSocketService.scala @@ -215,18 +215,6 @@ class EntranceWebSocketService s"Your job's execution code is (after variable substitution and code check) " ) ) - entranceServer.getEntranceContext - .getOrCreateLogManager() - .onLogUpdate( - job, - "************************************SCRIPT CODE************************************" - ) - entranceServer.getEntranceContext - .getOrCreateLogManager() - .onLogUpdate( - job, - "************************************SCRIPT CODE************************************" - ) entranceServer.getEntranceContext .getOrCreateLogManager() .onLogUpdate( 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 e053d3793c..270376911b 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 @@ -220,9 +220,31 @@ object EntranceConfiguration { val CREATOR_IP_SWITCH = CommonVars("wds.linkis.entrance.user.creator.ip.interceptor.switch", false) + val TEMPLATE_CONF_SWITCH = + CommonVars("linkis.entrance.template.conf.interceptor.switch", false) + val ENABLE_ENTRANCE_DIRTY_DATA_CLEAR = CommonVars("linkis.entrance.auto.clean.dirty.data.enable", false) + val ENTRANCE_CREATOR_JOB_LIMIT: CommonVars[Int] = + CommonVars[Int]( + "linkis.entrance.creator.job.concurrency.limit", + 10000, + "Creator task concurrency limit parameters" + ) + + val ENTRANCE_CREATOR_JOB_LIMIT_CONF_CACHE = + CommonVars("linkis.entrance.creator.job.concurrency.limit.conf.cache.time", 30L) + + val ENTRANCE_TASK_TIMEOUT = + CommonVars("linkis.entrance.task.timeout", new TimeType("48h")) + + val ENTRANCE_TASK_TIMEOUT_SCAN = + CommonVars("linkis.entrance.task.timeout.scan", new TimeType("12h")) + + val ENABLE_HDFS_JVM_USER = + CommonVars[Boolean]("linkis.entrance.enable.hdfs.jvm.user", true).getValue + val ENTRANCE_FAILOVER_ENABLED = CommonVars("linkis.entrance.failover.enable", true).getValue val ENTRANCE_FAILOVER_SCAN_INIT_TIME = diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CommentInterceptor.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CommentInterceptor.scala index 627ab82b8e..34bd6ead01 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CommentInterceptor.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/CommentInterceptor.scala @@ -18,7 +18,6 @@ package org.apache.linkis.entrance.interceptor.impl import org.apache.linkis.common.utils.CodeAndRunTypeUtils -import org.apache.linkis.entrance.conf.EntranceConfiguration import org.apache.linkis.entrance.interceptor.EntranceInterceptor import org.apache.linkis.governance.common.entity.job.JobRequest import org.apache.linkis.manager.label.utils.LabelUtil diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/Explain.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/Explain.scala index 8436ccc711..35b40db339 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/Explain.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/Explain.scala @@ -57,6 +57,8 @@ object SparkExplain extends Explain { private val sy = Pattern.compile("sys\\.") private val scCancelAllJobs = Pattern.compile("sc\\.cancelAllJobs(\\s*)") private val runtime = Pattern.compile("Runtime\\.getRuntime") + private val LINE_BREAK = "\n" + private val LOG: Logger = LoggerFactory.getLogger(getClass) override def authPass(code: String, error: StringBuilder): Boolean = { if (EntranceConfiguration.SKIP_AUTH.getHotValue()) { @@ -99,6 +101,7 @@ object SQLExplain extends Explain { private val LIMIT: String = "limit" private val LIMIT_UPPERCASE: String = "LIMIT" private val IDE_ALLOW_NO_LIMIT = "--set wds.linkis.engine.no.limit.allow=true" + private val LOG: Logger = LoggerFactory.getLogger(getClass) override def authPass(code: String, error: StringBuilder): Boolean = { true @@ -131,6 +134,8 @@ object SQLExplain extends Explain { .generateWarn("please pay attention ,SQL full export mode opened(请注意,SQL全量导出模式打开)\n") ) } + var isFirstTimePrintingLimit = true + var isFirstTimePrintingOverLimit = true if (tempCode.contains("""\;""")) { val semicolonIndexes = findRealSemicolonIndex(tempCode) var oldIndex = 0 @@ -140,20 +145,27 @@ object SQLExplain extends Explain { if (isSelectCmd(singleCode)) { val trimCode = singleCode.trim if (isSelectCmdNoLimit(trimCode) && !isNoLimitAllowed) { - logAppender.append( - LogUtils.generateWarn( - s"You submitted a sql without limit, DSS will add limit 5000 to your sql" - ) + "\n" - ) + if (isFirstTimePrintingLimit) { + logAppender.append( + LogUtils.generateWarn( + s"You submitted a sql without limit, DSS will add limit 5000 to your sql" + ) + "\n" + ) + isFirstTimePrintingLimit = false + } + // 将注释先干掉,然后再进行添加limit val realCode = cleanComment(trimCode) fixedCode += (realCode + SQL_APPEND_LIMIT) } else if (isSelectOverLimit(singleCode) && !isNoLimitAllowed) { val trimCode = singleCode.trim - logAppender.append( - LogUtils.generateWarn( - s"You submitted a sql with limit exceeding 5000, it is not allowed. DSS will change your limit to 5000" - ) + "\n" - ) + if (isFirstTimePrintingOverLimit) { + logAppender.append( + LogUtils.generateWarn( + s"You submitted a sql with limit exceeding 5000, it is not allowed. DSS will change your limit to 5000" + ) + "\n" + ) + isFirstTimePrintingOverLimit = false + } fixedCode += repairSelectOverLimit(trimCode) } else { fixedCode += singleCode.trim @@ -167,20 +179,27 @@ object SQLExplain extends Explain { if (isSelectCmd(singleCode)) { val trimCode = singleCode.trim if (isSelectCmdNoLimit(trimCode) && !isNoLimitAllowed) { - logAppender.append( - LogUtils.generateWarn( - s"You submitted a sql without limit, DSS will add limit 5000 to your sql" - ) + "\n" - ) + if (isFirstTimePrintingLimit) { + logAppender.append( + LogUtils.generateWarn( + s"You submitted a sql without limit, DSS will add limit 5000 to your sql" + ) + "\n" + ) + isFirstTimePrintingLimit = false + } + // 将注释先干掉,然后再进行添加limit val realCode = cleanComment(trimCode) fixedCode += (realCode + SQL_APPEND_LIMIT) } else if (isSelectOverLimit(singleCode) && !isNoLimitAllowed) { val trimCode = singleCode.trim - logAppender.append( - LogUtils.generateWarn( - s"You submitted a sql with limit exceeding 5000, it is not allowed. DSS will change your limit to 5000" - ) + "\n" - ) + if (isFirstTimePrintingOverLimit) { + logAppender.append( + LogUtils.generateWarn( + s"You submitted a sql with limit exceeding 5000, it is not allowed. DSS will change your limit to 5000" + ) + "\n" + ) + isFirstTimePrintingOverLimit = false + } fixedCode += repairSelectOverLimit(trimCode) } else { fixedCode += singleCode.trim @@ -210,6 +229,8 @@ object SQLExplain extends Explain { array.toArray } + private def addNoLimit(code: String) = code + NO_LIMIT_STRING + protected def needNoLimit(code: String): Boolean = code.endsWith(NO_LIMIT_STRING) def isSelectCmd(code: String): Boolean = { @@ -217,16 +238,17 @@ object SQLExplain extends Explain { return false } val realCode = cleanComment(code) - realCode.trim.split("\\s+")(0).toLowerCase().contains("select") + realCode.trim.split("\\s+")(0).toLowerCase(Locale.getDefault).contains("select") } - def continueWhenError: Boolean = false + // def continueWhenError = false def isSelectCmdNoLimit(cmd: String): Boolean = { if (StringUtils.isEmpty(cmd)) { return false } val realCode = cmd.trim + // limit is often the last in a sql statement, so you need to make a final judgment val arr = realCode.split("\\s+") val words = new ArrayBuffer[String]() arr foreach { w => @@ -235,8 +257,10 @@ object SQLExplain extends Explain { val a = words.toArray val length = a.length if (a.length > 1) { - val second_last = a(length - 2) - !"limit".equals(second_last.toLowerCase()) + val second_last = a(length - 2).toLowerCase(Locale.getDefault) + // for some case eg:"SELECT * from dual WHERE (1=1)LIMIT 1;" + val result = !("limit".equals(second_last) || second_last.contains(")limit")) + result } else { false } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfInterceptor.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfInterceptor.scala new file mode 100644 index 0000000000..6accd30bd5 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfInterceptor.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.entrance.interceptor.impl + +import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.entrance.interceptor.EntranceInterceptor +import org.apache.linkis.governance.common.entity.job.JobRequest + +import java.lang + +class TemplateConfInterceptor extends EntranceInterceptor { + + override def apply(jobRequest: JobRequest, logAppender: lang.StringBuilder): JobRequest = { + if (EntranceConfiguration.TEMPLATE_CONF_SWITCH.getValue) { + TemplateConfUtils.dealWithTemplateConf(jobRequest, logAppender) + } else { + jobRequest + } + } + +} diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfUtils.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfUtils.scala new file mode 100644 index 0000000000..cdcbe01e85 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfUtils.scala @@ -0,0 +1,276 @@ +/* + * 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.interceptor.impl + +import org.apache.linkis.common.conf.Configuration +import org.apache.linkis.common.exception.LinkisCommonErrorException +import org.apache.linkis.common.log.LogUtils +import org.apache.linkis.common.utils.{CodeAndRunTypeUtils, Logging, Utils} +import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.governance.common.entity.TemplateConfKey +import org.apache.linkis.governance.common.entity.job.JobRequest +import org.apache.linkis.governance.common.protocol.conf.{TemplateConfRequest, TemplateConfResponse} +import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext +import org.apache.linkis.manager.label.constant.LabelKeyConstant +import org.apache.linkis.manager.label.entity.entrance.ExecuteOnceLabel +import org.apache.linkis.manager.label.utils.LabelUtil +import org.apache.linkis.protocol.utils.TaskUtils +import org.apache.linkis.rpc.Sender + +import org.apache.commons.lang3.StringUtils + +import java.{lang, util} +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ + +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} + +object TemplateConfUtils extends Logging { + + val confTemplateNameKey = "ec.resource.name" + + private val templateCache: LoadingCache[String, util.List[TemplateConfKey]] = CacheBuilder + .newBuilder() + .maximumSize(1000) + .expireAfterWrite(5, TimeUnit.MINUTES) + .build(new CacheLoader[String, util.List[TemplateConfKey]]() { + + override def load(templateUuid: String): util.List[TemplateConfKey] = { + var templateList = Utils.tryAndWarn { + val sender: Sender = Sender + .getSender(Configuration.CLOUD_CONSOLE_CONFIGURATION_SPRING_APPLICATION_NAME.getValue) + + logger.info(s"load template configuration data templateUuid:$templateUuid") + val res = sender.ask(new TemplateConfRequest(templateUuid)) match { + case response: TemplateConfResponse => + logger + .debug(s"${response.getList()}") + response.getList + case _ => + logger + .warn(s"load template configuration data templateUuid:$templateUuid loading failed") + new util.ArrayList[TemplateConfKey](0) + } + res + } + if (templateList.size() == 0) { + logger.warn(s"template configuration data loading failed, plaese check warn log") + } + templateList + } + + }) + + private val templateCacheName: LoadingCache[String, util.List[TemplateConfKey]] = CacheBuilder + .newBuilder() + .maximumSize(1000) + .expireAfterWrite(5, TimeUnit.MINUTES) + .build(new CacheLoader[String, util.List[TemplateConfKey]]() { + + override def load(templateName: String): util.List[TemplateConfKey] = { + var templateList = Utils.tryAndWarn { + val sender: Sender = Sender + .getSender(Configuration.CLOUD_CONSOLE_CONFIGURATION_SPRING_APPLICATION_NAME.getValue) + + logger.info(s"load template configuration data templateName:$templateName") + val res = sender.ask(new TemplateConfRequest(null, templateName)) match { + case response: TemplateConfResponse => + logger + .debug(s"${response.getList()}") + response.getList + case _ => + logger + .warn(s"load template configuration data templateName:$templateName loading failed") + new util.ArrayList[TemplateConfKey](0) + } + res + } + + if (templateList.size() == 0) { + logger.warn(s"template configuration data loading failed, plaese check warn log") + } + templateList + } + + }) + + /** + * Get user-defined template conf name value + * + * @param code + * :code + * @param codeType + * :sql,hql,scala + * @return + * String the last one of template conf name + */ + def getCustomTemplateConfName(code: String, codeType: String): String = { + var templateConfName = ""; + + var varString: String = null + var errString: String = null + var rightVarString: String = null + + val languageType = CodeAndRunTypeUtils.getLanguageTypeByCodeType(codeType) + + languageType match { + case CodeAndRunTypeUtils.LANGUAGE_TYPE_SQL => + varString = s"""\\s*---@set ${confTemplateNameKey}=\\s*.+\\s*""" + errString = """\s*---@.*""" + case CodeAndRunTypeUtils.LANGUAGE_TYPE_PYTHON | CodeAndRunTypeUtils.LANGUAGE_TYPE_SHELL => + varString = s"""\\s*##@set ${confTemplateNameKey}=\\s*.+\\s*""" + errString = """\s*##@""" + case CodeAndRunTypeUtils.LANGUAGE_TYPE_SCALA => + varString = s"""\\s*///@set ${confTemplateNameKey}=\\s*.+\\s*""" + errString = """\s*///@.+""" + case _ => + return templateConfName + } + + val customRegex = varString.r.unanchored + val errRegex = errString.r.unanchored + var codeRes = code.replaceAll("\r\n", "\n") + // only allow set at fisrt line + val res = codeRes.split("\n") + if (res.size > 0) { + val str = res(0) + str match { + case customRegex() => + val clearStr = if (str.endsWith(";")) str.substring(0, str.length - 1) else str + val res: Array[String] = clearStr.split("=") + if (res != null && res.length == 2) { + templateConfName = res(1).trim + logger.info(s"get template conf name $templateConfName") + } else { + if (res.length > 2) { + throw new LinkisCommonErrorException( + 20044, + s"$str template conf name var defined uncorrectly" + ) + } else { + throw new LinkisCommonErrorException( + 20045, + s"template conf name var was defined uncorrectly:$str" + ) + } + } + case errRegex() => + logger.warn( + s"The template conf name var definition is incorrect:$str,if it is not used, it will not run the error, but it is recommended to use the correct specification to define" + ) + case _ => + } + } + templateConfName + } + + def dealWithTemplateConf(jobRequest: JobRequest, logAppender: lang.StringBuilder): JobRequest = { + jobRequest match { + case requestPersistTask: JobRequest => + val params = requestPersistTask.getParams + val startMap = TaskUtils.getStartupMap(params) + + var templateConflist: util.List[TemplateConfKey] = new util.ArrayList[TemplateConfKey]() + var templateName: String = "" + // only for Creator:IDE, try to get template conf name from code string. eg:---@set ec.resource.name=xxxx + val (user, creator) = LabelUtil.getUserCreator(jobRequest.getLabels) + if (EntranceConfiguration.DEFAULT_REQUEST_APPLICATION_NAME.getValue.equals(creator)) { + val codeType = LabelUtil.getCodeType(jobRequest.getLabels) + templateName = + TemplateConfUtils.getCustomTemplateConfName(jobRequest.getExecutionCode, codeType) + } + + // code template name > start params template uuid + if (StringUtils.isBlank(templateName)) { + logger.debug("jobRequest startMap param template name is empty") + + logger.info("jobRequest startMap params :{} ", startMap) + val templateUuid = startMap.getOrDefault(LabelKeyConstant.TEMPLATE_CONF_KEY, "").toString + + if (StringUtils.isBlank(templateUuid)) { + logger.debug("jobRequest startMap param template id is empty") + } else { + logger.info("try to get template conf list with template uid:{} ", templateUuid) + logAppender.append( + LogUtils + .generateInfo(s"Try to get template conf data with template uid:$templateUuid\nn") + ) + templateConflist = templateCache.get(templateUuid) + if (templateConflist == null || templateConflist.size() == 0) { + logAppender.append( + LogUtils.generateWarn( + s"Can not get any template conf data with template uid:$templateUuid\n" + ) + ) + } + } + } else { + logger.info("Try to get template conf list with template name:[{}]", templateName) + logAppender.append( + LogUtils + .generateInfo(s"Try to get template conf data with template name:[$templateName]\n") + ) + templateConflist = templateCacheName.get(templateName) + if (templateConflist == null || templateConflist.size() == 0) { + logAppender.append( + LogUtils.generateWarn( + s"Can not get any template conf data with template name:$templateName\n" + ) + ) + } else { + // to remove metedata start param + TaskUtils.clearStartupMap(params) + + val onceLabel = + LabelBuilderFactoryContext.getLabelBuilderFactory.createLabel( + classOf[ExecuteOnceLabel] + ) + logger.info("Add once label for task id:{}", requestPersistTask.getId.toString) + requestPersistTask.getLabels.add(onceLabel) + } + } + + if (templateConflist != null && templateConflist.size() > 0) { + val keyList = new util.HashMap[String, AnyRef]() + templateConflist.asScala.foreach(ele => { + val key = ele.getKey + val oldValue = startMap.get(key) + if (oldValue != null && StringUtils.isNotBlank(oldValue.toString)) { + logger.info(s"key:$key value:$oldValue not empty, skip to deal") + } else { + val newValue = ele.getConfigValue + logger.info(s"key:$key value:$newValue will add to startMap params") + if (TaskUtils.isWithDebugInfo(params)) { + logAppender.append(LogUtils.generateInfo(s"add $key=$newValue\n")) + } + keyList.put(key, newValue) + } + + }) + if (keyList.size() > 0) { + TaskUtils.addStartupMap(params, keyList) + } + } + + case _ => + } + jobRequest + } + +} diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/VarSubstitutionInterceptor.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/VarSubstitutionInterceptor.scala index 0487a238cf..72d40305a6 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/VarSubstitutionInterceptor.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/interceptor/impl/VarSubstitutionInterceptor.scala @@ -41,10 +41,26 @@ class VarSubstitutionInterceptor extends EntranceInterceptor { LogUtils.generateInfo("Program is substituting variables for you") + "\n" ) val codeType = LabelUtil.getCodeType(jobRequest.getLabels) - jobRequest.setExecutionCode(CustomVariableUtils.replaceCustomVar(jobRequest, codeType)) + val realCode = CustomVariableUtils.replaceCustomVar(jobRequest, codeType) + jobRequest.setExecutionCode(realCode) logAppender.append( LogUtils.generateInfo("Variables substitution ended successfully") + "\n" ) + // print code after variables substitution + logAppender.append( + LogUtils.generateInfo( + "You have submitted a new job, script code (after variable substitution) is" + ) + "\n" + ); + logAppender.append( + "************************************SCRIPT CODE************************************" + "\n" + ) + logAppender.append(realCode); + logAppender.append("\n"); + logAppender.append( + "************************************SCRIPT CODE************************************" + "\n" + ); + jobRequest } { case e: VarSubstitutionException => diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogReader.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogReader.scala index 483cf9ab43..748f82df4b 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogReader.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogReader.scala @@ -19,7 +19,11 @@ package org.apache.linkis.entrance.log import org.apache.linkis.common.io.{Fs, FsPath} import org.apache.linkis.common.utils.Utils +import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.entrance.exception.LogReadFailedException import org.apache.linkis.storage.FSFactory +import org.apache.linkis.storage.fs.FileSystem +import org.apache.linkis.storage.utils.StorageUtils import java.io.{InputStream, IOException} import java.util @@ -36,13 +40,26 @@ class CacheLogReader(logPath: String, charset: String, sharedCache: Cache, user: var closed = false private def createInputStream: InputStream = { + if (!logPath.contains(user)) { + throw new LogReadFailedException( + s"${user} does not have permission to read the path $logPath" + ) + } + val fsPath = new FsPath(logPath) if (fileSystem == null) lock synchronized { if (fileSystem == null) { - fileSystem = FSFactory.getFsByProxyUser(new FsPath(logPath), user) + + fileSystem = + if (StorageUtils.isHDFSPath(fsPath) && EntranceConfiguration.ENABLE_HDFS_JVM_USER) { + FSFactory.getFs(new FsPath(logPath)).asInstanceOf[FileSystem] + } else { + FSFactory.getFsByProxyUser(new FsPath(logPath), user).asInstanceOf[FileSystem] + } + fileSystem.init(new util.HashMap[String, String]()) } } - val inputStream: InputStream = fileSystem.read(new FsPath(logPath)) + val inputStream: InputStream = fileSystem.read(fsPath) inputStream } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogWriter.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogWriter.scala index 9028c469ab..b54dc757cd 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogWriter.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/CacheLogWriter.scala @@ -41,6 +41,8 @@ class CacheLogWriter(logPath: String, charset: String, sharedCache: Cache, user: val sb = new StringBuilder if (removed != null) sb.append(removed).append("\n") logs.filter(_ != null).foreach(log => sb.append(log).append("\n")) + // need append latest msg before clear + sb.append(msg).append("\n") sharedCache.cachedLogs.fakeClear() super.write(sb.toString()) pushTime.setTime( diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/ErrorCodeManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/ErrorCodeManager.scala index 54914b6002..4b082342ce 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/ErrorCodeManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/ErrorCodeManager.scala @@ -18,6 +18,12 @@ package org.apache.linkis.entrance.log import org.apache.linkis.errorcode.client.handler.LinkisErrorCodeHandler +import org.apache.linkis.errorcode.client.manager.LinkisErrorCodeManager +import org.apache.linkis.errorcode.common.LinkisErrorCode + +import java.util + +import scala.collection.JavaConverters.asScalaBufferConverter abstract class ErrorCodeManager { @@ -35,6 +41,22 @@ abstract class ErrorCodeManager { None } + def errorMatchAndGetContent(log: String): Option[(String, String, String)] = { + getErrorCodes.foreach(e => + if (e.regex.findFirstIn(log).isDefined) { + val matched = e.regex.unapplySeq(log) + if (matched.nonEmpty) { + return Some( + e.code, + e.message.format(matched.get: _*), + e.regex.findFirstIn(log).getOrElse("") + ) + } else Some(e.code, e.message, "") + } + ) + None + } + } /** @@ -44,7 +66,24 @@ object FlexibleErrorCodeManager extends ErrorCodeManager { private val errorCodeHandler = LinkisErrorCodeHandler.getInstance() - override def getErrorCodes: Array[ErrorCode] = Array.empty + private val linkisErrorCodeManager = LinkisErrorCodeManager.getInstance + + override def getErrorCodes: Array[ErrorCode] = { + val errorCodes: util.List[LinkisErrorCode] = linkisErrorCodeManager.getLinkisErrorCodes + if (errorCodes == null) { + Array.empty + } else { + errorCodes.asScala + .map(linkisErrorCode => + ErrorCode( + linkisErrorCode.getErrorRegex, + linkisErrorCode.getErrorCode, + linkisErrorCode.getErrorDesc + ) + ) + .toArray + } + } override def errorMatch(log: String): Option[(String, String)] = { val errorCodes = errorCodeHandler.handle(log) diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/HDFSCacheLogWriter.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/HDFSCacheLogWriter.scala index 24633dfbb2..4f37ff1040 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/HDFSCacheLogWriter.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/HDFSCacheLogWriter.scala @@ -37,11 +37,15 @@ import java.util class HDFSCacheLogWriter(logPath: String, charset: String, sharedCache: Cache, user: String) extends LogWriter(charset) { - if (StringUtils.isBlank(logPath)) + if (StringUtils.isBlank(logPath)) { throw new EntranceErrorException(LOGPATH_NOT_NULL.getErrorCode, LOGPATH_NOT_NULL.getErrorDesc) + } - protected var fileSystem = + protected var fileSystem = if (EntranceConfiguration.ENABLE_HDFS_JVM_USER) { + FSFactory.getFs(new FsPath(logPath)).asInstanceOf[FileSystem] + } else { FSFactory.getFsByProxyUser(new FsPath(logPath), user).asInstanceOf[FileSystem] + } override protected var outputStream: OutputStream = null @@ -55,7 +59,12 @@ class HDFSCacheLogWriter(logPath: String, charset: String, sharedCache: Cache, u private def init(): Unit = { fileSystem.init(new util.HashMap[String, String]()) - FileSystemUtils.createNewFileWithFileSystem(fileSystem, new FsPath(logPath), user, true) + FileSystemUtils.createNewFileAndSetOwnerWithFileSystem( + fileSystem, + new FsPath(logPath), + user, + true + ) } @throws[IOException] @@ -99,6 +108,8 @@ class HDFSCacheLogWriter(logPath: String, charset: String, sharedCache: Cache, u val sb = new StringBuilder if (removed != null) sb.append(removed).append("\n") logs.filter(_ != null).foreach(log => sb.append(log).append("\n")) + // need append latest msg before fake clear + sb.append(msg).append("\n") sharedCache.cachedLogs.fakeClear() writeToFile(sb.toString()) pushTime.setTime( diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LogManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LogManager.scala index 626a643a0b..19f4c5c6ad 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LogManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LogManager.scala @@ -17,6 +17,7 @@ package org.apache.linkis.entrance.log +import org.apache.linkis.common.log.LogUtils import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.entrance.EntranceContext import org.apache.linkis.entrance.job.EntranceExecutionJob @@ -59,10 +60,19 @@ abstract class LogManager extends LogListener with Logging { } } } - entranceExecutionJob.getLogWriter.foreach(logWriter => logWriter.write(log)) - errorCodeManager.foreach(_.errorMatch(log).foreach { case (code, errorMsg) => - errorCodeListener.foreach(_.onErrorCodeCreated(job, code, errorMsg)) + var writeLog = log + errorCodeManager.foreach(_.errorMatchAndGetContent(log).foreach { + case (code, errorMsg, targetMsg) => + if (!targetMsg.contains(LogUtils.ERROR_STR) && log.contains(LogUtils.ERROR_STR)) { + writeLog = LogUtils.generateERROR( + s"error code: $code, errorMsg: $errorMsg, errorLine: $targetMsg \n" + log + ) + } + errorCodeListener.foreach(_.onErrorCodeCreated(job, code, errorMsg)) + case _ => }) + entranceExecutionJob.getLogWriter.foreach(logWriter => logWriter.write(writeLog)) + case _ => } } { diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LoopArray.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LoopArray.scala index 155d8c7bd5..5b62a49aa1 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LoopArray.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/log/LoopArray.scala @@ -51,7 +51,7 @@ class LoopArray[T](maxCapacity: Int) { } else if (index > _max) { throw new IllegalArgumentException("The index " + index + " must be less than " + _max) } - val _index = (flag + (index - realSize)) % maxCapacity + val _index = (flag + (index - realSize + maxCapacity - 1)) % maxCapacity eventQueue(_index).asInstanceOf[T] } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/orchestrator/plugin/EntranceUserParallelOrchestratorPlugin.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/orchestrator/plugin/EntranceUserParallelOrchestratorPlugin.scala index 4b9b4570f1..e5c657023e 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/orchestrator/plugin/EntranceUserParallelOrchestratorPlugin.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/orchestrator/plugin/EntranceUserParallelOrchestratorPlugin.scala @@ -20,19 +20,15 @@ package org.apache.linkis.entrance.orchestrator.plugin import org.apache.linkis.common.conf.Configuration import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.entrance.utils.EntranceUtils import org.apache.linkis.governance.common.protocol.conf.{ RequestQueryEngineConfigWithGlobalConfig, ResponseQueryConfig } -import org.apache.linkis.manager.label.builder.factory.LabelBuilderFactoryContext -import org.apache.linkis.manager.label.constant.LabelKeyConstant import org.apache.linkis.manager.label.entity.Label import org.apache.linkis.manager.label.entity.engine.{EngineTypeLabel, UserCreatorLabel} import org.apache.linkis.orchestrator.plugin.UserParallelOrchestratorPlugin import org.apache.linkis.rpc.Sender -import org.apache.linkis.server.BDPJettyServerHelper - -import org.apache.commons.lang3.StringUtils import java.util import java.util.concurrent.TimeUnit @@ -43,10 +39,6 @@ import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} class EntranceUserParallelOrchestratorPlugin extends UserParallelOrchestratorPlugin with Logging { - private val SPLIT = "," - - private val labelFactory = LabelBuilderFactoryContext.getLabelBuilderFactory - private def getDefaultMaxRuningNum: Int = { EntranceConfiguration.WDS_LINKIS_INSTANCE.getHotValue() } @@ -62,7 +54,7 @@ class EntranceUserParallelOrchestratorPlugin extends UserParallelOrchestratorPlu .build(new CacheLoader[String, Integer]() { override def load(key: String): Integer = { - val (userCreatorLabel, engineTypeLabel) = fromKeyGetLabels(key) + val (userCreatorLabel, engineTypeLabel) = EntranceUtils.fromKeyGetLabels(key) val keyAndValue = Utils.tryAndWarnMsg { sender .ask(RequestQueryEngineConfigWithGlobalConfig(userCreatorLabel, engineTypeLabel)) @@ -75,10 +67,8 @@ class EntranceUserParallelOrchestratorPlugin extends UserParallelOrchestratorPlu null == keyAndValue || !keyAndValue .containsKey(EntranceConfiguration.WDS_LINKIS_INSTANCE.key) ) { - logger.error( - s"cannot found user configuration key:${EntranceConfiguration.WDS_LINKIS_INSTANCE.key}," + - s"will use default value ${EntranceConfiguration.WDS_LINKIS_INSTANCE.getHotValue()}。All config map: ${BDPJettyServerHelper.gson - .toJson(keyAndValue)}" + logger.warn( + s"cannot found user configuration key:${EntranceConfiguration.WDS_LINKIS_INSTANCE.key}," + s"will use default value " ) } val maxRunningJobs = EntranceConfiguration.WDS_LINKIS_INSTANCE.getValue(keyAndValue, true) @@ -102,27 +92,7 @@ class EntranceUserParallelOrchestratorPlugin extends UserParallelOrchestratorPlu if (null == userCreatorLabel || null == engineTypeLabel) { return getDefaultMaxRuningNum } - configCache.get(getKey(userCreatorLabel, engineTypeLabel)) - } - - private def getKey( - userCreatorLabel: UserCreatorLabel, - engineTypeLabel: EngineTypeLabel - ): String = { - userCreatorLabel.getStringValue + SPLIT + engineTypeLabel.getStringValue - } - - private def fromKeyGetLabels(key: String): (UserCreatorLabel, EngineTypeLabel) = { - if (StringUtils.isBlank(key)) (null, null) - else { - val labelStringValues = key.split(SPLIT) - if (labelStringValues.length < 2) return (null, null) - val userCreatorLabel = labelFactory - .createLabel[UserCreatorLabel](LabelKeyConstant.USER_CREATOR_TYPE_KEY, labelStringValues(0)) - val engineTypeLabel = labelFactory - .createLabel[EngineTypeLabel](LabelKeyConstant.ENGINE_TYPE_KEY, labelStringValues(1)) - (userCreatorLabel, engineTypeLabel) - } + configCache.get(EntranceUtils.getUserCreatorEcTypeKey(userCreatorLabel, engineTypeLabel)) } override def isReady: Boolean = true diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/persistence/EntranceResultSetEngine.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/persistence/EntranceResultSetEngine.scala index 02d1a6a08e..2ba98438e8 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/persistence/EntranceResultSetEngine.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/persistence/EntranceResultSetEngine.scala @@ -17,20 +17,12 @@ package org.apache.linkis.entrance.persistence -import org.apache.linkis.common.io.{FsPath, MetaData, Record} -import org.apache.linkis.common.io.resultset.ResultSet -import org.apache.linkis.common.utils.{Logging, Utils} +import org.apache.linkis.common.utils.Logging import org.apache.linkis.entrance.exception.{EntranceErrorCode, EntranceErrorException} -import org.apache.linkis.entrance.execute.StorePathExecuteRequest -import org.apache.linkis.entrance.job.{EntranceExecuteRequest, EntranceExecutionJob} -import org.apache.linkis.entrance.scheduler.cache.CacheOutputExecuteResponse -import org.apache.linkis.governance.common.entity.job.SubJobDetail import org.apache.linkis.scheduler.executer.{AliasOutputExecuteResponse, OutputExecuteResponse} import org.apache.linkis.scheduler.queue.Job -import org.apache.linkis.storage.resultset.{ResultSetFactory, ResultSetWriterFactory} -import org.apache.linkis.storage.utils.FileSystemUtils +import org.apache.linkis.storage.resultset.ResultSetFactory -import org.apache.commons.io.IOUtils import org.apache.commons.lang3.StringUtils class EntranceResultSetEngine extends ResultSetEngine with Logging { @@ -46,15 +38,11 @@ class EntranceResultSetEngine extends ResultSetEngine with Logging { EntranceErrorCode.RESULT_NOT_PERSISTED_ERROR.getDesc ) } - case CacheOutputExecuteResponse(alias, output) => - if (ResultSetFactory.getInstance.isResultSetPath(output)) { - getDir(output) - } else { - throw new EntranceErrorException( - EntranceErrorCode.RESULT_NOT_PERSISTED_ERROR.getErrCode, - EntranceErrorCode.RESULT_NOT_PERSISTED_ERROR.getDesc - ) - } + case _ => + throw new EntranceErrorException( + EntranceErrorCode.RESULT_NOT_PERSISTED_ERROR.getErrCode, + EntranceErrorCode.RESULT_NOT_PERSISTED_ERROR.getDesc + ) } } @@ -64,7 +52,7 @@ class EntranceResultSetEngine extends ResultSetEngine with Logging { } else { val arr = str.split("/").filter(StringUtils.isNotBlank) if (arr.length <= 2) { - return str + str } else { str.substring(0, str.lastIndexOf("/")) } 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 e2f0ab1d5a..26d8a60c4c 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 @@ -17,7 +17,7 @@ package org.apache.linkis.entrance.scheduler -import org.apache.linkis.common.utils.Utils +import org.apache.linkis.common.utils.{Logging, Utils} import org.apache.linkis.entrance.conf.EntranceConfiguration import org.apache.linkis.entrance.job.EntranceExecutionJob import org.apache.linkis.entrance.utils.JobHistoryHelper @@ -34,7 +34,8 @@ class EntranceFIFOUserConsumer( schedulerContext: SchedulerContext, executeService: ExecutorService, private var group: Group -) extends FIFOUserConsumer(schedulerContext, executeService, group) { +) extends FIFOUserConsumer(schedulerContext, executeService, group) + with Logging { override def loop(): Unit = { // When offlineFlag=true, the unsubmitted tasks will be failover, and the running tasks will wait for completion. @@ -67,4 +68,39 @@ class EntranceFIFOUserConsumer( } + override def runScheduleIntercept: Boolean = { + val consumers = getSchedulerContext.getOrCreateConsumerManager.listConsumers + var creatorRunningJobNum = 0 + // APP_TEST_hadoop_hive or IDE_hadoop_hive + val groupNameStr = getGroup.getGroupName + val groupNames = groupNameStr.split("_") + val length = groupNames.length + if (length < 3) return true + // APP_TEST + val lastIndex = groupNameStr.lastIndexOf("_") + val secondLastIndex = groupNameStr.lastIndexOf("_", lastIndex - 1) + val creatorName = groupNameStr.substring(0, secondLastIndex) + // hive + val ecType = groupNames(length - 1) + for (consumer <- consumers) { + val groupName = consumer.getGroup.getGroupName + if (groupName.startsWith(creatorName) && groupName.endsWith(ecType)) { + creatorRunningJobNum += consumer.getRunningEvents.length + } + } + val creatorECTypeMaxRunningJobs = + CreatorECTypeDefaultConf.getCreatorECTypeMaxRunningJobs(creatorName, ecType) + if (logger.isDebugEnabled) { + logger.debug( + s"Creator: $creatorName EC:$ecType there are currently:$creatorRunningJobNum jobs running and maximum limit: $creatorECTypeMaxRunningJobs" + ) + } + if (creatorRunningJobNum > creatorECTypeMaxRunningJobs) { + logger.error( + s"Creator: $creatorName EC:$ecType there are currently:$creatorRunningJobNum jobs running that exceed the maximum limit: $creatorECTypeMaxRunningJobs" + ) + false + } else 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 0f31351b48..de4c025e30 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 @@ -23,19 +23,14 @@ import org.apache.linkis.entrance.conf.EntranceConfiguration import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary._ import org.apache.linkis.entrance.exception.{EntranceErrorCode, EntranceErrorException} import org.apache.linkis.entrance.execute.EntranceJob +import org.apache.linkis.entrance.utils.EntranceUtils 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.engine.{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 @@ -46,8 +41,6 @@ import java.util import java.util.concurrent.TimeUnit import java.util.regex.Pattern -import scala.collection.JavaConverters._ - import com.google.common.cache.{Cache, CacheBuilder} class EntranceGroupFactory extends GroupFactory with Logging { @@ -58,7 +51,7 @@ class EntranceGroupFactory extends GroupFactory with Logging { .maximumSize(EntranceConfiguration.GROUP_CACHE_MAX.getValue) .build() - private val GROUP_MAX_CAPACITY = CommonVars("wds.linkis.entrance.max.capacity", 2000) + private val GROUP_MAX_CAPACITY = CommonVars("wds.linkis.entrance.max.capacity", 1000) private val SPECIFIED_USERNAME_REGEX = CommonVars("wds.linkis.entrance.specified.username.regex", "hduser.*") @@ -76,29 +69,16 @@ class EntranceGroupFactory extends GroupFactory with Logging { } override def getOrCreateGroup(event: SchedulerEvent): Group = { - val (labels, params) = event match { + val labels = event match { case job: EntranceJob => - (job.getJobRequest.getLabels, job.getJobRequest.getParams) + job.getJobRequest.getLabels + case _ => + throw new EntranceErrorException(LABEL_NOT_NULL.getErrorCode, LABEL_NOT_NULL.getErrorDesc) } - val groupName = EntranceGroupFactory.getGroupNameByLabels(labels, params) + val groupName = EntranceGroupFactory.getGroupNameByLabels(labels) val cacheGroup = groupNameToGroups.getIfPresent(groupName) if (null == cacheGroup) synchronized { val maxAskExecutorTimes = EntranceConfiguration.MAX_ASK_EXECUTOR_TIME.getValue.toLong - if (groupName.startsWith(EntranceGroupFactory.CONCURRENT)) { - if (null == groupNameToGroups.getIfPresent(groupName)) synchronized { - if (null == groupNameToGroups.getIfPresent(groupName)) { - val group = new ParallelGroup( - groupName, - 100, - EntranceConfiguration.CONCURRENT_FACTORY_MAX_CAPACITY.getValue - ) - group.setMaxRunningJobs(EntranceConfiguration.CONCURRENT_MAX_RUNNING_JOBS.getValue) - group.setMaxAskExecutorTimes(EntranceConfiguration.CONCURRENT_EXECUTOR_TIME.getValue) - groupNameToGroups.put(groupName, group) - return group - } - } - } val sender: Sender = Sender.getSender(Configuration.CLOUD_CONSOLE_CONFIGURATION_SPRING_APPLICATION_NAME.getValue) val userCreatorLabel: UserCreatorLabel = LabelUtil.getUserCreatorLabel(labels) @@ -136,8 +116,11 @@ class EntranceGroupFactory extends GroupFactory with Logging { group.setMaxRunningJobs(maxRunningJobs) group.setMaxAskExecutorTimes(maxAskExecutorTimes) groupNameToGroups.put(groupName, group) + group + } + else { + cacheGroup } - groupNameToGroups.getIfPresent(groupName) } override def getGroup(groupName: String): Group = { @@ -151,10 +134,18 @@ class EntranceGroupFactory extends GroupFactory with Logging { group } + /** + * User task concurrency control is controlled for multiple Entrances, which will be evenly + * distributed based on the number of existing Entrances + * @param keyAndValue + * @return + */ private def getUserMaxRunningJobs(keyAndValue: util.Map[String, String]): Int = { + val userDefinedRunningJobs = EntranceConfiguration.WDS_LINKIS_INSTANCE.getValue(keyAndValue) + val entranceNum = EntranceUtils.getRunningEntranceNumber() Math.max( EntranceConfiguration.ENTRANCE_INSTANCE_MIN.getValue, - EntranceConfiguration.WDS_LINKIS_INSTANCE.getValue(keyAndValue) + userDefinedRunningJobs / entranceNum ) } @@ -162,64 +153,21 @@ class EntranceGroupFactory extends GroupFactory with Logging { object EntranceGroupFactory { - val CACHE = "_Cache" - - val CONCURRENT = "Concurrent_" - - def getGroupName( - creator: String, - user: String, - params: util.Map[String, AnyRef] = new util.HashMap[String, AnyRef] - ): String = { - val runtime = TaskUtils.getRuntimeMap(params) - val cache = - if ( - runtime.get(TaskConstant.READ_FROM_CACHE) != null && runtime - .get(TaskConstant.READ_FROM_CACHE) - .asInstanceOf[Boolean] - ) { - CACHE - } else "" - if (StringUtils.isNotEmpty(creator)) creator + "_" + user + cache - else EntranceConfiguration.DEFAULT_REQUEST_APPLICATION_NAME.getValue + "_" + user + cache - } - - def getGroupNameByLabels( - labels: java.util.List[Label[_]], - params: util.Map[String, AnyRef] = new util.HashMap[String, AnyRef] - ): String = { - - val userCreator = labels.asScala.find(_.isInstanceOf[UserCreatorLabel]) - val engineType = labels.asScala.find(_.isInstanceOf[EngineTypeLabel]) - val concurrent = labels.asScala.find(_.isInstanceOf[ConcurrentEngineConnLabel]) - if (userCreator.isEmpty || engineType.isEmpty) { + /** + * Entrance group rule creator_username_engineType eg:IDE_PEACEWONG_SPARK + * @param labels + * @param params + * @return + */ + def getGroupNameByLabels(labels: java.util.List[Label[_]]): String = { + val userCreatorLabel = LabelUtil.getUserCreatorLabel(labels) + val engineTypeLabel = LabelUtil.getEngineTypeLabel(labels) + if (null == userCreatorLabel || null == engineTypeLabel) { throw new EntranceErrorException(LABEL_NOT_NULL.getErrorCode, LABEL_NOT_NULL.getErrorDesc) } - - if (concurrent.isDefined) { - - val engineTypeLabel = engineType.get.asInstanceOf[EngineTypeLabel] - val groupName = CONCURRENT + engineTypeLabel.getEngineType - groupName - - } else { - val userCreatorLabel = userCreator.get.asInstanceOf[UserCreatorLabel] - - val engineTypeLabel = engineType.get.asInstanceOf[EngineTypeLabel] - - val runtime = TaskUtils.getRuntimeMap(params) - val cache = - if ( - runtime.get(TaskConstant.READ_FROM_CACHE) != null && runtime - .get(TaskConstant.READ_FROM_CACHE) - .asInstanceOf[Boolean] - ) { - CACHE - } else "" - val groupName = - userCreatorLabel.getCreator + "_" + userCreatorLabel.getUser + "_" + engineTypeLabel.getEngineType + cache - groupName - } + val groupName = + userCreatorLabel.getCreator + "_" + userCreatorLabel.getUser + "_" + engineTypeLabel.getEngineType + groupName } } 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 2cdee97cc7..789e2ca2b1 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 @@ -20,6 +20,7 @@ package org.apache.linkis.entrance.scheduler import org.apache.linkis.common.ServiceInstance import org.apache.linkis.common.utils.Utils import org.apache.linkis.entrance.conf.EntranceConfiguration +import org.apache.linkis.entrance.utils.EntranceUtils 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} @@ -45,29 +46,10 @@ class EntranceParallelConsumerManager(maxParallelismUsers: Int, schedulerName: S if (EntranceConfiguration.ENTRANCE_GROUP_SCAN_ENABLED.getValue) { Utils.defaultScheduler.scheduleAtFixedRate( new Runnable { - override def run(): Unit = { - 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 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") - } + override def run(): Unit = Utils.tryAndWarn { + // refresh all group maxAllowRunningJobs + refreshAllGroupMaxAllowRunningJobs(EntranceUtils.getRunningEntranceNumber()) + logger.info("Finished to refresh consumer group maxAllowRunningJobs") } }, EntranceConfiguration.ENTRANCE_GROUP_SCAN_INIT_TIME.getValue, diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala deleted file mode 100644 index 65bbbd39b4..0000000000 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumer.scala +++ /dev/null @@ -1,158 +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.entrance.scheduler.cache - -import org.apache.linkis.common.io.FsPath -import org.apache.linkis.common.utils.Utils -import org.apache.linkis.entrance.errorcode.EntranceErrorCodeSummary._ -import org.apache.linkis.entrance.exception.CacheNotReadyException -import org.apache.linkis.entrance.execute.EntranceJob -import org.apache.linkis.entrance.persistence.PersistenceManager -import org.apache.linkis.entrance.utils.JobHistoryHelper -import org.apache.linkis.governance.common.entity.job.JobRequest -import org.apache.linkis.manager.label.constant.LabelKeyConstant -import org.apache.linkis.protocol.constants.TaskConstant -import org.apache.linkis.protocol.utils.TaskUtils -import org.apache.linkis.scheduler.SchedulerContext -import org.apache.linkis.scheduler.errorcode.LinkisSchedulerErrorCodeSummary._ -import org.apache.linkis.scheduler.exception.SchedulerErrorException -import org.apache.linkis.scheduler.executer.SuccessExecuteResponse -import org.apache.linkis.scheduler.queue.Group -import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer -import org.apache.linkis.server.BDPJettyServerHelper -import org.apache.linkis.storage.FSFactory -import org.apache.linkis.storage.fs.FileSystem - -import org.apache.commons.io.FilenameUtils -import org.apache.commons.lang3.StringUtils - -import java.util.concurrent.ExecutorService - -import scala.collection.JavaConverters._ - -import com.google.common.collect.Lists - -class ReadCacheConsumer( - schedulerContext: SchedulerContext, - executeService: ExecutorService, - private var group: Group, - persistenceManager: PersistenceManager -) extends FIFOUserConsumer(schedulerContext, executeService, group) { - - override protected def loop(): Unit = { - val event = Option(getConsumeQueue.take()) - event.foreach { - case job: EntranceJob => - job.getJobRequest match { - case jobRequest: JobRequest => - Utils.tryCatch { - val engineTpyeLabel = jobRequest.getLabels.asScala - .filter(l => l.getLabelKey.equalsIgnoreCase(LabelKeyConstant.ENGINE_TYPE_KEY)) - .headOption - .getOrElse(null) - val labelStrList = jobRequest.getLabels.asScala.map { case l => - l.getStringValue - }.toList - if (null == engineTpyeLabel) { - logger.error( - "Invalid engineType null, cannot process. jobReq : " + BDPJettyServerHelper.gson - .toJson(jobRequest) - ) - throw CacheNotReadyException( - INVALID_ENGINETYPE_NULL.getErrorCode, - INVALID_ENGINETYPE_NULL.getErrorDesc - ) - } - val readCacheBefore: Long = TaskUtils - .getRuntimeMap(job.getParams) - .getOrDefault(TaskConstant.READ_CACHE_BEFORE, 300L: java.lang.Long) - .asInstanceOf[Long] - val cacheResult = JobHistoryHelper.getCache( - jobRequest.getExecutionCode, - jobRequest.getExecuteUser, - labelStrList.asJava, - readCacheBefore - ) - if (cacheResult != null && StringUtils.isNotBlank(cacheResult.getResultLocation)) { - val resultSets = listResults(cacheResult.getResultLocation, job.getUser) - if (resultSets.size() > 0) { - for (resultSet: FsPath <- resultSets.asScala) { - val alias = FilenameUtils.getBaseName(resultSet.getPath) - val output = FsPath - .getFsPath( - cacheResult.getResultLocation, - FilenameUtils.getName(resultSet.getPath) - ) - .getSchemaPath -// persistenceManager.onResultSetCreated(job, new CacheOutputExecuteResponse(alias, output)) - throw CacheNotReadyException( - INVALID_RESULTSETS.getErrorCode, - INVALID_RESULTSETS.getErrorDesc - ) - // todo check - } -// persistenceManager.onResultSizeCreated(job, resultSets.size()) - } - val runtime = TaskUtils.getRuntimeMap(job.getParams) - runtime.put(TaskConstant.CACHE, java.lang.Boolean.FALSE) - TaskUtils.addRuntimeMap(job.getParams, runtime) - job.transitionCompleted(SuccessExecuteResponse(), "Result found in cache") - } else { - logger.info("Cache not found, submit to normal consumer.") - submitToExecute(job) - } - } { t => - logger.warn("Read cache failed, submit to normal consumer: ", t) - submitToExecute(job) - } - case _ => - } - case _ => - } - } - - private def listResults(resultLocation: String, user: String) = { - val dirPath = FsPath.getFsPath(resultLocation) - val fileSystem = FSFactory.getFsByProxyUser(dirPath, user).asInstanceOf[FileSystem] - Utils.tryFinally { - fileSystem.init(null) - if (fileSystem.exists(dirPath)) { - fileSystem.listPathWithError(dirPath).getFsPaths - } else { - Lists.newArrayList[FsPath]() - } - }(Utils.tryQuietly(fileSystem.close())) - } - - private def submitToExecute(job: EntranceJob): Unit = { - val runtime = TaskUtils.getRuntimeMap(job.getParams) - runtime.put(TaskConstant.READ_FROM_CACHE, java.lang.Boolean.FALSE) - TaskUtils.addRuntimeMap(job.getParams, runtime) - val groupName = schedulerContext.getOrCreateGroupFactory.getOrCreateGroup(job).getGroupName - val consumer = schedulerContext.getOrCreateConsumerManager.getOrCreateConsumer(groupName) - val index = consumer.getConsumeQueue.offer(job) - // index.map(getEventId(_, groupName)).foreach(job.setId) - if (index.isEmpty) { - throw new SchedulerErrorException( - JOB_QUEUE_IS_FULL.getErrorCode, - JOB_QUEUE_IS_FULL.getErrorDesc - ) - } - } - -} diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumerManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumerManager.scala deleted file mode 100644 index a4cba19f34..0000000000 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/scheduler/cache/ReadCacheConsumerManager.scala +++ /dev/null @@ -1,44 +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.entrance.scheduler.cache - -import org.apache.linkis.entrance.persistence.PersistenceManager -import org.apache.linkis.entrance.scheduler.EntranceGroupFactory -import org.apache.linkis.scheduler.queue.fifoqueue.FIFOUserConsumer -import org.apache.linkis.scheduler.queue.parallelqueue.ParallelConsumerManager - -class ReadCacheConsumerManager(maxParallelismUsers: Int, persistenceManager: PersistenceManager) - extends ParallelConsumerManager(maxParallelismUsers) { - - override protected def createConsumer(groupName: String): FIFOUserConsumer = { - val group = getSchedulerContext.getOrCreateGroupFactory.getGroup(groupName) - if (groupName.endsWith(EntranceGroupFactory.CACHE)) { - logger.info("Create cache consumer with group: " + groupName) - new ReadCacheConsumer( - getSchedulerContext, - getOrCreateExecutorService, - group, - persistenceManager - ) - } else { - logger.info("Create normal consumer with group: " + groupName) - new FIFOUserConsumer(getSchedulerContext, getOrCreateExecutorService, group) - } - } - -} diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/timeout/JobTimeoutManager.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/timeout/JobTimeoutManager.scala index aaaf131bd8..4e62430316 100644 --- a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/timeout/JobTimeoutManager.scala +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/timeout/JobTimeoutManager.scala @@ -38,8 +38,8 @@ class JobTimeoutManager extends Logging { private[this] final val timeoutJobByName: ConcurrentMap[String, EntranceJob] = new ConcurrentHashMap[String, EntranceJob] - val timeoutCheck: Boolean = EntranceConfiguration.ENABLE_JOB_TIMEOUT_CHECK.getValue - val timeoutScanInterval: Int = EntranceConfiguration.TIMEOUT_SCAN_INTERVAL.getValue + private val timeoutCheck: Boolean = EntranceConfiguration.ENABLE_JOB_TIMEOUT_CHECK.getValue + private val timeoutScanInterval: Int = EntranceConfiguration.TIMEOUT_SCAN_INTERVAL.getValue def add(jobKey: String, job: EntranceJob): Unit = { logger.info(s"Adding timeout job: ${job.getId()}") @@ -77,75 +77,75 @@ class JobTimeoutManager extends Logging { } private def timeoutDetective(): Unit = { - if (timeoutCheck) { - def checkAndSwitch(job: EntranceJob): Unit = { - logger.info(s"Checking whether the job id ${job.getJobRequest.getId()} timed out. ") - val currentTimeSeconds = System.currentTimeMillis() / 1000 - // job.isWaiting == job in queue - val jobScheduleStartTimeSeconds = - if (job.isWaiting) job.createTime / 1000 else currentTimeSeconds - val queuingTimeSeconds = currentTimeSeconds - jobScheduleStartTimeSeconds - val jobRunningStartTimeSeconds = - if (job.getStartTime > 0) job.getStartTime / 1000 else currentTimeSeconds - val runningTimeSeconds = currentTimeSeconds - jobRunningStartTimeSeconds - if (!job.isCompleted) { - job.jobRequest.getLabels.asScala foreach { - case queueTimeOutLabel: JobQueuingTimeoutLabel => - if ( - job.isWaiting && queueTimeOutLabel.getQueuingTimeout > 0 && queuingTimeSeconds >= queueTimeOutLabel.getQueuingTimeout - ) { - logger.warn( - s"Job ${job.getJobRequest.getId()} queued time : ${queuingTimeSeconds} seconds, which was over queueTimeOut : ${queueTimeOutLabel.getQueuingTimeout} seconds, cancel it now! " - ) - job.onFailure( - s"Job queued ${queuingTimeSeconds} seconds over max queue time : ${queueTimeOutLabel.getQueuingTimeout} seconds.", - null - ) - } - case jobRunningTimeoutLabel: JobRunningTimeoutLabel => - if ( - job.isRunning && jobRunningTimeoutLabel.getRunningTimeout > 0 && runningTimeSeconds >= jobRunningTimeoutLabel.getRunningTimeout - ) { - logger.warn( - s"Job ${job.getJobRequest.getId()} run timeout ${runningTimeSeconds} seconds, which was over runTimeOut : ${jobRunningTimeoutLabel.getRunningTimeout} seconds, cancel it now! " - ) - job.onFailure( - s"Job run ${runningTimeSeconds} seconds over max run time : ${jobRunningTimeoutLabel.getRunningTimeout} seconds.", - null - ) - } - case _ => - } + def checkAndSwitch(job: EntranceJob): Unit = { + logger.info(s"Checking whether the job id ${job.getJobRequest.getId()} timed out. ") + val currentTimeSeconds = System.currentTimeMillis() / 1000 + // job.isWaiting == job in queue + val jobScheduleStartTimeSeconds = + if (job.isWaiting) job.createTime / 1000 else currentTimeSeconds + val queuingTimeSeconds = currentTimeSeconds - jobScheduleStartTimeSeconds + val jobRunningStartTimeSeconds = + if (job.getStartTime > 0) job.getStartTime / 1000 else currentTimeSeconds + val runningTimeSeconds = currentTimeSeconds - jobRunningStartTimeSeconds + if (!job.isCompleted) { + job.jobRequest.getLabels.asScala foreach { + case queueTimeOutLabel: JobQueuingTimeoutLabel => + if ( + job.isWaiting && queueTimeOutLabel.getQueuingTimeout > 0 && queuingTimeSeconds >= queueTimeOutLabel.getQueuingTimeout + ) { + logger.warn( + s"Job ${job.getJobRequest.getId()} queued time : ${queuingTimeSeconds} seconds, which was over queueTimeOut : ${queueTimeOutLabel.getQueuingTimeout} seconds, cancel it now! " + ) + job.onFailure( + s"Job queued ${queuingTimeSeconds} seconds over max queue time : ${queueTimeOutLabel.getQueuingTimeout} seconds.", + null + ) + } + case jobRunningTimeoutLabel: JobRunningTimeoutLabel => + if ( + job.isRunning && jobRunningTimeoutLabel.getRunningTimeout > 0 && runningTimeSeconds >= jobRunningTimeoutLabel.getRunningTimeout + ) { + logger.warn( + s"Job ${job.getJobRequest.getId()} run timeout ${runningTimeSeconds} seconds, which was over runTimeOut : ${jobRunningTimeoutLabel.getRunningTimeout} seconds, cancel it now! " + ) + job.onFailure( + s"Job run ${runningTimeSeconds} seconds over max run time : ${jobRunningTimeoutLabel.getRunningTimeout} seconds.", + null + ) + } + case _ => } } - - timeoutJobByName.asScala.foreach(item => { - logger.info(s"Running timeout detection!") - synchronized { - jobCompleteDelete(item._1) - if (jobExist(item._1)) checkAndSwitch(item._2) - } - }) } + + timeoutJobByName.asScala.foreach(item => { + logger.info(s"Running timeout detection!") + synchronized { + jobCompleteDelete(item._1) + if (jobExist(item._1)) checkAndSwitch(item._2) + } + }) } // Thread periodic scan timeout task - val woker = Utils.defaultScheduler.scheduleAtFixedRate( - new Runnable() { - - override def run(): Unit = { - Utils.tryCatch { - timeoutDetective() - } { case t: Throwable => - logger.error(s"TimeoutDetective task failed. ${t.getMessage}", t) + if (timeoutCheck) { + val woker = Utils.defaultScheduler.scheduleAtFixedRate( + new Runnable() { + + override def run(): Unit = { + Utils.tryCatch { + timeoutDetective() + } { case t: Throwable => + logger.warn(s"TimeoutDetective task failed. ${t.getMessage}", t) + } } - } - }, - 0, - timeoutScanInterval, - TimeUnit.SECONDS - ) + }, + 0, + timeoutScanInterval, + TimeUnit.SECONDS + ) + } } diff --git a/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/EntranceUtils.scala b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/EntranceUtils.scala new file mode 100644 index 0000000000..13dcefa9f9 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/main/scala/org/apache/linkis/entrance/utils/EntranceUtils.scala @@ -0,0 +1,108 @@ +/* + * 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.utils + +import org.apache.linkis.common.ServiceInstance +import org.apache.linkis.common.utils.{Logging, Utils} +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.engine.{EngineTypeLabel, UserCreatorLabel} +import org.apache.linkis.manager.label.entity.route.RouteLabel +import org.apache.linkis.manager.label.utils.EngineTypeLabelCreator +import org.apache.linkis.rpc.Sender + +import org.apache.commons.lang3.StringUtils + +import java.util + +import scala.collection.JavaConverters.asScalaBufferConverter + +object EntranceUtils extends Logging { + + private val SPLIT = "," + + private val labelFactory = LabelBuilderFactoryContext.getLabelBuilderFactory + + def getUserCreatorEcTypeKey( + userCreatorLabel: UserCreatorLabel, + engineTypeLabel: EngineTypeLabel + ): String = { + userCreatorLabel.getStringValue + SPLIT + engineTypeLabel.getStringValue + } + + def fromKeyGetLabels(key: String): (UserCreatorLabel, EngineTypeLabel) = { + if (StringUtils.isBlank(key)) (null, null) + else { + val labelStringValues = key.split(SPLIT) + if (labelStringValues.length < 2) return (null, null) + val userCreatorLabel = labelFactory + .createLabel[UserCreatorLabel](LabelKeyConstant.USER_CREATOR_TYPE_KEY, labelStringValues(0)) + val engineTypeLabel = labelFactory + .createLabel[EngineTypeLabel](LabelKeyConstant.ENGINE_TYPE_KEY, labelStringValues(1)) + (userCreatorLabel, engineTypeLabel) + } + } + + def getDefaultCreatorECTypeKey(creator: String, ecType: String): String = { + val userCreatorLabel = + labelFactory.createLabel[UserCreatorLabel](LabelKeyConstant.USER_CREATOR_TYPE_KEY) + val ecTypeLabel = EngineTypeLabelCreator.createEngineTypeLabel(ecType) + userCreatorLabel.setUser("*") + userCreatorLabel.setCreator(creator) + getUserCreatorEcTypeKey(userCreatorLabel, ecTypeLabel) + } + + def getRunningEntranceNumber(): Int = { + val 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 + } + val entranceRealNumber = if (null != offlineIns) { + logger.info(s"There are ${offlineIns.length} offlining instance.") + entranceNum - offlineIns.length + } else { + entranceNum + } + /* + Sender.getInstances may get 0 instances due to cache in Sender. So this instance is the one instance. + */ + if (entranceRealNumber <= 0) { + logger.error( + s"Got ${entranceRealNumber} ${Sender.getThisServiceInstance.getApplicationName} instances." + ) + 1 + } else { + entranceRealNumber + } + } + +} diff --git a/linkis-computation-governance/linkis-entrance/src/test/java/org/apache/linkis/entrance/interceptor/impl/SQLExplainTest.java b/linkis-computation-governance/linkis-entrance/src/test/java/org/apache/linkis/entrance/interceptor/impl/SQLExplainTest.java new file mode 100644 index 0000000000..c5efb5633e --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/test/java/org/apache/linkis/entrance/interceptor/impl/SQLExplainTest.java @@ -0,0 +1,55 @@ +/* + * 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.interceptor.impl; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class SQLExplainTest { + + @Test + void isSelectCmdNoLimit() { + + String code = "SELECT * from dual WHERE (1=1)LIMIT 1;"; + boolean res = SQLExplain.isSelectCmdNoLimit(code); + Assertions.assertEquals(false, res); + + code = "SELECT * from dual"; + res = SQLExplain.isSelectCmdNoLimit(code); + Assertions.assertEquals(true, res); + + code = "SELECT * from dual LIMIT 1;"; + res = SQLExplain.isSelectCmdNoLimit(code); + Assertions.assertEquals(false, res); + } + + @Test + void isSelectOverLimit() { + String code = "SELECT * from dual WHERE (1=1)LIMIT 5001;"; + boolean res = SQLExplain.isSelectOverLimit(code); + Assertions.assertEquals(true, res); + + code = "SELECT * from dual"; + res = SQLExplain.isSelectOverLimit(code); + Assertions.assertEquals(false, res); + + code = "SELECT * from dual LIMIT 4000;"; + res = SQLExplain.isSelectOverLimit(code); + Assertions.assertEquals(false, res); + } +} diff --git a/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfUtilsTest.java b/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfUtilsTest.java new file mode 100644 index 0000000000..8e4674d241 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TemplateConfUtilsTest.java @@ -0,0 +1,123 @@ +/* + * 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.interceptor.impl; + +import static org.junit.jupiter.api.Assertions.*; + +import org.junit.jupiter.api.Test; + +class TemplateConfUtilsTest { + + @Test + void getCustomTemplateConfName() { + String sqlCode = "" + + "--注解\n" + + "select * from table;\n" + + " --注解 \n" + + "--注解\n" + + " select \"--注解\" as test\n" + + " --@set yy=123\n" + + " --注解"; + + String res = TemplateConfUtils.getCustomTemplateConfName(sqlCode, "sql"); + assertEquals(res, ""); + + String sqlCode2 = "" + + "---@set 123=注解\n" + + "select * from table;\n" + + " --注解 \n" + + "--注解\n" + + " select \"--注解\" as test\n" + + " --@set yy=123\n" + + " --注解"; + + res = TemplateConfUtils.getCustomTemplateConfName(sqlCode2, "sql"); + assertEquals(res, ""); + + String sqlCode3 = "" + + "---@set ec.resource.name=345\n" + + "select * from table;\n" + + " --注解 \n" + + "--注解\n" + + "---@set ec.resource.name=456\n" + + " select \"--注解\" as test\n" + + " --@set yy=123\n" + + " --注解"; + + res = TemplateConfUtils.getCustomTemplateConfName(sqlCode3, "sql"); + assertEquals(res, "345"); + + String sqlCode4 = "" + + "---@set ec.resource.name= name1 \n" + + " select \"--注解\" as test\n" + + " --@set yy=123\n" + + " --注解"; + + res = TemplateConfUtils.getCustomTemplateConfName(sqlCode4, "sql"); + assertEquals(res, "name1"); + + String sqlCode5 = "" + + "##@set ec.resource.name=pyname1\n" + + "select * from table;\n" + + " --注解 \n" + + "#注解\n" + + "##@set ec.resource.name= 123 \n" + + " select \"--注解\" as test\n" + + "#@set yy=123\n" + + " #注解"; + + res = TemplateConfUtils.getCustomTemplateConfName(sqlCode5, "python"); + assertEquals(res, "pyname1"); + + String sqlCode6 = "" + + "///@set ec.resource.name= scalaname1 \n" + + " select \"//注解\" as test\n" + + "//@set yy=123\n" + + " #注解"; + + res = TemplateConfUtils.getCustomTemplateConfName(sqlCode6, "scala"); + assertEquals(res, "scalaname1"); + + String sqlCode7 = "" + + "---@set ec.resource.name= hqlname1 \n" + + " select \"--注解\" as test\n" + + " --@set yy=123\n" + + " --注解"; + + res = TemplateConfUtils.getCustomTemplateConfName(sqlCode7, "hql"); + assertEquals(res, "hqlname1"); + + String sqlCode8 = "---@set ec.resource.name=linkis_test2;\n" + + " ---@set ec.resource.name=scriptis_test hive;\n" + + " select * from dss autotest.demo data limit 100;"; + res=TemplateConfUtils.getCustomTemplateConfName(sqlCode8,"hql"); + assertEquals(res,"linkis_test2"); + + + } + @Test + void getCustomTemplateConfName2() { + + + String sqlCode9= "---@set ec.resource.name=linkis_test2;\r\n---@set ec.resource.name=scriptis_test_hive;\r\n--@set limitn=100\r\nselect * from dss_autotest.demo_data limit ${limitn};\r\n"; + + + + String res=TemplateConfUtils.getCustomTemplateConfName(sqlCode9,"hql"); + assertEquals(res,"linkis_test2"); + } +} \ No newline at end of file diff --git a/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TestCommentHelper.java b/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TestCommentHelper.java new file mode 100644 index 0000000000..34482d293c --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TestCommentHelper.java @@ -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.entrance.interceptor.impl; + +import static org.junit.jupiter.api.Assertions.*; + +import java.util.Arrays; +import org.apache.linkis.common.utils.CodeAndRunTypeUtils; +import org.junit.jupiter.api.Test; +import org.junit.platform.commons.util.StringUtils; +import scala.reflect.internal.TypeDebugging; + +public class TestCommentHelper { + String sqlCode="" + + "--注解\n" + + "select * from table;\n" + + " --注解 \n" + + "--注解\n" + + " select \"--注解\" as test\n" + + " --@set yy=123\n" + + " --注解"; + + String scalaCode="" + + "// 注解\n" + + "print(1+1)\n" + + "//@set yy=123\n" + + " print(2)\n" + + " // 注解 \n" + + "// test\n" + + "print(\"//注解测试\")"; + + String scalaCodeRes="print(1+1)\n" + + "print(2)\n" + + "print(\"//注解测试\")"; + + @Test + void sqlDealCommentTest() { + String code=SQLCommentHelper.dealComment(sqlCode); + //System.out.println(code); + } + + @Test + void scalaDealCommentTest() { + String code=ScalaCommentHelper.dealComment(scalaCode); + String[] lines = Arrays.stream(code.split("\n")) + .map(String::trim) + .filter(x -> StringUtils.isNotBlank(x)) + .toArray(String[]::new); + String result=String.join("\n",lines); + // assertEquals(result,scalaCodeRes); + } + + +} \ No newline at end of file diff --git a/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TestHDFSCacheLogWriter.java b/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TestHDFSCacheLogWriter.java new file mode 100644 index 0000000000..34826ff222 --- /dev/null +++ b/linkis-computation-governance/linkis-entrance/src/test/scala/org/apache/linkis/entrance/interceptor/impl/TestHDFSCacheLogWriter.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.entrance.interceptor.impl; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.linkis.cs.common.entity.source.ContextID; +import org.apache.linkis.entrance.log.Cache; +import org.apache.linkis.entrance.log.HDFSCacheLogWriter; +import org.apache.linkis.rpc.BaseRPCSender; +import org.apache.linkis.rpc.Sender; +import org.apache.linkis.storage.fs.FileSystem; +import org.apache.linkis.storage.fs.impl.LocalFileSystem; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.Mockito; +import oshi.util.FileUtil; + +class TestHDFSCacheLogWriter { + + + @Test + void write() throws IOException { + + Cache cache = new Cache(5); + String fileName= UUID.randomUUID().toString().replace("-", "")+"-test.log"; + String logPath = System.getProperty("java.io.tmpdir")+ File.separator+fileName; + System.out.println(logPath); + String chartSet="utf-8"; + String username=System.getProperty("user.name"); + + File file=new File(logPath); + file.createNewFile(); + + HDFSCacheLogWriter logWriter =new HDFSCacheLogWriter( + //"D:\\DataSphere\\linkis\\docs\\test.log", + logPath, + chartSet, + cache, + username + ); + + String[] msgArr =new String[]{"1","2","3","4","5","6","7","8","9","10","11","12","13","14","15","16","17","18"}; + + List msgList = new ArrayList(Arrays.asList(msgArr)); + String msg=String.join("\n", msgList); + + logWriter.write(msg); + logWriter.flush(); + + List list = FileUtil.readFile(logPath); + String res=String.join("\n", list); + + res=res.replace("\n\n","\n"); + res=StringUtils.strip(res, " \n"); + Assertions.assertEquals(res,msg); + + + } +} \ No newline at end of file diff --git a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelKeyConstant.java b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelKeyConstant.java index 8021b35851..fbaea61811 100644 --- a/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelKeyConstant.java +++ b/linkis-computation-governance/linkis-manager/linkis-label-common/src/main/java/org/apache/linkis/manager/label/constant/LabelKeyConstant.java @@ -66,5 +66,9 @@ public class LabelKeyConstant { public static final String ENGINGE_CONN_RUNTIME_MODE_KEY = "engingeConnRuntimeMode"; + public static final String TEMPLATE_CONF_KEY = "ec.conf.templateId"; + + public static final String TEMPLATE_CONF_NAME_KEY = "ec.resource.name"; + public static final String MANAGER_KEY = "manager"; }