From d7798a6dbacba34ea9e80edbc4f8717b974d74a1 Mon Sep 17 00:00:00 2001 From: Guangdong Liu <804167098@qq.com> Date: Mon, 28 Oct 2024 10:52:37 +0800 Subject: [PATCH] [Improve][RestApi] Unify the code logic of REST API V1 and V2 (#7920) --- .../rest/RestHttpGetCommandProcessor.java | 821 ++---------------- .../rest/RestHttpPostCommandProcessor.java | 241 +---- .../server/rest/service/BaseLogService.java | 165 ++++ .../server/rest/service/BaseService.java | 570 ++++++++++++ .../rest/service/EncryptConfigService.java | 42 + .../server/rest/service/JobInfoService.java | 183 ++++ .../server/rest/service/LogService.java | 128 +++ .../server/rest/service/OverviewService.java | 61 ++ .../rest/service/RunningThreadService.java | 45 + .../rest/service/SystemMonitoringService.java | 31 + .../rest/service/ThreadDumpService.java | 51 ++ .../rest/service/UpdateTagsService.java | 54 ++ .../rest/servlet/AllLogNameServlet.java | 17 +- .../rest/servlet/AllNodeLogServlet.java | 67 +- .../server/rest/servlet/BaseServlet.java | 518 ----------- .../rest/servlet/CurrentNodeLogServlet.java | 19 +- .../rest/servlet/EncryptConfigServlet.java | 22 +- .../rest/servlet/FinishedJobsServlet.java | 62 +- .../server/rest/servlet/JobInfoServlet.java | 51 +- .../server/rest/servlet/LogBaseServlet.java | 134 --- .../server/rest/servlet/OverviewServlet.java | 34 +- .../rest/servlet/RunningJobsServlet.java | 19 +- .../rest/servlet/RunningThreadsServlet.java | 25 +- .../server/rest/servlet/StopJobServlet.java | 16 +- .../server/rest/servlet/StopJobsServlet.java | 29 +- .../server/rest/servlet/SubmitJobServlet.java | 26 +- .../rest/servlet/SubmitJobsServlet.java | 38 +- .../rest/servlet/SystemMonitoringServlet.java | 9 +- .../rest/servlet/ThreadDumpServlet.java | 27 +- .../rest/servlet/UpdateTagsServlet.java | 32 +- 30 files changed, 1511 insertions(+), 2026 deletions(-) create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseLogService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/EncryptConfigService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/JobInfoService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/LogService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/OverviewService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/RunningThreadService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/SystemMonitoringService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/ThreadDumpService.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/UpdateTagsService.java diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index 1b9cb65a6a..85c4ed6b36 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -17,98 +17,40 @@ package org.apache.seatunnel.engine.server.rest; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; - -import org.apache.seatunnel.api.common.metrics.JobMetrics; -import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; -import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.FileUtils; import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.engine.common.Constant; -import org.apache.seatunnel.engine.common.env.EnvironmentUtil; -import org.apache.seatunnel.engine.common.env.Version; -import org.apache.seatunnel.engine.core.classloader.ClassLoaderService; -import org.apache.seatunnel.engine.core.dag.logical.LogicalDag; -import org.apache.seatunnel.engine.core.job.JobDAGInfo; -import org.apache.seatunnel.engine.core.job.JobImmutableInformation; -import org.apache.seatunnel.engine.core.job.JobInfo; -import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.NodeExtension; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.dag.DAGUtils; import org.apache.seatunnel.engine.server.log.FormatType; import org.apache.seatunnel.engine.server.log.Log4j2HttpGetCommandProcessor; -import org.apache.seatunnel.engine.server.master.JobHistoryService.JobState; -import org.apache.seatunnel.engine.server.operation.GetClusterHealthMetricsOperation; -import org.apache.seatunnel.engine.server.operation.GetJobMetricsOperation; -import org.apache.seatunnel.engine.server.operation.GetJobStatusOperation; -import org.apache.seatunnel.engine.server.resourcemanager.opeartion.GetOverviewOperation; -import org.apache.seatunnel.engine.server.resourcemanager.resource.OverviewInfo; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; +import org.apache.seatunnel.engine.server.rest.service.LogService; +import org.apache.seatunnel.engine.server.rest.service.OverviewService; +import org.apache.seatunnel.engine.server.rest.service.RunningThreadService; +import org.apache.seatunnel.engine.server.rest.service.SystemMonitoringService; +import org.apache.seatunnel.engine.server.rest.service.ThreadDumpService; -import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.core.LoggerContext; -import org.apache.logging.log4j.core.config.builder.api.Component; -import org.apache.logging.log4j.core.config.builder.impl.BuiltConfiguration; -import org.apache.logging.log4j.core.config.properties.PropertiesConfiguration; -import org.apache.logging.log4j.core.lookup.StrSubstitutor; -import com.hazelcast.cluster.Address; -import com.hazelcast.cluster.Cluster; -import com.hazelcast.cluster.Member; import com.hazelcast.internal.ascii.TextCommandService; import com.hazelcast.internal.ascii.rest.HttpCommandProcessor; import com.hazelcast.internal.ascii.rest.HttpGetCommand; import com.hazelcast.internal.ascii.rest.RestValue; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.internal.json.JsonObject; -import com.hazelcast.internal.json.JsonValue; import com.hazelcast.internal.util.JsonUtil; import com.hazelcast.internal.util.StringUtil; -import com.hazelcast.jet.datamodel.Tuple3; -import com.hazelcast.jet.impl.execution.init.CustomClassLoadedObject; -import com.hazelcast.map.IMap; -import com.hazelcast.spi.impl.NodeEngine; import com.hazelcast.spi.impl.NodeEngineImpl; import io.prometheus.client.exporter.common.TextFormat; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; -import java.io.InputStream; import java.io.StringWriter; -import java.lang.reflect.Field; -import java.net.HttpURLConnection; -import java.net.URL; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_400; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; import static org.apache.seatunnel.engine.server.rest.RestConstant.CONTEXT_PATH; import static org.apache.seatunnel.engine.server.rest.RestConstant.FINISHED_JOBS_INFO; import static org.apache.seatunnel.engine.server.rest.RestConstant.GET_ALL_LOG_NAME; @@ -120,14 +62,6 @@ import static org.apache.seatunnel.engine.server.rest.RestConstant.RUNNING_JOB_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.RUNNING_THREADS; import static org.apache.seatunnel.engine.server.rest.RestConstant.SYSTEM_MONITORING_INFORMATION; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_BYTES; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_BYTES_PER_SECONDS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_COUNT; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_QPS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_BYTES; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_COUNT; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_QPS; import static org.apache.seatunnel.engine.server.rest.RestConstant.TELEMETRY_METRICS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.TELEMETRY_OPEN_METRICS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.THREAD_DUMP; @@ -135,11 +69,24 @@ public class RestHttpGetCommandProcessor extends HttpCommandProcessor { private final Log4j2HttpGetCommandProcessor original; - private NodeEngine nodeEngine; + private NodeEngineImpl nodeEngine; + private OverviewService overviewService; + private JobInfoService jobInfoService; + private SystemMonitoringService systemMonitoringService; + private ThreadDumpService threadDumpService; + private RunningThreadService runningThreadService; + private LogService logService; public RestHttpGetCommandProcessor(TextCommandService textCommandService) { this(textCommandService, new Log4j2HttpGetCommandProcessor(textCommandService)); + this.nodeEngine = this.textCommandService.getNode().getNodeEngine(); + this.overviewService = new OverviewService(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); + this.systemMonitoringService = new SystemMonitoringService(nodeEngine); + this.threadDumpService = new ThreadDumpService(nodeEngine); + this.runningThreadService = new RunningThreadService(nodeEngine); + this.logService = new LogService(nodeEngine); } public RestHttpGetCommandProcessor( @@ -149,6 +96,13 @@ public RestHttpGetCommandProcessor( textCommandService, textCommandService.getNode().getLogger(Log4j2HttpGetCommandProcessor.class)); this.original = log4j2HttpGetCommandProcessor; + this.nodeEngine = this.textCommandService.getNode().getNodeEngine(); + this.overviewService = new OverviewService(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); + this.systemMonitoringService = new SystemMonitoringService(nodeEngine); + this.threadDumpService = new ThreadDumpService(nodeEngine); + this.runningThreadService = new RunningThreadService(nodeEngine); + this.logService = new LogService(nodeEngine); } @Override @@ -215,109 +169,26 @@ public void overView(HttpGetCommand command, String uri) { .map(variable -> variable.split("=", 2)) .filter(pair -> pair.length == 2) .collect(Collectors.toMap(pair -> pair[0], pair -> pair[1])); - Version version = EnvironmentUtil.getVersion(); - - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); - - OverviewInfo overviewInfo; - - if (seaTunnelServer == null) { - overviewInfo = - (OverviewInfo) - NodeEngineUtil.sendOperationToMasterNode( - getNode().nodeEngine, new GetOverviewOperation(tags)) - .join(); - overviewInfo.setProjectVersion(version.getProjectVersion()); - overviewInfo.setGitCommitAbbrev(version.getGitCommitAbbrev()); - } else { - - NodeEngineImpl nodeEngine = this.textCommandService.getNode().getNodeEngine(); - overviewInfo = GetOverviewOperation.getOverviewInfo(seaTunnelServer, nodeEngine, tags); - overviewInfo.setProjectVersion(version.getProjectVersion()); - overviewInfo.setGitCommitAbbrev(version.getGitCommitAbbrev()); - } this.prepareResponse( command, - JsonUtil.toJsonObject(JsonUtils.toMap(JsonUtils.toJsonString(overviewInfo)))); + JsonUtil.toJsonObject( + JsonUtils.toMap( + JsonUtils.toJsonString(overviewService.getOverviewInfo(tags))))); } public void getThreadDump(HttpGetCommand command) { - Map threadStacks = Thread.getAllStackTraces(); - JsonArray threadInfoList = new JsonArray(); - for (Map.Entry entry : threadStacks.entrySet()) { - StringBuilder stackTraceBuilder = new StringBuilder(); - for (StackTraceElement element : entry.getValue()) { - stackTraceBuilder.append(element.toString()).append("\n"); - } - String stackTrace = stackTraceBuilder.toString().trim(); - JsonObject threadInfo = new JsonObject(); - threadInfo.add("threadName", entry.getKey().getName()); - threadInfo.add("threadId", entry.getKey().getId()); - threadInfo.add("threadState", entry.getKey().getState().name()); - threadInfo.add("stackTrace", stackTrace); - threadInfoList.add(threadInfo); - } - this.prepareResponse(command, threadInfoList); + this.prepareResponse(command, threadDumpService.getThreadDump()); } private void getSystemMonitoringInformation(HttpGetCommand command) { - JsonArray jsonValues = getSystemMonitoringInformationJsonValues(); - this.prepareResponse(command, jsonValues); - } - - private JsonArray getSystemMonitoringInformationJsonValues() { - Cluster cluster = textCommandService.getNode().hazelcastInstance.getCluster(); - nodeEngine = textCommandService.getNode().hazelcastInstance.node.nodeEngine; - - Set members = cluster.getMembers(); - JsonArray jsonValues = - members.stream() - .map( - member -> { - Address address = member.getAddress(); - String input = null; - try { - input = - (String) - NodeEngineUtil.sendOperationToMemberNode( - nodeEngine, - new GetClusterHealthMetricsOperation(), - address) - .get(); - } catch (InterruptedException | ExecutionException e) { - logger.severe("get system monitoring information fail", e); - } - String[] parts = input.split(", "); - JsonObject jobInfo = new JsonObject(); - Arrays.stream(parts) - .forEach( - part -> { - String[] keyValue = part.split("="); - jobInfo.add(keyValue[0], keyValue[1]); - }); - return jobInfo; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - return jsonValues; + this.prepareResponse( + command, systemMonitoringService.getSystemMonitoringInformationJsonValues()); } private void handleRunningJobsInfo(HttpGetCommand command) { - IMap values = - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_RUNNING_JOB_INFO); - JsonArray jobs = - values.entrySet().stream() - .map( - jobInfoEntry -> - convertToJson( - jobInfoEntry.getValue(), jobInfoEntry.getKey())) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - this.prepareResponse(command, jobs); + this.prepareResponse(command, jobInfoService.getRunningJobsJson()); } private void handleFinishedJobsInfo(HttpGetCommand command, String uri) { @@ -332,318 +203,18 @@ private void handleFinishedJobsInfo(HttpGetCommand command, String uri) { state = uri.substring(indexEnd + 1); } - IMap finishedJob = - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_STATE); - - IMap finishedJobMetrics = - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_METRICS); - - IMap finishedJobDAGInfo = - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); - JsonArray jobs = - finishedJob.values().stream() - .filter( - jobState -> { - if (state.isEmpty()) { - return true; - } - return jobState.getJobStatus() - .name() - .equals(state.toUpperCase()); - }) - .sorted(Comparator.comparing(JobState::getFinishTime)) - .map( - jobState -> { - Long jobId = jobState.getJobId(); - String jobMetrics; - if (seaTunnelServer == null) { - jobMetrics = - (String) - NodeEngineUtil.sendOperationToMasterNode( - getNode().nodeEngine, - new GetJobMetricsOperation( - jobId)) - .join(); - } else { - jobMetrics = - seaTunnelServer - .getCoordinatorService() - .getJobMetrics(jobId) - .toJsonString(); - } - return getJobInfoJson( - jobState, jobMetrics, finishedJobDAGInfo.get(jobId)); - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - - this.prepareResponse(command, jobs); + this.prepareResponse(command, jobInfoService.getJobsByStateJson(state)); } private void handleJobInfoById(HttpGetCommand command, String uri) { uri = StringUtil.stripTrailingSlash(uri); int indexEnd = uri.indexOf('/', URI_MAPS.length()); String jobId = uri.substring(indexEnd + 1); - IMap jobInfoMap = - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_RUNNING_JOB_INFO); - JobInfo jobInfo = (JobInfo) jobInfoMap.get(Long.valueOf(jobId)); - IMap finishedJobStateMap = - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_STATE); - JobState finishedJobState = (JobState) finishedJobStateMap.get(Long.valueOf(jobId)); - if (!jobId.isEmpty() && jobInfo != null) { - this.prepareResponse(command, convertToJson(jobInfo, Long.parseLong(jobId))); - } else if (!jobId.isEmpty() && finishedJobState != null) { - JobMetrics finishedJobMetrics = - (JobMetrics) - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_METRICS) - .get(Long.valueOf(jobId)); - JobDAGInfo finishedJobDAGInfo = - (JobDAGInfo) - this.textCommandService - .getNode() - .getNodeEngine() - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO) - .get(Long.valueOf(jobId)); - this.prepareResponse( - command, - getJobInfoJson( - finishedJobState, - finishedJobMetrics.toJsonString(), - finishedJobDAGInfo)); - } else { - this.prepareResponse(command, new JsonObject().add(RestConstant.JOB_ID, jobId)); - } + this.prepareResponse(command, jobInfoService.getJobInfoJson(Long.valueOf(jobId))); } private void getRunningThread(HttpGetCommand command) { - this.prepareResponse( - command, - Thread.getAllStackTraces().keySet().stream() - .sorted(Comparator.comparing(Thread::getName)) - .map( - stackTraceElements -> { - JsonObject jobInfoJson = new JsonObject(); - jobInfoJson.add("threadName", stackTraceElements.getName()); - jobInfoJson.add( - "classLoader", - String.valueOf( - stackTraceElements.getContextClassLoader())); - return jobInfoJson; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add)); - } - - private Map getJobMetrics(String jobMetrics) { - Map metricsMap = new HashMap<>(); - // To add metrics, populate the corresponding array, - String[] countMetricsNames = { - SOURCE_RECEIVED_COUNT, SINK_WRITE_COUNT, SOURCE_RECEIVED_BYTES, SINK_WRITE_BYTES - }; - String[] rateMetricsNames = { - SOURCE_RECEIVED_QPS, - SINK_WRITE_QPS, - SOURCE_RECEIVED_BYTES_PER_SECONDS, - SINK_WRITE_BYTES_PER_SECONDS - }; - String[] tableCountMetricsNames = { - TABLE_SOURCE_RECEIVED_COUNT, - TABLE_SINK_WRITE_COUNT, - TABLE_SOURCE_RECEIVED_BYTES, - TABLE_SINK_WRITE_BYTES - }; - String[] tableRateMetricsNames = { - TABLE_SOURCE_RECEIVED_QPS, - TABLE_SINK_WRITE_QPS, - TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS, - TABLE_SINK_WRITE_BYTES_PER_SECONDS - }; - Long[] metricsSums = - Stream.generate(() -> 0L).limit(countMetricsNames.length).toArray(Long[]::new); - Double[] metricsRates = - Stream.generate(() -> 0D).limit(rateMetricsNames.length).toArray(Double[]::new); - - // Used to store various indicators at the table - Map[] tableMetricsMaps = - new Map[] { - new HashMap<>(), // Source Received Count - new HashMap<>(), // Sink Write Count - new HashMap<>(), // Source Received Bytes - new HashMap<>(), // Sink Write Bytes - new HashMap<>(), // Source Received QPS - new HashMap<>(), // Sink Write QPS - new HashMap<>(), // Source Received Bytes Per Second - new HashMap<>() // Sink Write Bytes Per Second - }; - - try { - JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); - - jobMetricsStr - .fieldNames() - .forEachRemaining( - metricName -> { - if (metricName.contains("#")) { - String tableName = - TablePath.of(metricName.split("#")[1]).getFullName(); - JsonNode metricNode = jobMetricsStr.get(metricName); - processMetric( - metricName, tableName, metricNode, tableMetricsMaps); - } - }); - - // Aggregation summary and rate metrics - aggregateMetrics( - jobMetricsStr, - metricsSums, - metricsRates, - ArrayUtils.addAll(countMetricsNames, rateMetricsNames)); - - } catch (JsonProcessingException e) { - return metricsMap; - } - - populateMetricsMap( - metricsMap, - tableMetricsMaps, - ArrayUtils.addAll(tableCountMetricsNames, tableRateMetricsNames), - countMetricsNames.length); - populateMetricsMap( - metricsMap, - Stream.concat(Arrays.stream(metricsSums), Arrays.stream(metricsRates)) - .toArray(Number[]::new), - ArrayUtils.addAll(countMetricsNames, rateMetricsNames), - metricsSums.length); - - return metricsMap; - } - - private void processMetric( - String metricName, - String tableName, - JsonNode metricNode, - Map[] tableMetricsMaps) { - if (metricNode == null) { - return; - } - - // Define index constant - final int SOURCE_COUNT_IDX = 0, - SINK_COUNT_IDX = 1, - SOURCE_BYTES_IDX = 2, - SINK_BYTES_IDX = 3, - SOURCE_QPS_IDX = 4, - SINK_QPS_IDX = 5, - SOURCE_BYTES_SEC_IDX = 6, - SINK_BYTES_SEC_IDX = 7; - if (metricName.startsWith(SOURCE_RECEIVED_COUNT + "#")) { - tableMetricsMaps[SOURCE_COUNT_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_COUNT + "#")) { - tableMetricsMaps[SINK_COUNT_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES + "#")) { - tableMetricsMaps[SOURCE_BYTES_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_BYTES + "#")) { - tableMetricsMaps[SINK_BYTES_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SOURCE_RECEIVED_QPS + "#")) { - tableMetricsMaps[SOURCE_QPS_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_QPS + "#")) { - tableMetricsMaps[SINK_QPS_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES_PER_SECONDS + "#")) { - tableMetricsMaps[SOURCE_BYTES_SEC_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_BYTES_PER_SECONDS + "#")) { - tableMetricsMaps[SINK_BYTES_SEC_IDX].put(tableName, metricNode); - } - } - - private void aggregateMetrics( - JsonNode jobMetricsStr, - Long[] metricsSums, - Double[] metricsRates, - String[] metricsNames) { - for (int i = 0; i < metricsNames.length; i++) { - JsonNode metricNode = jobMetricsStr.get(metricsNames[i]); - if (metricNode != null && metricNode.isArray()) { - for (JsonNode node : metricNode) { - // Match Rate Metrics vs. Value Metrics - if (i < metricsSums.length) { - metricsSums[i] += node.path("value").asLong(); - } else { - metricsRates[i - metricsSums.length] += node.path("value").asDouble(); - } - } - } - } - } - - private void populateMetricsMap( - Map metricsMap, - Object[] metrics, - String[] metricNames, - int countMetricNames) { - for (int i = 0; i < metrics.length; i++) { - if (metrics[i] != null) { - if (metrics[i] instanceof Map) { - metricsMap.put( - metricNames[i], - aggregateMap( - (Map) metrics[i], i >= countMetricNames)); - } else { - metricsMap.put(metricNames[i], metrics[i]); - } - } - } - } - - private Map aggregateMap(Map inputMap, boolean isRate) { - return isRate - ? inputMap.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> - StreamSupport.stream( - entry.getValue().spliterator(), - false) - .mapToDouble( - node -> - node.path("value") - .asDouble()) - .sum())) - : inputMap.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> - StreamSupport.stream( - entry.getValue().spliterator(), - false) - .mapToLong( - node -> node.path("value").asLong()) - .sum())); + this.prepareResponse(command, runningThreadService.getRunningThread()); } private void handleMetrics(HttpGetCommand httpGetCommand, String contentType) { @@ -668,146 +239,7 @@ private void handleMetrics(HttpGetCommand httpGetCommand, String contentType) { } } - private SeaTunnelServer getSeaTunnelServer(boolean shouldBeMaster) { - Map extensionServices = - this.textCommandService.getNode().getNodeExtension().createExtensionServices(); - SeaTunnelServer seaTunnelServer = - (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); - if (shouldBeMaster && !seaTunnelServer.isMasterNode()) { - return null; - } - return seaTunnelServer; - } - - private JsonObject convertToJson(JobInfo jobInfo, long jobId) { - - JsonObject jobInfoJson = new JsonObject(); - JobImmutableInformation jobImmutableInformation = - this.textCommandService - .getNode() - .getNodeEngine() - .getSerializationService() - .toObject( - this.textCommandService - .getNode() - .getNodeEngine() - .getSerializationService() - .toObject(jobInfo.getJobImmutableInformation())); - - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); - ClassLoaderService classLoaderService = - seaTunnelServer == null - ? getSeaTunnelServer(false).getClassLoaderService() - : seaTunnelServer.getClassLoaderService(); - ClassLoader classLoader = - classLoaderService.getClassLoader( - jobId, jobImmutableInformation.getPluginJarsUrls()); - LogicalDag logicalDag = - CustomClassLoadedObject.deserializeWithCustomClassLoader( - this.textCommandService.getNode().getNodeEngine().getSerializationService(), - classLoader, - jobImmutableInformation.getLogicalDag()); - classLoaderService.releaseClassLoader(jobId, jobImmutableInformation.getPluginJarsUrls()); - - String jobMetrics; - JobStatus jobStatus; - if (seaTunnelServer == null) { - jobMetrics = - (String) - NodeEngineUtil.sendOperationToMasterNode( - getNode().nodeEngine, new GetJobMetricsOperation(jobId)) - .join(); - jobStatus = - JobStatus.values()[ - (int) - NodeEngineUtil.sendOperationToMasterNode( - getNode().nodeEngine, - new GetJobStatusOperation(jobId)) - .join()]; - } else { - jobMetrics = - seaTunnelServer.getCoordinatorService().getJobMetrics(jobId).toJsonString(); - jobStatus = seaTunnelServer.getCoordinatorService().getJobStatus(jobId); - } - - JobDAGInfo jobDAGInfo = - DAGUtils.getJobDAGInfo( - logicalDag, - jobImmutableInformation, - getSeaTunnelServer(false).getSeaTunnelConfig().getEngineConfig(), - true); - - jobInfoJson - .add(RestConstant.JOB_ID, String.valueOf(jobId)) - .add(RestConstant.JOB_NAME, logicalDag.getJobConfig().getName()) - .add(RestConstant.JOB_STATUS, jobStatus.toString()) - .add( - RestConstant.CREATE_TIME, - DateTimeUtils.toString( - jobImmutableInformation.getCreateTime(), - DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) - .add(RestConstant.JOB_DAG, jobDAGInfo.toJsonObject()) - .add( - RestConstant.PLUGIN_JARS_URLS, - (JsonValue) - jobImmutableInformation.getPluginJarsUrls().stream() - .map( - url -> { - JsonObject jarUrl = new JsonObject(); - jarUrl.add( - RestConstant.JAR_PATH, url.toString()); - return jarUrl; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add)) - .add( - RestConstant.IS_START_WITH_SAVE_POINT, - jobImmutableInformation.isStartWithSavePoint()) - .add(RestConstant.METRICS, toJsonObject(getJobMetrics(jobMetrics))); - - return jobInfoJson; - } - - private JsonObject toJsonObject(Map jobMetrics) { - JsonObject members = new JsonObject(); - jobMetrics.forEach( - (key, value) -> { - if (value instanceof Map) { - members.add(key, toJsonObject((Map) value)); - } else { - members.add(key, value.toString()); - } - }); - return members; - } - - private JsonObject getJobInfoJson(JobState jobState, String jobMetrics, JobDAGInfo jobDAGInfo) { - return new JsonObject() - .add(RestConstant.JOB_ID, String.valueOf(jobState.getJobId())) - .add(RestConstant.JOB_NAME, jobState.getJobName()) - .add(RestConstant.JOB_STATUS, jobState.getJobStatus().toString()) - .add(RestConstant.ERROR_MSG, jobState.getErrorMessage()) - .add( - RestConstant.CREATE_TIME, - DateTimeUtils.toString( - jobState.getSubmitTime(), - DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) - .add( - RestConstant.FINISH_TIME, - DateTimeUtils.toString( - jobState.getFinishTime(), - DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) - .add(RestConstant.JOB_DAG, jobDAGInfo.toJsonObject()) - .add(RestConstant.PLUGIN_JARS_URLS, new JsonArray()) - .add(RestConstant.METRICS, toJsonObject(getJobMetrics(jobMetrics))); - } - - private PropertiesConfiguration getLogConfiguration() { - LoggerContext context = (LoggerContext) LogManager.getContext(false); - return (PropertiesConfiguration) context.getConfiguration(); - } - - private void getAllNodeLog(HttpGetCommand httpGetCommand, String uri) - throws NoSuchFieldException, IllegalAccessException { + private void getAllNodeLog(HttpGetCommand httpGetCommand, String uri) { // Analysis uri, get logName and jobId param String param = getParam(uri); @@ -815,73 +247,24 @@ private void getAllNodeLog(HttpGetCommand httpGetCommand, String uri) String logName = isLogFile ? param : StringUtils.EMPTY; String jobId = !isLogFile ? param : StringUtils.EMPTY; - String logPath = getLogPath(); + String logPath = logService.getLogPath(); if (StringUtils.isBlank(logPath)) { logger.warning( "Log file path is empty, no log file path configured in the current configuration file"); httpGetCommand.send404(); return; } - JsonArray systemMonitoringInformationJsonValues = - getSystemMonitoringInformationJsonValues(); if (StringUtils.isBlank(logName)) { - StringBuffer logLink = new StringBuffer(); - ArrayList> allLogNameList = new ArrayList<>(); - - systemMonitoringInformationJsonValues.forEach( - systemMonitoringInformation -> { - String host = systemMonitoringInformation.asObject().get("host").asString(); - int port = - Integer.valueOf( - systemMonitoringInformation - .asObject() - .get("port") - .asString()); - String url = "http://" + host + ":" + port + CONTEXT_PATH; - String allName = sendGet(url + GET_ALL_LOG_NAME); - logger.fine(String.format("Request: %s , Result: %s", url, allName)); - ArrayNode jsonNodes = JsonUtils.parseArray(allName); - - jsonNodes.forEach( - jsonNode -> { - String fileName = jsonNode.asText(); - if (StringUtils.isNotBlank(jobId) - && !fileName.contains(jobId)) { - return; - } - allLogNameList.add( - Tuple3.tuple3( - host + ":" + port, - url + GET_LOGS + "/" + fileName, - fileName)); - }); - }); FormatType formatType = getFormatType(uri); switch (formatType) { case JSON: - JsonArray jsonArray = - allLogNameList.stream() - .map( - tuple -> { - JsonObject jsonObject = new JsonObject(); - jsonObject.add("node", tuple.f0()); - jsonObject.add("logLink", tuple.f1()); - jsonObject.add("logName", tuple.f2()); - return jsonObject; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - this.prepareResponse(httpGetCommand, jsonArray); + this.prepareResponse(httpGetCommand, logService.allNodeLogFormatJson(jobId)); return; case HTML: default: - allLogNameList.forEach( - tuple -> - logLink.append( - buildLogLink( - tuple.f1(), tuple.f0() + "-" + tuple.f2()))); - String logContent = buildWebSiteContent(logLink); - this.prepareResponse(httpGetCommand, getRestValue(logContent)); + this.prepareResponse( + httpGetCommand, getRestValue(logService.allNodeLogFormatHtml(jobId))); } } else { prepareLogResponse(httpGetCommand, logPath, logName); @@ -919,69 +302,14 @@ private static RestValue getRestValue(String logContent) { return restValue; } - private static String buildWebSiteContent(StringBuffer logLink) { - return "Seatunnel log\n" - + "\n" - + "

Seatunnel log

\n" - + "
    \n" - + logLink.toString() - + "
\n" - + ""; - } - - private String getFileLogPath(PropertiesConfiguration config) - throws NoSuchFieldException, IllegalAccessException { - Field propertiesField = BuiltConfiguration.class.getDeclaredField("appendersComponent"); - propertiesField.setAccessible(true); - Component propertiesComponent = (Component) propertiesField.get(config); - StrSubstitutor substitutor = config.getStrSubstitutor(); - return propertiesComponent.getComponents().stream() - .filter(component -> "fileAppender".equals(component.getAttributes().get("name"))) - .map(component -> substitutor.replace(component.getAttributes().get("fileName"))) - .findFirst() - .orElse(null); - } - - /** Get configuration log path */ - private String getLogPath() throws NoSuchFieldException, IllegalAccessException { - String routingAppender = "routingAppender"; - String fileAppender = "fileAppender"; - PropertiesConfiguration config = getLogConfiguration(); - // Get routingAppender log file path - String routingLogFilePath = getRoutingLogFilePath(config); - - // Get fileAppender log file path - String fileLogPath = getFileLogPath(config); - String logRef = - config.getLoggerConfig(StringUtils.EMPTY).getAppenderRefs().stream() - .map(Object::toString) - .filter(ref -> ref.contains(routingAppender) || ref.contains(fileAppender)) - .findFirst() - .orElse(StringUtils.EMPTY); - if (logRef.equals(routingAppender)) { - return routingLogFilePath.substring(0, routingLogFilePath.lastIndexOf("/")); - } else if (logRef.equals(fileAppender)) { - return fileLogPath.substring(0, routingLogFilePath.lastIndexOf("/")); - } else { - logger.warning(String.format("Log file path is empty, get logRef : %s", logRef)); - return null; - } - } - /** Get Current Node Log By /log request */ - private void getCurrentNodeLog(HttpGetCommand httpGetCommand, String uri) - throws NoSuchFieldException, IllegalAccessException { + private void getCurrentNodeLog(HttpGetCommand httpGetCommand, String uri) { String logName = getParam(uri); - String logPath = getLogPath(); + String logPath = logService.getLogPath(); if (StringUtils.isBlank(logName)) { // Get Current Node Log List - List logFileList = FileUtils.listFile(logPath); - StringBuffer logLink = new StringBuffer(); - for (File file : logFileList) { - logLink.append(buildLogLink("log/" + file.getName(), file.getName())); - } - this.prepareResponse(httpGetCommand, getRestValue(buildWebSiteContent(logLink))); + this.prepareResponse(httpGetCommand, getRestValue(logService.currentNodeLog(uri))); } else { // Get Current Node Log Content prepareLogResponse(httpGetCommand, logPath, logName); @@ -1002,64 +330,15 @@ private void prepareLogResponse(HttpGetCommand httpGetCommand, String logPath, S } } - public String buildLogLink(String href, String name) { - return "
  • " + name + "
  • \n"; - } + private void getAllLogName(HttpGetCommand httpGetCommand) { - private static String sendGet(String urlString) { try { - HttpURLConnection connection = (HttpURLConnection) new URL(urlString).openConnection(); - connection.setRequestMethod("GET"); - connection.setConnectTimeout(5000); - connection.setReadTimeout(5000); - connection.connect(); - - if (connection.getResponseCode() == 200) { - try (InputStream is = connection.getInputStream(); - ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - byte[] buffer = new byte[1024]; - int len; - while ((len = is.read(buffer)) != -1) { - baos.write(buffer, 0, len); - } - return baos.toString(); - } - } - } catch (IOException e) { - e.printStackTrace(); - } - return null; - } - - private void getAllLogName(HttpGetCommand httpGetCommand) - throws NoSuchFieldException, IllegalAccessException { - String logPath = getLogPath(); - List logFileList = FileUtils.listFile(logPath); - List fileNameList = - logFileList.stream().map(File::getName).collect(Collectors.toList()); - try { - this.prepareResponse(httpGetCommand, JsonUtils.toJsonString(fileNameList)); + this.prepareResponse(httpGetCommand, JsonUtils.toJsonString(logService.allLogName())); } catch (SeaTunnelRuntimeException e) { httpGetCommand.send400(); - logger.warning(String.format("Log file name get failed, get log path: %s", logPath)); + logger.warning( + String.format( + "Log file name get failed, get log path: %s", logService.getLogPath())); } } - - private static String getRoutingLogFilePath(PropertiesConfiguration config) - throws NoSuchFieldException, IllegalAccessException { - Field propertiesField = BuiltConfiguration.class.getDeclaredField("appendersComponent"); - propertiesField.setAccessible(true); - Component propertiesComponent = (Component) propertiesField.get(config); - StrSubstitutor substitutor = config.getStrSubstitutor(); - return propertiesComponent.getComponents().stream() - .filter( - component -> - "routingAppender".equals(component.getAttributes().get("name"))) - .flatMap(component -> component.getComponents().stream()) - .flatMap(component -> component.getComponents().stream()) - .flatMap(component -> component.getComponents().stream()) - .map(component -> substitutor.replace(component.getAttributes().get("fileName"))) - .findFirst() - .orElse(null); - } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java index ed855568e8..47776d11db 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpPostCommandProcessor.java @@ -17,47 +17,19 @@ package org.apache.seatunnel.engine.server.rest; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigRenderOptions; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.env.EnvCommonOptions; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.core.starter.utils.ConfigShadeUtils; -import org.apache.seatunnel.engine.common.Constant; -import org.apache.seatunnel.engine.common.config.JobConfig; -import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; -import org.apache.seatunnel.engine.core.job.JobImmutableInformation; -import org.apache.seatunnel.engine.server.CoordinatorService; -import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.log.Log4j2HttpPostCommandProcessor; -import org.apache.seatunnel.engine.server.operation.CancelJobOperation; -import org.apache.seatunnel.engine.server.operation.SavePointJobOperation; -import org.apache.seatunnel.engine.server.operation.SubmitJobOperation; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.seatunnel.engine.server.rest.service.EncryptConfigService; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; +import org.apache.seatunnel.engine.server.rest.service.UpdateTagsService; import org.apache.seatunnel.engine.server.utils.RestUtil; -import org.apache.commons.lang3.StringUtils; - -import com.hazelcast.cluster.impl.MemberImpl; -import com.hazelcast.instance.impl.Node; import com.hazelcast.internal.ascii.TextCommandService; import com.hazelcast.internal.ascii.rest.HttpCommandProcessor; import com.hazelcast.internal.ascii.rest.HttpPostCommand; -import com.hazelcast.internal.json.Json; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.internal.json.JsonObject; -import com.hazelcast.internal.serialization.Data; -import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; -import scala.Tuple2; -import java.io.IOException; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_400; import static com.hazelcast.internal.ascii.rest.HttpStatusCode.SC_500; @@ -73,9 +45,17 @@ public class RestHttpPostCommandProcessor extends HttpCommandProcessor { private final Log4j2HttpPostCommandProcessor original; + private JobInfoService jobInfoService; + private EncryptConfigService encryptConfigService; + private UpdateTagsService updateTagsService; public RestHttpPostCommandProcessor(TextCommandService textCommandService) { this(textCommandService, new Log4j2HttpPostCommandProcessor(textCommandService)); + this.jobInfoService = new JobInfoService(this.textCommandService.getNode().getNodeEngine()); + this.encryptConfigService = + new EncryptConfigService(this.textCommandService.getNode().getNodeEngine()); + this.updateTagsService = + new UpdateTagsService(this.textCommandService.getNode().getNodeEngine()); } protected RestHttpPostCommandProcessor( @@ -85,6 +65,11 @@ protected RestHttpPostCommandProcessor( textCommandService, textCommandService.getNode().getLogger(Log4j2HttpPostCommandProcessor.class)); this.original = log4j2HttpPostCommandProcessor; + this.jobInfoService = new JobInfoService(this.textCommandService.getNode().getNodeEngine()); + this.encryptConfigService = + new EncryptConfigService(this.textCommandService.getNode().getNodeEngine()); + this.updateTagsService = + new UpdateTagsService(this.textCommandService.getNode().getNodeEngine()); } @Override @@ -115,217 +100,41 @@ public void handle(HttpPostCommand httpPostCommand) { this.textCommandService.sendResponse(httpPostCommand); } - private SeaTunnelServer getSeaTunnelServer() { - Map extensionServices = - this.textCommandService.getNode().getNodeExtension().createExtensionServices(); - return (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); - } - private void handleSubmitJobs(HttpPostCommand httpPostCommand) throws IllegalArgumentException { - List, Config>> configTuples = - RestUtil.buildConfigList(requestHandle(httpPostCommand.getData()), false); - - JsonArray jsonArray = - configTuples.stream() - .map( - tuple -> { - String urlParams = mapToUrlParams(tuple._1); - Map requestParams = new HashMap<>(); - RestUtil.buildRequestParams(requestParams, urlParams); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); - Node node = textCommandService.getNode(); - return submitJobInternal( - tuple._2, requestParams, seaTunnelServer, node); - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - prepareResponse(httpPostCommand, jsonArray); - } - - private String mapToUrlParams(Map params) { - return params.entrySet().stream() - .map(entry -> entry.getKey() + "=" + entry.getValue()) - .collect(Collectors.joining("&", "?", "")); + prepareResponse(httpPostCommand, jobInfoService.submitJobs(httpPostCommand.getData())); } private void handleSubmitJob(HttpPostCommand httpPostCommand, String uri) throws IllegalArgumentException { Map requestParams = new HashMap<>(); RestUtil.buildRequestParams(requestParams, uri); - Config config = RestUtil.buildConfig(requestHandle(httpPostCommand.getData()), false); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); - Node node = textCommandService.getNode(); - JsonObject jsonObject = submitJobInternal(config, requestParams, seaTunnelServer, node); - this.prepareResponse(httpPostCommand, jsonObject); - } - - private JsonObject submitJobInternal( - Config config, - Map requestParams, - SeaTunnelServer seaTunnelServer, - Node node) { - ReadonlyConfig envOptions = ReadonlyConfig.fromConfig(config.getConfig("env")); - String jobName = envOptions.get(EnvCommonOptions.JOB_NAME); - - JobConfig jobConfig = new JobConfig(); - jobConfig.setName( - StringUtils.isEmpty(requestParams.get(RestConstant.JOB_NAME)) - ? jobName - : requestParams.get(RestConstant.JOB_NAME)); - - boolean startWithSavePoint = - Boolean.parseBoolean(requestParams.get(RestConstant.IS_START_WITH_SAVE_POINT)); - String jobIdStr = requestParams.get(RestConstant.JOB_ID); - Long finalJobId = StringUtils.isNotBlank(jobIdStr) ? Long.parseLong(jobIdStr) : null; - RestJobExecutionEnvironment restJobExecutionEnvironment = - new RestJobExecutionEnvironment( - seaTunnelServer, jobConfig, config, node, startWithSavePoint, finalJobId); - JobImmutableInformation jobImmutableInformation = restJobExecutionEnvironment.build(); - long jobId = jobImmutableInformation.getJobId(); - if (!seaTunnelServer.isMasterNode()) { - - NodeEngineUtil.sendOperationToMasterNode( - node.nodeEngine, - new SubmitJobOperation( - jobId, - node.nodeEngine.toData(jobImmutableInformation), - jobImmutableInformation.isStartWithSavePoint())) - .join(); - - } else { - submitJob(node, seaTunnelServer, jobImmutableInformation, jobConfig); - } - - return new JsonObject() - .add(RestConstant.JOB_ID, String.valueOf(jobId)) - .add(RestConstant.JOB_NAME, jobConfig.getName()); + this.prepareResponse( + httpPostCommand, + jobInfoService.submitJob(requestParams, httpPostCommand.getData())); } private void handleStopJobs(HttpPostCommand command) { - List jobList = - JsonUtils.toList(requestHandle(command.getData()).toString(), Map.class); - JsonArray jsonResponse = new JsonArray(); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); - Node node = textCommandService.getNode(); - jobList.forEach( - job -> { - handleStopJob(job, seaTunnelServer, node); - jsonResponse.add( - new JsonObject() - .add(RestConstant.JOB_ID, (Long) job.get(RestConstant.JOB_ID))); - }); - this.prepareResponse(command, jsonResponse); + this.prepareResponse(command, jobInfoService.stopJobs(command.getData())); } private void handleStopJob(HttpPostCommand httpPostCommand) { - Map map = JsonUtils.toMap(requestHandle(httpPostCommand.getData())); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); - Node node = textCommandService.getNode(); - handleStopJob(map, seaTunnelServer, node); - this.prepareResponse( - httpPostCommand, - new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString())); - } - - private void handleStopJob( - Map map, SeaTunnelServer seaTunnelServer, Node node) { - boolean isStopWithSavePoint = false; - if (map.get(RestConstant.JOB_ID) == null) { - throw new IllegalArgumentException("jobId cannot be empty."); - } - long jobId = Long.parseLong(map.get(RestConstant.JOB_ID).toString()); - if (map.get(RestConstant.IS_STOP_WITH_SAVE_POINT) != null) { - isStopWithSavePoint = - Boolean.parseBoolean(map.get(RestConstant.IS_STOP_WITH_SAVE_POINT).toString()); - } - - if (!seaTunnelServer.isMasterNode()) { - if (isStopWithSavePoint) { - NodeEngineUtil.sendOperationToMasterNode( - node.nodeEngine, new SavePointJobOperation(jobId)) - .join(); - } else { - NodeEngineUtil.sendOperationToMasterNode( - node.nodeEngine, new CancelJobOperation(jobId)) - .join(); - } - - } else { - CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); - - if (isStopWithSavePoint) { - coordinatorService.savePoint(jobId); - } else { - coordinatorService.cancelJob(jobId); - } - } - log.info("Stop job with jobId: " + jobId); + this.prepareResponse(httpPostCommand, jobInfoService.stopJob(httpPostCommand.getData())); } private void handleEncrypt(HttpPostCommand httpPostCommand) { - Config config = RestUtil.buildConfig(requestHandle(httpPostCommand.getData()), true); - Config encryptConfig = ConfigShadeUtils.encryptConfig(config); - String encryptString = - encryptConfig.root().render(ConfigRenderOptions.concise().setJson(true)); - JsonObject jsonObject = Json.parse(encryptString).asObject(); - this.prepareResponse(httpPostCommand, jsonObject); + this.prepareResponse( + httpPostCommand, encryptConfigService.encryptConfig(httpPostCommand.getData())); } private void handleUpdateTags(HttpPostCommand httpPostCommand) { - Map params = JsonUtils.toMap(requestHandle(httpPostCommand.getData())); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(); - - NodeEngineImpl nodeEngine = seaTunnelServer.getNodeEngine(); - MemberImpl localMember = nodeEngine.getLocalMember(); - - Map tags = - params.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - value -> - value.getValue() != null - ? value.getValue().toString() - : "")); - localMember.updateAttribute(tags); this.prepareResponse( - httpPostCommand, - new JsonObject() - .add("status", ResponseType.SUCCESS.toString()) - .add("message", "update node tags done.")); + httpPostCommand, updateTagsService.updateTags(httpPostCommand.getData())); } @Override public void handleRejection(HttpPostCommand httpPostCommand) { handle(httpPostCommand); } - - private JsonNode requestHandle(byte[] requestBody) { - if (requestBody.length == 0) { - throw new IllegalArgumentException("Request body is empty."); - } - JsonNode requestBodyJsonNode; - try { - requestBodyJsonNode = RestUtil.convertByteToJsonNode(requestBody); - } catch (IOException e) { - throw new IllegalArgumentException("Invalid JSON format in request body."); - } - return requestBodyJsonNode; - } - - private void submitJob( - Node node, - SeaTunnelServer seaTunnelServer, - JobImmutableInformation jobImmutableInformation, - JobConfig jobConfig) { - CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); - Data data = node.nodeEngine.getSerializationService().toData(jobImmutableInformation); - PassiveCompletableFuture voidPassiveCompletableFuture = - coordinatorService.submitJob( - Long.parseLong(jobConfig.getJobContext().getJobId()), - data, - jobImmutableInformation.isStartWithSavePoint()); - voidPassiveCompletableFuture.join(); - } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseLogService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseLogService.java new file mode 100644 index 0000000000..7d582bf589 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseLogService.java @@ -0,0 +1,165 @@ +/* + * 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.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.common.utils.ExceptionUtils; + +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.LoggerContext; +import org.apache.logging.log4j.core.config.builder.api.Component; +import org.apache.logging.log4j.core.config.builder.impl.BuiltConfiguration; +import org.apache.logging.log4j.core.config.properties.PropertiesConfiguration; +import org.apache.logging.log4j.core.lookup.StrSubstitutor; + +import com.hazelcast.internal.util.StringUtil; +import com.hazelcast.spi.impl.NodeEngineImpl; +import lombok.extern.slf4j.Slf4j; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Field; +import java.net.HttpURLConnection; +import java.net.URL; + +@Slf4j +public class BaseLogService extends BaseService { + public BaseLogService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + /** Get configuration log path */ + public String getLogPath() { + try { + String routingAppender = "routingAppender"; + String fileAppender = "fileAppender"; + PropertiesConfiguration config = getLogConfiguration(); + // Get routingAppender log file path + String routingLogFilePath = getRoutingLogFilePath(config); + + // Get fileAppender log file path + String fileLogPath = getFileLogPath(config); + String logRef = + config.getLoggerConfig(StringUtils.EMPTY).getAppenderRefs().stream() + .map(Object::toString) + .filter( + ref -> + ref.contains(routingAppender) + || ref.contains(fileAppender)) + .findFirst() + .orElse(StringUtils.EMPTY); + if (logRef.equals(routingAppender)) { + return routingLogFilePath.substring(0, routingLogFilePath.lastIndexOf("/")); + } else if (logRef.equals(fileAppender)) { + return fileLogPath.substring(0, routingLogFilePath.lastIndexOf("/")); + } else { + log.warn(String.format("Log file path is empty, get logRef : %s", logRef)); + return null; + } + } catch (NoSuchFieldException | IllegalAccessException e) { + log.error("Get log path error,{}", ExceptionUtils.getMessage(e)); + return null; + } + } + + private String getFileLogPath(PropertiesConfiguration config) + throws NoSuchFieldException, IllegalAccessException { + Field propertiesField = BuiltConfiguration.class.getDeclaredField("appendersComponent"); + propertiesField.setAccessible(true); + Component propertiesComponent = (Component) propertiesField.get(config); + StrSubstitutor substitutor = config.getStrSubstitutor(); + return propertiesComponent.getComponents().stream() + .filter(component -> "fileAppender".equals(component.getAttributes().get("name"))) + .map(component -> substitutor.replace(component.getAttributes().get("fileName"))) + .findFirst() + .orElse(null); + } + + private String getRoutingLogFilePath(PropertiesConfiguration config) + throws NoSuchFieldException, IllegalAccessException { + Field propertiesField = BuiltConfiguration.class.getDeclaredField("appendersComponent"); + propertiesField.setAccessible(true); + Component propertiesComponent = (Component) propertiesField.get(config); + StrSubstitutor substitutor = config.getStrSubstitutor(); + return propertiesComponent.getComponents().stream() + .filter( + component -> + "routingAppender".equals(component.getAttributes().get("name"))) + .flatMap(component -> component.getComponents().stream()) + .flatMap(component -> component.getComponents().stream()) + .flatMap(component -> component.getComponents().stream()) + .map(component -> substitutor.replace(component.getAttributes().get("fileName"))) + .findFirst() + .orElse(null); + } + + private PropertiesConfiguration getLogConfiguration() { + LoggerContext context = (LoggerContext) LogManager.getContext(false); + return (PropertiesConfiguration) context.getConfiguration(); + } + + protected String sendGet(String urlString) { + try { + HttpURLConnection connection = (HttpURLConnection) new URL(urlString).openConnection(); + connection.setRequestMethod("GET"); + connection.setConnectTimeout(5000); + connection.setReadTimeout(5000); + connection.connect(); + + if (connection.getResponseCode() == 200) { + try (InputStream is = connection.getInputStream(); + ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + byte[] buffer = new byte[1024]; + int len; + while ((len = is.read(buffer)) != -1) { + baos.write(buffer, 0, len); + } + return baos.toString(); + } + } + } catch (IOException e) { + log.error("Send get Fail.{}", ExceptionUtils.getMessage(e)); + } + return null; + } + + public String getLogParam(String uri, String contextPath) { + uri = uri.substring(uri.indexOf(contextPath) + contextPath.length()); + uri = StringUtil.stripTrailingSlash(uri).substring(1); + int indexEnd = uri.indexOf('/'); + if (indexEnd != -1) { + return uri.substring(indexEnd + 1); + } + return ""; + } + + protected String buildLogLink(String href, String name) { + return "
  • " + name + "
  • \n"; + } + + protected String buildWebSiteContent(StringBuffer logLink) { + return "Seatunnel log\n" + + "\n" + + "

    Seatunnel log

    \n" + + "
      \n" + + logLink.toString() + + "
    \n" + + ""; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseService.java new file mode 100644 index 0000000000..b304f7fdde --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/BaseService.java @@ -0,0 +1,570 @@ +/* + * 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.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.env.EnvCommonOptions; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.DateTimeUtils; +import org.apache.seatunnel.engine.common.Constant; +import org.apache.seatunnel.engine.common.config.JobConfig; +import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; +import org.apache.seatunnel.engine.core.classloader.ClassLoaderService; +import org.apache.seatunnel.engine.core.dag.logical.LogicalDag; +import org.apache.seatunnel.engine.core.job.JobDAGInfo; +import org.apache.seatunnel.engine.core.job.JobImmutableInformation; +import org.apache.seatunnel.engine.core.job.JobInfo; +import org.apache.seatunnel.engine.core.job.JobStatus; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.dag.DAGUtils; +import org.apache.seatunnel.engine.server.master.JobHistoryService; +import org.apache.seatunnel.engine.server.operation.CancelJobOperation; +import org.apache.seatunnel.engine.server.operation.GetClusterHealthMetricsOperation; +import org.apache.seatunnel.engine.server.operation.GetJobMetricsOperation; +import org.apache.seatunnel.engine.server.operation.GetJobStatusOperation; +import org.apache.seatunnel.engine.server.operation.SavePointJobOperation; +import org.apache.seatunnel.engine.server.operation.SubmitJobOperation; +import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.apache.seatunnel.engine.server.rest.RestJobExecutionEnvironment; +import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.seatunnel.engine.server.utils.RestUtil; + +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.StringUtils; + +import com.hazelcast.cluster.Address; +import com.hazelcast.cluster.Cluster; +import com.hazelcast.cluster.Member; +import com.hazelcast.instance.impl.Node; +import com.hazelcast.internal.json.JsonArray; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.internal.json.JsonValue; +import com.hazelcast.internal.serialization.Data; +import com.hazelcast.internal.util.JsonUtil; +import com.hazelcast.jet.impl.execution.init.CustomClassLoadedObject; +import com.hazelcast.spi.impl.NodeEngineImpl; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_BYTES; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_BYTES_PER_SECONDS; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_COUNT; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_QPS; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_BYTES; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_COUNT; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_QPS; + +@Slf4j +public abstract class BaseService { + + protected final NodeEngineImpl nodeEngine; + + public BaseService(NodeEngineImpl nodeEngine) { + this.nodeEngine = nodeEngine; + } + + protected SeaTunnelServer getSeaTunnelServer(boolean shouldBeMaster) { + Map extensionServices = + nodeEngine.getNode().getNodeExtension().createExtensionServices(); + SeaTunnelServer seaTunnelServer = + (SeaTunnelServer) extensionServices.get(Constant.SEATUNNEL_SERVICE_NAME); + if (shouldBeMaster && !seaTunnelServer.isMasterNode()) { + return null; + } + return seaTunnelServer; + } + + protected JsonObject convertToJson(JobInfo jobInfo, long jobId) { + + JsonObject jobInfoJson = new JsonObject(); + JobImmutableInformation jobImmutableInformation = + nodeEngine + .getSerializationService() + .toObject( + nodeEngine + .getSerializationService() + .toObject(jobInfo.getJobImmutableInformation())); + + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); + ClassLoaderService classLoaderService = + seaTunnelServer == null + ? getSeaTunnelServer(false).getClassLoaderService() + : seaTunnelServer.getClassLoaderService(); + ClassLoader classLoader = + classLoaderService.getClassLoader( + jobId, jobImmutableInformation.getPluginJarsUrls()); + LogicalDag logicalDag = + CustomClassLoadedObject.deserializeWithCustomClassLoader( + nodeEngine.getSerializationService(), + classLoader, + jobImmutableInformation.getLogicalDag()); + classLoaderService.releaseClassLoader(jobId, jobImmutableInformation.getPluginJarsUrls()); + + String jobMetrics; + JobStatus jobStatus; + if (seaTunnelServer == null) { + jobMetrics = + (String) + NodeEngineUtil.sendOperationToMasterNode( + nodeEngine, new GetJobMetricsOperation(jobId)) + .join(); + jobStatus = + JobStatus.values()[ + (int) + NodeEngineUtil.sendOperationToMasterNode( + nodeEngine, new GetJobStatusOperation(jobId)) + .join()]; + } else { + jobMetrics = + seaTunnelServer.getCoordinatorService().getJobMetrics(jobId).toJsonString(); + jobStatus = seaTunnelServer.getCoordinatorService().getJobStatus(jobId); + } + + JobDAGInfo jobDAGInfo = + DAGUtils.getJobDAGInfo( + logicalDag, + jobImmutableInformation, + getSeaTunnelServer(false).getSeaTunnelConfig().getEngineConfig(), + true); + + jobInfoJson + .add(RestConstant.JOB_ID, String.valueOf(jobId)) + .add(RestConstant.JOB_NAME, logicalDag.getJobConfig().getName()) + .add(RestConstant.JOB_STATUS, jobStatus.toString()) + .add( + RestConstant.ENV_OPTIONS, + JsonUtil.toJsonObject(logicalDag.getJobConfig().getEnvOptions())) + .add( + RestConstant.CREATE_TIME, + DateTimeUtils.toString( + jobImmutableInformation.getCreateTime(), + DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) + .add(RestConstant.JOB_DAG, jobDAGInfo.toJsonObject()) + .add( + RestConstant.PLUGIN_JARS_URLS, + (JsonValue) + jobImmutableInformation.getPluginJarsUrls().stream() + .map( + url -> { + JsonObject jarUrl = new JsonObject(); + jarUrl.add( + RestConstant.JAR_PATH, url.toString()); + return jarUrl; + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add)) + .add( + RestConstant.IS_START_WITH_SAVE_POINT, + jobImmutableInformation.isStartWithSavePoint()) + .add(RestConstant.METRICS, metricsToJsonObject(getJobMetrics(jobMetrics))); + + return jobInfoJson; + } + + protected JsonObject getJobInfoJson( + JobHistoryService.JobState jobState, String jobMetrics, JobDAGInfo jobDAGInfo) { + return new JsonObject() + .add(RestConstant.JOB_ID, String.valueOf(jobState.getJobId())) + .add(RestConstant.JOB_NAME, jobState.getJobName()) + .add(RestConstant.JOB_STATUS, jobState.getJobStatus().toString()) + .add(RestConstant.ERROR_MSG, jobState.getErrorMessage()) + .add( + RestConstant.CREATE_TIME, + DateTimeUtils.toString( + jobState.getSubmitTime(), + DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) + .add( + RestConstant.FINISH_TIME, + DateTimeUtils.toString( + jobState.getFinishTime(), + DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) + .add(RestConstant.JOB_DAG, jobDAGInfo.toJsonObject()) + .add(RestConstant.PLUGIN_JARS_URLS, new JsonArray()) + .add(RestConstant.METRICS, metricsToJsonObject(getJobMetrics(jobMetrics))); + } + + private Map getJobMetrics(String jobMetrics) { + Map metricsMap = new HashMap<>(); + // To add metrics, populate the corresponding array, + String[] countMetricsNames = { + SOURCE_RECEIVED_COUNT, SINK_WRITE_COUNT, SOURCE_RECEIVED_BYTES, SINK_WRITE_BYTES + }; + String[] rateMetricsNames = { + SOURCE_RECEIVED_QPS, + SINK_WRITE_QPS, + SOURCE_RECEIVED_BYTES_PER_SECONDS, + SINK_WRITE_BYTES_PER_SECONDS + }; + String[] tableCountMetricsNames = { + TABLE_SOURCE_RECEIVED_COUNT, + TABLE_SINK_WRITE_COUNT, + TABLE_SOURCE_RECEIVED_BYTES, + TABLE_SINK_WRITE_BYTES + }; + String[] tableRateMetricsNames = { + TABLE_SOURCE_RECEIVED_QPS, + TABLE_SINK_WRITE_QPS, + TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS, + TABLE_SINK_WRITE_BYTES_PER_SECONDS + }; + Long[] metricsSums = + Stream.generate(() -> 0L).limit(countMetricsNames.length).toArray(Long[]::new); + Double[] metricsRates = + Stream.generate(() -> 0D).limit(rateMetricsNames.length).toArray(Double[]::new); + + // Used to store various indicators at the table + Map[] tableMetricsMaps = + new Map[] { + new HashMap<>(), // Source Received Count + new HashMap<>(), // Sink Write Count + new HashMap<>(), // Source Received Bytes + new HashMap<>(), // Sink Write Bytes + new HashMap<>(), // Source Received QPS + new HashMap<>(), // Sink Write QPS + new HashMap<>(), // Source Received Bytes Per Second + new HashMap<>() // Sink Write Bytes Per Second + }; + + try { + JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); + + jobMetricsStr + .fieldNames() + .forEachRemaining( + metricName -> { + if (metricName.contains("#")) { + String tableName = + TablePath.of(metricName.split("#")[1]).getFullName(); + JsonNode metricNode = jobMetricsStr.get(metricName); + processMetric( + metricName, tableName, metricNode, tableMetricsMaps); + } + }); + + // Aggregation summary and rate metrics + aggregateMetrics( + jobMetricsStr, + metricsSums, + metricsRates, + ArrayUtils.addAll(countMetricsNames, rateMetricsNames)); + + } catch (JsonProcessingException e) { + return metricsMap; + } + + populateMetricsMap( + metricsMap, + tableMetricsMaps, + ArrayUtils.addAll(tableCountMetricsNames, tableRateMetricsNames), + countMetricsNames.length); + populateMetricsMap( + metricsMap, + Stream.concat(Arrays.stream(metricsSums), Arrays.stream(metricsRates)) + .toArray(Number[]::new), + ArrayUtils.addAll(countMetricsNames, rateMetricsNames), + metricsSums.length); + + return metricsMap; + } + + private void processMetric( + String metricName, + String tableName, + JsonNode metricNode, + Map[] tableMetricsMaps) { + if (metricNode == null) { + return; + } + + // Define index constant + final int SOURCE_COUNT_IDX = 0, + SINK_COUNT_IDX = 1, + SOURCE_BYTES_IDX = 2, + SINK_BYTES_IDX = 3, + SOURCE_QPS_IDX = 4, + SINK_QPS_IDX = 5, + SOURCE_BYTES_SEC_IDX = 6, + SINK_BYTES_SEC_IDX = 7; + if (metricName.startsWith(SOURCE_RECEIVED_COUNT + "#")) { + tableMetricsMaps[SOURCE_COUNT_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_COUNT + "#")) { + tableMetricsMaps[SINK_COUNT_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES + "#")) { + tableMetricsMaps[SOURCE_BYTES_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_BYTES + "#")) { + tableMetricsMaps[SINK_BYTES_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SOURCE_RECEIVED_QPS + "#")) { + tableMetricsMaps[SOURCE_QPS_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_QPS + "#")) { + tableMetricsMaps[SINK_QPS_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES_PER_SECONDS + "#")) { + tableMetricsMaps[SOURCE_BYTES_SEC_IDX].put(tableName, metricNode); + } else if (metricName.startsWith(SINK_WRITE_BYTES_PER_SECONDS + "#")) { + tableMetricsMaps[SINK_BYTES_SEC_IDX].put(tableName, metricNode); + } + } + + private void aggregateMetrics( + JsonNode jobMetricsStr, + Long[] metricsSums, + Double[] metricsRates, + String[] metricsNames) { + for (int i = 0; i < metricsNames.length; i++) { + JsonNode metricNode = jobMetricsStr.get(metricsNames[i]); + if (metricNode != null && metricNode.isArray()) { + for (JsonNode node : metricNode) { + // Match Rate Metrics vs. Value Metrics + if (i < metricsSums.length) { + metricsSums[i] += node.path("value").asLong(); + } else { + metricsRates[i - metricsSums.length] += node.path("value").asDouble(); + } + } + } + } + } + + private void populateMetricsMap( + Map metricsMap, + Object[] metrics, + String[] metricNames, + int countMetricNames) { + for (int i = 0; i < metrics.length; i++) { + if (metrics[i] != null) { + if (metrics[i] instanceof Map) { + metricsMap.put( + metricNames[i], + aggregateMap( + (Map) metrics[i], i >= countMetricNames)); + } else { + metricsMap.put(metricNames[i], metrics[i]); + } + } + } + } + + private Map aggregateMap(Map inputMap, boolean isRate) { + return isRate + ? inputMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToDouble( + node -> + node.path("value") + .asDouble()) + .sum())) + : inputMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.path("value").asLong()) + .sum())); + } + + private JsonObject metricsToJsonObject(Map jobMetrics) { + JsonObject members = new JsonObject(); + jobMetrics.forEach( + (key, value) -> { + if (value instanceof Map) { + members.add(key, metricsToJsonObject((Map) value)); + } else { + members.add(key, value.toString()); + } + }); + return members; + } + + protected JsonNode requestHandle(byte[] requestBody) { + if (requestBody.length == 0) { + throw new IllegalArgumentException("Request body is empty."); + } + JsonNode requestBodyJsonNode; + try { + requestBodyJsonNode = RestUtil.convertByteToJsonNode(requestBody); + } catch (IOException e) { + throw new IllegalArgumentException("Invalid JSON format in request body."); + } + return requestBodyJsonNode; + } + + protected void handleStopJob( + Map map, SeaTunnelServer seaTunnelServer, Node node) { + boolean isStopWithSavePoint = false; + if (map.get(RestConstant.JOB_ID) == null) { + throw new IllegalArgumentException("jobId cannot be empty."); + } + long jobId = Long.parseLong(map.get(RestConstant.JOB_ID).toString()); + if (map.get(RestConstant.IS_STOP_WITH_SAVE_POINT) != null) { + isStopWithSavePoint = + Boolean.parseBoolean(map.get(RestConstant.IS_STOP_WITH_SAVE_POINT).toString()); + } + + if (!seaTunnelServer.isMasterNode()) { + if (isStopWithSavePoint) { + NodeEngineUtil.sendOperationToMasterNode( + node.nodeEngine, new SavePointJobOperation(jobId)) + .join(); + } else { + NodeEngineUtil.sendOperationToMasterNode( + node.nodeEngine, new CancelJobOperation(jobId)) + .join(); + } + + } else { + CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); + + if (isStopWithSavePoint) { + coordinatorService.savePoint(jobId); + } else { + coordinatorService.cancelJob(jobId); + } + } + } + + protected String mapToUrlParams(Map params) { + return params.entrySet().stream() + .map(entry -> entry.getKey() + "=" + entry.getValue()) + .collect(Collectors.joining("&", "?", "")); + } + + protected JsonObject submitJobInternal( + Config config, + Map requestParams, + SeaTunnelServer seaTunnelServer, + Node node) { + ReadonlyConfig envOptions = ReadonlyConfig.fromConfig(config.getConfig("env")); + String jobName = envOptions.get(EnvCommonOptions.JOB_NAME); + + JobConfig jobConfig = new JobConfig(); + jobConfig.setName( + StringUtils.isEmpty(requestParams.get(RestConstant.JOB_NAME)) + ? jobName + : requestParams.get(RestConstant.JOB_NAME)); + + boolean startWithSavePoint = + Boolean.parseBoolean(requestParams.get(RestConstant.IS_START_WITH_SAVE_POINT)); + String jobIdStr = requestParams.get(RestConstant.JOB_ID); + Long finalJobId = StringUtils.isNotBlank(jobIdStr) ? Long.parseLong(jobIdStr) : null; + RestJobExecutionEnvironment restJobExecutionEnvironment = + new RestJobExecutionEnvironment( + seaTunnelServer, jobConfig, config, node, startWithSavePoint, finalJobId); + JobImmutableInformation jobImmutableInformation = restJobExecutionEnvironment.build(); + long jobId = jobImmutableInformation.getJobId(); + if (!seaTunnelServer.isMasterNode()) { + + NodeEngineUtil.sendOperationToMasterNode( + node.nodeEngine, + new SubmitJobOperation( + jobId, + node.nodeEngine.toData(jobImmutableInformation), + jobImmutableInformation.isStartWithSavePoint())) + .join(); + + } else { + submitJob(node, seaTunnelServer, jobImmutableInformation, jobConfig); + } + + return new JsonObject() + .add(RestConstant.JOB_ID, String.valueOf(jobId)) + .add(RestConstant.JOB_NAME, jobConfig.getName()); + } + + private void submitJob( + Node node, + SeaTunnelServer seaTunnelServer, + JobImmutableInformation jobImmutableInformation, + JobConfig jobConfig) { + CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); + Data data = node.nodeEngine.getSerializationService().toData(jobImmutableInformation); + PassiveCompletableFuture voidPassiveCompletableFuture = + coordinatorService.submitJob( + Long.parseLong(jobConfig.getJobContext().getJobId()), + data, + jobImmutableInformation.isStartWithSavePoint()); + voidPassiveCompletableFuture.join(); + } + + protected JsonArray getSystemMonitoringInformationJsonValues() { + Cluster cluster = nodeEngine.getHazelcastInstance().getCluster(); + + Set members = cluster.getMembers(); + JsonArray jsonValues = + members.stream() + .map( + member -> { + Address address = member.getAddress(); + String input = null; + try { + input = + (String) + NodeEngineUtil.sendOperationToMemberNode( + nodeEngine, + new GetClusterHealthMetricsOperation(), + address) + .get(); + } catch (InterruptedException | ExecutionException e) { + + log.error("Failed to get cluster health metrics", e); + } + String[] parts = input.split(", "); + JsonObject jobInfo = new JsonObject(); + Arrays.stream(parts) + .forEach( + part -> { + String[] keyValue = part.split("="); + jobInfo.add(keyValue[0], keyValue[1]); + }); + return jobInfo; + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + return jsonValues; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/EncryptConfigService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/EncryptConfigService.java new file mode 100644 index 0000000000..bbe8fe980b --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/EncryptConfigService.java @@ -0,0 +1,42 @@ +/* + * 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.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigRenderOptions; + +import org.apache.seatunnel.core.starter.utils.ConfigShadeUtils; +import org.apache.seatunnel.engine.server.utils.RestUtil; + +import com.hazelcast.internal.json.Json; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.spi.impl.NodeEngineImpl; + +public class EncryptConfigService extends BaseService { + public EncryptConfigService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public JsonObject encryptConfig(byte[] requestBody) { + Config config = RestUtil.buildConfig(requestHandle(requestBody), true); + Config encryptConfig = ConfigShadeUtils.encryptConfig(config); + String encryptString = + encryptConfig.root().render(ConfigRenderOptions.concise().setJson(true)); + return Json.parse(encryptString).asObject(); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/JobInfoService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/JobInfoService.java new file mode 100644 index 0000000000..2c497510ca --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/JobInfoService.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.common.metrics.JobMetrics; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.engine.common.Constant; +import org.apache.seatunnel.engine.core.job.JobDAGInfo; +import org.apache.seatunnel.engine.core.job.JobInfo; +import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.master.JobHistoryService.JobState; +import org.apache.seatunnel.engine.server.operation.GetJobMetricsOperation; +import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.seatunnel.engine.server.utils.RestUtil; + +import com.hazelcast.internal.json.JsonArray; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.map.IMap; +import com.hazelcast.spi.impl.NodeEngineImpl; +import scala.Tuple2; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class JobInfoService extends BaseService { + + public JobInfoService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public JsonObject getJobInfoJson(Long jobId) { + IMap jobInfoMap = + nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_RUNNING_JOB_INFO); + JobInfo jobInfo = (JobInfo) jobInfoMap.get(jobId); + + IMap finishedJobStateMap = + nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_STATE); + JobState finishedJobState = (JobState) finishedJobStateMap.get(jobId); + + if (jobInfo != null) { + return convertToJson(jobInfo, jobId); + } else if (finishedJobState != null) { + JobMetrics finishedJobMetrics = + (JobMetrics) + nodeEngine + .getHazelcastInstance() + .getMap(Constant.IMAP_FINISHED_JOB_METRICS) + .get(jobId); + JobDAGInfo finishedJobDAGInfo = + (JobDAGInfo) + nodeEngine + .getHazelcastInstance() + .getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO) + .get(jobId); + return getJobInfoJson( + finishedJobState, finishedJobMetrics.toJsonString(), finishedJobDAGInfo); + } else { + return new JsonObject().add(RestConstant.JOB_ID, jobId.toString()); + } + } + + public JsonArray getJobsByStateJson(String state) { + IMap finishedJob = + nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_STATE); + + IMap finishedJobDAGInfo = + nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO); + + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); + + return finishedJob.values().stream() + .filter( + jobState -> { + if (state.isEmpty()) { + return true; + } + return jobState.getJobStatus().name().equals(state.toUpperCase()); + }) + .sorted(Comparator.comparing(JobState::getFinishTime)) + .map( + jobState -> { + Long jobId = jobState.getJobId(); + String jobMetrics; + if (seaTunnelServer == null) { + jobMetrics = + (String) + NodeEngineUtil.sendOperationToMasterNode( + nodeEngine, + new GetJobMetricsOperation(jobId)) + .join(); + } else { + jobMetrics = + seaTunnelServer + .getCoordinatorService() + .getJobMetrics(jobId) + .toJsonString(); + } + return getJobInfoJson( + jobState, jobMetrics, finishedJobDAGInfo.get(jobId)); + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + } + + public JsonArray getRunningJobsJson() { + IMap values = + nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_RUNNING_JOB_INFO); + return values.entrySet().stream() + .map(jobInfoEntry -> convertToJson(jobInfoEntry.getValue(), jobInfoEntry.getKey())) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + } + + public JsonObject stopJob(byte[] requestBody) { + Map map = JsonUtils.toMap(requestHandle(requestBody)); + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); + handleStopJob(map, seaTunnelServer, nodeEngine.getNode()); + return new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString()); + } + + public JsonArray stopJobs(byte[] requestBody) { + JsonArray jsonResponse = new JsonArray(); + List jobList = JsonUtils.toList(requestHandle(requestBody).toString(), Map.class); + + jobList.forEach( + job -> { + handleStopJob(job, getSeaTunnelServer(false), nodeEngine.getNode()); + jsonResponse.add( + new JsonObject() + .add(RestConstant.JOB_ID, (Long) job.get(RestConstant.JOB_ID))); + }); + + return jsonResponse; + } + + public JsonObject submitJob(Map requestParams, byte[] requestBody) { + + if (Boolean.parseBoolean(requestParams.get(RestConstant.IS_START_WITH_SAVE_POINT)) + && requestParams.get(RestConstant.JOB_ID) == null) { + throw new IllegalArgumentException("Please provide jobId when start with save point."); + } + + Config config = RestUtil.buildConfig(requestHandle(requestBody), false); + + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); + return submitJobInternal(config, requestParams, seaTunnelServer, nodeEngine.getNode()); + } + + public JsonArray submitJobs(byte[] requestBody) { + List, Config>> configTuples = + RestUtil.buildConfigList(requestHandle(requestBody), false); + + return configTuples.stream() + .map( + tuple -> { + String urlParams = mapToUrlParams(tuple._1); + Map requestParams = new HashMap<>(); + RestUtil.buildRequestParams(requestParams, urlParams); + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); + return submitJobInternal( + tuple._2, requestParams, seaTunnelServer, nodeEngine.getNode()); + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/LogService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/LogService.java new file mode 100644 index 0000000000..ea1ee57188 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/LogService.java @@ -0,0 +1,128 @@ +/* + * 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.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; + +import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.engine.common.config.server.HttpConfig; +import org.apache.seatunnel.engine.server.SeaTunnelServer; + +import org.apache.commons.lang3.StringUtils; + +import com.hazelcast.internal.json.JsonArray; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.spi.impl.NodeEngineImpl; +import lombok.extern.slf4j.Slf4j; +import scala.Tuple3; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.engine.server.rest.RestConstant.GET_ALL_LOG_NAME; +import static org.apache.seatunnel.engine.server.rest.RestConstant.GET_LOGS; + +@Slf4j +public class LogService extends BaseLogService { + public LogService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public List allLogName() { + String logPath = getLogPath(); + List logFileList = FileUtils.listFile(logPath); + if (logFileList == null) { + return null; + } + return logFileList.stream().map(File::getName).collect(Collectors.toList()); + } + + public List> allLogNameList(String jobId) { + + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); + HttpConfig httpConfig = + seaTunnelServer.getSeaTunnelConfig().getEngineConfig().getHttpConfig(); + String contextPath = httpConfig.getContextPath(); + int port = httpConfig.getPort(); + + List> allLogNameList = new ArrayList<>(); + + JsonArray systemMonitoringInformationJsonValues = + getSystemMonitoringInformationJsonValues(); + systemMonitoringInformationJsonValues.forEach( + systemMonitoringInformation -> { + String host = systemMonitoringInformation.asObject().get("host").asString(); + String url = "http://" + host + ":" + port + contextPath; + String allName = sendGet(url + GET_ALL_LOG_NAME); + log.debug(String.format("Request: %s , Result: %s", url, allName)); + ArrayNode jsonNodes = JsonUtils.parseArray(allName); + + jsonNodes.forEach( + jsonNode -> { + String fileName = jsonNode.asText(); + if (StringUtils.isNotBlank(jobId) && !fileName.contains(jobId)) { + return; + } + allLogNameList.add( + new Tuple3<>( + host + ":" + port, + url + GET_LOGS + "/" + fileName, + fileName)); + }); + }); + + return allLogNameList; + } + + public JsonArray allNodeLogFormatJson(String jobId) { + + return allLogNameList(jobId).stream() + .map( + tuple -> { + JsonObject jsonObject = new JsonObject(); + jsonObject.add("node", tuple._1()); + jsonObject.add("logLink", tuple._2()); + jsonObject.add("logName", tuple._3()); + return jsonObject; + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + } + + public String allNodeLogFormatHtml(String jobId) { + StringBuffer logLink = new StringBuffer(); + + allLogNameList(jobId) + .forEach(tuple -> logLink.append(buildLogLink(tuple._2(), tuple._3()))); + return buildWebSiteContent(logLink); + } + + public String currentNodeLog(String uri) { + List logFileList = FileUtils.listFile(getLogPath()); + StringBuffer logLink = new StringBuffer(); + if (logFileList != null) { + for (File file : logFileList) { + logLink.append(buildLogLink("log/" + file.getName(), file.getName())); + } + } + + return buildWebSiteContent(logLink); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/OverviewService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/OverviewService.java new file mode 100644 index 0000000000..ab92ff8291 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/OverviewService.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.engine.common.env.EnvironmentUtil; +import org.apache.seatunnel.engine.common.env.Version; +import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.resourcemanager.opeartion.GetOverviewOperation; +import org.apache.seatunnel.engine.server.resourcemanager.resource.OverviewInfo; +import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; + +import com.hazelcast.spi.impl.NodeEngineImpl; + +import java.util.Map; + +public class OverviewService extends BaseService { + + private final NodeEngineImpl nodeEngine; + + public OverviewService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + this.nodeEngine = nodeEngine; + } + + public OverviewInfo getOverviewInfo(Map tags) { + Version version = EnvironmentUtil.getVersion(); + OverviewInfo overviewInfo; + + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); + + if (seaTunnelServer == null) { + overviewInfo = + (OverviewInfo) + NodeEngineUtil.sendOperationToMasterNode( + nodeEngine, new GetOverviewOperation(tags)) + .join(); + } else { + overviewInfo = GetOverviewOperation.getOverviewInfo(seaTunnelServer, nodeEngine, tags); + } + + overviewInfo.setProjectVersion(version.getProjectVersion()); + overviewInfo.setGitCommitAbbrev(version.getGitCommitAbbrev()); + + return overviewInfo; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/RunningThreadService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/RunningThreadService.java new file mode 100644 index 0000000000..f7f1b2b5c3 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/RunningThreadService.java @@ -0,0 +1,45 @@ +/* + * 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.seatunnel.engine.server.rest.service; + +import com.hazelcast.internal.json.JsonArray; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.spi.impl.NodeEngineImpl; + +import java.util.Comparator; + +public class RunningThreadService extends BaseService { + public RunningThreadService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public JsonArray getRunningThread() { + return Thread.getAllStackTraces().keySet().stream() + .sorted(Comparator.comparing(Thread::getName)) + .map( + stackTraceElements -> { + JsonObject jobInfoJson = new JsonObject(); + jobInfoJson.add("threadName", stackTraceElements.getName()); + jobInfoJson.add( + "classLoader", + String.valueOf(stackTraceElements.getContextClassLoader())); + return jobInfoJson; + }) + .collect(JsonArray::new, JsonArray::add, JsonArray::add); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/SystemMonitoringService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/SystemMonitoringService.java new file mode 100644 index 0000000000..f007b1f8c4 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/SystemMonitoringService.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.rest.service; + +import com.hazelcast.internal.json.JsonArray; +import com.hazelcast.spi.impl.NodeEngineImpl; + +public class SystemMonitoringService extends BaseService { + public SystemMonitoringService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public JsonArray getSystemMonitoringInformationJsonValues() { + return super.getSystemMonitoringInformationJsonValues(); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/ThreadDumpService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/ThreadDumpService.java new file mode 100644 index 0000000000..53900d4385 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/ThreadDumpService.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.rest.service; + +import com.hazelcast.internal.json.JsonArray; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.spi.impl.NodeEngineImpl; + +import java.util.Map; + +public class ThreadDumpService extends BaseService { + public ThreadDumpService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public JsonArray getThreadDump() { + + Map threadStacks = Thread.getAllStackTraces(); + JsonArray threadInfoList = new JsonArray(); + for (Map.Entry entry : threadStacks.entrySet()) { + StringBuilder stackTraceBuilder = new StringBuilder(); + for (StackTraceElement element : entry.getValue()) { + stackTraceBuilder.append(element.toString()).append("\n"); + } + String stackTrace = stackTraceBuilder.toString().trim(); + JsonObject threadInfo = new JsonObject(); + threadInfo.add("threadName", entry.getKey().getName()); + threadInfo.add("threadId", entry.getKey().getId()); + threadInfo.add("threadState", entry.getKey().getState().name()); + threadInfo.add("stackTrace", stackTrace); + threadInfoList.add(threadInfo); + } + + return threadInfoList; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/UpdateTagsService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/UpdateTagsService.java new file mode 100644 index 0000000000..2b8fa692b7 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/service/UpdateTagsService.java @@ -0,0 +1,54 @@ +/* + * 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.seatunnel.engine.server.rest.service; + +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.engine.server.SeaTunnelServer; + +import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.internal.json.JsonObject; +import com.hazelcast.spi.impl.NodeEngineImpl; + +import java.util.Map; +import java.util.stream.Collectors; + +public class UpdateTagsService extends BaseService { + public UpdateTagsService(NodeEngineImpl nodeEngine) { + super(nodeEngine); + } + + public JsonObject updateTags(byte[] requestBody) { + Map params = JsonUtils.toMap(requestHandle(requestBody)); + SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); + + NodeEngineImpl nodeEngine = seaTunnelServer.getNodeEngine(); + MemberImpl localMember = nodeEngine.getLocalMember(); + + Map tags = + params.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + value -> + value.getValue() != null + ? value.getValue().toString() + : "")); + localMember.updateAttribute(tags); + return new JsonObject().add("status", "success").add("message", "update node tags done."); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllLogNameServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllLogNameServlet.java index 2022095886..1135c4bf92 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllLogNameServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllLogNameServlet.java @@ -18,7 +18,7 @@ package org.apache.seatunnel.engine.server.rest.servlet; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; -import org.apache.seatunnel.common.utils.FileUtils; +import org.apache.seatunnel.engine.server.rest.service.LogService; import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; @@ -27,31 +27,26 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import java.io.File; import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; @Slf4j public class AllLogNameServlet extends LogBaseServlet { + private final LogService logService; + public AllLogNameServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.logService = new LogService(nodeEngine); } @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - - String logPath = getLogPath(); - List logFileList = FileUtils.listFile(logPath); - List fileNameList = - logFileList.stream().map(File::getName).collect(Collectors.toList()); try { - writeJson(resp, fileNameList); + writeJson(resp, logService.allLogName()); } catch (SeaTunnelRuntimeException e) { resp.setStatus(HttpServletResponse.SC_NOT_FOUND); - log.warn(String.format("Log file name get failed, get log path: %s", logPath)); + log.warn("Log file name get failed, get log path: {}", logService.getLogPath()); } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllNodeLogServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllNodeLogServlet.java index a9932d85b3..fbc1434d91 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllNodeLogServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/AllNodeLogServlet.java @@ -17,18 +17,13 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.node.ArrayNode; - -import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.engine.common.config.server.HttpConfig; import org.apache.seatunnel.engine.server.SeaTunnelServer; import org.apache.seatunnel.engine.server.log.FormatType; +import org.apache.seatunnel.engine.server.rest.service.LogService; import org.apache.commons.lang3.StringUtils; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.internal.json.JsonObject; -import com.hazelcast.jet.datamodel.Tuple3; import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; @@ -37,16 +32,15 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.ArrayList; - -import static org.apache.seatunnel.engine.server.rest.RestConstant.GET_ALL_LOG_NAME; -import static org.apache.seatunnel.engine.server.rest.RestConstant.GET_LOGS; @Slf4j public class AllNodeLogServlet extends LogBaseServlet { + private final LogService logService; + public AllNodeLogServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.logService = new LogService(nodeEngine); } @Override @@ -57,71 +51,26 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) HttpConfig httpConfig = seaTunnelServer.getSeaTunnelConfig().getEngineConfig().getHttpConfig(); String contextPath = httpConfig.getContextPath(); - int port = httpConfig.getPort(); String uri = req.getRequestURI(); // Analysis uri, get logName and jobId param - String param = getLogParam(uri, contextPath); + String param = logService.getLogParam(uri, contextPath); boolean isLogFile = param.contains(".log"); String logName = isLogFile ? param : StringUtils.EMPTY; String jobId = !isLogFile ? param : StringUtils.EMPTY; - String logPath = getLogPath(); - JsonArray systemMonitoringInformationJsonValues = - getSystemMonitoringInformationJsonValues(); + String logPath = logService.getLogPath(); if (StringUtils.isBlank(logName)) { - StringBuffer logLink = new StringBuffer(); - ArrayList> allLogNameList = new ArrayList<>(); - - systemMonitoringInformationJsonValues.forEach( - systemMonitoringInformation -> { - String host = systemMonitoringInformation.asObject().get("host").asString(); - String url = "http://" + host + ":" + port + contextPath; - String allName = sendGet(url + GET_ALL_LOG_NAME); - log.debug(String.format("Request: %s , Result: %s", url, allName)); - ArrayNode jsonNodes = JsonUtils.parseArray(allName); - - jsonNodes.forEach( - jsonNode -> { - String fileName = jsonNode.asText(); - if (StringUtils.isNotBlank(jobId) - && !fileName.contains(jobId)) { - return; - } - allLogNameList.add( - Tuple3.tuple3( - host + ":" + port, - url + GET_LOGS + "/" + fileName, - fileName)); - }); - }); FormatType formatType = FormatType.fromString(req.getParameter("format")); switch (formatType) { case JSON: - JsonArray jsonArray = - allLogNameList.stream() - .map( - tuple -> { - JsonObject jsonObject = new JsonObject(); - jsonObject.add("node", tuple.f0()); - jsonObject.add("logLink", tuple.f1()); - jsonObject.add("logName", tuple.f2()); - return jsonObject; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - writeJson(resp, jsonArray); + writeJson(resp, logService.allNodeLogFormatHtml(jobId)); return; case HTML: default: - allLogNameList.forEach( - tuple -> - logLink.append( - buildLogLink( - tuple.f1(), tuple.f0() + "-" + tuple.f2()))); - String logContent = buildWebSiteContent(logLink); - writeHtml(resp, logContent); + writeHtml(resp, logService.allNodeLogFormatHtml(jobId)); } } else { prepareLogResponse(resp, logPath, logName); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/BaseServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/BaseServlet.java index df59741146..b37883f7c3 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/BaseServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/BaseServlet.java @@ -17,53 +17,12 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.shade.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; -import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.env.EnvCommonOptions; -import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.engine.common.Constant; -import org.apache.seatunnel.engine.common.config.JobConfig; -import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture; -import org.apache.seatunnel.engine.core.classloader.ClassLoaderService; -import org.apache.seatunnel.engine.core.dag.logical.LogicalDag; -import org.apache.seatunnel.engine.core.job.JobDAGInfo; -import org.apache.seatunnel.engine.core.job.JobImmutableInformation; -import org.apache.seatunnel.engine.core.job.JobInfo; -import org.apache.seatunnel.engine.core.job.JobStatus; -import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.dag.DAGUtils; -import org.apache.seatunnel.engine.server.master.JobHistoryService; -import org.apache.seatunnel.engine.server.operation.CancelJobOperation; -import org.apache.seatunnel.engine.server.operation.GetClusterHealthMetricsOperation; -import org.apache.seatunnel.engine.server.operation.GetJobMetricsOperation; -import org.apache.seatunnel.engine.server.operation.GetJobStatusOperation; -import org.apache.seatunnel.engine.server.operation.SavePointJobOperation; -import org.apache.seatunnel.engine.server.operation.SubmitJobOperation; -import org.apache.seatunnel.engine.server.rest.RestConstant; -import org.apache.seatunnel.engine.server.rest.RestJobExecutionEnvironment; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; -import org.apache.seatunnel.engine.server.utils.RestUtil; - -import org.apache.commons.lang3.ArrayUtils; -import org.apache.commons.lang3.StringUtils; import com.google.gson.Gson; -import com.hazelcast.cluster.Address; -import com.hazelcast.cluster.Cluster; -import com.hazelcast.cluster.Member; -import com.hazelcast.instance.impl.Node; import com.hazelcast.internal.json.JsonArray; import com.hazelcast.internal.json.JsonObject; -import com.hazelcast.internal.json.JsonValue; -import com.hazelcast.internal.serialization.Data; -import com.hazelcast.internal.util.JsonUtil; -import com.hazelcast.jet.impl.execution.init.CustomClassLoadedObject; import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; @@ -74,31 +33,8 @@ import java.io.BufferedReader; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; - -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_BYTES_PER_SECONDS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_COUNT; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SINK_WRITE_QPS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_BYTES_PER_SECONDS; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_COUNT; -import static org.apache.seatunnel.api.common.metrics.MetricNames.SOURCE_RECEIVED_QPS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_BYTES; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_BYTES_PER_SECONDS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_COUNT; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SINK_WRITE_QPS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_BYTES; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_COUNT; -import static org.apache.seatunnel.engine.server.rest.RestConstant.TABLE_SOURCE_RECEIVED_QPS; @Slf4j public class BaseServlet extends HttpServlet { @@ -155,92 +91,6 @@ protected void writeHtml(HttpServletResponse resp, Object obj) throws IOExceptio resp.getWriter().write(obj.toString()); } - protected JsonObject convertToJson(JobInfo jobInfo, long jobId) { - - JsonObject jobInfoJson = new JsonObject(); - JobImmutableInformation jobImmutableInformation = - nodeEngine - .getSerializationService() - .toObject( - nodeEngine - .getSerializationService() - .toObject(jobInfo.getJobImmutableInformation())); - - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); - ClassLoaderService classLoaderService = - seaTunnelServer == null - ? getSeaTunnelServer(false).getClassLoaderService() - : seaTunnelServer.getClassLoaderService(); - ClassLoader classLoader = - classLoaderService.getClassLoader( - jobId, jobImmutableInformation.getPluginJarsUrls()); - LogicalDag logicalDag = - CustomClassLoadedObject.deserializeWithCustomClassLoader( - nodeEngine.getSerializationService(), - classLoader, - jobImmutableInformation.getLogicalDag()); - classLoaderService.releaseClassLoader(jobId, jobImmutableInformation.getPluginJarsUrls()); - - String jobMetrics; - JobStatus jobStatus; - if (seaTunnelServer == null) { - jobMetrics = - (String) - NodeEngineUtil.sendOperationToMasterNode( - nodeEngine, new GetJobMetricsOperation(jobId)) - .join(); - jobStatus = - JobStatus.values()[ - (int) - NodeEngineUtil.sendOperationToMasterNode( - nodeEngine, new GetJobStatusOperation(jobId)) - .join()]; - } else { - jobMetrics = - seaTunnelServer.getCoordinatorService().getJobMetrics(jobId).toJsonString(); - jobStatus = seaTunnelServer.getCoordinatorService().getJobStatus(jobId); - } - - JobDAGInfo jobDAGInfo = - DAGUtils.getJobDAGInfo( - logicalDag, - jobImmutableInformation, - getSeaTunnelServer(false).getSeaTunnelConfig().getEngineConfig(), - true); - - jobInfoJson - .add(RestConstant.JOB_ID, String.valueOf(jobId)) - .add(RestConstant.JOB_NAME, logicalDag.getJobConfig().getName()) - .add(RestConstant.JOB_STATUS, jobStatus.toString()) - .add( - RestConstant.ENV_OPTIONS, - JsonUtil.toJsonObject(logicalDag.getJobConfig().getEnvOptions())) - .add( - RestConstant.CREATE_TIME, - DateTimeUtils.toString( - jobImmutableInformation.getCreateTime(), - DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) - .add(RestConstant.JOB_DAG, jobDAGInfo.toJsonObject()) - .add( - RestConstant.PLUGIN_JARS_URLS, - (JsonValue) - jobImmutableInformation.getPluginJarsUrls().stream() - .map( - url -> { - JsonObject jarUrl = new JsonObject(); - jarUrl.add( - RestConstant.JAR_PATH, url.toString()); - return jarUrl; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add)) - .add( - RestConstant.IS_START_WITH_SAVE_POINT, - jobImmutableInformation.isStartWithSavePoint()) - .add(RestConstant.METRICS, metricsToJsonObject(getJobMetrics(jobMetrics))); - - return jobInfoJson; - } - protected SeaTunnelServer getSeaTunnelServer(boolean shouldBeMaster) { Map extensionServices = nodeEngine.getNode().getNodeExtension().createExtensionServices(); @@ -281,372 +131,4 @@ protected Map getParameterMap(HttpServletRequest req) { } return reqParameterMap; } - - protected JsonNode requestHandle(byte[] requestBody) { - if (requestBody.length == 0) { - throw new IllegalArgumentException("Request body is empty."); - } - JsonNode requestBodyJsonNode; - try { - requestBodyJsonNode = RestUtil.convertByteToJsonNode(requestBody); - } catch (IOException e) { - throw new IllegalArgumentException("Invalid JSON format in request body."); - } - return requestBodyJsonNode; - } - - protected JsonObject getJobInfoJson( - JobHistoryService.JobState jobState, String jobMetrics, JobDAGInfo jobDAGInfo) { - return new JsonObject() - .add(RestConstant.JOB_ID, String.valueOf(jobState.getJobId())) - .add(RestConstant.JOB_NAME, jobState.getJobName()) - .add(RestConstant.JOB_STATUS, jobState.getJobStatus().toString()) - .add(RestConstant.ERROR_MSG, jobState.getErrorMessage()) - .add( - RestConstant.CREATE_TIME, - DateTimeUtils.toString( - jobState.getSubmitTime(), - DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) - .add( - RestConstant.FINISH_TIME, - DateTimeUtils.toString( - jobState.getFinishTime(), - DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS)) - .add(RestConstant.JOB_DAG, jobDAGInfo.toJsonObject()) - .add(RestConstant.PLUGIN_JARS_URLS, new JsonArray()) - .add(RestConstant.METRICS, metricsToJsonObject(getJobMetrics(jobMetrics))); - } - - private Map getJobMetrics(String jobMetrics) { - Map metricsMap = new HashMap<>(); - // To add metrics, populate the corresponding array, - String[] countMetricsNames = { - SOURCE_RECEIVED_COUNT, SINK_WRITE_COUNT, SOURCE_RECEIVED_BYTES, SINK_WRITE_BYTES - }; - String[] rateMetricsNames = { - SOURCE_RECEIVED_QPS, - SINK_WRITE_QPS, - SOURCE_RECEIVED_BYTES_PER_SECONDS, - SINK_WRITE_BYTES_PER_SECONDS - }; - String[] tableCountMetricsNames = { - TABLE_SOURCE_RECEIVED_COUNT, - TABLE_SINK_WRITE_COUNT, - TABLE_SOURCE_RECEIVED_BYTES, - TABLE_SINK_WRITE_BYTES - }; - String[] tableRateMetricsNames = { - TABLE_SOURCE_RECEIVED_QPS, - TABLE_SINK_WRITE_QPS, - TABLE_SOURCE_RECEIVED_BYTES_PER_SECONDS, - TABLE_SINK_WRITE_BYTES_PER_SECONDS - }; - Long[] metricsSums = - Stream.generate(() -> 0L).limit(countMetricsNames.length).toArray(Long[]::new); - Double[] metricsRates = - Stream.generate(() -> 0D).limit(rateMetricsNames.length).toArray(Double[]::new); - - // Used to store various indicators at the table - Map[] tableMetricsMaps = - new Map[] { - new HashMap<>(), // Source Received Count - new HashMap<>(), // Sink Write Count - new HashMap<>(), // Source Received Bytes - new HashMap<>(), // Sink Write Bytes - new HashMap<>(), // Source Received QPS - new HashMap<>(), // Sink Write QPS - new HashMap<>(), // Source Received Bytes Per Second - new HashMap<>() // Sink Write Bytes Per Second - }; - - try { - JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); - - jobMetricsStr - .fieldNames() - .forEachRemaining( - metricName -> { - if (metricName.contains("#")) { - String tableName = - TablePath.of(metricName.split("#")[1]).getFullName(); - JsonNode metricNode = jobMetricsStr.get(metricName); - processMetric( - metricName, tableName, metricNode, tableMetricsMaps); - } - }); - - // Aggregation summary and rate metrics - aggregateMetrics( - jobMetricsStr, - metricsSums, - metricsRates, - ArrayUtils.addAll(countMetricsNames, rateMetricsNames)); - - } catch (JsonProcessingException e) { - return metricsMap; - } - - populateMetricsMap( - metricsMap, - tableMetricsMaps, - ArrayUtils.addAll(tableCountMetricsNames, tableRateMetricsNames), - countMetricsNames.length); - populateMetricsMap( - metricsMap, - Stream.concat(Arrays.stream(metricsSums), Arrays.stream(metricsRates)) - .toArray(Number[]::new), - ArrayUtils.addAll(countMetricsNames, rateMetricsNames), - metricsSums.length); - - return metricsMap; - } - - private void processMetric( - String metricName, - String tableName, - JsonNode metricNode, - Map[] tableMetricsMaps) { - if (metricNode == null) { - return; - } - - // Define index constant - final int SOURCE_COUNT_IDX = 0, - SINK_COUNT_IDX = 1, - SOURCE_BYTES_IDX = 2, - SINK_BYTES_IDX = 3, - SOURCE_QPS_IDX = 4, - SINK_QPS_IDX = 5, - SOURCE_BYTES_SEC_IDX = 6, - SINK_BYTES_SEC_IDX = 7; - if (metricName.startsWith(SOURCE_RECEIVED_COUNT + "#")) { - tableMetricsMaps[SOURCE_COUNT_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_COUNT + "#")) { - tableMetricsMaps[SINK_COUNT_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES + "#")) { - tableMetricsMaps[SOURCE_BYTES_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_BYTES + "#")) { - tableMetricsMaps[SINK_BYTES_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SOURCE_RECEIVED_QPS + "#")) { - tableMetricsMaps[SOURCE_QPS_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_QPS + "#")) { - tableMetricsMaps[SINK_QPS_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SOURCE_RECEIVED_BYTES_PER_SECONDS + "#")) { - tableMetricsMaps[SOURCE_BYTES_SEC_IDX].put(tableName, metricNode); - } else if (metricName.startsWith(SINK_WRITE_BYTES_PER_SECONDS + "#")) { - tableMetricsMaps[SINK_BYTES_SEC_IDX].put(tableName, metricNode); - } - } - - private void aggregateMetrics( - JsonNode jobMetricsStr, - Long[] metricsSums, - Double[] metricsRates, - String[] metricsNames) { - for (int i = 0; i < metricsNames.length; i++) { - JsonNode metricNode = jobMetricsStr.get(metricsNames[i]); - if (metricNode != null && metricNode.isArray()) { - for (JsonNode node : metricNode) { - // Match Rate Metrics vs. Value Metrics - if (i < metricsSums.length) { - metricsSums[i] += node.path("value").asLong(); - } else { - metricsRates[i - metricsSums.length] += node.path("value").asDouble(); - } - } - } - } - } - - private void populateMetricsMap( - Map metricsMap, - Object[] metrics, - String[] metricNames, - int countMetricNames) { - for (int i = 0; i < metrics.length; i++) { - if (metrics[i] != null) { - if (metrics[i] instanceof Map) { - metricsMap.put( - metricNames[i], - aggregateMap( - (Map) metrics[i], i >= countMetricNames)); - } else { - metricsMap.put(metricNames[i], metrics[i]); - } - } - } - } - - private Map aggregateMap(Map inputMap, boolean isRate) { - return isRate - ? inputMap.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> - StreamSupport.stream( - entry.getValue().spliterator(), - false) - .mapToDouble( - node -> - node.path("value") - .asDouble()) - .sum())) - : inputMap.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> - StreamSupport.stream( - entry.getValue().spliterator(), - false) - .mapToLong( - node -> node.path("value").asLong()) - .sum())); - } - - protected void handleStopJob( - Map map, SeaTunnelServer seaTunnelServer, Node node) { - boolean isStopWithSavePoint = false; - if (map.get(RestConstant.JOB_ID) == null) { - throw new IllegalArgumentException("jobId cannot be empty."); - } - long jobId = Long.parseLong(map.get(RestConstant.JOB_ID).toString()); - if (map.get(RestConstant.IS_STOP_WITH_SAVE_POINT) != null) { - isStopWithSavePoint = - Boolean.parseBoolean(map.get(RestConstant.IS_STOP_WITH_SAVE_POINT).toString()); - } - - if (!seaTunnelServer.isMasterNode()) { - if (isStopWithSavePoint) { - NodeEngineUtil.sendOperationToMasterNode( - node.nodeEngine, new SavePointJobOperation(jobId)) - .join(); - } else { - NodeEngineUtil.sendOperationToMasterNode( - node.nodeEngine, new CancelJobOperation(jobId)) - .join(); - } - - } else { - CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); - - if (isStopWithSavePoint) { - coordinatorService.savePoint(jobId); - } else { - coordinatorService.cancelJob(jobId); - } - } - } - - protected String mapToUrlParams(Map params) { - return params.entrySet().stream() - .map(entry -> entry.getKey() + "=" + entry.getValue()) - .collect(Collectors.joining("&", "?", "")); - } - - protected JsonObject submitJobInternal( - Config config, - Map requestParams, - SeaTunnelServer seaTunnelServer, - Node node) { - ReadonlyConfig envOptions = ReadonlyConfig.fromConfig(config.getConfig("env")); - String jobName = envOptions.get(EnvCommonOptions.JOB_NAME); - - JobConfig jobConfig = new JobConfig(); - jobConfig.setName( - StringUtils.isEmpty(requestParams.get(RestConstant.JOB_NAME)) - ? jobName - : requestParams.get(RestConstant.JOB_NAME)); - - boolean startWithSavePoint = - Boolean.parseBoolean(requestParams.get(RestConstant.IS_START_WITH_SAVE_POINT)); - String jobIdStr = requestParams.get(RestConstant.JOB_ID); - Long finalJobId = StringUtils.isNotBlank(jobIdStr) ? Long.parseLong(jobIdStr) : null; - RestJobExecutionEnvironment restJobExecutionEnvironment = - new RestJobExecutionEnvironment( - seaTunnelServer, jobConfig, config, node, startWithSavePoint, finalJobId); - JobImmutableInformation jobImmutableInformation = restJobExecutionEnvironment.build(); - long jobId = jobImmutableInformation.getJobId(); - if (!seaTunnelServer.isMasterNode()) { - - NodeEngineUtil.sendOperationToMasterNode( - node.nodeEngine, - new SubmitJobOperation( - jobId, - node.nodeEngine.toData(jobImmutableInformation), - jobImmutableInformation.isStartWithSavePoint())) - .join(); - - } else { - submitJob(node, seaTunnelServer, jobImmutableInformation, jobConfig); - } - - return new JsonObject() - .add(RestConstant.JOB_ID, String.valueOf(jobId)) - .add(RestConstant.JOB_NAME, jobConfig.getName()); - } - - private void submitJob( - Node node, - SeaTunnelServer seaTunnelServer, - JobImmutableInformation jobImmutableInformation, - JobConfig jobConfig) { - CoordinatorService coordinatorService = seaTunnelServer.getCoordinatorService(); - Data data = node.nodeEngine.getSerializationService().toData(jobImmutableInformation); - PassiveCompletableFuture voidPassiveCompletableFuture = - coordinatorService.submitJob( - Long.parseLong(jobConfig.getJobContext().getJobId()), - data, - jobImmutableInformation.isStartWithSavePoint()); - voidPassiveCompletableFuture.join(); - } - - protected JsonArray getSystemMonitoringInformationJsonValues() { - Cluster cluster = nodeEngine.getHazelcastInstance().getCluster(); - - Set members = cluster.getMembers(); - JsonArray jsonValues = - members.stream() - .map( - member -> { - Address address = member.getAddress(); - String input = null; - try { - input = - (String) - NodeEngineUtil.sendOperationToMemberNode( - nodeEngine, - new GetClusterHealthMetricsOperation(), - address) - .get(); - } catch (InterruptedException | ExecutionException e) { - log.error("Failed to get cluster health metrics", e); - } - String[] parts = input.split(", "); - JsonObject jobInfo = new JsonObject(); - Arrays.stream(parts) - .forEach( - part -> { - String[] keyValue = part.split("="); - jobInfo.add(keyValue[0], keyValue[1]); - }); - return jobInfo; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - return jsonValues; - } - - private JsonObject metricsToJsonObject(Map jobMetrics) { - JsonObject members = new JsonObject(); - jobMetrics.forEach( - (key, value) -> { - if (value instanceof Map) { - members.add(key, metricsToJsonObject((Map) value)); - } else { - members.add(key, value.toString()); - } - }); - return members; - } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/CurrentNodeLogServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/CurrentNodeLogServlet.java index d73c3d785d..3931899300 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/CurrentNodeLogServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/CurrentNodeLogServlet.java @@ -17,9 +17,9 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.common.utils.FileUtils; import org.apache.seatunnel.engine.common.config.server.HttpConfig; import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.rest.service.LogService; import org.apache.commons.lang3.StringUtils; @@ -30,15 +30,16 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import java.io.File; import java.io.IOException; -import java.util.List; @Slf4j public class CurrentNodeLogServlet extends LogBaseServlet { + private final LogService logService; + public CurrentNodeLogServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.logService = new LogService(nodeEngine); } @Override @@ -50,17 +51,11 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) seaTunnelServer.getSeaTunnelConfig().getEngineConfig().getHttpConfig(); String contextPath = httpConfig.getContextPath(); String uri = req.getRequestURI(); - String logName = getLogParam(uri, contextPath); - String logPath = getLogPath(); + String logName = logService.getLogParam(uri, contextPath); + String logPath = logService.getLogPath(); if (StringUtils.isBlank(logName)) { - // Get Current Node Log List - List logFileList = FileUtils.listFile(logPath); - StringBuffer logLink = new StringBuffer(); - for (File file : logFileList) { - logLink.append(buildLogLink("log/" + file.getName(), file.getName())); - } - writeHtml(resp, buildWebSiteContent(logLink)); + writeHtml(resp, logService.currentNodeLog(uri)); } else { // Get Current Node Log Content prepareLogResponse(resp, logPath, logName); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/EncryptConfigServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/EncryptConfigServlet.java index d5a169ec53..007a0192ba 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/EncryptConfigServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/EncryptConfigServlet.java @@ -17,17 +17,10 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigRenderOptions; +import org.apache.seatunnel.engine.server.rest.service.EncryptConfigService; -import org.apache.seatunnel.core.starter.utils.ConfigShadeUtils; -import org.apache.seatunnel.engine.server.utils.RestUtil; - -import com.hazelcast.internal.json.Json; -import com.hazelcast.internal.json.JsonObject; import com.hazelcast.spi.impl.NodeEngineImpl; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -35,18 +28,15 @@ public class EncryptConfigServlet extends BaseServlet { + private final EncryptConfigService encryptConfigService; + public EncryptConfigServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.encryptConfigService = new EncryptConfigService(nodeEngine); } @Override - public void doPost(HttpServletRequest req, HttpServletResponse resp) - throws ServletException, IOException { - Config config = RestUtil.buildConfig(requestHandle(requestBody(req)), true); - Config encryptConfig = ConfigShadeUtils.encryptConfig(config); - String encryptString = - encryptConfig.root().render(ConfigRenderOptions.concise().setJson(true)); - JsonObject jsonObject = Json.parse(encryptString).asObject(); - writeJson(resp, jsonObject); + public void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { + writeJson(resp, encryptConfigService.encryptConfig(requestBody(req))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/FinishedJobsServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/FinishedJobsServlet.java index 515d2c4b03..9f0e4e4c49 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/FinishedJobsServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/FinishedJobsServlet.java @@ -17,16 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.api.common.metrics.JobMetrics; -import org.apache.seatunnel.engine.common.Constant; -import org.apache.seatunnel.engine.core.job.JobDAGInfo; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.master.JobHistoryService.JobState; -import org.apache.seatunnel.engine.server.operation.GetJobMetricsOperation; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.map.IMap; import com.hazelcast.spi.impl.NodeEngineImpl; import javax.servlet.ServletException; @@ -34,14 +26,16 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Comparator; public class FinishedJobsServlet extends BaseServlet { private static final long serialVersionUID = 1L; + private final JobInfoService jobInfoService; + public FinishedJobsServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override @@ -56,52 +50,6 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) state = ""; } - IMap finishedJob = - nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_STATE); - - IMap finishedJobMetrics = - nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_METRICS); - - IMap finishedJobDAGInfo = - nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); - String finalState = state; - JsonArray jobs = - finishedJob.values().stream() - .filter( - jobState -> { - if (finalState.isEmpty()) { - return true; - } - return jobState.getJobStatus() - .name() - .equals(finalState.toUpperCase()); - }) - .sorted(Comparator.comparing(JobState::getFinishTime)) - .map( - jobState -> { - Long jobId = jobState.getJobId(); - String jobMetrics; - if (seaTunnelServer == null) { - jobMetrics = - (String) - NodeEngineUtil.sendOperationToMasterNode( - nodeEngine, - new GetJobMetricsOperation( - jobId)) - .join(); - } else { - jobMetrics = - seaTunnelServer - .getCoordinatorService() - .getJobMetrics(jobId) - .toJsonString(); - } - return getJobInfoJson( - jobState, jobMetrics, finishedJobDAGInfo.get(jobId)); - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - - writeJson(resp, jobs); + writeJson(resp, jobInfoService.getJobsByStateJson(state)); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/JobInfoServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/JobInfoServlet.java index d41635a9eb..1ae2288ed2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/JobInfoServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/JobInfoServlet.java @@ -17,15 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.api.common.metrics.JobMetrics; -import org.apache.seatunnel.engine.common.Constant; -import org.apache.seatunnel.engine.core.job.JobDAGInfo; -import org.apache.seatunnel.engine.core.job.JobInfo; -import org.apache.seatunnel.engine.server.master.JobHistoryService.JobState; -import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import com.hazelcast.internal.json.JsonObject; -import com.hazelcast.map.IMap; import com.hazelcast.spi.impl.NodeEngineImpl; import javax.servlet.ServletException; @@ -35,51 +28,27 @@ import java.io.IOException; public class JobInfoServlet extends BaseServlet { + + private final JobInfoService jobInfoService; + public JobInfoServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - String jobId = req.getPathInfo(); + String jobIdStr = req.getPathInfo(); - if (jobId != null && jobId.length() > 1) { - jobId = jobId.substring(1); + if (jobIdStr != null && jobIdStr.length() > 1) { + jobIdStr = jobIdStr.substring(1); } else { throw new IllegalArgumentException("The jobId must not be empty."); } + Long jobId = Long.valueOf(jobIdStr); - IMap jobInfoMap = - nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_RUNNING_JOB_INFO); - Object jobInfo = jobInfoMap.get(Long.valueOf(jobId)); - IMap finishedJobStateMap = - nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_FINISHED_JOB_STATE); - Object finishedJobState = finishedJobStateMap.get(Long.valueOf(jobId)); - if (jobInfo != null) { - writeJson(resp, convertToJson((JobInfo) jobInfo, Long.parseLong(jobId))); - } else if (finishedJobState != null) { - JobMetrics finishedJobMetrics = - (JobMetrics) - nodeEngine - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_METRICS) - .get(Long.valueOf(jobId)); - JobDAGInfo finishedJobDAGInfo = - (JobDAGInfo) - nodeEngine - .getHazelcastInstance() - .getMap(Constant.IMAP_FINISHED_JOB_VERTEX_INFO) - .get(Long.valueOf(jobId)); - writeJson( - resp, - getJobInfoJson( - (JobState) finishedJobState, - finishedJobMetrics.toJsonString(), - finishedJobDAGInfo)); - } else { - writeJson(resp, new JsonObject().add(RestConstant.JOB_ID, jobId)); - } + writeJson(resp, jobInfoService.getJobInfoJson(jobId)); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/LogBaseServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/LogBaseServlet.java index 42eb815980..2bb80fd897 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/LogBaseServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/LogBaseServlet.java @@ -18,30 +18,17 @@ package org.apache.seatunnel.engine.server.rest.servlet; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; -import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.common.utils.FileUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.core.LoggerContext; -import org.apache.logging.log4j.core.config.builder.api.Component; -import org.apache.logging.log4j.core.config.builder.impl.BuiltConfiguration; -import org.apache.logging.log4j.core.config.properties.PropertiesConfiguration; -import org.apache.logging.log4j.core.lookup.StrSubstitutor; -import com.hazelcast.internal.util.StringUtil; import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; import javax.servlet.http.HttpServletResponse; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.lang.reflect.Field; -import java.net.HttpURLConnection; -import java.net.URL; @Slf4j public class LogBaseServlet extends BaseServlet { @@ -49,127 +36,6 @@ public class LogBaseServlet extends BaseServlet { public LogBaseServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); } - - protected String getLogParam(String uri, String contextPath) { - uri = uri.substring(uri.indexOf(contextPath) + contextPath.length()); - uri = StringUtil.stripTrailingSlash(uri).substring(1); - int indexEnd = uri.indexOf('/'); - if (indexEnd != -1) { - String param = uri.substring(indexEnd + 1); - return param; - } - return ""; - } - - /** Get configuration log path */ - protected String getLogPath() { - try { - String routingAppender = "routingAppender"; - String fileAppender = "fileAppender"; - PropertiesConfiguration config = getLogConfiguration(); - // Get routingAppender log file path - String routingLogFilePath = getRoutingLogFilePath(config); - - // Get fileAppender log file path - String fileLogPath = getFileLogPath(config); - String logRef = - config.getLoggerConfig(StringUtils.EMPTY).getAppenderRefs().stream() - .map(Object::toString) - .filter( - ref -> - ref.contains(routingAppender) - || ref.contains(fileAppender)) - .findFirst() - .orElse(StringUtils.EMPTY); - if (logRef.equals(routingAppender)) { - return routingLogFilePath.substring(0, routingLogFilePath.lastIndexOf("/")); - } else if (logRef.equals(fileAppender)) { - return fileLogPath.substring(0, routingLogFilePath.lastIndexOf("/")); - } else { - log.warn(String.format("Log file path is empty, get logRef : %s", logRef)); - return null; - } - } catch (NoSuchFieldException | IllegalAccessException e) { - log.error("Get log path error", ExceptionUtils.getMessage(e)); - return null; - } - } - - private String getFileLogPath(PropertiesConfiguration config) - throws NoSuchFieldException, IllegalAccessException { - Field propertiesField = BuiltConfiguration.class.getDeclaredField("appendersComponent"); - propertiesField.setAccessible(true); - Component propertiesComponent = (Component) propertiesField.get(config); - StrSubstitutor substitutor = config.getStrSubstitutor(); - return propertiesComponent.getComponents().stream() - .filter(component -> "fileAppender".equals(component.getAttributes().get("name"))) - .map(component -> substitutor.replace(component.getAttributes().get("fileName"))) - .findFirst() - .orElse(null); - } - - private String getRoutingLogFilePath(PropertiesConfiguration config) - throws NoSuchFieldException, IllegalAccessException { - Field propertiesField = BuiltConfiguration.class.getDeclaredField("appendersComponent"); - propertiesField.setAccessible(true); - Component propertiesComponent = (Component) propertiesField.get(config); - StrSubstitutor substitutor = config.getStrSubstitutor(); - return propertiesComponent.getComponents().stream() - .filter( - component -> - "routingAppender".equals(component.getAttributes().get("name"))) - .flatMap(component -> component.getComponents().stream()) - .flatMap(component -> component.getComponents().stream()) - .flatMap(component -> component.getComponents().stream()) - .map(component -> substitutor.replace(component.getAttributes().get("fileName"))) - .findFirst() - .orElse(null); - } - - private PropertiesConfiguration getLogConfiguration() { - LoggerContext context = (LoggerContext) LogManager.getContext(false); - return (PropertiesConfiguration) context.getConfiguration(); - } - - protected String sendGet(String urlString) { - try { - HttpURLConnection connection = (HttpURLConnection) new URL(urlString).openConnection(); - connection.setRequestMethod("GET"); - connection.setConnectTimeout(5000); - connection.setReadTimeout(5000); - connection.connect(); - - if (connection.getResponseCode() == 200) { - try (InputStream is = connection.getInputStream(); - ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - byte[] buffer = new byte[1024]; - int len; - while ((len = is.read(buffer)) != -1) { - baos.write(buffer, 0, len); - } - return baos.toString(); - } - } - } catch (IOException e) { - log.error("Send get Fail.", ExceptionUtils.getMessage(e)); - } - return null; - } - - protected String buildLogLink(String href, String name) { - return "
  • " + name + "
  • \n"; - } - - protected String buildWebSiteContent(StringBuffer logLink) { - return "Seatunnel log\n" - + "\n" - + "

    Seatunnel log

    \n" - + "
      \n" - + logLink.toString() - + "
    \n" - + ""; - } - /** Prepare Log Response */ protected void prepareLogResponse(HttpServletResponse resp, String logPath, String logName) { if (StringUtils.isBlank(logPath)) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/OverviewServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/OverviewServlet.java index 46542f6fda..37d66824b0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/OverviewServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/OverviewServlet.java @@ -18,12 +18,7 @@ package org.apache.seatunnel.engine.server.rest.servlet; import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.engine.common.env.EnvironmentUtil; -import org.apache.seatunnel.engine.common.env.Version; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.resourcemanager.opeartion.GetOverviewOperation; -import org.apache.seatunnel.engine.server.resourcemanager.resource.OverviewInfo; -import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; +import org.apache.seatunnel.engine.server.rest.service.OverviewService; import com.hazelcast.internal.util.JsonUtil; import com.hazelcast.spi.impl.NodeEngineImpl; @@ -37,8 +32,11 @@ public class OverviewServlet extends BaseServlet { + private final OverviewService overviewService; + public OverviewServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.overviewService = new OverviewService(nodeEngine); } @Override @@ -47,26 +45,10 @@ protected void doGet(HttpServletRequest req, HttpServletResponse resp) Map tags = getParameterMap(req); - Version version = EnvironmentUtil.getVersion(); - - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(true); - - OverviewInfo overviewInfo; - - if (seaTunnelServer == null) { - overviewInfo = - (OverviewInfo) - NodeEngineUtil.sendOperationToMasterNode( - nodeEngine, new GetOverviewOperation(tags)) - .join(); - } else { - - overviewInfo = GetOverviewOperation.getOverviewInfo(seaTunnelServer, nodeEngine, tags); - } - overviewInfo.setProjectVersion(version.getProjectVersion()); - overviewInfo.setGitCommitAbbrev(version.getGitCommitAbbrev()); - writeJson( - resp, JsonUtil.toJsonObject(JsonUtils.toMap(JsonUtils.toJsonString(overviewInfo)))); + resp, + JsonUtil.toJsonObject( + JsonUtils.toMap( + JsonUtils.toJsonString(overviewService.getOverviewInfo(tags))))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningJobsServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningJobsServlet.java index 69cd150d46..5eed9bd58c 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningJobsServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningJobsServlet.java @@ -17,11 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.engine.common.Constant; -import org.apache.seatunnel.engine.core.job.JobInfo; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.map.IMap; import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; @@ -34,23 +31,17 @@ @Slf4j public class RunningJobsServlet extends BaseServlet { + private final JobInfoService jobInfoService; + public RunningJobsServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - IMap values = - nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_RUNNING_JOB_INFO); - JsonArray jobs = - values.entrySet().stream() - .map( - jobInfoEntry -> - convertToJson( - jobInfoEntry.getValue(), jobInfoEntry.getKey())) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - writeJson(resp, jobs); + writeJson(resp, jobInfoService.getRunningJobsJson()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningThreadsServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningThreadsServlet.java index 7e33034858..9069fa88b1 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningThreadsServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/RunningThreadsServlet.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.internal.json.JsonObject; +import org.apache.seatunnel.engine.server.rest.service.RunningThreadService; + import com.hazelcast.spi.impl.NodeEngineImpl; import javax.servlet.ServletException; @@ -26,33 +26,20 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Comparator; public class RunningThreadsServlet extends BaseServlet { + private final RunningThreadService runningThreadService; + public RunningThreadsServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.runningThreadService = new RunningThreadService(nodeEngine); } @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - JsonArray runningThreads = - Thread.getAllStackTraces().keySet().stream() - .sorted(Comparator.comparing(Thread::getName)) - .map( - stackTraceElements -> { - JsonObject jobInfoJson = new JsonObject(); - jobInfoJson.add("threadName", stackTraceElements.getName()); - jobInfoJson.add( - "classLoader", - String.valueOf( - stackTraceElements.getContextClassLoader())); - return jobInfoJson; - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); - - writeJson(resp, runningThreads); + writeJson(resp, runningThreadService.getRunningThread()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobServlet.java index bb1c0acb77..bd9b2d2942 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobServlet.java @@ -17,11 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import com.hazelcast.internal.json.JsonObject; import com.hazelcast.spi.impl.NodeEngineImpl; import javax.servlet.ServletException; @@ -29,21 +26,18 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Map; public class StopJobServlet extends BaseServlet { + private final JobInfoService jobInfoService; + public StopJobServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override public void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - Map map = JsonUtils.toMap(requestHandle(requestBody(req))); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); - handleStopJob(map, seaTunnelServer, nodeEngine.getNode()); - writeJson( - resp, - new JsonObject().add(RestConstant.JOB_ID, map.get(RestConstant.JOB_ID).toString())); + writeJson(resp, jobInfoService.stopJob(requestBody(req))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobsServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobsServlet.java index fbd5c79ed3..f2966ed3c5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobsServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/StopJobsServlet.java @@ -17,40 +17,27 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.internal.json.JsonObject; import com.hazelcast.spi.impl.NodeEngineImpl; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.List; -import java.util.Map; public class StopJobsServlet extends BaseServlet { + + private final JobInfoService jobInfoService; + public StopJobsServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override - public void doPost(HttpServletRequest req, HttpServletResponse resp) - throws ServletException, IOException { - List jobList = JsonUtils.toList(requestHandle(requestBody(req)).toString(), Map.class); - JsonArray jsonResponse = new JsonArray(); - - jobList.forEach( - job -> { - handleStopJob(job, getSeaTunnelServer(false), nodeEngine.getNode()); - jsonResponse.add( - new JsonObject() - .add(RestConstant.JOB_ID, (Long) job.get(RestConstant.JOB_ID))); - }); - - writeJson(resp, jsonResponse); + public void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { + + writeJson(resp, jobInfoService.stopJobs(requestBody(req))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobServlet.java index d7d2a754da..b4e61c959c 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobServlet.java @@ -17,16 +17,10 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.rest.RestConstant; -import org.apache.seatunnel.engine.server.utils.RestUtil; - -import com.hazelcast.internal.json.JsonObject; import com.hazelcast.spi.impl.NodeEngineImpl; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -34,26 +28,18 @@ import java.util.Map; public class SubmitJobServlet extends BaseServlet { + private final JobInfoService jobInfoService; + public SubmitJobServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override - public void doPost(HttpServletRequest req, HttpServletResponse resp) - throws ServletException, IOException { + public void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { Map requestParams = getParameterMap(req); - if (Boolean.parseBoolean(requestParams.get(RestConstant.IS_START_WITH_SAVE_POINT)) - && requestParams.get(RestConstant.JOB_ID) == null) { - throw new IllegalArgumentException("Please provide jobId when start with save point."); - } - - Config config = RestUtil.buildConfig(requestHandle(requestBody(req)), false); - - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); - JsonObject jsonObject = - submitJobInternal(config, requestParams, seaTunnelServer, nodeEngine.getNode()); - writeJson(resp, jsonObject); + writeJson(resp, jobInfoService.submitJob(requestParams, requestBody(req))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobsServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobsServlet.java index 90c3be600a..5ed0d0a8c2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobsServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SubmitJobsServlet.java @@ -17,51 +17,27 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.engine.server.rest.service.JobInfoService; -import org.apache.seatunnel.engine.server.SeaTunnelServer; -import org.apache.seatunnel.engine.server.utils.RestUtil; - -import com.hazelcast.internal.json.JsonArray; import com.hazelcast.spi.impl.NodeEngineImpl; -import scala.Tuple2; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; public class SubmitJobsServlet extends BaseServlet { + + private final JobInfoService jobInfoService; + public SubmitJobsServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.jobInfoService = new JobInfoService(nodeEngine); } @Override - public void doPost(HttpServletRequest req, HttpServletResponse resp) - throws ServletException, IOException { - List, Config>> configTuples = - RestUtil.buildConfigList(requestHandle(requestBody(req)), false); - - JsonArray jsonArray = - configTuples.stream() - .map( - tuple -> { - String urlParams = mapToUrlParams(tuple._1); - Map requestParams = new HashMap<>(); - RestUtil.buildRequestParams(requestParams, urlParams); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); - return submitJobInternal( - tuple._2, - requestParams, - seaTunnelServer, - nodeEngine.getNode()); - }) - .collect(JsonArray::new, JsonArray::add, JsonArray::add); + public void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { - writeJson(resp, jsonArray); + writeJson(resp, jobInfoService.submitJobs(requestBody(req))); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SystemMonitoringServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SystemMonitoringServlet.java index 9439c9e58e..29a4b576d0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SystemMonitoringServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/SystemMonitoringServlet.java @@ -17,7 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import com.hazelcast.internal.json.JsonArray; +import org.apache.seatunnel.engine.server.rest.service.SystemMonitoringService; + import com.hazelcast.spi.impl.NodeEngineImpl; import lombok.extern.slf4j.Slf4j; @@ -29,15 +30,17 @@ @Slf4j public class SystemMonitoringServlet extends BaseServlet { + private final SystemMonitoringService systemMonitoringService; + public SystemMonitoringServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.systemMonitoringService = new SystemMonitoringService(nodeEngine); } @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - JsonArray jsonValues = getSystemMonitoringInformationJsonValues(); - writeJson(resp, jsonValues); + writeJson(resp, systemMonitoringService.getSystemMonitoringInformationJsonValues()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/ThreadDumpServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/ThreadDumpServlet.java index c556a6dbcf..e32d7fef3d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/ThreadDumpServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/ThreadDumpServlet.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import com.hazelcast.internal.json.JsonArray; -import com.hazelcast.internal.json.JsonObject; +import org.apache.seatunnel.engine.server.rest.service.ThreadDumpService; + import com.hazelcast.spi.impl.NodeEngineImpl; import javax.servlet.ServletException; @@ -26,33 +26,20 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Map; public class ThreadDumpServlet extends BaseServlet { + private final ThreadDumpService threadDumpService; + public ThreadDumpServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.threadDumpService = new ThreadDumpService(nodeEngine); } @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - Map threadStacks = Thread.getAllStackTraces(); - JsonArray threadInfoList = new JsonArray(); - for (Map.Entry entry : threadStacks.entrySet()) { - StringBuilder stackTraceBuilder = new StringBuilder(); - for (StackTraceElement element : entry.getValue()) { - stackTraceBuilder.append(element.toString()).append("\n"); - } - String stackTrace = stackTraceBuilder.toString().trim(); - JsonObject threadInfo = new JsonObject(); - threadInfo.add("threadName", entry.getKey().getName()); - threadInfo.add("threadId", entry.getKey().getId()); - threadInfo.add("threadState", entry.getKey().getState().name()); - threadInfo.add("stackTrace", stackTrace); - threadInfoList.add(threadInfo); - } - - writeJson(resp, threadInfoList); + + writeJson(resp, threadDumpService.getThreadDump()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/UpdateTagsServlet.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/UpdateTagsServlet.java index 86f147224d..eed0d91a94 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/UpdateTagsServlet.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/servlet/UpdateTagsServlet.java @@ -17,46 +17,26 @@ package org.apache.seatunnel.engine.server.rest.servlet; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.engine.server.SeaTunnelServer; +import org.apache.seatunnel.engine.server.rest.service.UpdateTagsService; -import com.hazelcast.cluster.impl.MemberImpl; -import com.hazelcast.internal.json.JsonObject; import com.hazelcast.spi.impl.NodeEngineImpl; -import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Map; -import java.util.stream.Collectors; public class UpdateTagsServlet extends BaseServlet { + private final UpdateTagsService updateTagsService; + public UpdateTagsServlet(NodeEngineImpl nodeEngine) { super(nodeEngine); + this.updateTagsService = new UpdateTagsService(nodeEngine); } @Override - public void doPost(HttpServletRequest req, HttpServletResponse resp) - throws ServletException, IOException { - Map params = JsonUtils.toMap(requestHandle(requestBody(req))); - SeaTunnelServer seaTunnelServer = getSeaTunnelServer(false); - - NodeEngineImpl nodeEngine = seaTunnelServer.getNodeEngine(); - MemberImpl localMember = nodeEngine.getLocalMember(); - - Map tags = - params.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - value -> - value.getValue() != null - ? value.getValue().toString() - : "")); - localMember.updateAttribute(tags); - writeJson(resp, new JsonObject().add("status", "success")); + public void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { + writeJson(resp, updateTagsService.updateTags(requestBody(req))); } }