From d234c9ce015fb536482727803825d20079f189b1 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 29 Nov 2024 15:51:23 +0800 Subject: [PATCH 01/56] [Feature][transform-v2] Support transform metrics --- .../api/common/metrics/MetricNames.java | 4 + .../api/common/metrics/MetricsContext.java | 2 + .../api/transform/SeaTunnelTransform.java | 5 ++ .../common/utils/StringFormatUtils.java | 8 +- .../command/ClientExecuteCommand.java | 76 +++++++++++------ .../engine/client/job/JobClient.java | 32 ++++++- .../engine/client/job/JobMetricsRunner.java | 31 ++++++- .../engine/server/task/SeaTunnelTask.java | 3 +- .../task/flow/TransformFlowLifeCycle.java | 8 +- ...bstractCatalogSupportFlatMapTransform.java | 18 ++-- .../AbstractCatalogSupportMapTransform.java | 21 +++-- .../AbstractMultiCatalogFlatMapTransform.java | 14 ++-- .../common/AbstractMultiCatalogTransform.java | 19 ++++- .../common/AbstractSeaTunnelTransform.java | 84 +++++++++++++++++-- .../transform/common/FilterRowTransform.java | 6 +- .../common/MultipleFieldOutputTransform.java | 12 ++- .../common/SingleFieldOutputTransform.java | 6 +- .../copy/CopyFieldMultiCatalogTransform.java | 2 +- .../transform/copy/CopyFieldTransform.java | 7 +- .../DynamicCompileTransform.java | 2 +- .../FieldMapperMultiCatalogTransform.java | 2 +- .../fieldmapper/FieldMapperTransform.java | 7 +- .../filter/FilterFieldTransform.java | 2 +- .../filterrowkind/FilterRowKindTransform.java | 2 +- .../JsonPathMultiCatalogTransform.java | 3 +- .../transform/jsonpath/JsonPathTransform.java | 10 ++- .../transform/metadata/MetadataTransform.java | 2 +- .../embadding/EmbeddingTransform.java | 2 +- .../transform/nlpmodel/llm/LLMTransform.java | 2 +- .../transform/replace/ReplaceTransform.java | 2 +- .../rowkind/RowKindExtractorTransform.java | 2 +- .../split/SplitMultiCatalogTransform.java | 2 +- .../transform/split/SplitTransform.java | 9 +- .../seatunnel/transform/sql/SQLTransform.java | 13 +-- 34 files changed, 304 insertions(+), 116 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java index b1fc60e0f16..69dc778426d 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java @@ -33,4 +33,8 @@ private MetricNames() {} public static final String SINK_WRITE_BYTES = "SinkWriteBytes"; public static final String SINK_WRITE_QPS = "SinkWriteQPS"; public static final String SINK_WRITE_BYTES_PER_SECONDS = "SinkWriteBytesPerSeconds"; + public static final String TRANSFORM_COUNT = "TransformCount"; + public static final String TRANSFORM_BYTES = "TransformBytes"; + public static final String TRANSFORM_QPS = "TransformQPS"; + public static final String TRANSFORM_BYTES_PER_SECONDS = "TransformBytesPerSeconds"; } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java index a8875d91d7d..ce4b486e23c 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.api.common.metrics; +import org.checkerframework.checker.units.qual.C; + public interface MetricsContext { /** diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index b12d69bafbb..679391bb5cb 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.api.transform; import org.apache.seatunnel.api.common.PluginIdentifierInterface; +import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.source.SeaTunnelJobAware; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; @@ -54,4 +55,8 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve /** call it when Transformer completed */ default void close() {} + + void setMetricsContext(MetricsContext metricsContext); + + MetricsContext getMetricsContext(); } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java index 9a52118145c..28e71d210be 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java @@ -32,15 +32,15 @@ public static String formatTable(Object... objects) { int kvNum = (objects.length - 1) / 2; String template = "\n" - + "***********************************************" + + "*************************************************************" + "\n" + String.join("", Collections.nCopies(blankNum, " ")) + "%s" + "\n" - + "***********************************************" + + "*************************************************************" + "\n" - + String.join("", Collections.nCopies(kvNum, "%-26s: %19s\n")) - + "***********************************************\n"; + + String.join("", Collections.nCopies(kvNum, "%-40s: %19s\n")) + + "*************************************************************\n"; return String.format(template, objects); } } diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java index f6ef4d76d02..61d45fee028 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java @@ -17,6 +17,13 @@ package org.apache.seatunnel.core.starter.seatunnel.command; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.core.HazelcastInstance; +import com.hazelcast.instance.impl.HazelcastInstanceFactory; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; import org.apache.seatunnel.core.starter.command.Command; @@ -40,19 +47,13 @@ import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelNodeContext; -import org.apache.commons.lang3.StringUtils; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.hazelcast.client.config.ClientConfig; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.instance.impl.HazelcastInstanceFactory; -import lombok.extern.slf4j.Slf4j; - import java.nio.file.Path; import java.time.Duration; import java.time.LocalDateTime; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; @@ -61,7 +62,9 @@ import static org.apache.seatunnel.core.starter.utils.FileUtils.checkConfigExist; -/** This command is used to execute the SeaTunnel engine job by SeaTunnel API. */ +/** + * This command is used to execute the SeaTunnel engine job by SeaTunnel API. + */ @Slf4j public class ClientExecuteCommand implements Command { @@ -227,24 +230,43 @@ public void execute() throws CommandExecuteException { } finally { if (jobMetricsSummary != null) { // print job statistics information when job finished - log.info( - StringFormatUtils.formatTable( - "Job Statistic Information", - "Start Time", - DateTimeUtils.toString( - startTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), - "End Time", - DateTimeUtils.toString( - endTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), - "Total Time(s)", - Duration.between(startTime, endTime).getSeconds(), - "Total Read Count", - jobMetricsSummary.getSourceReadCount(), - "Total Write Count", - jobMetricsSummary.getSinkWriteCount(), - "Total Failed Count", - jobMetricsSummary.getSourceReadCount() - - jobMetricsSummary.getSinkWriteCount())); + StringBuilder logMessage = new StringBuilder(); + logMessage.append(StringFormatUtils.formatTable( + "Job Statistic Information", + "Start Time", + DateTimeUtils.toString( + startTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), + "End Time", + DateTimeUtils.toString( + endTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), + "Total Time(s)", + Duration.between(startTime, endTime).getSeconds(), + "Total Read Count", + jobMetricsSummary.getSourceReadCount(), + "Total Write Count", + jobMetricsSummary.getSinkWriteCount(), + "Total Failed Count", + jobMetricsSummary.getSourceReadCount() + - jobMetricsSummary.getSinkWriteCount())); + String[] transformInfos = null; + if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformCountMap())) { + transformInfos = + new String + [jobMetricsSummary.getTransformCountMap().entrySet().size() * 2 + + 1]; + transformInfos[0] = "Transform Information"; + int index = 0; + for (Map.Entry entry : + jobMetricsSummary.getTransformCountMap().entrySet()) { + transformInfos[++index] = entry.getKey(); + transformInfos[++index] = String.valueOf(entry.getValue()); + } + } + + if (Objects.nonNull(transformInfos)) { + logMessage.append(StringFormatUtils.formatTable(transformInfos)); + } + log.info("{}", logMessage); } closeClient(); } diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java index 2421829ea9e..c88b3ea6cd6 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java @@ -43,7 +43,11 @@ import lombok.NonNull; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.TreeMap; public class JobClient { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -154,6 +158,7 @@ public JobDAGInfo getJobInfo(Long jobId) { public JobMetricsRunner.JobMetricsSummary getJobMetricsSummary(Long jobId) { long sourceReadCount = 0L; long sinkWriteCount = 0L; + Map transformCountMap = new HashMap<>(); String jobMetrics = getJobMetrics(jobId); try { JsonNode jsonNode = OBJECT_MAPPER.readTree(jobMetrics); @@ -165,13 +170,36 @@ public JobMetricsRunner.JobMetricsSummary getJobMetricsSummary(Long jobId) { sourceReadCount += sourceReader.get("value").asLong(); sinkWriteCount += sinkWriter.get("value").asLong(); } - return new JobMetricsRunner.JobMetricsSummary(sourceReadCount, sinkWriteCount); + transformCountMap = extractTransformKeys(jsonNode); + return new JobMetricsRunner.JobMetricsSummary( + sourceReadCount, sinkWriteCount, transformCountMap); // Add NullPointerException because of metrics information can be empty like {} } catch (JsonProcessingException | NullPointerException e) { - return new JobMetricsRunner.JobMetricsSummary(sourceReadCount, sinkWriteCount); + return new JobMetricsRunner.JobMetricsSummary( + sourceReadCount, sinkWriteCount, transformCountMap); } } + private Map extractTransformKeys(JsonNode rootNode) { + Map transformCountMap = new TreeMap<>(); + Iterator> fields = rootNode.fields(); + while (fields.hasNext()) { + Map.Entry field = fields.next(); + String key = field.getKey(); + JsonNode transforms = field.getValue(); + long transformCount = 0L; + + if (key.startsWith("TransformCount-")) { + for (int i = 0; i < transforms.size(); i++) { + JsonNode transform = transforms.get(i); + transformCount += transform.get("value").asLong(); + } + transformCountMap.put(key, transformCount); + } + } + return transformCountMap; + } + public List getCheckpointData(Long jobId) { return hazelcastClient .getSerializationService() diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java index f413a5bfc52..a08121aadfc 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.engine.client.job; +import org.apache.commons.collections4.MapUtils; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; import org.apache.seatunnel.engine.client.SeaTunnelClient; @@ -27,6 +28,9 @@ import java.time.Duration; import java.time.LocalDateTime; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; @Slf4j public class JobMetricsRunner implements Runnable { @@ -35,6 +39,7 @@ public class JobMetricsRunner implements Runnable { private LocalDateTime lastRunTime = LocalDateTime.now(); private Long lastReadCount = 0L; private Long lastWriteCount = 0L; + private Map transformCountMap = new HashMap<>(); public JobMetricsRunner(SeaTunnelClient seaTunnelClient, Long jobId) { this.seaTunnelClient = seaTunnelClient; @@ -50,7 +55,8 @@ public void run() { long seconds = Duration.between(lastRunTime, now).getSeconds(); long averageRead = (jobMetricsSummary.getSourceReadCount() - lastReadCount) / seconds; long averageWrite = (jobMetricsSummary.getSinkWriteCount() - lastWriteCount) / seconds; - log.info( + StringBuilder logMessage = new StringBuilder(); + logMessage.append( StringFormatUtils.formatTable( "Job Progress Information", "Job Id", @@ -69,9 +75,31 @@ public void run() { "Current Statistic Time", DateTimeUtils.toString( now, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS))); + + String[] transformInfos = null; + if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformCountMap())) { + transformInfos = + new String + [jobMetricsSummary.getTransformCountMap().entrySet().size() * 2 + + 1]; + transformInfos[0] = "Transform Information"; + int index = 0; + for (Map.Entry entry : + jobMetricsSummary.getTransformCountMap().entrySet()) { + transformInfos[++index] = entry.getKey(); + transformInfos[++index] = String.valueOf(entry.getValue()); + } + } + + if (Objects.nonNull(transformInfos)) { + logMessage.append(StringFormatUtils.formatTable(transformInfos)); + } + + log.info("{}", logMessage); lastRunTime = now; lastReadCount = jobMetricsSummary.getSourceReadCount(); lastWriteCount = jobMetricsSummary.getSinkWriteCount(); + transformCountMap = jobMetricsSummary.getTransformCountMap(); } catch (Exception e) { log.warn("Failed to get job metrics summary, it maybe first-run"); } @@ -82,5 +110,6 @@ public void run() { public static class JobMetricsSummary { private long sourceReadCount; private long sinkWriteCount; + private Map transformCountMap; } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java index d66921c7423..668fc9f976e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java @@ -236,7 +236,8 @@ private FlowLifeCycle convertFlowToActionLifeCycle(@NonNull Flow flow) throws Ex (TransformChainAction) f.getAction(), this, new SeaTunnelTransformCollector(flowLifeCycles), - completableFuture); + completableFuture, + this.getMetricsContext()); } else if (f.getAction() instanceof ShuffleAction) { ShuffleAction shuffleAction = (ShuffleAction) f.getAction(); HazelcastInstance hazelcastInstance = getExecutionContext().getInstance(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 632e529169e..b4ed2abd4f6 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.engine.server.task.flow; +import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.api.transform.Collector; @@ -54,11 +55,15 @@ public TransformFlowLifeCycle( TransformChainAction action, SeaTunnelTask runningTask, Collector> collector, - CompletableFuture completableFuture) { + CompletableFuture completableFuture, + MetricsContext metricsContext) { super(action, runningTask, completableFuture); this.action = action; this.transform = action.getTransforms(); this.collector = collector; + for (SeaTunnelTransform t : transform) { + t.setMetricsContext(metricsContext); + } } @Override @@ -173,7 +178,6 @@ public List transform(T inputData) { dataList = nextInputDataList; } - return dataList; } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index 71b7b536be9..8336a950ab3 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -16,13 +16,13 @@ */ package org.apache.seatunnel.transform.common; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; - import java.util.List; @Slf4j @@ -30,17 +30,15 @@ public abstract class AbstractCatalogSupportFlatMapTransform extends AbstractSeaTunnelTransform> implements SeaTunnelFlatMapTransform { - public AbstractCatalogSupportFlatMapTransform(@NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); - } - public AbstractCatalogSupportFlatMapTransform( - @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { - super(inputCatalogTable, rowErrorHandleWay); + @NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { + super(config, catalogTable); } @Override public List flatMap(SeaTunnelRow row) { - return transform(row); + List results = transform(row); + hazelcastMetric(results.size()); + return results; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 6380d885397..8ace6467495 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -17,28 +17,33 @@ package org.apache.seatunnel.transform.common; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; - @Slf4j public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform implements SeaTunnelMapTransform { - public AbstractCatalogSupportMapTransform(@NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + public AbstractCatalogSupportMapTransform( + @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(config, inputCatalogTable); } public AbstractCatalogSupportMapTransform( - @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { - super(inputCatalogTable, rowErrorHandleWay); + @NonNull ReadonlyConfig config, + @NonNull CatalogTable inputCatalogTable, + ErrorHandleWay rowErrorHandleWay) { + super(config, inputCatalogTable, rowErrorHandleWay); } @Override public SeaTunnelRow map(SeaTunnelRow row) { - return transform(row); + SeaTunnelRow result = transform(row); + hazelcastMetric(); + return result; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java index b1deb453d6d..5b7fab0f0f8 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java @@ -36,11 +36,15 @@ public AbstractMultiCatalogFlatMapTransform( @Override public List flatMap(SeaTunnelRow row) { if (transformMap.size() == 1) { - return ((SeaTunnelFlatMapTransform) - transformMap.values().iterator().next()) - .flatMap(row); + SeaTunnelFlatMapTransform transform = + ((SeaTunnelFlatMapTransform) + transformMap.values().iterator().next()); + transform.setMetricsContext(metricsContext); + return transform.flatMap(row); } - return ((SeaTunnelFlatMapTransform) transformMap.get(row.getTableId())) - .flatMap(row); + SeaTunnelFlatMapTransform transform = + (SeaTunnelFlatMapTransform) transformMap.get(row.getTableId()); + transform.setMetricsContext(metricsContext); + return transform.flatMap(row); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index 49fa561979f..9cb14dbdb0e 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.common; +import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; @@ -44,6 +45,8 @@ public abstract class AbstractMultiCatalogTransform implements SeaTunnelTransfor protected Map> transformMap; + protected MetricsContext metricsContext; + public AbstractMultiCatalogTransform( List inputCatalogTables, ReadonlyConfig config) { this.inputCatalogTables = inputCatalogTables; @@ -73,7 +76,7 @@ public AbstractMultiCatalogTransform( if (tableConfig != null) { transformMap.put(tableId, buildTransform(inputCatalogTable, tableConfig)); } else { - transformMap.put(tableId, new IdentityTransform(inputCatalogTable)); + transformMap.put(tableId, new IdentityTransform(config, inputCatalogTable)); } }); @@ -88,6 +91,16 @@ public AbstractMultiCatalogTransform( .collect(Collectors.toList()); } + @Override + public void setMetricsContext(MetricsContext metricsContext) { + this.metricsContext = metricsContext; + } + + @Override + public MetricsContext getMetricsContext() { + return metricsContext; + } + protected abstract SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config); @@ -112,8 +125,8 @@ public String getPluginName() { return "Identity"; } - public IdentityTransform(CatalogTable catalogTable) { - super(catalogTable); + public IdentityTransform(ReadonlyConfig config, CatalogTable catalogTable) { + super(config, catalogTable); this.catalogTable = catalogTable; } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 01ce7eaf0a2..e8c238e0c6f 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -16,6 +16,14 @@ */ package org.apache.seatunnel.transform.common; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.groovy.parser.antlr4.util.StringUtils; +import org.apache.seatunnel.api.common.CommonOptions; +import org.apache.seatunnel.api.common.metrics.Counter; +import org.apache.seatunnel.api.common.metrics.MetricNames; +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; @@ -23,9 +31,6 @@ import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.transform.exception.ErrorDataTransformException; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; - import java.util.Collections; import java.util.List; @@ -37,14 +42,44 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected volatile CatalogTable outputCatalogTable; - public AbstractSeaTunnelTransform(@NonNull CatalogTable inputCatalogTable) { - this(inputCatalogTable, TransformCommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + protected String inputTableName; + + protected String outTableName; + + protected MetricsContext metricsContext; + + protected volatile Counter counter; + + @Override + public void open() { } public AbstractSeaTunnelTransform( - @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { - this.inputCatalogTable = inputCatalogTable; + @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + this( + config, + inputCatalogTable, + TransformCommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + } + + public AbstractSeaTunnelTransform( + @NonNull ReadonlyConfig config, + @NonNull CatalogTable catalogTable, + ErrorHandleWay rowErrorHandleWay) { + this.inputCatalogTable = catalogTable; this.rowErrorHandleWay = rowErrorHandleWay; + List pluginInputIdentifiers = config.get(CommonOptions.PLUGIN_INPUT); + String pluginOutIdentifiers = config.get(CommonOptions.PLUGIN_OUTPUT); + if (pluginInputIdentifiers != null && !pluginInputIdentifiers.isEmpty()) { + this.inputTableName = pluginInputIdentifiers.get(0); + } else { + this.inputTableName = catalogTable.getTableId().getTableName(); + } + if (!StringUtils.isEmpty(pluginOutIdentifiers)) { + this.outTableName = pluginOutIdentifiers; + } else { + this.outTableName = catalogTable.getTableId().getTableName(); + } } public CatalogTable getProducedCatalogTable() { @@ -105,4 +140,39 @@ public R transform(SeaTunnelRow row) { protected abstract TableSchema transformTableSchema(); protected abstract TableIdentifier transformTableIdentifier(); + + @Override + public void setMetricsContext(MetricsContext metricsContext) { + this.metricsContext = metricsContext; + } + + @Override + public MetricsContext getMetricsContext() { + return metricsContext; + } + + protected void hazelcastMetric(long size) { + if (metricsContext != null) { + metricsContext.counter(getTransformMetricName(this)).inc(size); + } + } + + protected void hazelcastMetric() { + if (metricsContext != null) { + metricsContext.counter(getTransformMetricName(this)).inc(); + } + } + + protected String getTransformMetricName(SeaTunnelTransform transformer) { + StringBuilder metricName = new StringBuilder(); + metricName + .append(MetricNames.TRANSFORM_COUNT) + .append("-") + .append(transformer.getPluginName()) + .append("-") + .append(inputTableName) + .append("-") + .append(outTableName); + return metricName.toString(); + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java index 855085a98b3..02f6e2c6287 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.common; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; @@ -25,8 +26,9 @@ public abstract class FilterRowTransform extends AbstractCatalogSupportMapTransform { - public FilterRowTransform(@NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + public FilterRowTransform( + @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(config, inputCatalogTable); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java index 84e3a9348d4..70b9b0bad86 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.common; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -42,13 +43,16 @@ public abstract class MultipleFieldOutputTransform extends AbstractCatalogSuppor private int[] fieldsIndex; private SeaTunnelRowContainerGenerator rowContainerGenerator; - public MultipleFieldOutputTransform(@NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + public MultipleFieldOutputTransform( + @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(config, inputCatalogTable); } public MultipleFieldOutputTransform( - @NonNull CatalogTable inputCatalogTable, ErrorHandleWay errorHandleWay) { - super(inputCatalogTable, errorHandleWay); + @NonNull ReadonlyConfig config, + @NonNull CatalogTable inputCatalogTable, + ErrorHandleWay errorHandleWay) { + super(config, inputCatalogTable, errorHandleWay); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java index 8768069ab84..921ab031f22 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.common; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -38,8 +39,9 @@ public abstract class SingleFieldOutputTransform extends AbstractCatalogSupportM private int fieldIndex; private SeaTunnelRowContainerGenerator rowContainerGenerator; - public SingleFieldOutputTransform(@NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + public SingleFieldOutputTransform( + @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { + super(config, inputCatalogTable); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java index 6ed09e211f2..cef7a2d74a4 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new CopyFieldTransform(CopyTransformConfig.of(config), inputCatalogTable); + return new CopyFieldTransform(config, inputCatalogTable); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java index 75712b5fc87..03517324a43 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.copy; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -46,9 +47,9 @@ public class CopyFieldTransform extends MultipleFieldOutputTransform { private List fieldOriginalIndexes; private List> fieldTypes; - public CopyFieldTransform(CopyTransformConfig copyTransformConfig, CatalogTable catalogTable) { - super(catalogTable); - this.config = copyTransformConfig; + public CopyFieldTransform(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { + super(readonlyConfig, catalogTable); + this.config = CopyTransformConfig.of(readonlyConfig); SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); initOutputFields(seaTunnelRowType, config.getFields()); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java index 7ff88d85d49..8bcb196333f 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java @@ -51,7 +51,7 @@ public class DynamicCompileTransform extends MultipleFieldOutputTransform { private AbstractParse DynamicCompileParse; public DynamicCompileTransform(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { - super(catalogTable); + super(readonlyConfig, catalogTable); CompileLanguage compileLanguage = readonlyConfig.get(DynamicCompileTransformConfig.COMPILE_LANGUAGE); // todo other compile diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java index 259f1b31634..36f99bc6773 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new FieldMapperTransform(FieldMapperTransformConfig.of(config), inputCatalogTable); + return new FieldMapperTransform(config, inputCatalogTable); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java index 5c9709c18a2..150ce64d777 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.fieldmapper; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -47,9 +48,9 @@ public class FieldMapperTransform extends AbstractCatalogSupportMapTransform { private List needReaderColIndex; public FieldMapperTransform( - @NonNull FieldMapperTransformConfig config, @NonNull CatalogTable catalogTable) { - super(catalogTable); - this.config = config; + @NonNull ReadonlyConfig readonlyConfig, @NonNull CatalogTable catalogTable) { + super(readonlyConfig, catalogTable); + this.config = FieldMapperTransformConfig.of(readonlyConfig); Map fieldMapper = config.getFieldMapper(); SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); List notFoundField = diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java index 53b66af3d84..03d761ebc18 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java @@ -54,7 +54,7 @@ public class FilterFieldTransform extends AbstractCatalogSupportMapTransform { public FilterFieldTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(catalogTable); + super(config, catalogTable); SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); includeFields = config.get(FilterFieldTransformConfig.INCLUDE_FIELDS); excludeFields = config.get(FilterFieldTransformConfig.EXCLUDE_FIELDS); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java index 9df88d50614..bbf4955eee2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java @@ -41,7 +41,7 @@ public class FilterRowKindTransform extends FilterRowTransform { public FilterRowKindTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + super(config, inputCatalogTable); initConfig(config); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java index f8e096d34f8..2afd5c1d5bc 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java @@ -34,8 +34,7 @@ public JsonPathMultiCatalogTransform( @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new JsonPathTransform( - JsonPathTransformConfig.of(config, inputCatalogTable), inputCatalogTable); + return new JsonPathTransform(config, inputCatalogTable); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java index 82c22bcb9a7..e3b542e71b4 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java @@ -18,6 +18,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -57,9 +58,12 @@ public class JsonPathTransform extends MultipleFieldOutputTransform { private int[] srcFieldIndexArr; - public JsonPathTransform(JsonPathTransformConfig config, CatalogTable catalogTable) { - super(catalogTable, config.getErrorHandleWay()); - this.config = config; + public JsonPathTransform(ReadonlyConfig config, CatalogTable catalogTable) { + super( + config, + catalogTable, + JsonPathTransformConfig.of(config, catalogTable).getErrorHandleWay()); + this.config = JsonPathTransformConfig.of(config, catalogTable); this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); init(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java index 2a1679b500d..d1209b0dd0c 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java @@ -44,7 +44,7 @@ public class MetadataTransform extends MultipleFieldOutputTransform { private Map metadataFieldMapping; public MetadataTransform(ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + super(config, inputCatalogTable); initOutputFields(inputCatalogTable, config.get(MetadataTransformConfig.METADATA_FIELDS)); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java index ce6d864da6f..020a5a81f5b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embadding/EmbeddingTransform.java @@ -54,7 +54,7 @@ public class EmbeddingTransform extends MultipleFieldOutputTransform { public EmbeddingTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + super(config, inputCatalogTable); this.config = config; initOutputFields( inputCatalogTable.getTableSchema().toPhysicalRowDataType(), diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java index c99b03776e9..fe069594e7a 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java @@ -47,7 +47,7 @@ public class LLMTransform extends SingleFieldOutputTransform { private Model model; public LLMTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + super(config, inputCatalogTable); this.config = config; this.outputDataType = SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType( diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java index b2c9fa44ce5..4491ef253a1 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java @@ -38,7 +38,7 @@ public class ReplaceTransform extends SingleFieldOutputTransform { public ReplaceTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + super(config, inputCatalogTable); this.config = config; initOutputFields( inputCatalogTable.getTableSchema().toPhysicalRowDataType(), diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java index 354e5a3dd5d..717836259f6 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java @@ -40,7 +40,7 @@ public class RowKindExtractorTransform extends SingleFieldOutputTransform { public RowKindExtractorTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(inputCatalogTable); + super(config, inputCatalogTable); this.config = config; this.transformType = config.get(RowKindExtractorTransformConfig.TRANSFORM_TYPE); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java index 52f033b10a0..06d247787e9 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new SplitTransform(SplitTransformConfig.of(config), inputCatalogTable); + return new SplitTransform(config, inputCatalogTable); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java index eac04aa1c9c..1f4f11f01b3 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.split; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -35,11 +36,9 @@ public class SplitTransform extends MultipleFieldOutputTransform { private final SplitTransformConfig splitTransformConfig; private final int splitFieldIndex; - public SplitTransform( - @NonNull SplitTransformConfig splitTransformConfig, - @NonNull CatalogTable catalogTable) { - super(catalogTable); - this.splitTransformConfig = splitTransformConfig; + public SplitTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { + super(config, catalogTable); + this.splitTransformConfig = SplitTransformConfig.of(config); SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); try { splitFieldIndex = seaTunnelRowType.indexOf(splitTransformConfig.getSplitField()); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index 230541a7bf2..990e3d3bcf0 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.sql; -import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.ReadonlyConfig; @@ -64,23 +63,14 @@ public class SQLTransform extends AbstractCatalogSupportFlatMapTransform { private transient SQLEngine sqlEngine; - private final String inputTableName; - public SQLTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(catalogTable); + super(config, catalogTable); this.query = config.get(KEY_QUERY); if (config.getOptional(KEY_ENGINE).isPresent()) { this.engineType = EngineType.valueOf(config.get(KEY_ENGINE).toUpperCase()); } else { this.engineType = ZETA; } - - List pluginInputIdentifiers = config.get(CommonOptions.PLUGIN_INPUT); - if (pluginInputIdentifiers != null && !pluginInputIdentifiers.isEmpty()) { - this.inputTableName = pluginInputIdentifiers.get(0); - } else { - this.inputTableName = catalogTable.getTableId().getTableName(); - } } @Override @@ -90,6 +80,7 @@ public String getPluginName() { @Override public void open() { + super.open(); sqlEngine = SQLEngineFactory.getSQLEngine(engineType); sqlEngine.init( inputTableName, From eac26d172b5bc9de664e79ed9fbdf23cdd348dea Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 29 Nov 2024 15:55:46 +0800 Subject: [PATCH 02/56] [Feature][transform-v2] Support transform metrics --- .../command/ClientExecuteCommand.java | 57 ++++++++++--------- .../engine/client/job/JobMetricsRunner.java | 3 +- ...bstractCatalogSupportFlatMapTransform.java | 5 +- .../AbstractCatalogSupportMapTransform.java | 5 +- .../common/AbstractSeaTunnelTransform.java | 11 ++-- .../seatunnel/transform/sql/SQLTransform.java | 1 - 6 files changed, 43 insertions(+), 39 deletions(-) diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java index 61d45fee028..40bc9bc3c05 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java @@ -17,13 +17,6 @@ package org.apache.seatunnel.core.starter.seatunnel.command; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.hazelcast.client.config.ClientConfig; -import com.hazelcast.core.HazelcastInstance; -import com.hazelcast.instance.impl.HazelcastInstanceFactory; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.collections4.MapUtils; -import org.apache.commons.lang3.StringUtils; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; import org.apache.seatunnel.core.starter.command.Command; @@ -47,6 +40,15 @@ import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelNodeContext; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.StringUtils; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.core.HazelcastInstance; +import com.hazelcast.instance.impl.HazelcastInstanceFactory; +import lombok.extern.slf4j.Slf4j; + import java.nio.file.Path; import java.time.Duration; import java.time.LocalDateTime; @@ -62,9 +64,7 @@ import static org.apache.seatunnel.core.starter.utils.FileUtils.checkConfigExist; -/** - * This command is used to execute the SeaTunnel engine job by SeaTunnel API. - */ +/** This command is used to execute the SeaTunnel engine job by SeaTunnel API. */ @Slf4j public class ClientExecuteCommand implements Command { @@ -231,29 +231,30 @@ public void execute() throws CommandExecuteException { if (jobMetricsSummary != null) { // print job statistics information when job finished StringBuilder logMessage = new StringBuilder(); - logMessage.append(StringFormatUtils.formatTable( - "Job Statistic Information", - "Start Time", - DateTimeUtils.toString( - startTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), - "End Time", - DateTimeUtils.toString( - endTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), - "Total Time(s)", - Duration.between(startTime, endTime).getSeconds(), - "Total Read Count", - jobMetricsSummary.getSourceReadCount(), - "Total Write Count", - jobMetricsSummary.getSinkWriteCount(), - "Total Failed Count", - jobMetricsSummary.getSourceReadCount() - - jobMetricsSummary.getSinkWriteCount())); + logMessage.append( + StringFormatUtils.formatTable( + "Job Statistic Information", + "Start Time", + DateTimeUtils.toString( + startTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), + "End Time", + DateTimeUtils.toString( + endTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), + "Total Time(s)", + Duration.between(startTime, endTime).getSeconds(), + "Total Read Count", + jobMetricsSummary.getSourceReadCount(), + "Total Write Count", + jobMetricsSummary.getSinkWriteCount(), + "Total Failed Count", + jobMetricsSummary.getSourceReadCount() + - jobMetricsSummary.getSinkWriteCount())); String[] transformInfos = null; if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformCountMap())) { transformInfos = new String [jobMetricsSummary.getTransformCountMap().entrySet().size() * 2 - + 1]; + + 1]; transformInfos[0] = "Transform Information"; int index = 0; for (Map.Entry entry : diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java index a08121aadfc..4e116c6cb55 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java @@ -17,11 +17,12 @@ package org.apache.seatunnel.engine.client.job; -import org.apache.commons.collections4.MapUtils; import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; import org.apache.seatunnel.engine.client.SeaTunnelClient; +import org.apache.commons.collections4.MapUtils; + import lombok.AllArgsConstructor; import lombok.Data; import lombok.extern.slf4j.Slf4j; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index 8336a950ab3..e0a5c4df059 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -16,13 +16,14 @@ */ package org.apache.seatunnel.transform.common; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + import java.util.List; @Slf4j diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 8ace6467495..4e400d1e117 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -17,13 +17,14 @@ package org.apache.seatunnel.transform.common; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + @Slf4j public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index e8c238e0c6f..dc40e3a27ac 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -16,9 +16,6 @@ */ package org.apache.seatunnel.transform.common; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.apache.groovy.parser.antlr4.util.StringUtils; import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.common.metrics.Counter; import org.apache.seatunnel.api.common.metrics.MetricNames; @@ -31,6 +28,11 @@ import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.transform.exception.ErrorDataTransformException; +import org.apache.groovy.parser.antlr4.util.StringUtils; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + import java.util.Collections; import java.util.List; @@ -51,8 +53,7 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected volatile Counter counter; @Override - public void open() { - } + public void open() {} public AbstractSeaTunnelTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index 990e3d3bcf0..90d4d371102 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -80,7 +80,6 @@ public String getPluginName() { @Override public void open() { - super.open(); sqlEngine = SQLEngineFactory.getSQLEngine(engineType); sqlEngine.init( inputTableName, From d17353b64bda0460a715b1d73ba84ae4f36ed37c Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 29 Nov 2024 15:58:11 +0800 Subject: [PATCH 03/56] [Feature][transform-v2] Support transform metrics --- .../org/apache/seatunnel/api/common/metrics/MetricsContext.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java index ce4b486e23c..a8875d91d7d 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricsContext.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.api.common.metrics; -import org.checkerframework.checker.units.qual.C; - public interface MetricsContext { /** From 7782507c1a10c547e022e20feb3dec4a286edbb4 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 29 Nov 2024 17:09:41 +0800 Subject: [PATCH 04/56] [Feature][transform-v2] Support transform metrics --- .../transform/JsonPathTransformTest.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java index c34af87e359..3b557fbc358 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java @@ -62,8 +62,7 @@ public void testJsonPath() { new SeaTunnelRowType( new String[] {"data"}, new SeaTunnelDataType[] {BasicType.STRING_TYPE})); - JsonPathTransform transform = - new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + JsonPathTransform transform = new JsonPathTransform(config, table); CatalogTable outputTable = transform.getProducedCatalogTable(); SeaTunnelRow outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f1\": 1}"})); @@ -88,8 +87,7 @@ public void testErrorHandleWay() { new SeaTunnelRowType( new String[] {"data"}, new SeaTunnelDataType[] {BasicType.STRING_TYPE})); - JsonPathTransform transform = - new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + JsonPathTransform transform = new JsonPathTransform(config, table); CatalogTable outputTable = transform.getProducedCatalogTable(); final JsonPathTransform finalTransform = transform; Assertions.assertThrows( @@ -109,7 +107,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.FAIL.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); JsonPathTransform finalTransform1 = transform; Assertions.assertThrows( @@ -129,7 +127,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); SeaTunnelRow outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNotNull(outputRow); @@ -148,7 +146,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP_ROW.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNull(outputRow); @@ -164,7 +162,7 @@ public void testErrorHandleWay() { JsonPathTransformConfig.PATH.key(), "$.f1", JsonPathTransformConfig.DEST_FIELD.key(), "f1"))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNull(outputRow); @@ -185,7 +183,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.FAIL.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); try { outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); @@ -210,7 +208,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNotNull(outputRow); @@ -232,7 +230,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP_ROW.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + transform = new JsonPathTransform(config, table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNull(outputRow); @@ -266,8 +264,7 @@ public void testOutputColumn() { new HashMap<>(), new ArrayList<>(), null); - JsonPathTransform transform = - new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); + JsonPathTransform transform = new JsonPathTransform(config, table); CatalogTable outputCatalogTable = transform.getProducedCatalogTable(); Column f1 = outputCatalogTable.getTableSchema().getColumn("f1"); Assertions.assertEquals(BasicType.STRING_TYPE, f1.getDataType()); From 5a68e50ed54a5409f14e3069f4ccba0fbbbd4613 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 29 Nov 2024 17:39:00 +0800 Subject: [PATCH 05/56] [Feature][transform-v2] Support transform metrics --- .../common/utils/StringFormatUtilsTest.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java index b813ab60dd4..a138a2ceafa 100644 --- a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java +++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java @@ -41,15 +41,15 @@ public void testStringFormat() { Assertions.assertEquals( s, "\n" - + "***********************************************\n" + + "*************************************************************\n" + " Job Statistic Information\n" - + "***********************************************\n" - + "Start Time : 2023-01-11 00:00:00\n" - + "End Time : 2023-01-11 00:00:00\n" - + "Total Time(s) : 0\n" - + "Total Read Count : 0\n" - + "Total Write Count : 0\n" - + "Total Failed Count : 0\n" - + "***********************************************\n"); + + "*************************************************************\n" + + "Start Time : 2023-01-11 00:00:00\n" + + "End Time : 2023-01-11 00:00:00\n" + + "Total Time(s) : 0\n" + + "Total Read Count : 0\n" + + "Total Write Count : 0\n" + + "Total Failed Count : 0\n" + + "*************************************************************\n"); } } From 11f4e3a4d805a190f0f6c3640202fa2157154867 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Mon, 2 Dec 2024 11:31:27 +0800 Subject: [PATCH 06/56] [Feature][transform-v2] Support transform metrics --- .../seatunnel/api/transform/SeaTunnelTransform.java | 2 -- .../transform/common/AbstractSeaTunnelTransform.java | 10 +--------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index 679391bb5cb..a1a59b7cd26 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -57,6 +57,4 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve default void close() {} void setMetricsContext(MetricsContext metricsContext); - - MetricsContext getMetricsContext(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index dc40e3a27ac..8fef128fd6d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.common.metrics.Counter; import org.apache.seatunnel.api.common.metrics.MetricNames; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.configuration.ReadonlyConfig; @@ -50,8 +49,6 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected MetricsContext metricsContext; - protected volatile Counter counter; - @Override public void open() {} @@ -147,11 +144,6 @@ public void setMetricsContext(MetricsContext metricsContext) { this.metricsContext = metricsContext; } - @Override - public MetricsContext getMetricsContext() { - return metricsContext; - } - protected void hazelcastMetric(long size) { if (metricsContext != null) { metricsContext.counter(getTransformMetricName(this)).inc(size); @@ -160,7 +152,7 @@ protected void hazelcastMetric(long size) { protected void hazelcastMetric() { if (metricsContext != null) { - metricsContext.counter(getTransformMetricName(this)).inc(); + hazelcastMetric(1); } } From 6f691245bdd82f323de2358cab21bd4f74f023fe Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Mon, 2 Dec 2024 11:43:40 +0800 Subject: [PATCH 07/56] [Feature][transform-v2] Support transform metrics --- .../transform/common/AbstractMultiCatalogTransform.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index 9cb14dbdb0e..19c86f10ce0 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -96,11 +96,6 @@ public void setMetricsContext(MetricsContext metricsContext) { this.metricsContext = metricsContext; } - @Override - public MetricsContext getMetricsContext() { - return metricsContext; - } - protected abstract SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config); From d2b2ec7c7663c6b58df065a4b215da4e09480ba6 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 3 Dec 2024 11:40:10 +0800 Subject: [PATCH 08/56] [Feature][transform-v2] Support transform metrics --- .../common/utils/StringFormatUtils.java | 8 +- .../command/ClientExecuteCommand.java | 35 +++--- .../engine/e2e/MultiTableMetricsIT.java | 44 ++++++- .../batch_fake_multi_table_to_console.conf | 35 +++++- .../engine/client/job/JobClient.java | 109 +++++++++++++++--- .../engine/client/job/JobMetricsRunner.java | 39 ++++--- .../server/rest/service/BaseService.java | 87 +++++++++++++- .../common/AbstractSeaTunnelTransform.java | 8 +- 8 files changed, 299 insertions(+), 66 deletions(-) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java index 28e71d210be..9a52118145c 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java @@ -32,15 +32,15 @@ public static String formatTable(Object... objects) { int kvNum = (objects.length - 1) / 2; String template = "\n" - + "*************************************************************" + + "***********************************************" + "\n" + String.join("", Collections.nCopies(blankNum, " ")) + "%s" + "\n" - + "*************************************************************" + + "***********************************************" + "\n" - + String.join("", Collections.nCopies(kvNum, "%-40s: %19s\n")) - + "*************************************************************\n"; + + String.join("", Collections.nCopies(kvNum, "%-26s: %19s\n")) + + "***********************************************\n"; return String.format(template, objects); } } diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java index 40bc9bc3c05..25d67a37c76 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java @@ -249,23 +249,26 @@ public void execute() throws CommandExecuteException { "Total Failed Count", jobMetricsSummary.getSourceReadCount() - jobMetricsSummary.getSinkWriteCount())); - String[] transformInfos = null; - if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformCountMap())) { - transformInfos = - new String - [jobMetricsSummary.getTransformCountMap().entrySet().size() * 2 - + 1]; - transformInfos[0] = "Transform Information"; - int index = 0; - for (Map.Entry entry : - jobMetricsSummary.getTransformCountMap().entrySet()) { - transformInfos[++index] = entry.getKey(); - transformInfos[++index] = String.valueOf(entry.getValue()); - } - } - if (Objects.nonNull(transformInfos)) { - logMessage.append(StringFormatUtils.formatTable(transformInfos)); + if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformMetricsMaps())) { + jobMetricsSummary + .getTransformMetricsMaps() + .forEach( + (tableName, metrics) -> { + String[] transformInfos = + new String[metrics.entrySet().size() * 2 + 1]; + transformInfos[0] = "Transform Information " + tableName; + int index = 0; + for (Map.Entry entry : metrics.entrySet()) { + transformInfos[++index] = entry.getKey(); + transformInfos[++index] = + String.valueOf(entry.getValue()); + } + if (Objects.nonNull(transformInfos)) { + logMessage.append( + StringFormatUtils.formatTable(transformInfos)); + } + }); } log.info("{}", logMessage); } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index 36fb2bef996..ca000b01173 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -132,7 +132,25 @@ public void multiTableMetrics() { equalTo(String.valueOf(dataSize * 10))) .body( "metrics.TableSinkWriteBytes.'fake.public.table2'", - equalTo(String.valueOf(dataSize * 5))); + equalTo(String.valueOf(dataSize * 5))) + .body( + "metrics.TransformCount.'fake.table1'.'fake1->fake3'", + equalTo("10")) + .body( + "metrics.TransformCount.'fake.table1'.'fake3->fake4'", + equalTo("10")) + .body( + "metrics.TransformCount.'fake.table1'.'fake4->fake5'", + equalTo("10")) + .body( + "metrics.TransformCount.'fake.public.table2'.'fake2->fake6'", + equalTo("5")) + .body( + "metrics.TransformCount.'fake.public.table2'.'fake6->fake7'", + equalTo("5")) + .body( + "metrics.TransformCount.'fake.public.table2'.'fake7->fake8'", + equalTo("5")); Assertions.assertTrue( Double.parseDouble(response.path("metrics.SourceReceivedQPS")) > 0 @@ -178,6 +196,30 @@ public void multiTableMetrics() { && Double.parseDouble( response.path( "metrics.TableSinkWriteBytesPerSeconds.'fake.public.table2'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TransformCount.'fake.table1'.'fake1->fake3'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TransformCount.'fake.table1'.'fake3->fake4'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TransformCount.'fake.table1'.'fake4->fake5'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TransformCount.'fake.public.table2'.'fake2->fake6'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TransformCount.'fake.public.table2'.'fake6->fake7'")) + > 0 + && Double.parseDouble( + response.path( + "metrics.TransformCount.'fake.public.table2'.'fake7->fake8'")) > 0); }); } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf index d453bf9ed62..b8230e4c9bd 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf @@ -116,13 +116,44 @@ source { } transform { + Sql { + plugin_input = "fake1" + plugin_output = "fake3" + query = "select * from fake1" + } + Sql { + plugin_input = "fake3" + plugin_output = "fake4" + query = "select * from fake3" + } + Sql { + plugin_input = "fake4" + plugin_output = "fake5" + query = "select * from fake4" + } + Sql { + plugin_input = "fake2" + plugin_output = "fake6" + query = "select * from fake2" + } + Sql { + plugin_input = "fake6" + plugin_output = "fake7" + query = "select * from fake6" + } + Sql { + plugin_input = "fake7" + plugin_output = "fake8" + query = "select * from fake7" + } + } sink { console { - plugin_input = "fake1" + plugin_input = "fake5" } console { - plugin_input = "fake2" + plugin_input = "fake8" } } diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java index c88b3ea6cd6..912edf480e0 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.engine.client.SeaTunnelHazelcastClient; import org.apache.seatunnel.engine.client.util.ContentFormatUtil; import org.apache.seatunnel.engine.common.Constant; @@ -41,13 +42,19 @@ import org.apache.seatunnel.engine.core.protocol.codec.SeaTunnelListJobStatusCodec; import org.apache.seatunnel.engine.core.protocol.codec.SeaTunnelSavePointJobCodec; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.collections4.map.HashedMap; + import lombok.NonNull; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.seatunnel.api.common.metrics.MetricNames.TRANSFORM_COUNT; public class JobClient { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -158,7 +165,7 @@ public JobDAGInfo getJobInfo(Long jobId) { public JobMetricsRunner.JobMetricsSummary getJobMetricsSummary(Long jobId) { long sourceReadCount = 0L; long sinkWriteCount = 0L; - Map transformCountMap = new HashMap<>(); + Map> transformCountMap = new HashMap<>(); String jobMetrics = getJobMetrics(jobId); try { JsonNode jsonNode = OBJECT_MAPPER.readTree(jobMetrics); @@ -180,26 +187,90 @@ public JobMetricsRunner.JobMetricsSummary getJobMetricsSummary(Long jobId) { } } - private Map extractTransformKeys(JsonNode rootNode) { - Map transformCountMap = new TreeMap<>(); - Iterator> fields = rootNode.fields(); - while (fields.hasNext()) { - Map.Entry field = fields.next(); - String key = field.getKey(); - JsonNode transforms = field.getValue(); - long transformCount = 0L; - - if (key.startsWith("TransformCount-")) { - for (int i = 0; i < transforms.size(); i++) { - JsonNode transform = transforms.get(i); - transformCount += transform.get("value").asLong(); - } - transformCountMap.put(key, transformCount); - } - } + private Map> extractTransformKeys(JsonNode rootNode) { + Map> transformCountMap = new TreeMap<>(); + Map>> transformMetricsMaps = new HashedMap(); + rootNode.fieldNames() + .forEachRemaining( + metricName -> { + if (metricName.contains("Transform")) { + processTransformMetric(transformMetricsMaps, metricName, rootNode); + } + }); + transformMetricsMaps.forEach( + (metricName, metricMap) -> { + metricMap.forEach( + (tableName, pathMap) -> { + transformCountMap.put(tableName, aggregateTreeMap(pathMap, false)); + }); + }); return transformCountMap; } + private Map aggregateTreeMap(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(), + (v1, v2) -> v1, + TreeMap::new)) + : inputMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.path("value").asLong()) + .sum(), + (v1, v2) -> v1, + TreeMap::new)); + } + + private void processTransformMetric( + Map>> transformMetricsMaps, + String metricName, + JsonNode jobMetricsStr) { + if (metricName.contains(TRANSFORM_COUNT)) { + processTransformMetric( + transformMetricsMaps, TRANSFORM_COUNT, metricName, jobMetricsStr); + } + } + + private void processTransformMetric( + Map>> transformMetricsMaps, + String key, + String metricName, + JsonNode jobMetricsStr) { + Map> transformMetricsMap = transformMetricsMaps.get(key); + if (MapUtils.isEmpty(transformMetricsMap)) { + transformMetricsMap = new TreeMap<>(); + transformMetricsMaps.put(key, transformMetricsMap); + } + String tableName = TablePath.of(metricName.split("#")[1]).getFullName(); + String path = metricName.split("#")[2]; + Map pathMap; + if (transformMetricsMap.containsKey(tableName)) { + pathMap = transformMetricsMap.get(tableName); + } else { + pathMap = new TreeMap<>(); + } + pathMap.put(path, jobMetricsStr.get(metricName)); + transformMetricsMap.put(tableName, pathMap); + } + public List getCheckpointData(Long jobId) { return hazelcastClient .getSerializationService() diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java index 4e116c6cb55..060f9af3699 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java @@ -77,30 +77,31 @@ public void run() { DateTimeUtils.toString( now, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS))); - String[] transformInfos = null; - if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformCountMap())) { - transformInfos = - new String - [jobMetricsSummary.getTransformCountMap().entrySet().size() * 2 - + 1]; - transformInfos[0] = "Transform Information"; - int index = 0; - for (Map.Entry entry : - jobMetricsSummary.getTransformCountMap().entrySet()) { - transformInfos[++index] = entry.getKey(); - transformInfos[++index] = String.valueOf(entry.getValue()); - } - } - - if (Objects.nonNull(transformInfos)) { - logMessage.append(StringFormatUtils.formatTable(transformInfos)); + if (MapUtils.isNotEmpty(jobMetricsSummary.getTransformMetricsMaps())) { + jobMetricsSummary + .getTransformMetricsMaps() + .forEach( + (tableName, metrics) -> { + String[] transformInfos = + new String[metrics.entrySet().size() * 2 + 1]; + transformInfos[0] = "Transform Information " + tableName; + int index = 0; + for (Map.Entry entry : metrics.entrySet()) { + transformInfos[++index] = entry.getKey(); + transformInfos[++index] = String.valueOf(entry.getValue()); + } + if (Objects.nonNull(transformInfos)) { + logMessage.append( + StringFormatUtils.formatTable(transformInfos)); + } + }); } log.info("{}", logMessage); lastRunTime = now; lastReadCount = jobMetricsSummary.getSourceReadCount(); lastWriteCount = jobMetricsSummary.getSinkWriteCount(); - transformCountMap = jobMetricsSummary.getTransformCountMap(); + // transformCountMap = jobMetricsSummary.getTransformCountMap(); } catch (Exception e) { log.warn("Failed to get job metrics summary, it maybe first-run"); } @@ -111,6 +112,6 @@ public void run() { public static class JobMetricsSummary { private long sourceReadCount; private long sinkWriteCount; - private Map transformCountMap; + private Map> transformMetricsMaps; } } 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 index fc99feb2649..10cd0cce83e 100644 --- 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 @@ -51,6 +51,8 @@ import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; import org.apache.seatunnel.engine.server.utils.RestUtil; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.collections4.map.HashedMap; import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; @@ -73,6 +75,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -86,6 +89,7 @@ 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.api.common.metrics.MetricNames.TRANSFORM_COUNT; 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; @@ -268,7 +272,7 @@ private Map getJobMetrics(String jobMetrics) { new HashMap<>(), // Source Received Bytes Per Second new HashMap<>() // Sink Write Bytes Per Second }; - + Map>> transformMetricsMaps = new HashedMap(); try { JsonNode jobMetricsStr = new ObjectMapper().readTree(jobMetrics); @@ -276,6 +280,11 @@ private Map getJobMetrics(String jobMetrics) { .fieldNames() .forEachRemaining( metricName -> { + if (metricName.contains("Transform")) { + processTransformMetric( + transformMetricsMaps, metricName, jobMetricsStr); + return; + } if (metricName.contains("#")) { String tableName = TablePath.of(metricName.split("#")[1]).getFullName(); @@ -308,9 +317,53 @@ private Map getJobMetrics(String jobMetrics) { ArrayUtils.addAll(countMetricsNames, rateMetricsNames), metricsSums.length); + transformMetricsMaps.forEach( + (metricName, metricMap) -> { + Map transformMetricsMap = new TreeMap<>(); + metricMap.forEach( + (tableName, pathMap) -> { + transformMetricsMap.put( + tableName, aggregateTreeMap(pathMap, false)); + }); + + metricsMap.put(metricName, transformMetricsMap); + }); + return metricsMap; } + private void processTransformMetric( + Map>> transformMetricsMaps, + String metricName, + JsonNode jobMetricsStr) { + if (metricName.contains(TRANSFORM_COUNT)) { + processTransformMetric( + transformMetricsMaps, TRANSFORM_COUNT, metricName, jobMetricsStr); + } + } + + private void processTransformMetric( + Map>> transformMetricsMaps, + String key, + String metricName, + JsonNode jobMetricsStr) { + Map> transformMetricsMap = transformMetricsMaps.get(key); + if (MapUtils.isEmpty(transformMetricsMap)) { + transformMetricsMap = new TreeMap<>(); + transformMetricsMaps.put(key, transformMetricsMap); + } + String tableName = TablePath.of(metricName.split("#")[1]).getFullName(); + String path = metricName.split("#")[2]; + Map pathMap; + if (transformMetricsMap.containsKey(tableName)) { + pathMap = transformMetricsMap.get(tableName); + } else { + pathMap = new TreeMap<>(); + } + pathMap.put(path, jobMetricsStr.get(metricName)); + transformMetricsMap.put(tableName, pathMap); + } + private void processMetric( String metricName, String tableName, @@ -415,6 +468,38 @@ private Map aggregateMap(Map inputMap, boolean .sum())); } + private Map aggregateTreeMap(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(), + (v1, v2) -> v1, + TreeMap::new)) + : inputMap.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> + StreamSupport.stream( + entry.getValue().spliterator(), + false) + .mapToLong( + node -> node.path("value").asLong()) + .sum(), + (v1, v2) -> v1, + TreeMap::new)); + } + private JsonObject metricsToJsonObject(Map jobMetrics) { JsonObject members = new JsonObject(); jobMetrics.forEach( diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 8fef128fd6d..3a6051326d1 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -160,11 +160,11 @@ protected String getTransformMetricName(SeaTunnelTransform transformer) { StringBuilder metricName = new StringBuilder(); metricName .append(MetricNames.TRANSFORM_COUNT) - .append("-") - .append(transformer.getPluginName()) - .append("-") + .append("#") + .append(inputCatalogTable.getTablePath().getFullName()) + .append("#") .append(inputTableName) - .append("-") + .append("->") .append(outTableName); return metricName.toString(); } From 4f4de4064b68952d4a026798b3e4b003c4bd56ac Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 3 Dec 2024 11:43:38 +0800 Subject: [PATCH 09/56] [Feature][transform-v2] Support transform metrics --- .../common/utils/StringFormatUtilsTest.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java index a138a2ceafa..b813ab60dd4 100644 --- a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java +++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/StringFormatUtilsTest.java @@ -41,15 +41,15 @@ public void testStringFormat() { Assertions.assertEquals( s, "\n" - + "*************************************************************\n" + + "***********************************************\n" + " Job Statistic Information\n" - + "*************************************************************\n" - + "Start Time : 2023-01-11 00:00:00\n" - + "End Time : 2023-01-11 00:00:00\n" - + "Total Time(s) : 0\n" - + "Total Read Count : 0\n" - + "Total Write Count : 0\n" - + "Total Failed Count : 0\n" - + "*************************************************************\n"); + + "***********************************************\n" + + "Start Time : 2023-01-11 00:00:00\n" + + "End Time : 2023-01-11 00:00:00\n" + + "Total Time(s) : 0\n" + + "Total Read Count : 0\n" + + "Total Write Count : 0\n" + + "Total Failed Count : 0\n" + + "***********************************************\n"); } } From 048e522f99281fa8afba561735ef5abd86da7b2b Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 3 Dec 2024 13:51:42 +0800 Subject: [PATCH 10/56] [Feature][transform-v2] Support transform metrics --- .../seatunnel/transform/fieldmapper/FieldMapperTransform.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java index 0b64dfa6bce..ea6ce0168a1 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java @@ -17,9 +17,9 @@ package org.apache.seatunnel.transform.fieldmapper; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.shade.com.google.common.collect.Lists; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; From fc0a42925b22259499a6521d6392a899ba0e457f Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 3 Dec 2024 16:23:32 +0800 Subject: [PATCH 11/56] [Feature][transform-v2] Support transform metrics --- .../common/AbstractCatalogSupportFlatMapTransform.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index e0a5c4df059..f7990a1abbe 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; +import org.apache.commons.collections4.CollectionUtils; + import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -39,7 +41,9 @@ public AbstractCatalogSupportFlatMapTransform( @Override public List flatMap(SeaTunnelRow row) { List results = transform(row); - hazelcastMetric(results.size()); + if (CollectionUtils.isNotEmpty(results)) { + hazelcastMetric(results.size()); + } return results; } } From 9bdbc3a5da93ead88497907e8d17bbdb55fd627c Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 3 Dec 2024 16:26:29 +0800 Subject: [PATCH 12/56] [Feature][transform-v2] Support transform metrics --- .../common/AbstractCatalogSupportMapTransform.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 4e400d1e117..b47600b3457 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -25,6 +25,8 @@ import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import java.util.Objects; + @Slf4j public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform @@ -44,7 +46,10 @@ public AbstractCatalogSupportMapTransform( @Override public SeaTunnelRow map(SeaTunnelRow row) { SeaTunnelRow result = transform(row); - hazelcastMetric(); + if (Objects.nonNull(result)) { + hazelcastMetric(); + } + return result; } } From 4c60cf3e328fbe37f648d9af0b5eeac30df6d689 Mon Sep 17 00:00:00 2001 From: CosmosNi <40288034+CosmosNi@users.noreply.github.com> Date: Tue, 3 Dec 2024 22:46:17 +0800 Subject: [PATCH 13/56] Update Hive.md --- docs/zh/connector-v2/source/Hive.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/connector-v2/source/Hive.md b/docs/zh/connector-v2/source/Hive.md index 00a322cc408..e32239ebac9 100644 --- a/docs/zh/connector-v2/source/Hive.md +++ b/docs/zh/connector-v2/source/Hive.md @@ -197,7 +197,7 @@ source { sink { Assert { - source_table_name = hive_source + plugin_input = hive_source rules { row_rules = [ { From ad5a696889d66b90f819a9fe04788f259c597fb9 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 4 Dec 2024 09:25:24 +0800 Subject: [PATCH 14/56] [Feature][transform-v2] update `plugin_input` and `plugin_output` --- docs/en/connector-v2/source/Hive.md | 4 ++-- docs/zh/connector-v2/source/Hive.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/connector-v2/source/Hive.md b/docs/en/connector-v2/source/Hive.md index b8fcea0d694..527a94fc94a 100644 --- a/docs/en/connector-v2/source/Hive.md +++ b/docs/en/connector-v2/source/Hive.md @@ -193,7 +193,7 @@ source { table_name = "default.test_hive_sink_on_hdfs_with_kerberos" metastore_uri = "thrift://metastore:9083" hive.hadoop.conf-path = "/tmp/hadoop" - result_table_name = hive_source + plugin_output = hive_source hive_site_path = "/tmp/hive-site.xml" kerberos_principal = "hive/metastore.seatunnel@EXAMPLE.COM" kerberos_keytab_path = "/tmp/hive.keytab" @@ -203,7 +203,7 @@ source { sink { Assert { - source_table_name = hive_source + plugin_input = hive_source rules { row_rules = [ { diff --git a/docs/zh/connector-v2/source/Hive.md b/docs/zh/connector-v2/source/Hive.md index e32239ebac9..094d701b455 100644 --- a/docs/zh/connector-v2/source/Hive.md +++ b/docs/zh/connector-v2/source/Hive.md @@ -158,7 +158,7 @@ source { table_name = "default.test_hive_sink_on_hdfs_with_kerberos" metastore_uri = "thrift://metastore:9083" hive.hadoop.conf-path = "/tmp/hadoop" - result_table_name = hive_source + plugin_output = hive_source hive_site_path = "/tmp/hive-site.xml" kerberos_principal = "hive/metastore.seatunnel@EXAMPLE.COM" kerberos_keytab_path = "/tmp/hive.keytab" @@ -187,7 +187,7 @@ source { table_name = "default.test_hive_sink_on_hdfs_with_kerberos" metastore_uri = "thrift://metastore:9083" hive.hadoop.conf-path = "/tmp/hadoop" - result_table_name = hive_source + plugin_output = hive_source hive_site_path = "/tmp/hive-site.xml" kerberos_principal = "hive/metastore.seatunnel@EXAMPLE.COM" kerberos_keytab_path = "/tmp/hive.keytab" From 0fe280dfe12420b5c405ec9108cd1857dd3679d7 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Thu, 5 Dec 2024 09:18:10 +0800 Subject: [PATCH 15/56] [Feature][transform-v2] resolve kafka it execute timeout --- .github/workflows/backend.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index f75a867b4c3..6e924096e8c 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -1187,7 +1187,7 @@ jobs: matrix: java: [ '8', '11' ] os: [ 'ubuntu-latest' ] - timeout-minutes: 120 + timeout-minutes: 150 steps: - uses: actions/checkout@v2 - name: Set up JDK ${{ matrix.java }} From 2fd9a0f5d32e6f0d6abd05ed76b9ccfaca5a4a59 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 6 Dec 2024 09:27:57 +0800 Subject: [PATCH 16/56] [Feature][transform-v2] resolve kafka it execute timeout --- .github/workflows/backend.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index e9a6da09adc..eff3cc4b418 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -1191,7 +1191,7 @@ jobs: matrix: java: [ '8', '11' ] os: [ 'ubuntu-latest' ] - timeout-minutes: 150 + timeout-minutes: 180 steps: - uses: actions/checkout@v2 - name: Set up JDK ${{ matrix.java }} From 8ce0afa458f6ede8d7f2e33f64b4835890b38628 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 6 Dec 2024 10:24:50 +0800 Subject: [PATCH 17/56] [Feature][transform-v2] TableRenameTransform support --- .../TableRenameMultiCatalogTransform.java | 2 +- .../rename/TableRenameTransform.java | 7 ++-- .../rename/TableRenameTransformTest.java | 42 +++++++++++++++---- 3 files changed, 38 insertions(+), 13 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java index 67cff881da3..24610e64a24 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable table, ReadonlyConfig config) { - return new TableRenameTransform(TableRenameConfig.of(config), table); + return new TableRenameTransform(config, table); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java index 7b56a2e2ba2..82a2f3f6c67 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -56,10 +57,10 @@ public class TableRenameTransform extends AbstractCatalogSupportMapTransform { private TablePath outputTablePath; private String outputTableId; - public TableRenameTransform(TableRenameConfig config, CatalogTable table) { - super(table); + public TableRenameTransform(ReadonlyConfig readonlyConfig, CatalogTable table) { + super(readonlyConfig, table); + config = TableRenameConfig.of(readonlyConfig); this.inputTable = table; - this.config = config; } @Override diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java index d1d756922e0..ca319cf7793 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.rename; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -33,6 +34,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; public class TableRenameTransformTest { @@ -93,7 +95,13 @@ public void testRename() { DEFAULT_TABLE.getTableId(), PhysicalColumn.of("f4", BasicType.LONG_TYPE, null, null, true, null, null)); - TableRenameConfig config = new TableRenameConfig().setConvertCase(ConvertCase.LOWER); + ReadonlyConfig config = + ReadonlyConfig.fromMap( + new HashMap() { + { + put("convert_case", "LOWER"); + } + }); TableRenameTransform transform = new TableRenameTransform(config, DEFAULT_TABLE); List outputCatalogTable = transform.getProducedCatalogTables(); @@ -106,7 +114,14 @@ public void testRename() { Assertions.assertEquals( "database-x.schema-x.table-x", outputEvent.tablePath().getFullName()); - config = new TableRenameConfig().setConvertCase(ConvertCase.UPPER); + config = + ReadonlyConfig.fromMap( + new HashMap() { + { + put("convert_case", "UPPER"); + } + }); + transform = new TableRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTables(); outputRow = transform.map(inputRow); @@ -117,8 +132,15 @@ public void testRename() { Assertions.assertEquals("DATABASE-X.SCHEMA-X.TABLE-X", outputRow.getTableId()); Assertions.assertEquals( "DATABASE-X.SCHEMA-X.TABLE-X", outputEvent.tablePath().getFullName()); + config = + ReadonlyConfig.fromMap( + new HashMap() { + { + put("prefix", "user-"); + put("suffix", "-table"); + } + }); - config = new TableRenameConfig().setPrefix("user-").setSuffix("-table"); transform = new TableRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTables(); outputRow = transform.map(inputRow); @@ -131,12 +153,14 @@ public void testRename() { "Database-x.Schema-x.user-Table-x-table", outputEvent.tablePath().getFullName()); config = - new TableRenameConfig() - .setReplacementsWithRegex( - Arrays.asList( - new TableRenameConfig.ReplacementsWithRegex("Table", "t1"), - new TableRenameConfig.ReplacementsWithRegex( - "Table", "t2"))); + ReadonlyConfig.fromMap( + new HashMap() { + { + put( + "replacements_with_regex", + "[{\"replace_from\" = \"Table\", \"replace_to\" = \"t1\"},{\"replace_from\" = \"Table\", \"replace_to\" = \"t2\"}]"); + } + }); transform = new TableRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTables(); outputRow = transform.map(inputRow); From 23a9027b882ece034a30c2c6af096c324b9fa26d Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 6 Dec 2024 10:38:07 +0800 Subject: [PATCH 18/56] [Feature][transform-v2] TableRenameTransform support --- .../FieldRenameMultiCatalogTransform.java | 2 +- .../rename/FieldRenameTransform.java | 7 +-- .../rename/FieldRenameTransformTest.java | 44 ++++++++++++++----- 3 files changed, 39 insertions(+), 14 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java index dd0f61e4e45..c433150e99b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable table, ReadonlyConfig config) { - return new FieldRenameTransform(FieldRenameConfig.of(config), table); + return new FieldRenameTransform(config, table); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java index ce52a6992e3..25726c287ec 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -59,9 +60,9 @@ public class FieldRenameTransform extends AbstractCatalogSupportMapTransform { private final FieldRenameConfig config; private TableSchemaChangeEventHandler tableSchemaChangeEventHandler; - public FieldRenameTransform(FieldRenameConfig config, CatalogTable table) { - super(table); - this.config = config; + public FieldRenameTransform(ReadonlyConfig readonlyConfig, CatalogTable table) { + super(readonlyConfig, table); + this.config = FieldRenameConfig.of(readonlyConfig); this.inputTable = table; this.tableSchemaChangeEventHandler = new TableSchemaChangeEventDispatcher(); } diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java index 02a8b01bcf8..77750b3bc5f 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.rename; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -34,6 +35,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.stream.Collectors; public class FieldRenameTransformTest { @@ -103,7 +105,14 @@ public void testRename() { AlterTableDropColumnEvent dropColumnEvent = new AlterTableDropColumnEvent(DEFAULT_TABLE.getTableId(), "f5"); - FieldRenameConfig config = new FieldRenameConfig().setConvertCase(ConvertCase.LOWER); + ReadonlyConfig config = + ReadonlyConfig.fromMap( + new HashMap() { + { + put("convert_case", "LOWER"); + } + }); + FieldRenameTransform transform = new FieldRenameTransform(config, DEFAULT_TABLE); CatalogTable outputCatalogTable = transform.getProducedCatalogTable(); AlterTableAddColumnEvent outputAddEvent = @@ -137,7 +146,13 @@ public void testRename() { Assertions.assertEquals("f5", outputChangeEvent.getColumn().getName()); Assertions.assertEquals("f5", outputDropEvent.getColumn()); - config = new FieldRenameConfig().setConvertCase(ConvertCase.UPPER); + config = + ReadonlyConfig.fromMap( + new HashMap() { + { + put("convert_case", "UPPER"); + } + }); transform = new FieldRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTable(); outputAddEvent = (AlterTableAddColumnEvent) transform.mapSchemaChangeEvent(addColumnEvent); @@ -169,7 +184,15 @@ public void testRename() { Assertions.assertEquals("f5", outputChangeEvent.getColumn().getName()); Assertions.assertEquals("F5", outputDropEvent.getColumn()); - config = new FieldRenameConfig().setPrefix("p-").setSuffix("-s"); + config = + ReadonlyConfig.fromMap( + new HashMap() { + { + put("prefix", "p-"); + put("suffix", "-s"); + } + }); + transform = new FieldRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTable(); outputAddEvent = (AlterTableAddColumnEvent) transform.mapSchemaChangeEvent(addColumnEvent); @@ -202,13 +225,14 @@ public void testRename() { Assertions.assertEquals("p-f5-s", outputDropEvent.getColumn()); config = - new FieldRenameConfig() - .setReplacementsWithRegex( - Arrays.asList( - new FieldRenameConfig.ReplacementsWithRegex( - "f1", "t1", true), - new FieldRenameConfig.ReplacementsWithRegex( - "f1", "t2", true))); + ReadonlyConfig.fromMap( + new HashMap() { + { + put( + "replacements_with_regex", + "[{\"replace_from\" = \"f1\", \"replace_to\" = \"t1\"},{\"replace_from\" = \"f1\", \"replace_to\" = \"t2\"}]"); + } + }); transform = new FieldRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTable(); outputAddEvent = (AlterTableAddColumnEvent) transform.mapSchemaChangeEvent(addColumnEvent); From c7654bff2572c3241422ea34dc115eebf741244d Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 6 Dec 2024 11:53:14 +0800 Subject: [PATCH 19/56] [Feature][transform-v2] TableRenameTransform support --- .../seatunnel/transform/rename/FieldRenameTransformTest.java | 3 ++- .../seatunnel/transform/rename/TableRenameTransformTest.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java index 77750b3bc5f..8d70761a128 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java @@ -230,7 +230,8 @@ public void testRename() { { put( "replacements_with_regex", - "[{\"replace_from\" = \"f1\", \"replace_to\" = \"t1\"},{\"replace_from\" = \"f1\", \"replace_to\" = \"t2\"}]"); + "[{\"replace_from\" : \"f1\", \"replace_to\" : \"t1\"},{\"replace_from\" : " + + "\"f1\", \"replace_to\" : \"t2\"}]"); } }); transform = new FieldRenameTransform(config, DEFAULT_TABLE); diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java index ca319cf7793..da6d8a3cdb4 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java @@ -158,7 +158,7 @@ public void testRename() { { put( "replacements_with_regex", - "[{\"replace_from\" = \"Table\", \"replace_to\" = \"t1\"},{\"replace_from\" = \"Table\", \"replace_to\" = \"t2\"}]"); + "[{\"replace_from\" : \"Table\", \"replace_to\" : \"t1\"},{\"replace_from\" : \"Table\", \"replace_to\" : \"t2\"}]"); } }); transform = new TableRenameTransform(config, DEFAULT_TABLE); From e0e6a03951c5a9b8b7e5af67c59f8438ee6cfcc2 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Mon, 9 Dec 2024 11:39:10 +0800 Subject: [PATCH 20/56] [Feature][transform-v2] Support transform metrics --- .../common/AbstractCatalogSupportFlatMapTransform.java | 8 +++----- .../common/AbstractCatalogSupportMapTransform.java | 9 +++------ 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index f7990a1abbe..604bd098d2e 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; -import org.apache.commons.collections4.CollectionUtils; - import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -41,9 +39,9 @@ public AbstractCatalogSupportFlatMapTransform( @Override public List flatMap(SeaTunnelRow row) { List results = transform(row); - if (CollectionUtils.isNotEmpty(results)) { - hazelcastMetric(results.size()); - } + // if (CollectionUtils.isNotEmpty(results)) { + // hazelcastMetric(results.size()); + // } return results; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index b47600b3457..85c06e84f91 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -25,8 +25,6 @@ import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import java.util.Objects; - @Slf4j public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform @@ -46,10 +44,9 @@ public AbstractCatalogSupportMapTransform( @Override public SeaTunnelRow map(SeaTunnelRow row) { SeaTunnelRow result = transform(row); - if (Objects.nonNull(result)) { - hazelcastMetric(); - } - + // if (Objects.nonNull(result)) { + // hazelcastMetric(); + // } return result; } } From c687fceea17158374174bce8ec0ec19f59e977dc Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Mon, 9 Dec 2024 15:15:09 +0800 Subject: [PATCH 21/56] [Feature][transform-v2] Support transform metrics --- .../AbstractCatalogSupportFlatMapTransform.java | 8 +++++--- .../common/AbstractCatalogSupportMapTransform.java | 8 +++++--- .../AbstractMultiCatalogFlatMapTransform.java | 14 +++++--------- .../common/AbstractMultiCatalogTransform.java | 8 ++++++++ .../common/AbstractSeaTunnelTransform.java | 4 ++-- 5 files changed, 25 insertions(+), 17 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index 604bd098d2e..f7990a1abbe 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; +import org.apache.commons.collections4.CollectionUtils; + import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -39,9 +41,9 @@ public AbstractCatalogSupportFlatMapTransform( @Override public List flatMap(SeaTunnelRow row) { List results = transform(row); - // if (CollectionUtils.isNotEmpty(results)) { - // hazelcastMetric(results.size()); - // } + if (CollectionUtils.isNotEmpty(results)) { + hazelcastMetric(results.size()); + } return results; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 85c06e84f91..330e11fd1da 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -25,6 +25,8 @@ import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import java.util.Objects; + @Slf4j public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform @@ -44,9 +46,9 @@ public AbstractCatalogSupportMapTransform( @Override public SeaTunnelRow map(SeaTunnelRow row) { SeaTunnelRow result = transform(row); - // if (Objects.nonNull(result)) { - // hazelcastMetric(); - // } + if (Objects.nonNull(result)) { + hazelcastMetric(); + } return result; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java index 5b7fab0f0f8..b1deb453d6d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogFlatMapTransform.java @@ -36,15 +36,11 @@ public AbstractMultiCatalogFlatMapTransform( @Override public List flatMap(SeaTunnelRow row) { if (transformMap.size() == 1) { - SeaTunnelFlatMapTransform transform = - ((SeaTunnelFlatMapTransform) - transformMap.values().iterator().next()); - transform.setMetricsContext(metricsContext); - return transform.flatMap(row); + return ((SeaTunnelFlatMapTransform) + transformMap.values().iterator().next()) + .flatMap(row); } - SeaTunnelFlatMapTransform transform = - (SeaTunnelFlatMapTransform) transformMap.get(row.getTableId()); - transform.setMetricsContext(metricsContext); - return transform.flatMap(row); + return ((SeaTunnelFlatMapTransform) transformMap.get(row.getTableId())) + .flatMap(row); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index 19c86f10ce0..7b505a90633 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -29,6 +29,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.function.Function; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -91,6 +92,13 @@ public AbstractMultiCatalogTransform( .collect(Collectors.toList()); } + @Override + public void open() { + if (Objects.nonNull(metricsContext)) { + transformMap.values().forEach(transform -> transform.setMetricsContext(metricsContext)); + } + } + @Override public void setMetricsContext(MetricsContext metricsContext) { this.metricsContext = metricsContext; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 3a6051326d1..8a35bd86d44 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -146,7 +146,7 @@ public void setMetricsContext(MetricsContext metricsContext) { protected void hazelcastMetric(long size) { if (metricsContext != null) { - metricsContext.counter(getTransformMetricName(this)).inc(size); + metricsContext.counter(getTransformMetricName()).inc(size); } } @@ -156,7 +156,7 @@ protected void hazelcastMetric() { } } - protected String getTransformMetricName(SeaTunnelTransform transformer) { + protected String getTransformMetricName() { StringBuilder metricName = new StringBuilder(); metricName .append(MetricNames.TRANSFORM_COUNT) From 8f32bfcbde8decf87376162417fe6862585610d3 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 10 Dec 2024 16:53:27 +0800 Subject: [PATCH 22/56] [Feature][transform-v2] Support transform metrics --- .github/workflows/backend.yml | 2 +- .../apache/seatunnel/api/event/EventType.java | 1 + .../api/transform/SeaTunnelTransform.java | 16 ++++++- .../transform/event/TransformCloseEvent.java | 40 +++++++++++++++++ .../engine/server/task/SeaTunnelTask.java | 1 + .../server/task/context/TransformContext.java | 44 +++++++++++++++++++ .../task/flow/TransformFlowLifeCycle.java | 32 ++++++++++++-- .../common/AbstractMultiCatalogTransform.java | 15 +++---- .../common/AbstractSeaTunnelTransform.java | 19 ++++---- 9 files changed, 144 insertions(+), 26 deletions(-) create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index eff3cc4b418..7bdcf6b165a 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -1191,7 +1191,7 @@ jobs: matrix: java: [ '8', '11' ] os: [ 'ubuntu-latest' ] - timeout-minutes: 180 + timeout-minutes: 120 steps: - uses: actions/checkout@v2 - name: Set up JDK ${{ matrix.java }} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java index edb1b72f36b..1bcb76b457b 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java @@ -29,5 +29,6 @@ public enum EventType { LIFECYCLE_READER_OPEN, LIFECYCLE_READER_CLOSE, LIFECYCLE_WRITER_CLOSE, + LIFECYCLE_TRANSFORM_CLOSE, READER_MESSAGE_DELAYED, } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index a1a59b7cd26..a2c1952e067 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.common.PluginIdentifierInterface; import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.source.SeaTunnelJobAware; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; @@ -56,5 +57,18 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve /** call it when Transformer completed */ default void close() {} - void setMetricsContext(MetricsContext metricsContext); + void loadContext(Context context); + + interface Context extends Serializable { + + /** @return metricsContext of this reader. */ + MetricsContext getMetricsContext(); + + /** + * Get the {@link EventListener} of this writer. + * + * @return + */ + EventListener getEventListener(); + } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java new file mode 100644 index 00000000000..9ae056d6b80 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.transform.event; + +import org.apache.seatunnel.api.event.EventType; +import org.apache.seatunnel.api.event.LifecycleEvent; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +@Getter +@Setter +@ToString +@AllArgsConstructor +public class TransformCloseEvent implements LifecycleEvent { + private long createdTime; + private String jobId; + private EventType eventType = EventType.LIFECYCLE_TRANSFORM_CLOSE; + + public TransformCloseEvent() { + this.createdTime = System.currentTimeMillis(); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java index a01364f07dd..3158e424280 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java @@ -236,6 +236,7 @@ private FlowLifeCycle convertFlowToActionLifeCycle(@NonNull Flow flow) throws Ex new TransformFlowLifeCycle( (TransformChainAction) f.getAction(), this, + taskLocation, new SeaTunnelTransformCollector(flowLifeCycles), completableFuture, this.getMetricsContext()); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java new file mode 100644 index 00000000000..147ae3235d1 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java @@ -0,0 +1,44 @@ +/* + * 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.task.context; + +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; +import org.apache.seatunnel.api.transform.SeaTunnelTransform; + +public class TransformContext implements SeaTunnelTransform.Context { + + private static final long serialVersionUID = -3082515319043725121L; + private final MetricsContext metricsContext; + private final EventListener eventListener; + + public TransformContext(MetricsContext metricsContext, EventListener eventListener) { + this.metricsContext = metricsContext; + this.eventListener = eventListener; + } + + @Override + public MetricsContext getMetricsContext() { + return metricsContext; + } + + @Override + public EventListener getEventListener() { + return eventListener; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index b4ed2abd4f6..b8a1f1f8534 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -18,17 +18,22 @@ package org.apache.seatunnel.engine.server.task.flow; import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.api.transform.Collector; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; import org.apache.seatunnel.api.transform.SeaTunnelTransform; +import org.apache.seatunnel.api.transform.event.TransformCloseEvent; import org.apache.seatunnel.engine.core.dag.actions.TransformChainAction; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; import org.apache.seatunnel.engine.server.checkpoint.ActionSubtaskState; import org.apache.seatunnel.engine.server.checkpoint.CheckpointBarrier; +import org.apache.seatunnel.engine.server.event.JobEventListener; +import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.task.SeaTunnelTask; +import org.apache.seatunnel.engine.server.task.context.TransformContext; import org.apache.seatunnel.engine.server.task.record.Barrier; import org.apache.commons.collections4.CollectionUtils; @@ -51,9 +56,16 @@ public class TransformFlowLifeCycle extends ActionFlowLifeCycle private final Collector> collector; + private EventListener eventListener; + + private SeaTunnelTransform.Context transformContext; + + private MetricsContext metricsContext; + public TransformFlowLifeCycle( TransformChainAction action, SeaTunnelTask runningTask, + TaskLocation taskLocation, Collector> collector, CompletableFuture completableFuture, MetricsContext metricsContext) { @@ -61,9 +73,8 @@ public TransformFlowLifeCycle( this.action = action; this.transform = action.getTransforms(); this.collector = collector; - for (SeaTunnelTransform t : transform) { - t.setMetricsContext(metricsContext); - } + this.metricsContext = metricsContext; + this.eventListener = new JobEventListener(taskLocation, runningTask.getExecutionContext()); } @Override @@ -183,7 +194,18 @@ public List transform(T inputData) { @Override public void restoreState(List actionStateList) throws Exception { - // nothing + this.transformContext = new TransformContext(metricsContext, eventListener); + for (SeaTunnelTransform t : transform) { + try { + t.loadContext(transformContext); + } catch (Exception e) { + log.error( + "restore transform: {} failed, cause: {}", + t.getPluginName(), + e.getMessage(), + e); + } + } } @Override @@ -199,6 +221,8 @@ public void close() throws IOException { e); } } + super.close(); + transformContext.getEventListener().onEvent(new TransformCloseEvent()); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index 7b505a90633..c24787770b6 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; @@ -29,7 +28,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.function.Function; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -46,7 +44,7 @@ public abstract class AbstractMultiCatalogTransform implements SeaTunnelTransfor protected Map> transformMap; - protected MetricsContext metricsContext; + private Context context; public AbstractMultiCatalogTransform( List inputCatalogTables, ReadonlyConfig config) { @@ -93,15 +91,12 @@ public AbstractMultiCatalogTransform( } @Override - public void open() { - if (Objects.nonNull(metricsContext)) { - transformMap.values().forEach(transform -> transform.setMetricsContext(metricsContext)); - } - } + public void open() {} @Override - public void setMetricsContext(MetricsContext metricsContext) { - this.metricsContext = metricsContext; + public void loadContext(Context context) { + this.context = context; + transformMap.values().forEach(transform -> transform.loadContext(context)); } protected abstract SeaTunnelTransform buildTransform( diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 8a35bd86d44..1da30795fad 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -18,7 +18,6 @@ import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.common.metrics.MetricNames; -import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; @@ -47,7 +46,7 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected String outTableName; - protected MetricsContext metricsContext; + private Context context; @Override public void open() {} @@ -97,6 +96,11 @@ public List getProducedCatalogTables() { return Collections.singletonList(getProducedCatalogTable()); } + @Override + public void loadContext(Context context) { + this.context = context; + } + private CatalogTable transformCatalogTable() { TableIdentifier tableIdentifier = transformTableIdentifier(); TableSchema tableSchema = transformTableSchema(); @@ -139,19 +143,14 @@ public R transform(SeaTunnelRow row) { protected abstract TableIdentifier transformTableIdentifier(); - @Override - public void setMetricsContext(MetricsContext metricsContext) { - this.metricsContext = metricsContext; - } - protected void hazelcastMetric(long size) { - if (metricsContext != null) { - metricsContext.counter(getTransformMetricName()).inc(size); + if (context.getMetricsContext() != null) { + context.getMetricsContext().counter(getTransformMetricName()).inc(size); } } protected void hazelcastMetric() { - if (metricsContext != null) { + if (context.getMetricsContext() != null) { hazelcastMetric(1); } } From 86fbe80130119ca5f4e44896f8f828b07ae17bde Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 10 Dec 2024 17:31:44 +0800 Subject: [PATCH 23/56] [Feature][transform-v2] Support transform metrics --- .../transform/common/AbstractSeaTunnelTransform.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 1da30795fad..8a81fda1acb 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -144,13 +144,13 @@ public R transform(SeaTunnelRow row) { protected abstract TableIdentifier transformTableIdentifier(); protected void hazelcastMetric(long size) { - if (context.getMetricsContext() != null) { + if (context != null && context.getMetricsContext() != null) { context.getMetricsContext().counter(getTransformMetricName()).inc(size); } } protected void hazelcastMetric() { - if (context.getMetricsContext() != null) { + if (context != null && context.getMetricsContext() != null) { hazelcastMetric(1); } } From d57e5385e4cae09d8f04d99f2acf7debae6cb510 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 11 Dec 2024 10:32:26 +0800 Subject: [PATCH 24/56] [Feature][transform-v2] Support transform metrics --- .../apache/seatunnel/api/event/EventType.java | 1 + .../transform/event/TransformOpenEvent.java | 40 +++++++++++++++++++ .../task/flow/TransformFlowLifeCycle.java | 37 +++++++++-------- 3 files changed, 61 insertions(+), 17 deletions(-) create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java index 1bcb76b457b..7b8b59aa9e0 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java @@ -30,5 +30,6 @@ public enum EventType { LIFECYCLE_READER_CLOSE, LIFECYCLE_WRITER_CLOSE, LIFECYCLE_TRANSFORM_CLOSE, + LIFECYCLE_TRANSFORM_OPEN, READER_MESSAGE_DELAYED, } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java new file mode 100644 index 00000000000..d7b2f12c8e7 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.api.transform.event; + +import org.apache.seatunnel.api.event.EventType; +import org.apache.seatunnel.api.event.LifecycleEvent; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +@Getter +@Setter +@ToString +@AllArgsConstructor +public class TransformOpenEvent implements LifecycleEvent { + private long createdTime; + private String jobId; + private EventType eventType = EventType.LIFECYCLE_TRANSFORM_OPEN; + + public TransformOpenEvent() { + this.createdTime = System.currentTimeMillis(); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index b8a1f1f8534..9242b910a59 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.api.transform.event.TransformCloseEvent; +import org.apache.seatunnel.api.transform.event.TransformOpenEvent; import org.apache.seatunnel.engine.core.dag.actions.TransformChainAction; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; import org.apache.seatunnel.engine.server.checkpoint.ActionSubtaskState; @@ -77,9 +78,25 @@ public TransformFlowLifeCycle( this.eventListener = new JobEventListener(taskLocation, runningTask.getExecutionContext()); } + @Override + public void init() throws Exception { + this.transformContext = new TransformContext(metricsContext, eventListener); + for (SeaTunnelTransform t : transform) { + try { + t.loadContext(transformContext); + } catch (Exception e) { + log.error( + "restore transform: {} failed, cause: {}", + t.getPluginName(), + e.getMessage(), + e); + } + } + } + @Override public void open() throws Exception { - super.open(); + transformContext.getEventListener().onEvent(new TransformOpenEvent()); for (SeaTunnelTransform t : transform) { try { t.open(); @@ -193,23 +210,11 @@ public List transform(T inputData) { } @Override - public void restoreState(List actionStateList) throws Exception { - this.transformContext = new TransformContext(metricsContext, eventListener); - for (SeaTunnelTransform t : transform) { - try { - t.loadContext(transformContext); - } catch (Exception e) { - log.error( - "restore transform: {} failed, cause: {}", - t.getPluginName(), - e.getMessage(), - e); - } - } - } + public void restoreState(List actionStateList) throws Exception {} @Override public void close() throws IOException { + transformContext.getEventListener().onEvent(new TransformCloseEvent()); for (SeaTunnelTransform t : transform) { try { t.close(); @@ -221,8 +226,6 @@ public void close() throws IOException { e); } } - super.close(); - transformContext.getEventListener().onEvent(new TransformCloseEvent()); } } From 64a19147814a8ff2a93ca297927b232d8c288942 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 11 Dec 2024 11:56:02 +0800 Subject: [PATCH 25/56] [Feature][transform-v2] Support transform metrics --- .../src/test/resources/kafka_sink_fake_to_kafka.conf | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf index 90825bbf340..b234dda7fc0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf @@ -34,6 +34,7 @@ source { array.size = 10 bytes.length = 10 string.length = 10 + plugin_output = "kafka_table" schema = { fields { c_map = "map" @@ -56,11 +57,20 @@ source { } } +transform { + Sql { + plugin_input = "kafka_table" + plugin_output = "kafka_table_transform" + query = "select Address.city,c_string from kafka_table" + } +} + sink { Kafka { bootstrap.servers = "kafkaCluster:9092" + plugin_input = "kafka_table_transform" topic = "test_topic" format = json partition_key_fields = ["c_map", "c_string"] } -} \ No newline at end of file +} From 2c1d0b9483b8fe5d6392607111f3320a3ca37ce7 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 11 Dec 2024 14:10:08 +0800 Subject: [PATCH 26/56] [Feature][transform-v2] Support transform metrics --- .../src/test/resources/kafka_sink_fake_to_kafka.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf index b234dda7fc0..86efc616690 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf @@ -61,7 +61,7 @@ transform { Sql { plugin_input = "kafka_table" plugin_output = "kafka_table_transform" - query = "select Address.city,c_string from kafka_table" + query = "select * from kafka_table" } } From 52219abed6b19929a6dfc302cae544d219cb16de Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 11 Dec 2024 15:41:11 +0800 Subject: [PATCH 27/56] [Feature][transform-v2] Support transform metrics --- .../common/utils/StringFormatUtils.java | 26 +++++++++++++++++++ .../command/ClientExecuteCommand.java | 10 ++++--- .../engine/client/job/JobMetricsRunner.java | 10 ++++--- 3 files changed, 38 insertions(+), 8 deletions(-) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java index 9a52118145c..32640c0cf2d 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.common.utils; +import lombok.SneakyThrows; + import java.util.Collections; public class StringFormatUtils { @@ -43,4 +45,28 @@ public static String formatTable(Object... objects) { + "***********************************************\n"; return String.format(template, objects); } + + @SneakyThrows + public static String transformFormatTable(Object... objects) { + String title = objects[0].toString(); + String tableName = objects[1].toString(); + int blankNum = (NUM - title.length()) / 2; + int tableBlankNum = (NUM - tableName.length()) / 2; + int kvNum = (objects.length - 2) / 2; + String template = + "\n" + + "***********************************************" + + "\n" + + String.join("", Collections.nCopies(blankNum, " ")) + + "%s" + + "\n" + + String.join("", Collections.nCopies(tableBlankNum, " ")) + + "%s" + + "\n" + + "***********************************************" + + "\n" + + String.join("", Collections.nCopies(kvNum, "%-26s: %19s\n")) + + "***********************************************\n"; + return String.format(template, objects); + } } diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java index 2a42e5f3f84..811b3c4449d 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java @@ -257,9 +257,10 @@ public void execute() throws CommandExecuteException { .forEach( (tableName, metrics) -> { String[] transformInfos = - new String[metrics.entrySet().size() * 2 + 1]; - transformInfos[0] = "Transform Information " + tableName; - int index = 0; + new String[metrics.entrySet().size() * 2 + 2]; + transformInfos[0] = "Transform Information"; + transformInfos[1] = tableName; + int index = 1; for (Map.Entry entry : metrics.entrySet()) { transformInfos[++index] = entry.getKey(); transformInfos[++index] = @@ -267,7 +268,8 @@ public void execute() throws CommandExecuteException { } if (Objects.nonNull(transformInfos)) { logMessage.append( - StringFormatUtils.formatTable(transformInfos)); + StringFormatUtils.transformFormatTable( + transformInfos)); } }); } diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java index 060f9af3699..ee41b994397 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java @@ -83,16 +83,18 @@ public void run() { .forEach( (tableName, metrics) -> { String[] transformInfos = - new String[metrics.entrySet().size() * 2 + 1]; - transformInfos[0] = "Transform Information " + tableName; - int index = 0; + new String[metrics.entrySet().size() * 2 + 2]; + transformInfos[0] = "Transform Information"; + transformInfos[1] = tableName; + int index = 1; for (Map.Entry entry : metrics.entrySet()) { transformInfos[++index] = entry.getKey(); transformInfos[++index] = String.valueOf(entry.getValue()); } if (Objects.nonNull(transformInfos)) { logMessage.append( - StringFormatUtils.formatTable(transformInfos)); + StringFormatUtils.transformFormatTable( + transformInfos)); } }); } From cd58795ef74a422f9a220979cb713db3cfe1dd81 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 11 Dec 2024 15:49:01 +0800 Subject: [PATCH 28/56] [Feature][transform-v2] Support transform metrics --- .../seatunnel/common/utils/StringFormatUtils.java | 2 +- .../src/test/resources/kafka_sink_fake_to_kafka.conf | 10 ---------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java index 32640c0cf2d..4685e623169 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java @@ -60,7 +60,7 @@ public static String transformFormatTable(Object... objects) { + String.join("", Collections.nCopies(blankNum, " ")) + "%s" + "\n" - + String.join("", Collections.nCopies(tableBlankNum, " ")) + + "originTable:" + "%s" + "\n" + "***********************************************" diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf index 86efc616690..00dc8fecfe5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf @@ -34,7 +34,6 @@ source { array.size = 10 bytes.length = 10 string.length = 10 - plugin_output = "kafka_table" schema = { fields { c_map = "map" @@ -57,18 +56,9 @@ source { } } -transform { - Sql { - plugin_input = "kafka_table" - plugin_output = "kafka_table_transform" - query = "select * from kafka_table" - } -} - sink { Kafka { bootstrap.servers = "kafkaCluster:9092" - plugin_input = "kafka_table_transform" topic = "test_topic" format = json partition_key_fields = ["c_map", "c_string"] From 38f61be30116344ae3c36d9014d27c6c9f82e819 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 11 Dec 2024 15:50:29 +0800 Subject: [PATCH 29/56] [Feature][transform-v2] Support transform metrics --- .../org/apache/seatunnel/api/common/metrics/MetricNames.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java index 69dc778426d..e3c9debae49 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java @@ -34,7 +34,4 @@ private MetricNames() {} public static final String SINK_WRITE_QPS = "SinkWriteQPS"; public static final String SINK_WRITE_BYTES_PER_SECONDS = "SinkWriteBytesPerSeconds"; public static final String TRANSFORM_COUNT = "TransformCount"; - public static final String TRANSFORM_BYTES = "TransformBytes"; - public static final String TRANSFORM_QPS = "TransformQPS"; - public static final String TRANSFORM_BYTES_PER_SECONDS = "TransformBytesPerSeconds"; } From 6bad18ecf2f52e79632cc474ff6ada80303daaeb Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 17 Dec 2024 16:59:34 +0800 Subject: [PATCH 30/56] [Feature][transform-v2] Support transform metrics --- .../api/common/metrics/MetricNames.java | 2 +- .../api/transform/SeaTunnelTransform.java | 17 ++++----- .../common/utils/StringFormatUtils.java | 26 ------------- .../command/ClientExecuteCommand.java | 8 ++-- .../engine/client/job/JobClient.java | 6 +-- .../engine/client/job/JobMetricsRunner.java | 10 ++--- .../server/rest/service/BaseService.java | 6 +-- .../engine/server/task/SeaTunnelTask.java | 1 - .../server/task/context/TransformContext.java | 11 +++--- .../task/flow/TransformFlowLifeCycle.java | 38 ++++--------------- .../common/AbstractSeaTunnelTransform.java | 8 ++-- 11 files changed, 37 insertions(+), 96 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java index e3c9debae49..0d1c6116cb3 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics/MetricNames.java @@ -33,5 +33,5 @@ private MetricNames() {} public static final String SINK_WRITE_BYTES = "SinkWriteBytes"; public static final String SINK_WRITE_QPS = "SinkWriteQPS"; public static final String SINK_WRITE_BYTES_PER_SECONDS = "SinkWriteBytesPerSeconds"; - public static final String TRANSFORM_COUNT = "TransformCount"; + public static final String TRANSFORM_OUTPUT_COUNT = "TransformOutputCount"; } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index a2c1952e067..a91a08eb5fb 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.common.PluginIdentifierInterface; import org.apache.seatunnel.api.common.metrics.MetricsContext; -import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.source.SeaTunnelJobAware; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; @@ -32,7 +31,8 @@ public interface SeaTunnelTransform extends Serializable, PluginIdentifierInterface, SeaTunnelJobAware { /** call it when Transformer initialed */ - default void open() {} + default void open() { + } /** * Set the data type info of input data. @@ -55,20 +55,17 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve } /** call it when Transformer completed */ - default void close() {} + default void close() { + } void loadContext(Context context); interface Context extends Serializable { - /** @return metricsContext of this reader. */ + /** @return metricsContext of this transform. */ MetricsContext getMetricsContext(); - /** - * Get the {@link EventListener} of this writer. - * - * @return - */ - EventListener getEventListener(); + /** @return name of this transform. */ + String getTransformName(); } } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java index 4685e623169..9a52118145c 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/StringFormatUtils.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.common.utils; -import lombok.SneakyThrows; - import java.util.Collections; public class StringFormatUtils { @@ -45,28 +43,4 @@ public static String formatTable(Object... objects) { + "***********************************************\n"; return String.format(template, objects); } - - @SneakyThrows - public static String transformFormatTable(Object... objects) { - String title = objects[0].toString(); - String tableName = objects[1].toString(); - int blankNum = (NUM - title.length()) / 2; - int tableBlankNum = (NUM - tableName.length()) / 2; - int kvNum = (objects.length - 2) / 2; - String template = - "\n" - + "***********************************************" - + "\n" - + String.join("", Collections.nCopies(blankNum, " ")) - + "%s" - + "\n" - + "originTable:" - + "%s" - + "\n" - + "***********************************************" - + "\n" - + String.join("", Collections.nCopies(kvNum, "%-26s: %19s\n")) - + "***********************************************\n"; - return String.format(template, objects); - } } diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java index 811b3c4449d..e502e0fb6b2 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ClientExecuteCommand.java @@ -258,9 +258,8 @@ public void execute() throws CommandExecuteException { (tableName, metrics) -> { String[] transformInfos = new String[metrics.entrySet().size() * 2 + 2]; - transformInfos[0] = "Transform Information"; - transformInfos[1] = tableName; - int index = 1; + transformInfos[0] = tableName + " Information"; + int index = 0; for (Map.Entry entry : metrics.entrySet()) { transformInfos[++index] = entry.getKey(); transformInfos[++index] = @@ -268,8 +267,7 @@ public void execute() throws CommandExecuteException { } if (Objects.nonNull(transformInfos)) { logMessage.append( - StringFormatUtils.transformFormatTable( - transformInfos)); + StringFormatUtils.formatTable(transformInfos)); } }); } diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java index 912edf480e0..0306a069064 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobClient.java @@ -54,7 +54,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import static org.apache.seatunnel.api.common.metrics.MetricNames.TRANSFORM_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.TRANSFORM_OUTPUT_COUNT; public class JobClient { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -243,9 +243,9 @@ private void processTransformMetric( Map>> transformMetricsMaps, String metricName, JsonNode jobMetricsStr) { - if (metricName.contains(TRANSFORM_COUNT)) { + if (metricName.contains(TRANSFORM_OUTPUT_COUNT)) { processTransformMetric( - transformMetricsMaps, TRANSFORM_COUNT, metricName, jobMetricsStr); + transformMetricsMaps, TRANSFORM_OUTPUT_COUNT, metricName, jobMetricsStr); } } diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java index ee41b994397..734e40c3f13 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java @@ -83,18 +83,16 @@ public void run() { .forEach( (tableName, metrics) -> { String[] transformInfos = - new String[metrics.entrySet().size() * 2 + 2]; - transformInfos[0] = "Transform Information"; - transformInfos[1] = tableName; - int index = 1; + new String[metrics.entrySet().size() * 2 + 1]; + transformInfos[0] = tableName + "Transform"; + int index = 0; for (Map.Entry entry : metrics.entrySet()) { transformInfos[++index] = entry.getKey(); transformInfos[++index] = String.valueOf(entry.getValue()); } if (Objects.nonNull(transformInfos)) { logMessage.append( - StringFormatUtils.transformFormatTable( - transformInfos)); + StringFormatUtils.formatTable(transformInfos)); } }); } 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 index 10cd0cce83e..f02a97b4380 100644 --- 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 @@ -89,7 +89,7 @@ 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.api.common.metrics.MetricNames.TRANSFORM_COUNT; +import static org.apache.seatunnel.api.common.metrics.MetricNames.TRANSFORM_OUTPUT_COUNT; 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; @@ -336,9 +336,9 @@ private void processTransformMetric( Map>> transformMetricsMaps, String metricName, JsonNode jobMetricsStr) { - if (metricName.contains(TRANSFORM_COUNT)) { + if (metricName.contains(TRANSFORM_OUTPUT_COUNT)) { processTransformMetric( - transformMetricsMaps, TRANSFORM_COUNT, metricName, jobMetricsStr); + transformMetricsMaps, TRANSFORM_OUTPUT_COUNT, metricName, jobMetricsStr); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java index 3158e424280..a01364f07dd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java @@ -236,7 +236,6 @@ private FlowLifeCycle convertFlowToActionLifeCycle(@NonNull Flow flow) throws Ex new TransformFlowLifeCycle( (TransformChainAction) f.getAction(), this, - taskLocation, new SeaTunnelTransformCollector(flowLifeCycles), completableFuture, this.getMetricsContext()); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java index 147ae3235d1..2ee27c2cd16 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java @@ -18,18 +18,17 @@ package org.apache.seatunnel.engine.server.task.context; import org.apache.seatunnel.api.common.metrics.MetricsContext; -import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.transform.SeaTunnelTransform; public class TransformContext implements SeaTunnelTransform.Context { private static final long serialVersionUID = -3082515319043725121L; private final MetricsContext metricsContext; - private final EventListener eventListener; + private final String transformName; - public TransformContext(MetricsContext metricsContext, EventListener eventListener) { + public TransformContext(MetricsContext metricsContext, String transformName) { this.metricsContext = metricsContext; - this.eventListener = eventListener; + this.transformName = transformName; } @Override @@ -38,7 +37,7 @@ public MetricsContext getMetricsContext() { } @Override - public EventListener getEventListener() { - return eventListener; + public String getTransformName() { + return transformName; } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 9242b910a59..9884d0ef205 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.engine.server.task.flow; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.collections4.CollectionUtils; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; @@ -25,8 +27,6 @@ import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; import org.apache.seatunnel.api.transform.SeaTunnelTransform; -import org.apache.seatunnel.api.transform.event.TransformCloseEvent; -import org.apache.seatunnel.api.transform.event.TransformOpenEvent; import org.apache.seatunnel.engine.core.dag.actions.TransformChainAction; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; import org.apache.seatunnel.engine.server.checkpoint.ActionSubtaskState; @@ -37,10 +37,6 @@ import org.apache.seatunnel.engine.server.task.context.TransformContext; import org.apache.seatunnel.engine.server.task.record.Barrier; -import org.apache.commons.collections4.CollectionUtils; - -import lombok.extern.slf4j.Slf4j; - import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -57,16 +53,12 @@ public class TransformFlowLifeCycle extends ActionFlowLifeCycle private final Collector> collector; - private EventListener eventListener; - - private SeaTunnelTransform.Context transformContext; private MetricsContext metricsContext; public TransformFlowLifeCycle( TransformChainAction action, SeaTunnelTask runningTask, - TaskLocation taskLocation, Collector> collector, CompletableFuture completableFuture, MetricsContext metricsContext) { @@ -75,30 +67,16 @@ public TransformFlowLifeCycle( this.transform = action.getTransforms(); this.collector = collector; this.metricsContext = metricsContext; - this.eventListener = new JobEventListener(taskLocation, runningTask.getExecutionContext()); - } - - @Override - public void init() throws Exception { - this.transformContext = new TransformContext(metricsContext, eventListener); - for (SeaTunnelTransform t : transform) { - try { - t.loadContext(transformContext); - } catch (Exception e) { - log.error( - "restore transform: {} failed, cause: {}", - t.getPluginName(), - e.getMessage(), - e); - } - } } @Override public void open() throws Exception { - transformContext.getEventListener().onEvent(new TransformOpenEvent()); + int index = 0; for (SeaTunnelTransform t : transform) { try { + String transformName = "Transform:" + ++index; + TransformContext transformContext = new TransformContext(metricsContext, transformName); + t.loadContext(transformContext); t.open(); } catch (Exception e) { log.error( @@ -210,11 +188,11 @@ public List transform(T inputData) { } @Override - public void restoreState(List actionStateList) throws Exception {} + public void restoreState(List actionStateList) throws Exception { + } @Override public void close() throws IOException { - transformContext.getEventListener().onEvent(new TransformCloseEvent()); for (SeaTunnelTransform t : transform) { try { t.close(); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 8a81fda1acb..e0934ff4f17 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -158,13 +158,11 @@ protected void hazelcastMetric() { protected String getTransformMetricName() { StringBuilder metricName = new StringBuilder(); metricName - .append(MetricNames.TRANSFORM_COUNT) + .append(MetricNames.TRANSFORM_OUTPUT_COUNT) .append("#") - .append(inputCatalogTable.getTablePath().getFullName()) + .append(context.getTransformName()) .append("#") - .append(inputTableName) - .append("->") - .append(outTableName); + .append(inputTableName); return metricName.toString(); } } From fcb4474943b28a42aa3d78d633caad36327cdd93 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 18 Dec 2024 14:32:55 +0800 Subject: [PATCH 31/56] [Feature][transform-v2] Support transform metrics --- .../api/transform/SeaTunnelTransform.java | 6 ++--- .../engine/e2e/MultiTableMetricsIT.java | 24 +++++++++---------- .../task/flow/TransformFlowLifeCycle.java | 16 ++++++------- 3 files changed, 21 insertions(+), 25 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index a91a08eb5fb..274acd3b7f5 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -31,8 +31,7 @@ public interface SeaTunnelTransform extends Serializable, PluginIdentifierInterface, SeaTunnelJobAware { /** call it when Transformer initialed */ - default void open() { - } + default void open() {} /** * Set the data type info of input data. @@ -55,8 +54,7 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve } /** call it when Transformer completed */ - default void close() { - } + default void close() {} void loadContext(Context context); diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index ca000b01173..d964bb83ae0 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -134,22 +134,22 @@ public void multiTableMetrics() { "metrics.TableSinkWriteBytes.'fake.public.table2'", equalTo(String.valueOf(dataSize * 5))) .body( - "metrics.TransformCount.'fake.table1'.'fake1->fake3'", + "metrics.TransformOutputCount.'transform1'.'fake1'", equalTo("10")) .body( - "metrics.TransformCount.'fake.table1'.'fake3->fake4'", + "metrics.TransformOutputCount.'transform2'.'fake3'", equalTo("10")) .body( - "metrics.TransformCount.'fake.table1'.'fake4->fake5'", + "metrics.TransformOutputCount.'transform3'.'fake4'", equalTo("10")) .body( - "metrics.TransformCount.'fake.public.table2'.'fake2->fake6'", + "metrics.TransformOutputCount.'transform1'.'fake2'", equalTo("5")) .body( - "metrics.TransformCount.'fake.public.table2'.'fake6->fake7'", + "metrics.TransformOutputCount.'transform2'.'fake6'", equalTo("5")) .body( - "metrics.TransformCount.'fake.public.table2'.'fake7->fake8'", + "metrics.TransformOutputCount.'transform3'.'fake7'", equalTo("5")); Assertions.assertTrue( Double.parseDouble(response.path("metrics.SourceReceivedQPS")) @@ -199,27 +199,27 @@ public void multiTableMetrics() { > 0 && Double.parseDouble( response.path( - "metrics.TransformCount.'fake.table1'.'fake1->fake3'")) + "metrics.TransformOutputCount.'transform1'.'fake1'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformCount.'fake.table1'.'fake3->fake4'")) + "metrics.TransformOutputCount.'transform2'.'fake3'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformCount.'fake.table1'.'fake4->fake5'")) + "metrics.TransformOutputCount.'transform3'.'fake4'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformCount.'fake.public.table2'.'fake2->fake6'")) + "metrics.TransformOutputCount.'transform1'.'fake2'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformCount.'fake.public.table2'.'fake6->fake7'")) + "metrics.TransformOutputCount.'transform2'.'fake6'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformCount.'fake.public.table2'.'fake7->fake8'")) + "metrics.TransformOutputCount.'transform3'.'fake7'")) > 0); }); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 9884d0ef205..e7df98ab05d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -17,10 +17,7 @@ package org.apache.seatunnel.engine.server.task.flow; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.collections4.CollectionUtils; import org.apache.seatunnel.api.common.metrics.MetricsContext; -import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.api.transform.Collector; @@ -31,12 +28,14 @@ import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; import org.apache.seatunnel.engine.server.checkpoint.ActionSubtaskState; import org.apache.seatunnel.engine.server.checkpoint.CheckpointBarrier; -import org.apache.seatunnel.engine.server.event.JobEventListener; -import org.apache.seatunnel.engine.server.execution.TaskLocation; import org.apache.seatunnel.engine.server.task.SeaTunnelTask; import org.apache.seatunnel.engine.server.task.context.TransformContext; import org.apache.seatunnel.engine.server.task.record.Barrier; +import org.apache.commons.collections4.CollectionUtils; + +import lombok.extern.slf4j.Slf4j; + import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -53,7 +52,6 @@ public class TransformFlowLifeCycle extends ActionFlowLifeCycle private final Collector> collector; - private MetricsContext metricsContext; public TransformFlowLifeCycle( @@ -75,7 +73,8 @@ public void open() throws Exception { for (SeaTunnelTransform t : transform) { try { String transformName = "Transform:" + ++index; - TransformContext transformContext = new TransformContext(metricsContext, transformName); + TransformContext transformContext = + new TransformContext(metricsContext, transformName); t.loadContext(transformContext); t.open(); } catch (Exception e) { @@ -188,8 +187,7 @@ public List transform(T inputData) { } @Override - public void restoreState(List actionStateList) throws Exception { - } + public void restoreState(List actionStateList) throws Exception {} @Override public void close() throws IOException { From 2f9d60b2d20357023980a46230b3f97d8f184d71 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 24 Dec 2024 10:34:39 +0800 Subject: [PATCH 32/56] [Feature][transform-v2] Support transform metrics --- .../engine/e2e/MultiTableMetricsIT.java | 24 +++++++++---------- .../task/flow/TransformFlowLifeCycle.java | 2 +- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index d964bb83ae0..0957fc0435d 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -134,22 +134,22 @@ public void multiTableMetrics() { "metrics.TableSinkWriteBytes.'fake.public.table2'", equalTo(String.valueOf(dataSize * 5))) .body( - "metrics.TransformOutputCount.'transform1'.'fake1'", + "metrics.TransformOutputCount.'Transform1'.'fake1'", equalTo("10")) .body( - "metrics.TransformOutputCount.'transform2'.'fake3'", + "metrics.TransformOutputCount.'Transform2'.'fake3'", equalTo("10")) .body( - "metrics.TransformOutputCount.'transform3'.'fake4'", + "metrics.TransformOutputCount.'Transform3'.'fake4'", equalTo("10")) .body( - "metrics.TransformOutputCount.'transform1'.'fake2'", + "metrics.TransformOutputCount.'Transform1'.'fake2'", equalTo("5")) .body( - "metrics.TransformOutputCount.'transform2'.'fake6'", + "metrics.TransformOutputCount.'Transform2'.'fake6'", equalTo("5")) .body( - "metrics.TransformOutputCount.'transform3'.'fake7'", + "metrics.TransformOutputCount.'Transform3'.'fake7'", equalTo("5")); Assertions.assertTrue( Double.parseDouble(response.path("metrics.SourceReceivedQPS")) @@ -199,27 +199,27 @@ public void multiTableMetrics() { > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'transform1'.'fake1'")) + "metrics.TransformOutputCount.'Transform1'.'fake1'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'transform2'.'fake3'")) + "metrics.TransformOutputCount.'Transform2'.'fake3'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'transform3'.'fake4'")) + "metrics.TransformOutputCount.'Transform3'.'fake4'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'transform1'.'fake2'")) + "metrics.TransformOutputCount.'Transform1'.'fake2'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'transform2'.'fake6'")) + "metrics.TransformOutputCount.'Transform2'.'fake6'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'transform3'.'fake7'")) + "metrics.TransformOutputCount.'Transform3'.'fake7'")) > 0); }); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index e7df98ab05d..df12e3ccd82 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -72,7 +72,7 @@ public void open() throws Exception { int index = 0; for (SeaTunnelTransform t : transform) { try { - String transformName = "Transform:" + ++index; + String transformName = "Transform" + ++index; TransformContext transformContext = new TransformContext(metricsContext, transformName); t.loadContext(transformContext); From 298282123281a3995f59569332776353a5945937 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 27 Dec 2024 16:10:14 +0800 Subject: [PATCH 33/56] [Feature][transform-v2] Support transform metrics --- .../seatunnel/transform/common/AbstractSeaTunnelTransform.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index e0934ff4f17..22fbacb6558 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -162,7 +162,7 @@ protected String getTransformMetricName() { .append("#") .append(context.getTransformName()) .append("#") - .append(inputTableName); + .append(outTableName); return metricName.toString(); } } From 6c1f35482c31690ca871ea3dc6ecd7da34d87e75 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Thu, 2 Jan 2025 09:24:29 +0800 Subject: [PATCH 34/56] [Feature][transform-v2] Support transform metrics --- .../engine/e2e/MultiTableMetricsIT.java | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index 0957fc0435d..a698afee1a9 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -134,22 +134,22 @@ public void multiTableMetrics() { "metrics.TableSinkWriteBytes.'fake.public.table2'", equalTo(String.valueOf(dataSize * 5))) .body( - "metrics.TransformOutputCount.'Transform1'.'fake1'", + "metrics.TransformOutputCount.'Transform1'.'fake3'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Transform2'.'fake3'", + "metrics.TransformOutputCount.'Transform2'.'fake4'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Transform3'.'fake4'", + "metrics.TransformOutputCount.'Transform3'.'fake5'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Transform1'.'fake2'", + "metrics.TransformOutputCount.'Transform1'.'fake6'", equalTo("5")) .body( - "metrics.TransformOutputCount.'Transform2'.'fake6'", + "metrics.TransformOutputCount.'Transform2'.'fake7'", equalTo("5")) .body( - "metrics.TransformOutputCount.'Transform3'.'fake7'", + "metrics.TransformOutputCount.'Transform3'.'fake8'", equalTo("5")); Assertions.assertTrue( Double.parseDouble(response.path("metrics.SourceReceivedQPS")) @@ -199,27 +199,32 @@ public void multiTableMetrics() { > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform1'.'fake1'")) + "metrics.TransformOutputCount.'Transform1'.'fake3'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform2'.'fake3'")) + "metrics.TransformOutputCount.'Transform2'" + + ".'fake4'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform3'.'fake4'")) + "metrics.TransformOutputCount.'Transform3'" + + ".'fake5'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform1'.'fake2'")) + "metrics.TransformOutputCount.'Transform1'" + + ".'fake6'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform2'.'fake6'")) + "metrics.TransformOutputCount.'Transform2'" + + ".'fake7'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform3'.'fake7'")) + "metrics.TransformOutputCount.'Transform3'" + + ".'fake8'")) > 0); }); } From ed2ed5cb620779531835e49734357e8bc0b5e856 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 10 Jan 2025 09:45:48 +0800 Subject: [PATCH 35/56] [Feature][transform-v2] Support transform metrics --- .../transform/table/TableMergeMultiCatalogTransform.java | 2 +- .../seatunnel/transform/table/TableMergeTransform.java | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java index c63219b03d1..6cd3b5d8bb3 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java @@ -48,7 +48,7 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable table, ReadonlyConfig config) { - return new TableMergeTransform(TableMergeConfig.of(config), table); + return new TableMergeTransform(config, table); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java index b51376a9ead..d1d1874172d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.table; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -31,11 +32,11 @@ public class TableMergeTransform extends AbstractCatalogSupportMapTransform { private final TablePath outputTablePath; private final String outputTableId; - public TableMergeTransform(TableMergeConfig config, CatalogTable table) { - super(table); + public TableMergeTransform(ReadonlyConfig config, CatalogTable table) { + super(config, table); this.inputTable = table; - this.outputTablePath = config.getTablePath(); - this.outputTableId = config.getTablePath().getFullName(); + this.outputTablePath = TableMergeConfig.of(config).getTablePath(); + this.outputTableId = TableMergeConfig.of(config).getTablePath().getFullName(); } @Override From 1cf7286cf95b638f328c6f6002ddc1141272843b Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Sat, 11 Jan 2025 12:41:18 +0800 Subject: [PATCH 36/56] [Feature][transform-v2] support transform metrics --- .../api/transform/SeaTunnelTransform.java | 4 +--- .../engine/e2e/MultiTableMetricsIT.java | 24 +++++++++---------- .../task/flow/TransformFlowLifeCycle.java | 5 ++-- .../AbstractCatalogSupportMapTransform.java | 2 +- .../common/AbstractMultiCatalogTransform.java | 7 ++---- .../common/AbstractSeaTunnelTransform.java | 11 ++++----- .../embedding/EmbeddingTransform.java | 9 +++++-- .../transform/nlpmodel/llm/LLMTransform.java | 9 +++++-- .../seatunnel/transform/sql/SQLTransform.java | 19 +++++++++------ 9 files changed, 48 insertions(+), 42 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index 274acd3b7f5..152b4bb50b7 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -31,7 +31,7 @@ public interface SeaTunnelTransform extends Serializable, PluginIdentifierInterface, SeaTunnelJobAware { /** call it when Transformer initialed */ - default void open() {} + default void open(Context context) {} /** * Set the data type info of input data. @@ -56,8 +56,6 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve /** call it when Transformer completed */ default void close() {} - void loadContext(Context context); - interface Context extends Serializable { /** @return metricsContext of this transform. */ diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index 7150d0a7d96..308e0ec5920 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -134,22 +134,22 @@ public void multiTableMetrics() { "metrics.TableSinkWriteBytes.'fake.public.table2'", equalTo(String.valueOf(dataSize * 5))) .body( - "metrics.TransformOutputCount.'Transform1'.'fake3'", + "metrics.TransformOutputCount.'Sql_1'.'fake3'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Transform2'.'fake4'", + "metrics.TransformOutputCount.'Sql_2'.'fake4'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Transform3'.'fake5'", + "metrics.TransformOutputCount.'Sql_3'.'fake5'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Transform1'.'fake6'", + "metrics.TransformOutputCount.'Sql_1'.'fake6'", equalTo("5")) .body( - "metrics.TransformOutputCount.'Transform2'.'fake7'", + "metrics.TransformOutputCount.'Sql_2'.'fake7'", equalTo("5")) .body( - "metrics.TransformOutputCount.'Transform3'.'fake8'", + "metrics.TransformOutputCount.'Sql_3'.'fake8'", equalTo("5")); Assertions.assertTrue( Double.parseDouble(response.path("metrics.SourceReceivedQPS")) @@ -199,31 +199,31 @@ public void multiTableMetrics() { > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform1'.'fake3'")) + "metrics.TransformOutputCount.'Sql_1'.'fake3'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform2'" + "metrics.TransformOutputCount.'Sql_2'" + ".'fake4'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform3'" + "metrics.TransformOutputCount.'Sql_3'" + ".'fake5'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform1'" + "metrics.TransformOutputCount.'Sql_1'" + ".'fake6'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform2'" + "metrics.TransformOutputCount.'Sql_2'" + ".'fake7'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Transform3'" + "metrics.TransformOutputCount.'Sql_3'" + ".'fake8'")) > 0); }); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 34929c07559..ef4ad8a020f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -72,11 +72,10 @@ public void open() throws Exception { int index = 0; for (SeaTunnelTransform t : transform) { try { - String transformName = "Transform" + ++index; + String transformName = t.getPluginName() + "_" + ++index; TransformContext transformContext = new TransformContext(metricsContext, transformName); - t.loadContext(transformContext); - t.open(); + t.open(transformContext); } catch (Exception e) { log.error( "Open transform: {} failed, cause: {}", diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 330e11fd1da..292bbbe76e8 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -47,7 +47,7 @@ public AbstractCatalogSupportMapTransform( public SeaTunnelRow map(SeaTunnelRow row) { SeaTunnelRow result = transform(row); if (Objects.nonNull(result)) { - hazelcastMetric(); + updateMetric(); } return result; } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index c24787770b6..fdc0f7612f1 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -91,12 +91,9 @@ public AbstractMultiCatalogTransform( } @Override - public void open() {} - - @Override - public void loadContext(Context context) { + public void open(Context context) { this.context = context; - transformMap.values().forEach(transform -> transform.loadContext(context)); + transformMap.values().forEach(transform -> transform.open(context)); } protected abstract SeaTunnelTransform buildTransform( diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 22fbacb6558..40d73c369b0 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -49,7 +49,9 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans private Context context; @Override - public void open() {} + public void open(Context context) { + this.context = context; + } public AbstractSeaTunnelTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { @@ -96,11 +98,6 @@ public List getProducedCatalogTables() { return Collections.singletonList(getProducedCatalogTable()); } - @Override - public void loadContext(Context context) { - this.context = context; - } - private CatalogTable transformCatalogTable() { TableIdentifier tableIdentifier = transformTableIdentifier(); TableSchema tableSchema = transformTableSchema(); @@ -149,7 +146,7 @@ protected void hazelcastMetric(long size) { } } - protected void hazelcastMetric() { + protected void updateMetric() { if (context != null && context.getMetricsContext() != null) { hazelcastMetric(1); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java index c1dff896f6f..10408599dc5 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java @@ -63,13 +63,18 @@ public EmbeddingTransform( private void tryOpen() { if (model == null) { - open(); + initModel(); } } @Override - public void open() { + public void open(Context context) { + super.open(context); // Initialize model + initModel(); + } + + private void initModel() { ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); try { switch (provider) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java index fe069594e7a..ece2a5619f6 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java @@ -56,7 +56,7 @@ public LLMTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable inputC private void tryOpen() { if (model == null) { - open(); + initModel(); } } @@ -66,7 +66,12 @@ public String getPluginName() { } @Override - public void open() { + public void open(Context context) { + super.open(context); + initModel(); + } + + private void initModel() { ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); switch (provider) { case CUSTOM: diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index 90d4d371102..6cbf4530f7b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -79,7 +79,18 @@ public String getPluginName() { } @Override - public void open() { + public void open(Context context) { + super.open(context); + initSqlEngine(); + } + + private void tryOpen() { + if (sqlEngine == null) { + initSqlEngine(); + } + } + + private void initSqlEngine() { sqlEngine = SQLEngineFactory.getSQLEngine(engineType); sqlEngine.init( inputTableName, @@ -88,12 +99,6 @@ public void open() { query); } - private void tryOpen() { - if (sqlEngine == null) { - open(); - } - } - @Override protected List transformRow(SeaTunnelRow inputRow) { tryOpen(); From 3b064bb6e098d112399561ec4ac20089d0316dfb Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 11:03:15 +0800 Subject: [PATCH 37/56] [Feature][transform-v2] Support transform metrics --- .../apache/seatunnel/api/event/EventType.java | 2 - .../api/transform/SeaTunnelTransform.java | 11 +++-- .../transform/event/TransformCloseEvent.java | 40 ----------------- .../transform/event/TransformOpenEvent.java | 40 ----------------- .../core/dag/actions/TransformAction.java | 13 +++++- .../dag/actions/TransformChainAction.java | 11 ++++- .../parse/MultipleTableJobConfigParser.java | 3 +- .../dag/execution/ExecutionPlanGenerator.java | 10 +++-- .../server/task/context/TransformContext.java | 43 ------------------- .../task/flow/TransformFlowLifeCycle.java | 38 +++++++++++++--- ...bstractCatalogSupportFlatMapTransform.java | 3 -- .../AbstractCatalogSupportMapTransform.java | 3 -- .../common/AbstractMultiCatalogTransform.java | 7 --- .../common/AbstractSeaTunnelTransform.java | 27 +----------- .../embedding/EmbeddingTransform.java | 3 +- .../transform/nlpmodel/llm/LLMTransform.java | 3 +- .../seatunnel/transform/sql/SQLTransform.java | 3 +- 17 files changed, 72 insertions(+), 188 deletions(-) delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java delete mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java index 7b8b59aa9e0..edb1b72f36b 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/event/EventType.java @@ -29,7 +29,5 @@ public enum EventType { LIFECYCLE_READER_OPEN, LIFECYCLE_READER_CLOSE, LIFECYCLE_WRITER_CLOSE, - LIFECYCLE_TRANSFORM_CLOSE, - LIFECYCLE_TRANSFORM_OPEN, READER_MESSAGE_DELAYED, } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index 152b4bb50b7..747da450f54 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.common.PluginIdentifierInterface; import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.source.SeaTunnelJobAware; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; @@ -31,7 +32,7 @@ public interface SeaTunnelTransform extends Serializable, PluginIdentifierInterface, SeaTunnelJobAware { /** call it when Transformer initialed */ - default void open(Context context) {} + default void open() {} /** * Set the data type info of input data. @@ -61,7 +62,11 @@ interface Context extends Serializable { /** @return metricsContext of this transform. */ MetricsContext getMetricsContext(); - /** @return name of this transform. */ - String getTransformName(); + /** + * Get the {@link EventListener} of this transform. + * + * @return + */ + EventListener getEventListener(); } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java deleted file mode 100644 index 9ae056d6b80..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformCloseEvent.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.transform.event; - -import org.apache.seatunnel.api.event.EventType; -import org.apache.seatunnel.api.event.LifecycleEvent; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.Setter; -import lombok.ToString; - -@Getter -@Setter -@ToString -@AllArgsConstructor -public class TransformCloseEvent implements LifecycleEvent { - private long createdTime; - private String jobId; - private EventType eventType = EventType.LIFECYCLE_TRANSFORM_CLOSE; - - public TransformCloseEvent() { - this.createdTime = System.currentTimeMillis(); - } -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java deleted file mode 100644 index d7b2f12c8e7..00000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/event/TransformOpenEvent.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.transform.event; - -import org.apache.seatunnel.api.event.EventType; -import org.apache.seatunnel.api.event.LifecycleEvent; - -import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.Setter; -import lombok.ToString; - -@Getter -@Setter -@ToString -@AllArgsConstructor -public class TransformOpenEvent implements LifecycleEvent { - private long createdTime; - private String jobId; - private EventType eventType = EventType.LIFECYCLE_TRANSFORM_OPEN; - - public TransformOpenEvent() { - this.createdTime = System.currentTimeMillis(); - } -} diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformAction.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformAction.java index 2d594eb0d62..59f53b627e5 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformAction.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformAction.java @@ -28,6 +28,7 @@ public class TransformAction extends AbstractAction { private final SeaTunnelTransform transform; + private final String pluginOutput; public TransformAction( long id, @@ -35,9 +36,11 @@ public TransformAction( @NonNull List upstreams, @NonNull SeaTunnelTransform transform, @NonNull Set jarUrls, - @NonNull Set connectorJarIdentifiers) { + @NonNull Set connectorJarIdentifiers, + String pluginOutput) { super(id, name, upstreams, jarUrls, connectorJarIdentifiers); this.transform = transform; + this.pluginOutput = pluginOutput; } public TransformAction( @@ -45,12 +48,18 @@ public TransformAction( @NonNull String name, @NonNull SeaTunnelTransform transform, @NonNull Set jarUrls, - @NonNull Set connectorJarIdentifiers) { + @NonNull Set connectorJarIdentifiers, + String pluginOutput) { super(id, name, jarUrls, connectorJarIdentifiers); this.transform = transform; + this.pluginOutput = pluginOutput; } public SeaTunnelTransform getTransform() { return transform; } + + public String getPluginOutput() { + return pluginOutput; + } } diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java index ae3c64dcef1..63601d6131d 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java @@ -30,6 +30,7 @@ public class TransformChainAction extends AbstractAction { private static final long serialVersionUID = -340174711145367535L; private final List> transforms; + private final List pluginOutputs; public TransformChainAction( long id, @@ -37,9 +38,10 @@ public TransformChainAction( @NonNull List upstreams, @NonNull Set jarUrls, @NonNull Set connectorJarIdentifiers, - @NonNull List> transforms) { + @NonNull List> transforms, List pluginOutputs) { super(id, name, upstreams, jarUrls, connectorJarIdentifiers); this.transforms = transforms; + this.pluginOutputs = pluginOutputs; } public TransformChainAction( @@ -47,12 +49,17 @@ public TransformChainAction( @NonNull String name, @NonNull Set jarUrls, @NonNull Set connectorJarIdentifiers, - @NonNull List> transforms) { + @NonNull List> transforms, List pluginOutputs) { super(id, name, jarUrls, connectorJarIdentifiers); this.transforms = transforms; + this.pluginOutputs = pluginOutputs; } public List> getTransforms() { return transforms; } + + public List getPluginOutputs() { + return pluginOutputs; + } } diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java index 22abdfbffdf..c780ee39112 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java @@ -497,7 +497,8 @@ private void parseTransform( new ArrayList<>(inputActions), transform, jarUrls, - new HashSet<>()); + new HashSet<>(), + readonlyConfig.get(CommonOptions.PLUGIN_OUTPUT)); transformAction.setParallelism(parallelism); List> actions = new ArrayList<>(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java index 3755477f45a..cd3f83659a0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java @@ -125,7 +125,8 @@ public static Action recreateAction(Action action, Long id, int parallelism) { action.getName(), ((TransformAction) action).getTransform(), action.getJarUrls(), - action.getConnectorJarIdentifiers()); + action.getConnectorJarIdentifiers(), + ((TransformAction) action).getPluginOutput()); } else if (action instanceof TransformChainAction) { newAction = new TransformChainAction( @@ -133,7 +134,8 @@ public static Action recreateAction(Action action, Long id, int parallelism) { action.getName(), action.getJarUrls(), action.getConnectorJarIdentifiers(), - ((TransformChainAction) action).getTransforms()); + ((TransformChainAction) action).getTransforms(), + ((TransformChainAction) action).getPluginOutputs()); } else { throw new UnknownActionException(action); } @@ -368,6 +370,7 @@ private void fillChainedTransformExecutionVertex( if (transformChainedVertices.size() > 0) { long newVertexId = idGenerator.getNextId(); List transforms = new ArrayList<>(transformChainedVertices.size()); + List pluginOutputs = new ArrayList<>(transformChainedVertices.size()); List names = new ArrayList<>(transformChainedVertices.size()); Set jars = new HashSet<>(); Set identifiers = new HashSet<>(); @@ -382,6 +385,7 @@ private void fillChainedTransformExecutionVertex( .forEach( action -> { transforms.add(action.getTransform()); + pluginOutputs.add(action.getPluginOutput()); jars.addAll(action.getJarUrls()); identifiers.addAll(action.getConnectorJarIdentifiers()); names.add(action.getName()); @@ -390,7 +394,7 @@ private void fillChainedTransformExecutionVertex( String.format("TransformChain[%s]", String.join("->", names)); TransformChainAction transformChainAction = new TransformChainAction( - newVertexId, transformChainActionName, jars, identifiers, transforms); + newVertexId, transformChainActionName, jars, identifiers, transforms,pluginOutputs); transformChainAction.setParallelism(currentVertex.getAction().getParallelism()); ExecutionVertex executionVertex = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java deleted file mode 100644 index 2ee27c2cd16..00000000000 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/context/TransformContext.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.engine.server.task.context; - -import org.apache.seatunnel.api.common.metrics.MetricsContext; -import org.apache.seatunnel.api.transform.SeaTunnelTransform; - -public class TransformContext implements SeaTunnelTransform.Context { - - private static final long serialVersionUID = -3082515319043725121L; - private final MetricsContext metricsContext; - private final String transformName; - - public TransformContext(MetricsContext metricsContext, String transformName) { - this.metricsContext = metricsContext; - this.transformName = transformName; - } - - @Override - public MetricsContext getMetricsContext() { - return metricsContext; - } - - @Override - public String getTransformName() { - return transformName; - } -} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index ef4ad8a020f..c83865204c2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -17,9 +17,11 @@ package org.apache.seatunnel.engine.server.task.flow; +import org.apache.seatunnel.api.common.metrics.MetricNames; import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; import org.apache.seatunnel.api.table.type.Record; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.Collector; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; @@ -30,7 +32,6 @@ import org.apache.seatunnel.engine.server.checkpoint.ActionSubtaskState; import org.apache.seatunnel.engine.server.checkpoint.CheckpointBarrier; import org.apache.seatunnel.engine.server.task.SeaTunnelTask; -import org.apache.seatunnel.engine.server.task.context.TransformContext; import org.apache.seatunnel.engine.server.task.record.Barrier; import org.apache.commons.collections4.CollectionUtils; @@ -50,6 +51,8 @@ public class TransformFlowLifeCycle extends ActionFlowLifeCycle private final List> transform; + private final List pluginOutputList; + private final Collector> collector; private MetricsContext metricsContext; @@ -63,19 +66,16 @@ public TransformFlowLifeCycle( super(action, runningTask, completableFuture); this.action = action; this.transform = action.getTransforms(); + this.pluginOutputList = action.getPluginOutputs(); this.collector = collector; this.metricsContext = metricsContext; } @Override public void open() throws Exception { - int index = 0; for (SeaTunnelTransform t : transform) { try { - String transformName = t.getPluginName() + "_" + ++index; - TransformContext transformContext = - new TransformContext(metricsContext, transformName); - t.open(transformContext); + t.open(); } catch (Exception e) { log.error( "Open transform: {} failed, cause: {}", @@ -145,8 +145,10 @@ public List transform(T inputData) { List dataList = new ArrayList<>(); dataList.add(inputData); - + int index = 0; for (SeaTunnelTransform transformer : transform) { + String pluginOutput = pluginOutputList.get(index); + String metricName = transformer.getPluginName() + "_" + ++index; List nextInputDataList = new ArrayList<>(); if (transformer instanceof SeaTunnelFlatMapTransform) { SeaTunnelFlatMapTransform transformDecorator = @@ -160,6 +162,12 @@ public List transform(T inputData) { outputDataArray); if (CollectionUtils.isNotEmpty(outputDataArray)) { nextInputDataList.addAll(outputDataArray); + for (T outputData : outputDataArray) { + if(outputData instanceof SeaTunnelRow) { + String tableId = pluginOutput==null?((SeaTunnelRow) outputData).getTableId():pluginOutput; + updateMetric(metricName,tableId); + } + } } } } else if (transformer instanceof SeaTunnelMapTransform) { @@ -177,6 +185,9 @@ public List transform(T inputData) { continue; } nextInputDataList.add(outputData); + if(outputData instanceof SeaTunnelRow) { + updateMetric(metricName,((SeaTunnelRow) outputData).getTableId()); + } } } @@ -203,4 +214,17 @@ public void close() throws IOException { } super.close(); } + + private void updateMetric(String metricName,String tableId) { + StringBuilder metricNameBuilder = new StringBuilder(); + metricNameBuilder + .append(MetricNames.TRANSFORM_OUTPUT_COUNT) + .append("#") + .append(metricName) + .append("#") + .append(tableId); + if (metricsContext != null) { + metricsContext.counter(metricNameBuilder.toString()).inc(); + } + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index f7990a1abbe..5ce12ffdda4 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -41,9 +41,6 @@ public AbstractCatalogSupportFlatMapTransform( @Override public List flatMap(SeaTunnelRow row) { List results = transform(row); - if (CollectionUtils.isNotEmpty(results)) { - hazelcastMetric(results.size()); - } return results; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 292bbbe76e8..e1ad4b00221 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -46,9 +46,6 @@ public AbstractCatalogSupportMapTransform( @Override public SeaTunnelRow map(SeaTunnelRow row) { SeaTunnelRow result = transform(row); - if (Objects.nonNull(result)) { - updateMetric(); - } return result; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index fdc0f7612f1..02cf777f75e 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -44,8 +44,6 @@ public abstract class AbstractMultiCatalogTransform implements SeaTunnelTransfor protected Map> transformMap; - private Context context; - public AbstractMultiCatalogTransform( List inputCatalogTables, ReadonlyConfig config) { this.inputCatalogTables = inputCatalogTables; @@ -90,11 +88,6 @@ public AbstractMultiCatalogTransform( .collect(Collectors.toList()); } - @Override - public void open(Context context) { - this.context = context; - transformMap.values().forEach(transform -> transform.open(context)); - } protected abstract SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 40d73c369b0..7ef8d19a7a5 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -46,11 +46,8 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected String outTableName; - private Context context; - @Override - public void open(Context context) { - this.context = context; + public void open() { } public AbstractSeaTunnelTransform( @@ -140,26 +137,4 @@ public R transform(SeaTunnelRow row) { protected abstract TableIdentifier transformTableIdentifier(); - protected void hazelcastMetric(long size) { - if (context != null && context.getMetricsContext() != null) { - context.getMetricsContext().counter(getTransformMetricName()).inc(size); - } - } - - protected void updateMetric() { - if (context != null && context.getMetricsContext() != null) { - hazelcastMetric(1); - } - } - - protected String getTransformMetricName() { - StringBuilder metricName = new StringBuilder(); - metricName - .append(MetricNames.TRANSFORM_OUTPUT_COUNT) - .append("#") - .append(context.getTransformName()) - .append("#") - .append(outTableName); - return metricName.toString(); - } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java index 10408599dc5..0d9ad9e2425 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java @@ -68,8 +68,7 @@ private void tryOpen() { } @Override - public void open(Context context) { - super.open(context); + public void open() { // Initialize model initModel(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java index ece2a5619f6..42e6020b54c 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java @@ -66,8 +66,7 @@ public String getPluginName() { } @Override - public void open(Context context) { - super.open(context); + public void open() { initModel(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index 6cbf4530f7b..246b6febf03 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -79,8 +79,7 @@ public String getPluginName() { } @Override - public void open(Context context) { - super.open(context); + public void open() { initSqlEngine(); } From 71a9828baeb6cacd713eb9510670314b8ca88d39 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 11:04:25 +0800 Subject: [PATCH 38/56] [Feature][transform-v2] Support transform metrics --- .../core/dag/actions/TransformChainAction.java | 6 ++++-- .../dag/execution/ExecutionPlanGenerator.java | 7 ++++++- .../server/task/flow/TransformFlowLifeCycle.java | 15 +++++++++------ .../AbstractCatalogSupportFlatMapTransform.java | 2 -- .../AbstractCatalogSupportMapTransform.java | 2 -- .../common/AbstractMultiCatalogTransform.java | 1 - .../common/AbstractSeaTunnelTransform.java | 5 +---- 7 files changed, 20 insertions(+), 18 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java index 63601d6131d..9431cf698fa 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java @@ -38,7 +38,8 @@ public TransformChainAction( @NonNull List upstreams, @NonNull Set jarUrls, @NonNull Set connectorJarIdentifiers, - @NonNull List> transforms, List pluginOutputs) { + @NonNull List> transforms, + List pluginOutputs) { super(id, name, upstreams, jarUrls, connectorJarIdentifiers); this.transforms = transforms; this.pluginOutputs = pluginOutputs; @@ -49,7 +50,8 @@ public TransformChainAction( @NonNull String name, @NonNull Set jarUrls, @NonNull Set connectorJarIdentifiers, - @NonNull List> transforms, List pluginOutputs) { + @NonNull List> transforms, + List pluginOutputs) { super(id, name, jarUrls, connectorJarIdentifiers); this.transforms = transforms; this.pluginOutputs = pluginOutputs; diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java index cd3f83659a0..fcc4c41b3ad 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java @@ -394,7 +394,12 @@ private void fillChainedTransformExecutionVertex( String.format("TransformChain[%s]", String.join("->", names)); TransformChainAction transformChainAction = new TransformChainAction( - newVertexId, transformChainActionName, jars, identifiers, transforms,pluginOutputs); + newVertexId, + transformChainActionName, + jars, + identifiers, + transforms, + pluginOutputs); transformChainAction.setParallelism(currentVertex.getAction().getParallelism()); ExecutionVertex executionVertex = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index c83865204c2..dfb1036fbc3 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -163,9 +163,12 @@ public List transform(T inputData) { if (CollectionUtils.isNotEmpty(outputDataArray)) { nextInputDataList.addAll(outputDataArray); for (T outputData : outputDataArray) { - if(outputData instanceof SeaTunnelRow) { - String tableId = pluginOutput==null?((SeaTunnelRow) outputData).getTableId():pluginOutput; - updateMetric(metricName,tableId); + if (outputData instanceof SeaTunnelRow) { + String tableId = + pluginOutput == null + ? ((SeaTunnelRow) outputData).getTableId() + : pluginOutput; + updateMetric(metricName, tableId); } } } @@ -185,8 +188,8 @@ public List transform(T inputData) { continue; } nextInputDataList.add(outputData); - if(outputData instanceof SeaTunnelRow) { - updateMetric(metricName,((SeaTunnelRow) outputData).getTableId()); + if (outputData instanceof SeaTunnelRow) { + updateMetric(metricName, ((SeaTunnelRow) outputData).getTableId()); } } } @@ -215,7 +218,7 @@ public void close() throws IOException { super.close(); } - private void updateMetric(String metricName,String tableId) { + private void updateMetric(String metricName, String tableId) { StringBuilder metricNameBuilder = new StringBuilder(); metricNameBuilder .append(MetricNames.TRANSFORM_OUTPUT_COUNT) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index 5ce12ffdda4..a62c2992dfa 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -21,8 +21,6 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; -import org.apache.commons.collections4.CollectionUtils; - import lombok.NonNull; import lombok.extern.slf4j.Slf4j; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index e1ad4b00221..401efbe7d8d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -25,8 +25,6 @@ import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import java.util.Objects; - @Slf4j public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index 02cf777f75e..2bd6b092f98 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -88,7 +88,6 @@ public AbstractMultiCatalogTransform( .collect(Collectors.toList()); } - protected abstract SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 7ef8d19a7a5..05d3845229d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.common.metrics.MetricNames; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; @@ -47,8 +46,7 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected String outTableName; @Override - public void open() { - } + public void open() {} public AbstractSeaTunnelTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { @@ -136,5 +134,4 @@ public R transform(SeaTunnelRow row) { protected abstract TableSchema transformTableSchema(); protected abstract TableIdentifier transformTableIdentifier(); - } From c80f8472959fc592738a90acb94033d6865fa438 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 16:12:13 +0800 Subject: [PATCH 39/56] [Feature][transform-v2] Support flink transform metrics --- .../api/transform/SeaTunnelTransform.java | 15 --- .../execution/TransformExecuteProcessor.java | 95 ++++++++++++++++--- .../execution/TransformExecuteProcessor.java | 15 ++- .../dag/actions/TransformChainAction.java | 13 ++- .../dag/execution/ExecutionPlanGenerator.java | 7 +- .../task/flow/TransformFlowLifeCycle.java | 6 +- .../flink/metric/FlinkJobMetricsSummary.java | 92 ++++++++++++++---- 7 files changed, 188 insertions(+), 55 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java index 747da450f54..b12d69bafbb 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelTransform.java @@ -18,8 +18,6 @@ package org.apache.seatunnel.api.transform; import org.apache.seatunnel.api.common.PluginIdentifierInterface; -import org.apache.seatunnel.api.common.metrics.MetricsContext; -import org.apache.seatunnel.api.event.EventListener; import org.apache.seatunnel.api.source.SeaTunnelJobAware; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; @@ -56,17 +54,4 @@ default SchemaChangeEvent mapSchemaChangeEvent(SchemaChangeEvent schemaChangeEve /** call it when Transformer completed */ default void close() {} - - interface Context extends Serializable { - - /** @return metricsContext of this transform. */ - MetricsContext getMetricsContext(); - - /** - * Get the {@link EventListener} of this transform. - * - * @return - */ - EventListener getEventListener(); - } } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index 615876c4173..97214e1a47f 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.common.metrics.MetricNames; +import org.apache.seatunnel.api.common.metrics.MetricsContext; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.ConfigValidator; import org.apache.seatunnel.api.table.factory.TableTransformFactory; @@ -32,12 +34,16 @@ import org.apache.seatunnel.core.starter.execution.PluginUtil; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; +import org.apache.seatunnel.translation.flink.metric.FlinkMetricContext; import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.util.Collector; import java.net.URL; @@ -103,24 +109,28 @@ public List execute(List upstreamDataS LinkedHashMap::new)); ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + int index = 0; for (int i = 0; i < plugins.size(); i++) { try { Config pluginConfig = pluginConfigs.get(i); + ReadonlyConfig options = ReadonlyConfig.fromConfig(pluginConfig); DataStreamTableInfo stream = fromSourceTable(pluginConfig, new ArrayList<>(outputTables.values())) .orElse(input); TableTransformFactory factory = plugins.get(i); TableTransformFactoryContext context = new TableTransformFactoryContext( - stream.getCatalogTables(), - ReadonlyConfig.fromConfig(pluginConfig), - classLoader); + stream.getCatalogTables(), options, classLoader); ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); + SeaTunnelTransform transform = factory.createTransform(context).createTransform(); transform.setJobContext(jobContext); + String metricName = + String.format("Transform[%s]-%s", transform.getPluginName(), index++); + String pluginOutput = options.get(PLUGIN_OUTPUT); DataStream inputStream = - flinkTransform(transform, stream.getDataStream()); + flinkTransform(transform, stream.getDataStream(), metricName, pluginOutput); String pluginOutputIdentifier = ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT); // TODO transform support multi tables @@ -142,10 +152,14 @@ public List execute(List upstreamDataS } protected DataStream flinkTransform( - SeaTunnelTransform transform, DataStream stream) { + SeaTunnelTransform transform, + DataStream stream, + String metricName, + String pluginOutput) { if (transform instanceof SeaTunnelFlatMapTransform) { return stream.flatMap( - new ArrayFlatMap(transform), TypeInformation.of(SeaTunnelRow.class)); + new ArrayFlatMap(transform, metricName, pluginOutput), + TypeInformation.of(SeaTunnelRow.class)); } return stream.transform( @@ -155,20 +169,59 @@ protected DataStream flinkTransform( flinkRuntimeEnvironment .getStreamExecutionEnvironment() .clean( - row -> - ((SeaTunnelMapTransform) - transform) - .map(row)))) + new FlinkRichMapFunction( + transform, metricName, pluginOutput)))) // null value shouldn't be passed to downstream .filter(Objects::nonNull); } - public static class ArrayFlatMap implements FlatMapFunction { + public static class FlinkRichMapFunction extends RichMapFunction { + private MetricsContext metricsContext; // + private SeaTunnelTransform transform; + private final String metricName; + private final String pluginOutput; + + public FlinkRichMapFunction( + SeaTunnelTransform transform, String metricName, String pluginOutput) { + this.transform = transform; + this.metricName = metricName; + this.pluginOutput = pluginOutput; + } + + @Override + public void open(Configuration parameters) throws Exception { + // 获取 RuntimeContext + metricsContext = new FlinkMetricContext((StreamingRuntimeContext) getRuntimeContext()); + } + @Override + public SeaTunnelRow map(SeaTunnelRow row) throws Exception { + if (Objects.isNull(row)) { + return null; + } + SeaTunnelRow rowResult = ((SeaTunnelMapTransform) transform).map(row); + String tableId = pluginOutput == null ? rowResult.getTableId() : pluginOutput; + updateMetric(metricName, tableId, metricsContext); + return rowResult; + } + } + + public static class ArrayFlatMap extends RichFlatMapFunction { + private MetricsContext metricsContext; // private SeaTunnelTransform transform; + private final String metricName; + private final String pluginOutput; - public ArrayFlatMap(SeaTunnelTransform transform) { + public ArrayFlatMap(SeaTunnelTransform transform, String metricName, String pluginOutput) { this.transform = transform; + this.metricName = metricName; + this.pluginOutput = pluginOutput; + } + + @Override + public void open(Configuration parameters) throws Exception { + // 获取 RuntimeContext + metricsContext = new FlinkMetricContext((StreamingRuntimeContext) getRuntimeContext()); } @Override @@ -177,9 +230,25 @@ public void flatMap(SeaTunnelRow row, Collector collector) { ((SeaTunnelFlatMapTransform) transform).flatMap(row); if (CollectionUtils.isNotEmpty(rows)) { for (SeaTunnelRow rowResult : rows) { + String tableId = pluginOutput == null ? rowResult.getTableId() : pluginOutput; + updateMetric(metricName, tableId, metricsContext); collector.collect(rowResult); } } } } + + private static void updateMetric( + String metricName, String tableId, MetricsContext metricsContext) { + StringBuilder metricNameBuilder = new StringBuilder(); + metricNameBuilder + .append(MetricNames.TRANSFORM_OUTPUT_COUNT) + .append("#") + .append(metricName) + .append("#") + .append(tableId); + if (metricsContext != null) { + metricsContext.counter(metricNameBuilder.toString()).inc(); + } + } } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index 492af1ad73d..03214b5e83e 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -112,9 +112,11 @@ public List execute(List upstreamDataStreams e -> e, (a, b) -> b, LinkedHashMap::new)); + int index = 0; for (int i = 0; i < plugins.size(); i++) { try { Config pluginConfig = pluginConfigs.get(i); + ReadonlyConfig options = ReadonlyConfig.fromConfig(pluginConfig); DatasetTableInfo dataset = fromSourceTable( pluginConfig, @@ -130,7 +132,12 @@ public List execute(List upstreamDataStreams ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); SeaTunnelTransform transform = factory.createTransform(context).createTransform(); - Dataset inputDataset = sparkTransform(transform, dataset); + String metricName = + String.format("Transform[%s]-%s", transform.getPluginName(), index++); + String pluginOutput = options.get(PLUGIN_OUTPUT); + + Dataset inputDataset = + sparkTransform(transform, dataset, metricName, pluginOutput); registerInputTempView(pluginConfig, inputDataset); String pluginOutputIdentifier = ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT); @@ -151,7 +158,11 @@ public List execute(List upstreamDataStreams return new ArrayList<>(outputTables.values()); } - private Dataset sparkTransform(SeaTunnelTransform transform, DatasetTableInfo tableInfo) { + private Dataset sparkTransform( + SeaTunnelTransform transform, + DatasetTableInfo tableInfo, + String metricName, + String pluginOutput) { MultiTableManager inputManager = new MultiTableManager(tableInfo.getCatalogTables().toArray(new CatalogTable[0])); MultiTableManager outputManager = diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java index 9431cf698fa..747a7e38af1 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/dag/actions/TransformChainAction.java @@ -31,6 +31,7 @@ public class TransformChainAction extends AbstractAction { private static final long serialVersionUID = -340174711145367535L; private final List> transforms; private final List pluginOutputs; + private final List transformNames; public TransformChainAction( long id, @@ -39,10 +40,12 @@ public TransformChainAction( @NonNull Set jarUrls, @NonNull Set connectorJarIdentifiers, @NonNull List> transforms, - List pluginOutputs) { + List pluginOutputs, + List transformNames) { super(id, name, upstreams, jarUrls, connectorJarIdentifiers); this.transforms = transforms; this.pluginOutputs = pluginOutputs; + this.transformNames = transformNames; } public TransformChainAction( @@ -51,10 +54,12 @@ public TransformChainAction( @NonNull Set jarUrls, @NonNull Set connectorJarIdentifiers, @NonNull List> transforms, - List pluginOutputs) { + List pluginOutputs, + List transformNames) { super(id, name, jarUrls, connectorJarIdentifiers); this.transforms = transforms; this.pluginOutputs = pluginOutputs; + this.transformNames = transformNames; } public List> getTransforms() { @@ -64,4 +69,8 @@ public List> getTransforms() { public List getPluginOutputs() { return pluginOutputs; } + + public List getTransformNames() { + return transformNames; + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java index fcc4c41b3ad..f2d01a0d238 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/execution/ExecutionPlanGenerator.java @@ -135,7 +135,8 @@ public static Action recreateAction(Action action, Long id, int parallelism) { action.getJarUrls(), action.getConnectorJarIdentifiers(), ((TransformChainAction) action).getTransforms(), - ((TransformChainAction) action).getPluginOutputs()); + ((TransformChainAction) action).getPluginOutputs(), + ((TransformChainAction) action).getTransformNames()); } else { throw new UnknownActionException(action); } @@ -392,6 +393,7 @@ private void fillChainedTransformExecutionVertex( }); String transformChainActionName = String.format("TransformChain[%s]", String.join("->", names)); + List transformNames = names; TransformChainAction transformChainAction = new TransformChainAction( newVertexId, @@ -399,7 +401,8 @@ private void fillChainedTransformExecutionVertex( jars, identifiers, transforms, - pluginOutputs); + pluginOutputs, + transformNames); transformChainAction.setParallelism(currentVertex.getAction().getParallelism()); ExecutionVertex executionVertex = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index dfb1036fbc3..8e619936fca 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -53,6 +53,8 @@ public class TransformFlowLifeCycle extends ActionFlowLifeCycle private final List pluginOutputList; + private final List transformNames; + private final Collector> collector; private MetricsContext metricsContext; @@ -67,6 +69,7 @@ public TransformFlowLifeCycle( this.action = action; this.transform = action.getTransforms(); this.pluginOutputList = action.getPluginOutputs(); + this.transformNames = action.getTransformNames(); this.collector = collector; this.metricsContext = metricsContext; } @@ -148,7 +151,8 @@ public List transform(T inputData) { int index = 0; for (SeaTunnelTransform transformer : transform) { String pluginOutput = pluginOutputList.get(index); - String metricName = transformer.getPluginName() + "_" + ++index; + String metricName = transformNames.get(index); + index++; List nextInputDataList = new ArrayList<>(); if (transformer instanceof SeaTunnelFlatMapTransform) { SeaTunnelFlatMapTransform transformDecorator = diff --git a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkJobMetricsSummary.java b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkJobMetricsSummary.java index 556f028ecf0..a5209308793 100644 --- a/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkJobMetricsSummary.java +++ b/seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/metric/FlinkJobMetricsSummary.java @@ -21,10 +21,17 @@ import org.apache.seatunnel.common.utils.DateTimeUtils; import org.apache.seatunnel.common.utils.StringFormatUtils; +import org.apache.commons.collections.MapUtils; import org.apache.flink.api.common.JobExecutionResult; import java.time.Duration; import java.time.LocalDateTime; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +import static org.apache.seatunnel.api.common.metrics.MetricNames.TRANSFORM_OUTPUT_COUNT; public final class FlinkJobMetricsSummary { @@ -82,25 +89,70 @@ public FlinkJobMetricsSummary build() { @Override public String toString() { - return StringFormatUtils.formatTable( - "Job Statistic Information", - "Start Time", - DateTimeUtils.toString(jobStartTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), - "End Time", - DateTimeUtils.toString(jobEndTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), - "Total Time(s)", - Duration.between(jobStartTime, jobEndTime).getSeconds(), - "Total Read Count", - jobExecutionResult - .getAllAccumulatorResults() - .get(MetricNames.SOURCE_RECEIVED_COUNT), - "Total Write Count", - jobExecutionResult.getAllAccumulatorResults().get(MetricNames.SINK_WRITE_COUNT), - "Total Read Bytes", - jobExecutionResult - .getAllAccumulatorResults() - .get(MetricNames.SOURCE_RECEIVED_BYTES), - "Total Write Bytes", - jobExecutionResult.getAllAccumulatorResults().get(MetricNames.SINK_WRITE_BYTES)); + Map> transformCountMap = new TreeMap<>(); + + // Transform metrics + jobExecutionResult.getAllAccumulatorResults().entrySet().stream() + .filter(entry -> entry.getKey().contains(TRANSFORM_OUTPUT_COUNT)) + .forEach( + (metric) -> { + String tableName = metric.getKey().split("#")[1]; + String path = metric.getKey().split("#")[2]; + if (transformCountMap.containsKey(tableName)) { + Map metricMap = transformCountMap.get(tableName); + metricMap.put(path, metric.getValue()); + } else { + Map metricMap = new HashMap<>(); + metricMap.put(path, metric.getValue()); + transformCountMap.put(tableName, metricMap); + } + }); + + StringBuilder logMessage = new StringBuilder(); + logMessage.append( + StringFormatUtils.formatTable( + "Job Statistic Information", + "Start Time", + DateTimeUtils.toString( + jobStartTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), + "End Time", + DateTimeUtils.toString( + jobEndTime, DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS), + "Total Time(s)", + Duration.between(jobStartTime, jobEndTime).getSeconds(), + "Total Read Count", + jobExecutionResult + .getAllAccumulatorResults() + .get(MetricNames.SOURCE_RECEIVED_COUNT), + "Total Write Count", + jobExecutionResult + .getAllAccumulatorResults() + .get(MetricNames.SINK_WRITE_COUNT), + "Total Read Bytes", + jobExecutionResult + .getAllAccumulatorResults() + .get(MetricNames.SOURCE_RECEIVED_BYTES), + "Total Write Bytes", + jobExecutionResult + .getAllAccumulatorResults() + .get(MetricNames.SINK_WRITE_BYTES))); + + if (MapUtils.isNotEmpty(transformCountMap)) { + transformCountMap.forEach( + (tableName, metrics) -> { + String[] transformInfos = new String[metrics.entrySet().size() * 2 + 2]; + transformInfos[0] = tableName + " Information"; + int index = 0; + for (Map.Entry entry : metrics.entrySet()) { + transformInfos[++index] = entry.getKey(); + transformInfos[++index] = String.valueOf(entry.getValue()); + } + if (Objects.nonNull(transformInfos)) { + logMessage.append(StringFormatUtils.formatTable(transformInfos)); + } + }); + } + + return logMessage.toString(); } } From 9e1be272f915ead8a7d5df7fa1d729a58dd539fb Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 16:15:13 +0800 Subject: [PATCH 40/56] [Feature][transform-v2] Support flink transform metrics --- .../engine/e2e/MultiTableMetricsIT.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java index 308e0ec5920..5ef5e7f23ab 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/MultiTableMetricsIT.java @@ -134,22 +134,22 @@ public void multiTableMetrics() { "metrics.TableSinkWriteBytes.'fake.public.table2'", equalTo(String.valueOf(dataSize * 5))) .body( - "metrics.TransformOutputCount.'Sql_1'.'fake3'", + "metrics.TransformOutputCount.'Transform[0]-Sql'.'fake3'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Sql_2'.'fake4'", + "metrics.TransformOutputCount.'Transform[1]-Sql'.'fake4'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Sql_3'.'fake5'", + "metrics.TransformOutputCount.'Transform[2]-Sql'.'fake5'", equalTo("10")) .body( - "metrics.TransformOutputCount.'Sql_1'.'fake6'", + "metrics.TransformOutputCount.'Transform[3]-Sql'.'fake6'", equalTo("5")) .body( - "metrics.TransformOutputCount.'Sql_2'.'fake7'", + "metrics.TransformOutputCount.'Transform[4]-Sql'.'fake7'", equalTo("5")) .body( - "metrics.TransformOutputCount.'Sql_3'.'fake8'", + "metrics.TransformOutputCount.'Transform[5]-Sql'.'fake8'", equalTo("5")); Assertions.assertTrue( Double.parseDouble(response.path("metrics.SourceReceivedQPS")) @@ -199,31 +199,31 @@ public void multiTableMetrics() { > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Sql_1'.'fake3'")) + "metrics.TransformOutputCount.'Transform[0]-Sql'.'fake3'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Sql_2'" + "metrics.TransformOutputCount.'Transform[1]-Sql'" + ".'fake4'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Sql_3'" + "metrics.TransformOutputCount.'Transform[2]-Sql'" + ".'fake5'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Sql_1'" + "metrics.TransformOutputCount.'Transform[3]-Sql'" + ".'fake6'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Sql_2'" + "metrics.TransformOutputCount.'Transform[4]-Sql'" + ".'fake7'")) > 0 && Double.parseDouble( response.path( - "metrics.TransformOutputCount.'Sql_3'" + "metrics.TransformOutputCount.'Transform[5]-Sql'" + ".'fake8'")) > 0); }); From f4edf091a6b6c8d24da6d08c903807653c045b07 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 16:51:21 +0800 Subject: [PATCH 41/56] [Feature][transform-v2] Support flink transform metrics --- .../core/starter/flink/execution/TransformExecuteProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index 97214e1a47f..c7570cb6322 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -190,7 +190,6 @@ public FlinkRichMapFunction( @Override public void open(Configuration parameters) throws Exception { - // 获取 RuntimeContext metricsContext = new FlinkMetricContext((StreamingRuntimeContext) getRuntimeContext()); } From ce8df7be230771273236508e6afbae4c186c176d Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 16:56:45 +0800 Subject: [PATCH 42/56] [Feature][transform-v2] Support flink transform metrics --- .../starter/flink/execution/TransformExecuteProcessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index c7570cb6322..d77434082fe 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -176,7 +176,7 @@ protected DataStream flinkTransform( } public static class FlinkRichMapFunction extends RichMapFunction { - private MetricsContext metricsContext; // + private MetricsContext metricsContext; private SeaTunnelTransform transform; private final String metricName; private final String pluginOutput; @@ -206,7 +206,7 @@ public SeaTunnelRow map(SeaTunnelRow row) throws Exception { } public static class ArrayFlatMap extends RichFlatMapFunction { - private MetricsContext metricsContext; // + private MetricsContext metricsContext; private SeaTunnelTransform transform; private final String metricName; private final String pluginOutput; From 2c7c683b91294b834b7dec22f57c857fddf0691b Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 16 Jan 2025 16:57:22 +0800 Subject: [PATCH 43/56] [Feature][transform-v2] Support flink transform metrics --- .../core/starter/flink/execution/TransformExecuteProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index d77434082fe..5365b230f37 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -219,7 +219,6 @@ public ArrayFlatMap(SeaTunnelTransform transform, String metricName, String plug @Override public void open(Configuration parameters) throws Exception { - // 获取 RuntimeContext metricsContext = new FlinkMetricContext((StreamingRuntimeContext) getRuntimeContext()); } From 6fd3d9dfbc54f0789cd6741acde550db0082fdf1 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Fri, 17 Jan 2025 11:45:29 +0800 Subject: [PATCH 44/56] [Feature][transform-v2] Support flink transform metrics --- .../starter/flink/execution/TransformExecuteProcessor.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index 5365b230f37..ea1c30c49f8 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -199,8 +199,10 @@ public SeaTunnelRow map(SeaTunnelRow row) throws Exception { return null; } SeaTunnelRow rowResult = ((SeaTunnelMapTransform) transform).map(row); - String tableId = pluginOutput == null ? rowResult.getTableId() : pluginOutput; - updateMetric(metricName, tableId, metricsContext); + if (rowResult != null) { + String tableId = pluginOutput == null ? rowResult.getTableId() : pluginOutput; + updateMetric(metricName, tableId, metricsContext); + } return rowResult; } } From 807c44e0523094d38dc67224dfa39eda62d689fa Mon Sep 17 00:00:00 2001 From: nijiahui Date: Mon, 20 Jan 2025 11:17:54 +0800 Subject: [PATCH 45/56] [Fix][doc] Repair doris dead link --- docs/en/connector-v2/sink/Doris.md | 4 ++-- docs/zh/connector-v2/sink/Doris.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/connector-v2/sink/Doris.md b/docs/en/connector-v2/sink/Doris.md index dc177a3962b..1f7571ac0ef 100644 --- a/docs/en/connector-v2/sink/Doris.md +++ b/docs/en/connector-v2/sink/Doris.md @@ -37,7 +37,7 @@ The internal implementation of Doris sink connector is cached and imported by st ## Sink Options -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |--------------------------------|---------|----------|------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | fenodes | String | Yes | - | `Doris` cluster fenodes address, the format is `"fe_ip:fe_http_port, ..."` | | query-port | int | No | 9030 | `Doris` Fenodes query_port | @@ -47,7 +47,7 @@ The internal implementation of Doris sink connector is cached and imported by st | table | String | Yes | - | The table name of `Doris` table, use `${table_name}` to represent the upstream table name | | table.identifier | String | Yes | - | The name of `Doris` table, it will deprecate after version 2.3.5, please use `database` and `table` instead. | | sink.label-prefix | String | Yes | - | The label prefix used by stream load imports. In the 2pc scenario, global uniqueness is required to ensure the EOS semantics of SeaTunnel. | -| sink.enable-2pc | bool | No | false | Whether to enable two-phase commit (2pc), the default is false. For two-phase commit, please refer to [here](https://doris.apache.org/docs/dev/sql-manual/sql-statements/Data-Manipulation-Statements/Load/STREAM-LOAD/). | +| sink.enable-2pc | bool | No | false | Whether to enable two-phase commit (2pc), the default is false. For two-phase commit, please refer to [here](https://doris.apache.org/docs/data-operate/transaction?_highlight=two&_highlight=phase#stream-load-2pc). | | sink.enable-delete | bool | No | - | Whether to enable deletion. This option requires Doris table to enable batch delete function (0.15+ version is enabled by default), and only supports Unique model. you can get more detail at this [link](https://doris.apache.org/docs/dev/data-operate/delete/batch-delete-manual/) | | sink.check-interval | int | No | 10000 | check exception with the interval while loading | | sink.max-retries | int | No | 3 | the max retry times if writing records to database failed | diff --git a/docs/zh/connector-v2/sink/Doris.md b/docs/zh/connector-v2/sink/Doris.md index 66fbe728ae5..f0504977aec 100644 --- a/docs/zh/connector-v2/sink/Doris.md +++ b/docs/zh/connector-v2/sink/Doris.md @@ -46,7 +46,7 @@ Doris Sink连接器的内部实现是通过stream load批量缓存和导入的 | table | String | Yes | - | `Doris` 表名, 使用 `${table_name}` 表示上游表名。 | | table.identifier | String | Yes | - | `Doris` 表的名称,2.3.5 版本后将弃用,请使用 `database` 和 `table` 代替。 | | sink.label-prefix | String | Yes | - | stream load导入使用的标签前缀。 在2pc场景下,需要全局唯一性来保证SeaTunnel的EOS语义。 | -| sink.enable-2pc | bool | No | false | 是否启用两阶段提交(2pc),默认为 false。 对于两阶段提交,请参考[此处](https://doris.apache.org/docs/dev/sql-manual/sql-statements/Data-Manipulation-Statements/Load/STREAM-LOAD/)。 | +| sink.enable-2pc | bool | No | false | 是否启用两阶段提交(2pc),默认为 false。 对于两阶段提交,请参考[此处](https://doris.apache.org/docs/data-operate/transaction?_highlight=two&_highlight=phase#stream-load-2pc)。 | | sink.enable-delete | bool | No | - | 是否启用删除。 该选项需要Doris表开启批量删除功能(0.15+版本默认开启),且仅支持Unique模型。 您可以在此[link](https://doris.apache.org/docs/dev/data-operate/delete/batch-delete-manual/)获得更多详细信息 | | sink.check-interval | int | No | 10000 | 加载过程中检查异常时间间隔。 | | sink.max-retries | int | No | 3 | 向数据库写入记录失败时的最大重试次数。 | From 30de2d4cedc461a9fc0a78c39ca13263fc1844ba Mon Sep 17 00:00:00 2001 From: nijiahui Date: Mon, 20 Jan 2025 15:26:26 +0800 Subject: [PATCH 46/56] [Feature][transform-v2] Support flink transform metrics --- .../server/task/flow/TransformFlowLifeCycle.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 8e619936fca..77b588195f8 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -168,6 +168,10 @@ public List transform(T inputData) { nextInputDataList.addAll(outputDataArray); for (T outputData : outputDataArray) { if (outputData instanceof SeaTunnelRow) { + log.info("==============pluginOutput:{}", pluginOutput); + log.info( + "==============tableId:{}", + ((SeaTunnelRow) outputData).getTableId()); String tableId = pluginOutput == null ? ((SeaTunnelRow) outputData).getTableId() @@ -193,7 +197,11 @@ public List transform(T inputData) { } nextInputDataList.add(outputData); if (outputData instanceof SeaTunnelRow) { - updateMetric(metricName, ((SeaTunnelRow) outputData).getTableId()); + String tableId = + pluginOutput == null + ? ((SeaTunnelRow) outputData).getTableId() + : pluginOutput; + updateMetric(metricName, tableId); } } } From 12af93e0cb57a33b837e1a3b02cb6a666e77eaf4 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Mon, 20 Jan 2025 15:30:36 +0800 Subject: [PATCH 47/56] [Feature][transform-v2] Support flink transform metrics --- .../engine/server/task/flow/TransformFlowLifeCycle.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index 77b588195f8..da21d63dbaa 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -168,10 +168,6 @@ public List transform(T inputData) { nextInputDataList.addAll(outputDataArray); for (T outputData : outputDataArray) { if (outputData instanceof SeaTunnelRow) { - log.info("==============pluginOutput:{}", pluginOutput); - log.info( - "==============tableId:{}", - ((SeaTunnelRow) outputData).getTableId()); String tableId = pluginOutput == null ? ((SeaTunnelRow) outputData).getTableId() From 76349c70d673c49669e7b29168d32319aa3c61a9 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Wed, 22 Jan 2025 09:03:01 +0800 Subject: [PATCH 48/56] [Feature][transform-v2] Support flink transform metrics --- docs/zh/connector-v2/sink/Doris.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/connector-v2/sink/Doris.md b/docs/zh/connector-v2/sink/Doris.md index f0504977aec..c49b08e34d2 100644 --- a/docs/zh/connector-v2/sink/Doris.md +++ b/docs/zh/connector-v2/sink/Doris.md @@ -106,7 +106,7 @@ DISTRIBUTED BY HASH (${rowtype_primary_key}) ```sql CREATE TABLE IF NOT EXISTS `${database}`.`${table_name}` -( +( id, ${rowtype_fields} ) ENGINE = OLAP UNIQUE KEY (${rowtype_primary_key}) @@ -360,4 +360,4 @@ sink { PR 4235 is an incompatible modification to PR 3856. Please refer to PR 4235 to use the new Doris connector -::: +::: \ No newline at end of file From 60cd7fe47713a82cb42a3587d5f24c01117abe81 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Wed, 22 Jan 2025 09:18:07 +0800 Subject: [PATCH 49/56] [Feature][transform-v2] Support flink transform metrics --- docs/zh/connector-v2/sink/Doris.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/connector-v2/sink/Doris.md b/docs/zh/connector-v2/sink/Doris.md index c49b08e34d2..f0504977aec 100644 --- a/docs/zh/connector-v2/sink/Doris.md +++ b/docs/zh/connector-v2/sink/Doris.md @@ -106,7 +106,7 @@ DISTRIBUTED BY HASH (${rowtype_primary_key}) ```sql CREATE TABLE IF NOT EXISTS `${database}`.`${table_name}` -( +( id, ${rowtype_fields} ) ENGINE = OLAP UNIQUE KEY (${rowtype_primary_key}) @@ -360,4 +360,4 @@ sink { PR 4235 is an incompatible modification to PR 3856. Please refer to PR 4235 to use the new Doris connector -::: \ No newline at end of file +::: From 633801ec4888fc4089213dc8a8a0b6b225b4e853 Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 23 Jan 2025 12:50:08 +0800 Subject: [PATCH 50/56] [Feature][transform-v2] Support transform metrics --- ...bstractCatalogSupportFlatMapTransform.java | 12 ++--- .../AbstractCatalogSupportMapTransform.java | 15 +++---- .../common/AbstractMultiCatalogTransform.java | 6 +-- .../common/AbstractSeaTunnelTransform.java | 37 ++------------- .../transform/common/FilterRowTransform.java | 6 +-- .../common/MultipleFieldOutputTransform.java | 12 ++--- .../common/SingleFieldOutputTransform.java | 6 +-- .../copy/CopyFieldMultiCatalogTransform.java | 2 +- .../transform/copy/CopyFieldTransform.java | 7 ++- .../DynamicCompileTransform.java | 2 +- .../FieldMapperMultiCatalogTransform.java | 2 +- .../fieldmapper/FieldMapperTransform.java | 7 ++- .../filter/FilterFieldTransform.java | 2 +- .../filterrowkind/FilterRowKindTransform.java | 2 +- .../JsonPathMultiCatalogTransform.java | 3 +- .../transform/jsonpath/JsonPathTransform.java | 10 ++--- .../transform/metadata/MetadataTransform.java | 2 +- .../embedding/EmbeddingTransform.java | 8 +--- .../transform/nlpmodel/llm/LLMTransform.java | 8 +--- .../FieldRenameMultiCatalogTransform.java | 2 +- .../rename/FieldRenameTransform.java | 7 ++- .../TableRenameMultiCatalogTransform.java | 2 +- .../rename/TableRenameTransform.java | 7 ++- .../transform/replace/ReplaceTransform.java | 2 +- .../rowkind/RowKindExtractorTransform.java | 2 +- .../split/SplitMultiCatalogTransform.java | 2 +- .../transform/split/SplitTransform.java | 9 ++-- .../seatunnel/transform/sql/SQLTransform.java | 28 +++++++----- .../TableMergeMultiCatalogTransform.java | 2 +- .../transform/table/TableMergeTransform.java | 9 ++-- .../transform/JsonPathTransformTest.java | 23 +++++----- .../rename/FieldRenameTransformTest.java | 45 +++++-------------- .../rename/TableRenameTransformTest.java | 42 ++++------------- 33 files changed, 118 insertions(+), 213 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java index a62c2992dfa..71b7b536be9 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportFlatMapTransform.java @@ -16,7 +16,6 @@ */ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform; @@ -31,14 +30,17 @@ public abstract class AbstractCatalogSupportFlatMapTransform extends AbstractSeaTunnelTransform> implements SeaTunnelFlatMapTransform { + public AbstractCatalogSupportFlatMapTransform(@NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); + } + public AbstractCatalogSupportFlatMapTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(config, catalogTable); + @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { + super(inputCatalogTable, rowErrorHandleWay); } @Override public List flatMap(SeaTunnelRow row) { - List results = transform(row); - return results; + return transform(row); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java index 401efbe7d8d..6380d885397 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportMapTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelMapTransform; @@ -29,21 +28,17 @@ public abstract class AbstractCatalogSupportMapTransform extends AbstractSeaTunnelTransform implements SeaTunnelMapTransform { - public AbstractCatalogSupportMapTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + public AbstractCatalogSupportMapTransform(@NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); } public AbstractCatalogSupportMapTransform( - @NonNull ReadonlyConfig config, - @NonNull CatalogTable inputCatalogTable, - ErrorHandleWay rowErrorHandleWay) { - super(config, inputCatalogTable, rowErrorHandleWay); + @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { + super(inputCatalogTable, rowErrorHandleWay); } @Override public SeaTunnelRow map(SeaTunnelRow row) { - SeaTunnelRow result = transform(row); - return result; + return transform(row); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java index 2bd6b092f98..49fa561979f 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractMultiCatalogTransform.java @@ -73,7 +73,7 @@ public AbstractMultiCatalogTransform( if (tableConfig != null) { transformMap.put(tableId, buildTransform(inputCatalogTable, tableConfig)); } else { - transformMap.put(tableId, new IdentityTransform(config, inputCatalogTable)); + transformMap.put(tableId, new IdentityTransform(inputCatalogTable)); } }); @@ -112,8 +112,8 @@ public String getPluginName() { return "Identity"; } - public IdentityTransform(ReadonlyConfig config, CatalogTable catalogTable) { - super(config, catalogTable); + public IdentityTransform(CatalogTable catalogTable) { + super(catalogTable); this.catalogTable = catalogTable; } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java index 05d3845229d..01ce7eaf0a2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -16,8 +16,6 @@ */ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; @@ -25,8 +23,6 @@ import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.transform.exception.ErrorDataTransformException; -import org.apache.groovy.parser.antlr4.util.StringUtils; - import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -41,39 +37,14 @@ public abstract class AbstractSeaTunnelTransform implements SeaTunnelTrans protected volatile CatalogTable outputCatalogTable; - protected String inputTableName; - - protected String outTableName; - - @Override - public void open() {} - - public AbstractSeaTunnelTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - this( - config, - inputCatalogTable, - TransformCommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + public AbstractSeaTunnelTransform(@NonNull CatalogTable inputCatalogTable) { + this(inputCatalogTable, TransformCommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); } public AbstractSeaTunnelTransform( - @NonNull ReadonlyConfig config, - @NonNull CatalogTable catalogTable, - ErrorHandleWay rowErrorHandleWay) { - this.inputCatalogTable = catalogTable; + @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { + this.inputCatalogTable = inputCatalogTable; this.rowErrorHandleWay = rowErrorHandleWay; - List pluginInputIdentifiers = config.get(CommonOptions.PLUGIN_INPUT); - String pluginOutIdentifiers = config.get(CommonOptions.PLUGIN_OUTPUT); - if (pluginInputIdentifiers != null && !pluginInputIdentifiers.isEmpty()) { - this.inputTableName = pluginInputIdentifiers.get(0); - } else { - this.inputTableName = catalogTable.getTableId().getTableName(); - } - if (!StringUtils.isEmpty(pluginOutIdentifiers)) { - this.outTableName = pluginOutIdentifiers; - } else { - this.outTableName = catalogTable.getTableId().getTableName(); - } } public CatalogTable getProducedCatalogTable() { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java index 02f6e2c6287..855085a98b3 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/FilterRowTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; @@ -26,9 +25,8 @@ public abstract class FilterRowTransform extends AbstractCatalogSupportMapTransform { - public FilterRowTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + public FilterRowTransform(@NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java index 70b9b0bad86..84e3a9348d4 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -43,16 +42,13 @@ public abstract class MultipleFieldOutputTransform extends AbstractCatalogSuppor private int[] fieldsIndex; private SeaTunnelRowContainerGenerator rowContainerGenerator; - public MultipleFieldOutputTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + public MultipleFieldOutputTransform(@NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); } public MultipleFieldOutputTransform( - @NonNull ReadonlyConfig config, - @NonNull CatalogTable inputCatalogTable, - ErrorHandleWay errorHandleWay) { - super(config, inputCatalogTable, errorHandleWay); + @NonNull CatalogTable inputCatalogTable, ErrorHandleWay errorHandleWay) { + super(inputCatalogTable, errorHandleWay); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java index 921ab031f22..8768069ab84 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.common; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -39,9 +38,8 @@ public abstract class SingleFieldOutputTransform extends AbstractCatalogSupportM private int fieldIndex; private SeaTunnelRowContainerGenerator rowContainerGenerator; - public SingleFieldOutputTransform( - @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + public SingleFieldOutputTransform(@NonNull CatalogTable inputCatalogTable) { + super(inputCatalogTable); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java index cef7a2d74a4..6ed09e211f2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new CopyFieldTransform(config, inputCatalogTable); + return new CopyFieldTransform(CopyTransformConfig.of(config), inputCatalogTable); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java index 03517324a43..75712b5fc87 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.copy; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -47,9 +46,9 @@ public class CopyFieldTransform extends MultipleFieldOutputTransform { private List fieldOriginalIndexes; private List> fieldTypes; - public CopyFieldTransform(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { - super(readonlyConfig, catalogTable); - this.config = CopyTransformConfig.of(readonlyConfig); + public CopyFieldTransform(CopyTransformConfig copyTransformConfig, CatalogTable catalogTable) { + super(catalogTable); + this.config = copyTransformConfig; SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); initOutputFields(seaTunnelRowType, config.getFields()); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java index 8bcb196333f..7ff88d85d49 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/dynamiccompile/DynamicCompileTransform.java @@ -51,7 +51,7 @@ public class DynamicCompileTransform extends MultipleFieldOutputTransform { private AbstractParse DynamicCompileParse; public DynamicCompileTransform(ReadonlyConfig readonlyConfig, CatalogTable catalogTable) { - super(readonlyConfig, catalogTable); + super(catalogTable); CompileLanguage compileLanguage = readonlyConfig.get(DynamicCompileTransformConfig.COMPILE_LANGUAGE); // todo other compile diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java index 36f99bc6773..259f1b31634 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new FieldMapperTransform(config, inputCatalogTable); + return new FieldMapperTransform(FieldMapperTransformConfig.of(config), inputCatalogTable); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java index ea6ce0168a1..e028ead5366 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.shade.com.google.common.collect.Lists; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -49,9 +48,9 @@ public class FieldMapperTransform extends AbstractCatalogSupportMapTransform { private List needReaderColIndex; public FieldMapperTransform( - @NonNull ReadonlyConfig readonlyConfig, @NonNull CatalogTable catalogTable) { - super(readonlyConfig, catalogTable); - this.config = FieldMapperTransformConfig.of(readonlyConfig); + @NonNull FieldMapperTransformConfig config, @NonNull CatalogTable catalogTable) { + super(catalogTable); + this.config = config; Map fieldMapper = config.getFieldMapper(); SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); List notFoundField = diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java index 03d761ebc18..53b66af3d84 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java @@ -54,7 +54,7 @@ public class FilterFieldTransform extends AbstractCatalogSupportMapTransform { public FilterFieldTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(config, catalogTable); + super(catalogTable); SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); includeFields = config.get(FilterFieldTransformConfig.INCLUDE_FIELDS); excludeFields = config.get(FilterFieldTransformConfig.EXCLUDE_FIELDS); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java index bbf4955eee2..9df88d50614 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java @@ -41,7 +41,7 @@ public class FilterRowKindTransform extends FilterRowTransform { public FilterRowKindTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + super(inputCatalogTable); initConfig(config); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java index 2afd5c1d5bc..f8e096d34f8 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathMultiCatalogTransform.java @@ -34,7 +34,8 @@ public JsonPathMultiCatalogTransform( @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new JsonPathTransform(config, inputCatalogTable); + return new JsonPathTransform( + JsonPathTransformConfig.of(config, inputCatalogTable), inputCatalogTable); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java index e3b542e71b4..82c22bcb9a7 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/jsonpath/JsonPathTransform.java @@ -18,7 +18,6 @@ import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -58,12 +57,9 @@ public class JsonPathTransform extends MultipleFieldOutputTransform { private int[] srcFieldIndexArr; - public JsonPathTransform(ReadonlyConfig config, CatalogTable catalogTable) { - super( - config, - catalogTable, - JsonPathTransformConfig.of(config, catalogTable).getErrorHandleWay()); - this.config = JsonPathTransformConfig.of(config, catalogTable); + public JsonPathTransform(JsonPathTransformConfig config, CatalogTable catalogTable) { + super(catalogTable, config.getErrorHandleWay()); + this.config = config; this.seaTunnelRowType = catalogTable.getSeaTunnelRowType(); init(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java index 225c79f3d01..f37028e0705 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/metadata/MetadataTransform.java @@ -45,7 +45,7 @@ public class MetadataTransform extends MultipleFieldOutputTransform { private Map metadataFieldMapping; public MetadataTransform(ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + super(inputCatalogTable); initOutputFields(inputCatalogTable, config.get(MetadataTransformConfig.METADATA_FIELDS)); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java index 0d9ad9e2425..c699c6bfe82 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/embedding/EmbeddingTransform.java @@ -54,7 +54,7 @@ public class EmbeddingTransform extends MultipleFieldOutputTransform { public EmbeddingTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + super(inputCatalogTable); this.config = config; initOutputFields( inputCatalogTable.getTableSchema().toPhysicalRowDataType(), @@ -63,17 +63,13 @@ public EmbeddingTransform( private void tryOpen() { if (model == null) { - initModel(); + open(); } } @Override public void open() { // Initialize model - initModel(); - } - - private void initModel() { ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); try { switch (provider) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java index d4fa62f1eda..346fd688084 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/nlpmodel/llm/LLMTransform.java @@ -47,7 +47,7 @@ public class LLMTransform extends SingleFieldOutputTransform { private Model model; public LLMTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + super(inputCatalogTable); this.config = config; this.outputDataType = SeaTunnelDataTypeConvertorUtil.deserializeSeaTunnelDataType( @@ -56,7 +56,7 @@ public LLMTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable inputC private void tryOpen() { if (model == null) { - initModel(); + open(); } } @@ -67,10 +67,6 @@ public String getPluginName() { @Override public void open() { - initModel(); - } - - private void initModel() { ModelProvider provider = config.get(ModelTransformConfig.MODEL_PROVIDER); switch (provider) { case CUSTOM: diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java index c433150e99b..dd0f61e4e45 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable table, ReadonlyConfig config) { - return new FieldRenameTransform(config, table); + return new FieldRenameTransform(FieldRenameConfig.of(config), table); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java index 25726c287ec..ce52a6992e3 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/FieldRenameTransform.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; @@ -60,9 +59,9 @@ public class FieldRenameTransform extends AbstractCatalogSupportMapTransform { private final FieldRenameConfig config; private TableSchemaChangeEventHandler tableSchemaChangeEventHandler; - public FieldRenameTransform(ReadonlyConfig readonlyConfig, CatalogTable table) { - super(readonlyConfig, table); - this.config = FieldRenameConfig.of(readonlyConfig); + public FieldRenameTransform(FieldRenameConfig config, CatalogTable table) { + super(table); + this.config = config; this.inputTable = table; this.tableSchemaChangeEventHandler = new TableSchemaChangeEventDispatcher(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java index 24610e64a24..67cff881da3 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable table, ReadonlyConfig config) { - return new TableRenameTransform(config, table); + return new TableRenameTransform(TableRenameConfig.of(config), table); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java index 82a2f3f6c67..7b56a2e2ba2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rename/TableRenameTransform.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -57,10 +56,10 @@ public class TableRenameTransform extends AbstractCatalogSupportMapTransform { private TablePath outputTablePath; private String outputTableId; - public TableRenameTransform(ReadonlyConfig readonlyConfig, CatalogTable table) { - super(readonlyConfig, table); - config = TableRenameConfig.of(readonlyConfig); + public TableRenameTransform(TableRenameConfig config, CatalogTable table) { + super(table); this.inputTable = table; + this.config = config; } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java index 4491ef253a1..b2c9fa44ce5 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java @@ -38,7 +38,7 @@ public class ReplaceTransform extends SingleFieldOutputTransform { public ReplaceTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + super(inputCatalogTable); this.config = config; initOutputFields( inputCatalogTable.getTableSchema().toPhysicalRowDataType(), diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java index 8b8062401cf..58b80c50c1b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/rowkind/RowKindExtractorTransform.java @@ -41,7 +41,7 @@ public class RowKindExtractorTransform extends SingleFieldOutputTransform { public RowKindExtractorTransform( @NonNull ReadonlyConfig config, @NonNull CatalogTable inputCatalogTable) { - super(config, inputCatalogTable); + super(inputCatalogTable); this.config = config; this.transformType = config.get(RowKindExtractorTransformConfig.TRANSFORM_TYPE); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java index 06d247787e9..52f033b10a0 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitMultiCatalogTransform.java @@ -40,6 +40,6 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable inputCatalogTable, ReadonlyConfig config) { - return new SplitTransform(config, inputCatalogTable); + return new SplitTransform(SplitTransformConfig.of(config), inputCatalogTable); } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java index 1f4f11f01b3..eac04aa1c9c 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.split; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -36,9 +35,11 @@ public class SplitTransform extends MultipleFieldOutputTransform { private final SplitTransformConfig splitTransformConfig; private final int splitFieldIndex; - public SplitTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(config, catalogTable); - this.splitTransformConfig = SplitTransformConfig.of(config); + public SplitTransform( + @NonNull SplitTransformConfig splitTransformConfig, + @NonNull CatalogTable catalogTable) { + super(catalogTable); + this.splitTransformConfig = splitTransformConfig; SeaTunnelRowType seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); try { splitFieldIndex = seaTunnelRowType.indexOf(splitTransformConfig.getSplitField()); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java index 246b6febf03..230541a7bf2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLTransform.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.transform.sql; +import org.apache.seatunnel.api.common.CommonOptions; import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.ReadonlyConfig; @@ -63,14 +64,23 @@ public class SQLTransform extends AbstractCatalogSupportFlatMapTransform { private transient SQLEngine sqlEngine; + private final String inputTableName; + public SQLTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(config, catalogTable); + super(catalogTable); this.query = config.get(KEY_QUERY); if (config.getOptional(KEY_ENGINE).isPresent()) { this.engineType = EngineType.valueOf(config.get(KEY_ENGINE).toUpperCase()); } else { this.engineType = ZETA; } + + List pluginInputIdentifiers = config.get(CommonOptions.PLUGIN_INPUT); + if (pluginInputIdentifiers != null && !pluginInputIdentifiers.isEmpty()) { + this.inputTableName = pluginInputIdentifiers.get(0); + } else { + this.inputTableName = catalogTable.getTableId().getTableName(); + } } @Override @@ -80,16 +90,6 @@ public String getPluginName() { @Override public void open() { - initSqlEngine(); - } - - private void tryOpen() { - if (sqlEngine == null) { - initSqlEngine(); - } - } - - private void initSqlEngine() { sqlEngine = SQLEngineFactory.getSQLEngine(engineType); sqlEngine.init( inputTableName, @@ -98,6 +98,12 @@ private void initSqlEngine() { query); } + private void tryOpen() { + if (sqlEngine == null) { + open(); + } + } + @Override protected List transformRow(SeaTunnelRow inputRow) { tryOpen(); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java index 6cd3b5d8bb3..c63219b03d1 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeMultiCatalogTransform.java @@ -48,7 +48,7 @@ public String getPluginName() { @Override protected SeaTunnelTransform buildTransform( CatalogTable table, ReadonlyConfig config) { - return new TableMergeTransform(config, table); + return new TableMergeTransform(TableMergeConfig.of(config), table); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java index d1d1874172d..b51376a9ead 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/table/TableMergeTransform.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.table; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; @@ -32,11 +31,11 @@ public class TableMergeTransform extends AbstractCatalogSupportMapTransform { private final TablePath outputTablePath; private final String outputTableId; - public TableMergeTransform(ReadonlyConfig config, CatalogTable table) { - super(config, table); + public TableMergeTransform(TableMergeConfig config, CatalogTable table) { + super(table); this.inputTable = table; - this.outputTablePath = TableMergeConfig.of(config).getTablePath(); - this.outputTableId = TableMergeConfig.of(config).getTablePath().getFullName(); + this.outputTablePath = config.getTablePath(); + this.outputTableId = config.getTablePath().getFullName(); } @Override diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java index 3b557fbc358..c34af87e359 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/JsonPathTransformTest.java @@ -62,7 +62,8 @@ public void testJsonPath() { new SeaTunnelRowType( new String[] {"data"}, new SeaTunnelDataType[] {BasicType.STRING_TYPE})); - JsonPathTransform transform = new JsonPathTransform(config, table); + JsonPathTransform transform = + new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); CatalogTable outputTable = transform.getProducedCatalogTable(); SeaTunnelRow outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f1\": 1}"})); @@ -87,7 +88,8 @@ public void testErrorHandleWay() { new SeaTunnelRowType( new String[] {"data"}, new SeaTunnelDataType[] {BasicType.STRING_TYPE})); - JsonPathTransform transform = new JsonPathTransform(config, table); + JsonPathTransform transform = + new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); CatalogTable outputTable = transform.getProducedCatalogTable(); final JsonPathTransform finalTransform = transform; Assertions.assertThrows( @@ -107,7 +109,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.FAIL.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); JsonPathTransform finalTransform1 = transform; Assertions.assertThrows( @@ -127,7 +129,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); SeaTunnelRow outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNotNull(outputRow); @@ -146,7 +148,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP_ROW.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNull(outputRow); @@ -162,7 +164,7 @@ public void testErrorHandleWay() { JsonPathTransformConfig.PATH.key(), "$.f1", JsonPathTransformConfig.DEST_FIELD.key(), "f1"))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNull(outputRow); @@ -183,7 +185,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.FAIL.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); try { outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); @@ -208,7 +210,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNotNull(outputRow); @@ -230,7 +232,7 @@ public void testErrorHandleWay() { TransformCommonOptions.COLUMN_ERROR_HANDLE_WAY_OPTION.key(), ErrorHandleWay.SKIP_ROW.name()))); config = ReadonlyConfig.fromMap(configMap); - transform = new JsonPathTransform(config, table); + transform = new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); outputTable = transform.getProducedCatalogTable(); outputRow = transform.map(new SeaTunnelRow(new Object[] {"{\"f2\": 1}"})); Assertions.assertNull(outputRow); @@ -264,7 +266,8 @@ public void testOutputColumn() { new HashMap<>(), new ArrayList<>(), null); - JsonPathTransform transform = new JsonPathTransform(config, table); + JsonPathTransform transform = + new JsonPathTransform(JsonPathTransformConfig.of(config, table), table); CatalogTable outputCatalogTable = transform.getProducedCatalogTable(); Column f1 = outputCatalogTable.getTableSchema().getColumn("f1"); Assertions.assertEquals(BasicType.STRING_TYPE, f1.getDataType()); diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java index 8d70761a128..02a8b01bcf8 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/FieldRenameTransformTest.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.rename; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -35,7 +34,6 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.stream.Collectors; public class FieldRenameTransformTest { @@ -105,14 +103,7 @@ public void testRename() { AlterTableDropColumnEvent dropColumnEvent = new AlterTableDropColumnEvent(DEFAULT_TABLE.getTableId(), "f5"); - ReadonlyConfig config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put("convert_case", "LOWER"); - } - }); - + FieldRenameConfig config = new FieldRenameConfig().setConvertCase(ConvertCase.LOWER); FieldRenameTransform transform = new FieldRenameTransform(config, DEFAULT_TABLE); CatalogTable outputCatalogTable = transform.getProducedCatalogTable(); AlterTableAddColumnEvent outputAddEvent = @@ -146,13 +137,7 @@ public void testRename() { Assertions.assertEquals("f5", outputChangeEvent.getColumn().getName()); Assertions.assertEquals("f5", outputDropEvent.getColumn()); - config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put("convert_case", "UPPER"); - } - }); + config = new FieldRenameConfig().setConvertCase(ConvertCase.UPPER); transform = new FieldRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTable(); outputAddEvent = (AlterTableAddColumnEvent) transform.mapSchemaChangeEvent(addColumnEvent); @@ -184,15 +169,7 @@ public void testRename() { Assertions.assertEquals("f5", outputChangeEvent.getColumn().getName()); Assertions.assertEquals("F5", outputDropEvent.getColumn()); - config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put("prefix", "p-"); - put("suffix", "-s"); - } - }); - + config = new FieldRenameConfig().setPrefix("p-").setSuffix("-s"); transform = new FieldRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTable(); outputAddEvent = (AlterTableAddColumnEvent) transform.mapSchemaChangeEvent(addColumnEvent); @@ -225,15 +202,13 @@ public void testRename() { Assertions.assertEquals("p-f5-s", outputDropEvent.getColumn()); config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put( - "replacements_with_regex", - "[{\"replace_from\" : \"f1\", \"replace_to\" : \"t1\"},{\"replace_from\" : " - + "\"f1\", \"replace_to\" : \"t2\"}]"); - } - }); + new FieldRenameConfig() + .setReplacementsWithRegex( + Arrays.asList( + new FieldRenameConfig.ReplacementsWithRegex( + "f1", "t1", true), + new FieldRenameConfig.ReplacementsWithRegex( + "f1", "t2", true))); transform = new FieldRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTable(); outputAddEvent = (AlterTableAddColumnEvent) transform.mapSchemaChangeEvent(addColumnEvent); diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java index da6d8a3cdb4..d1d756922e0 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/rename/TableRenameTransformTest.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.transform.rename; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PhysicalColumn; @@ -34,7 +33,6 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; public class TableRenameTransformTest { @@ -95,13 +93,7 @@ public void testRename() { DEFAULT_TABLE.getTableId(), PhysicalColumn.of("f4", BasicType.LONG_TYPE, null, null, true, null, null)); - ReadonlyConfig config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put("convert_case", "LOWER"); - } - }); + TableRenameConfig config = new TableRenameConfig().setConvertCase(ConvertCase.LOWER); TableRenameTransform transform = new TableRenameTransform(config, DEFAULT_TABLE); List outputCatalogTable = transform.getProducedCatalogTables(); @@ -114,14 +106,7 @@ public void testRename() { Assertions.assertEquals( "database-x.schema-x.table-x", outputEvent.tablePath().getFullName()); - config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put("convert_case", "UPPER"); - } - }); - + config = new TableRenameConfig().setConvertCase(ConvertCase.UPPER); transform = new TableRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTables(); outputRow = transform.map(inputRow); @@ -132,15 +117,8 @@ public void testRename() { Assertions.assertEquals("DATABASE-X.SCHEMA-X.TABLE-X", outputRow.getTableId()); Assertions.assertEquals( "DATABASE-X.SCHEMA-X.TABLE-X", outputEvent.tablePath().getFullName()); - config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put("prefix", "user-"); - put("suffix", "-table"); - } - }); + config = new TableRenameConfig().setPrefix("user-").setSuffix("-table"); transform = new TableRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTables(); outputRow = transform.map(inputRow); @@ -153,14 +131,12 @@ public void testRename() { "Database-x.Schema-x.user-Table-x-table", outputEvent.tablePath().getFullName()); config = - ReadonlyConfig.fromMap( - new HashMap() { - { - put( - "replacements_with_regex", - "[{\"replace_from\" : \"Table\", \"replace_to\" : \"t1\"},{\"replace_from\" : \"Table\", \"replace_to\" : \"t2\"}]"); - } - }); + new TableRenameConfig() + .setReplacementsWithRegex( + Arrays.asList( + new TableRenameConfig.ReplacementsWithRegex("Table", "t1"), + new TableRenameConfig.ReplacementsWithRegex( + "Table", "t2"))); transform = new TableRenameTransform(config, DEFAULT_TABLE); outputCatalogTable = transform.getProducedCatalogTables(); outputRow = transform.map(inputRow); From 3a37d6e319df729a4af23cdccf0baefd26d296fc Mon Sep 17 00:00:00 2001 From: nijiahui Date: Thu, 23 Jan 2025 12:53:05 +0800 Subject: [PATCH 51/56] [Feature][transform-v2] Support transform metrics --- .../src/test/resources/kafka_sink_fake_to_kafka.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf index 00dc8fecfe5..90825bbf340 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka_sink_fake_to_kafka.conf @@ -63,4 +63,4 @@ sink { format = json partition_key_fields = ["c_map", "c_string"] } -} +} \ No newline at end of file From af224b1d0eb3f994b8dd696415d3de7a5a112b13 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Thu, 23 Jan 2025 12:58:22 +0800 Subject: [PATCH 52/56] [Feature][transform-v2] add metric doc --- docs/en/seatunnel-engine/rest-api-v1.md | 1 + docs/en/seatunnel-engine/rest-api-v2.md | 2 ++ 2 files changed, 3 insertions(+) diff --git a/docs/en/seatunnel-engine/rest-api-v1.md b/docs/en/seatunnel-engine/rest-api-v1.md index 8859faa32f1..897be73639f 100644 --- a/docs/en/seatunnel-engine/rest-api-v1.md +++ b/docs/en/seatunnel-engine/rest-api-v1.md @@ -259,6 +259,7 @@ This API has been deprecated, please use /hazelcast/rest/maps/job-info/:jobId in "SinkWriteQPS": "", "SinkWriteBytes": "", "SinkWriteBytesPerSeconds": "", + "TransformOutputCount": {}, "TableSourceReceivedCount": {}, "TableSourceReceivedBytes": {}, "TableSourceReceivedBytesPerSeconds": {}, diff --git a/docs/en/seatunnel-engine/rest-api-v2.md b/docs/en/seatunnel-engine/rest-api-v2.md index 01dc9479117..d21cdb9e25e 100644 --- a/docs/en/seatunnel-engine/rest-api-v2.md +++ b/docs/en/seatunnel-engine/rest-api-v2.md @@ -165,6 +165,7 @@ seatunnel: "SinkWriteQPS": "", "SinkWriteBytes": "", "SinkWriteBytesPerSeconds": "", + "TransformOutputCount": {}, "TableSourceReceivedCount": {}, "TableSourceReceivedBytes": {}, "TableSourceReceivedBytesPerSeconds": {}, @@ -245,6 +246,7 @@ This API has been deprecated, please use /job-info/:jobId instead "SinkWriteQPS": "", "SinkWriteBytes": "", "SinkWriteBytesPerSeconds": "", + "TransformOutputCount": {}, "TableSourceReceivedCount": {}, "TableSourceReceivedBytes": {}, "TableSourceReceivedBytesPerSeconds": {}, From b23bc44ce3f65840bfdbc0d740866c35e0b57f3f Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Sun, 26 Jan 2025 15:24:27 +0800 Subject: [PATCH 53/56] [Fix][doris-e2e] fix init be 127.0.0.1 connect refuse --- .../seatunnel/e2e/connector/doris/AbstractDorisIT.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java index d8647f0f072..7777d9cd24c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java @@ -71,13 +71,10 @@ public abstract class AbstractDorisIT extends TestSuiteBase implements TestResou protected static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; protected static final String DRIVER_JAR = "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; - private static final boolean isGithubActionsEnv = - "true".equalsIgnoreCase(System.getenv("GITHUB_ACTIONS")); @BeforeAll @Override public void startUp() { - log.info("isGithubActionsEnv: {}", isGithubActionsEnv); container = new GenericContainer<>(DOCKER_IMAGE) .withNetwork(NETWORK) @@ -109,9 +106,7 @@ protected void initializeJdbcConnection() props.put("user", USERNAME); props.put("password", PASSWORD); jdbcConnection = driver.connect(String.format(URL, container.getHost()), props); - if (isGithubActionsEnv) { - initializeBE(); - } + initializeBE(); try (Statement statement = jdbcConnection.createStatement()) { statement.execute(SET_SQL); statement.execute(SET_CONNECTIONS); From 094bba6f92f7a1c6d5dee268267003534b852882 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Sun, 26 Jan 2025 16:17:02 +0800 Subject: [PATCH 54/56] [Fix][doris-e2e] fix init be 127.0.0.1 connect refuse --- .../e2e/connector/doris/AbstractDorisIT.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java index 7777d9cd24c..e6ddb111717 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java @@ -71,10 +71,13 @@ public abstract class AbstractDorisIT extends TestSuiteBase implements TestResou protected static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; protected static final String DRIVER_JAR = "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + private static final boolean isGithubActionsEnv = + "true".equalsIgnoreCase(System.getenv("GITHUB_ACTIONS")); @BeforeAll @Override public void startUp() { + log.info("isGithubActionsEnv: {}", isGithubActionsEnv); container = new GenericContainer<>(DOCKER_IMAGE) .withNetwork(NETWORK) @@ -96,7 +99,7 @@ public void startUp() { protected void initializeJdbcConnection() throws SQLException, ClassNotFoundException, MalformedURLException, - InstantiationException, IllegalAccessException { + InstantiationException, IllegalAccessException { log.info("doris initializing ..."); URLClassLoader urlClassLoader = new URLClassLoader(new URL[] {new URL(DRIVER_JAR)}, DorisIT.class.getClassLoader()); @@ -106,7 +109,9 @@ protected void initializeJdbcConnection() props.put("user", USERNAME); props.put("password", PASSWORD); jdbcConnection = driver.connect(String.format(URL, container.getHost()), props); - initializeBE(); + if (isGithubActionsEnv) { + initializeBE(); + } try (Statement statement = jdbcConnection.createStatement()) { statement.execute(SET_SQL); statement.execute(SET_CONNECTIONS); @@ -164,4 +169,4 @@ public void tearDown() throws Exception { jdbcConnection.close(); } } -} +} \ No newline at end of file From c1c0feb0f49a85fafa42b464626d845cd032760e Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Sun, 26 Jan 2025 16:17:49 +0800 Subject: [PATCH 55/56] [Fix][doris-e2e] fix init be 127.0.0.1 connect refuse --- .../apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java index e6ddb111717..d8647f0f072 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/AbstractDorisIT.java @@ -99,7 +99,7 @@ public void startUp() { protected void initializeJdbcConnection() throws SQLException, ClassNotFoundException, MalformedURLException, - InstantiationException, IllegalAccessException { + InstantiationException, IllegalAccessException { log.info("doris initializing ..."); URLClassLoader urlClassLoader = new URLClassLoader(new URL[] {new URL(DRIVER_JAR)}, DorisIT.class.getClassLoader()); @@ -169,4 +169,4 @@ public void tearDown() throws Exception { jdbcConnection.close(); } } -} \ No newline at end of file +} From e703f289b4a670c5607292214a033ca867e86f39 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Sun, 26 Jan 2025 17:01:05 +0800 Subject: [PATCH 56/56] [Feature][transform-v2] Support transform metrics --- docs/zh/seatunnel-engine/rest-api-v1.md | 1 + docs/zh/seatunnel-engine/rest-api-v2.md | 1 + .../execution/TransformExecuteProcessor.java | 1 - .../execution/TransformExecuteProcessor.java | 15 ++------------- .../batch_fake_multi_table_to_console.conf | 2 +- .../engine/client/job/JobMetricsRunner.java | 1 - .../server/task/flow/TransformFlowLifeCycle.java | 1 + 7 files changed, 6 insertions(+), 16 deletions(-) diff --git a/docs/zh/seatunnel-engine/rest-api-v1.md b/docs/zh/seatunnel-engine/rest-api-v1.md index f23021cdb94..919e9ad0848 100644 --- a/docs/zh/seatunnel-engine/rest-api-v1.md +++ b/docs/zh/seatunnel-engine/rest-api-v1.md @@ -191,6 +191,7 @@ network: "SinkWriteQPS": "", "SinkWriteBytes": "", "SinkWriteBytesPerSeconds": "", + "TransformOutputCount": {}, "TableSourceReceivedCount": {}, "TableSourceReceivedBytes": {}, "TableSourceReceivedBytesPerSeconds": {}, diff --git a/docs/zh/seatunnel-engine/rest-api-v2.md b/docs/zh/seatunnel-engine/rest-api-v2.md index ed3c6803c7a..59f7f31618b 100644 --- a/docs/zh/seatunnel-engine/rest-api-v2.md +++ b/docs/zh/seatunnel-engine/rest-api-v2.md @@ -161,6 +161,7 @@ seatunnel: "SinkWriteQPS": "", "SinkWriteBytes": "", "SinkWriteBytesPerSeconds": "", + "TransformOutputCount": {}, "TableSourceReceivedCount": {}, "TableSourceReceivedBytes": {}, "TableSourceReceivedBytesPerSeconds": {}, diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index ea1c30c49f8..1808ffeff84 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -122,7 +122,6 @@ public List execute(List upstreamDataS new TableTransformFactoryContext( stream.getCatalogTables(), options, classLoader); ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); - SeaTunnelTransform transform = factory.createTransform(context).createTransform(); transform.setJobContext(jobContext); diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index 03214b5e83e..492af1ad73d 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -112,11 +112,9 @@ public List execute(List upstreamDataStreams e -> e, (a, b) -> b, LinkedHashMap::new)); - int index = 0; for (int i = 0; i < plugins.size(); i++) { try { Config pluginConfig = pluginConfigs.get(i); - ReadonlyConfig options = ReadonlyConfig.fromConfig(pluginConfig); DatasetTableInfo dataset = fromSourceTable( pluginConfig, @@ -132,12 +130,7 @@ public List execute(List upstreamDataStreams ConfigValidator.of(context.getOptions()).validate(factory.optionRule()); SeaTunnelTransform transform = factory.createTransform(context).createTransform(); - String metricName = - String.format("Transform[%s]-%s", transform.getPluginName(), index++); - String pluginOutput = options.get(PLUGIN_OUTPUT); - - Dataset inputDataset = - sparkTransform(transform, dataset, metricName, pluginOutput); + Dataset inputDataset = sparkTransform(transform, dataset); registerInputTempView(pluginConfig, inputDataset); String pluginOutputIdentifier = ReadonlyConfig.fromConfig(pluginConfig).get(PLUGIN_OUTPUT); @@ -158,11 +151,7 @@ public List execute(List upstreamDataStreams return new ArrayList<>(outputTables.values()); } - private Dataset sparkTransform( - SeaTunnelTransform transform, - DatasetTableInfo tableInfo, - String metricName, - String pluginOutput) { + private Dataset sparkTransform(SeaTunnelTransform transform, DatasetTableInfo tableInfo) { MultiTableManager inputManager = new MultiTableManager(tableInfo.getCatalogTables().toArray(new CatalogTable[0])); MultiTableManager outputManager = diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf index b8230e4c9bd..dc3bd95fd3f 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/batch_fake_multi_table_to_console.conf @@ -156,4 +156,4 @@ sink { console { plugin_input = "fake8" } -} +} \ No newline at end of file diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java index 734e40c3f13..829f4c1160c 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobMetricsRunner.java @@ -101,7 +101,6 @@ public void run() { lastRunTime = now; lastReadCount = jobMetricsSummary.getSourceReadCount(); lastWriteCount = jobMetricsSummary.getSinkWriteCount(); - // transformCountMap = jobMetricsSummary.getTransformCountMap(); } catch (Exception e) { log.warn("Failed to get job metrics summary, it maybe first-run"); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java index da21d63dbaa..afa43ece7f2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/TransformFlowLifeCycle.java @@ -76,6 +76,7 @@ public TransformFlowLifeCycle( @Override public void open() throws Exception { + super.open(); for (SeaTunnelTransform t : transform) { try { t.open();