From 9b09ad8c4960291d7b776119e97394cf52631f87 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Mon, 28 Oct 2024 12:50:46 +0800 Subject: [PATCH 01/29] [Feature][transform] transform support explode --- plugin-mapping.properties | 1 + .../transform/SeaTunnelMultiRowTransform.java | 35 ++++ .../execution/TransformExecuteProcessor.java | 26 +++ .../execution/TransformExecuteProcessor.java | 51 +++-- .../e2e/transform/TestExplodeIT.java | 34 ++++ .../src/test/resources/explode_transform.conf | 92 +++++++++ .../task/flow/TransformFlowLifeCycle.java | 63 ++++-- .../AbstractCatalogMultiRowTransform.java | 104 ++++++++++ .../transform/explode/ExplodeTransform.java | 165 +++++++++++++++ .../explode/ExplodeTransformConfig.java | 31 +++ .../explode/ExplodeTransformFactory.java | 49 +++++ .../ExplodeTransformFactoryTest.java | 30 +++ .../explode/ExplodeTransformTest.java | 188 ++++++++++++++++++ 13 files changed, 830 insertions(+), 39 deletions(-) create mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java create mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java diff --git a/plugin-mapping.properties b/plugin-mapping.properties index e314ef86613..0a5809c1752 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -152,3 +152,4 @@ seatunnel.transform.DynamicCompile = seatunnel-transforms-v2 seatunnel.transform.LLM = seatunnel-transforms-v2 seatunnel.transform.Embedding = seatunnel-transforms-v2 seatunnel.transform.RowKindExtractor = seatunnel-transforms-v2 +seatunnel.transform.Explode = seatunnel-transforms-v2 diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java new file mode 100644 index 00000000000..2f4e1026ad1 --- /dev/null +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.api.transform; + +import java.util.List; + +public interface SeaTunnelMultiRowTransform extends SeaTunnelTransform { + + /** + * Transform input data to {@link this#getProducedCatalogTable().getSeaTunnelRowType()} types + * data. + * + * @param row the data need be transformed. + * @return transformed data. + */ + List MultiRowMap(T row); + + default T map(T row) { + throw new UnsupportedOperationException("Heads-up conversion is not supported"); + } +} 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 00a5046e431..2e5cdfb6039 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 @@ -25,15 +25,18 @@ import org.apache.seatunnel.api.table.factory.TableTransformFactory; import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform; import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; 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.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.util.Collector; import java.net.URL; import java.util.ArrayList; @@ -140,6 +143,11 @@ public List execute(List upstreamDataS protected DataStream flinkTransform( SeaTunnelTransform transform, DataStream stream) { + if (transform instanceof SeaTunnelMultiRowTransform) { + return stream.flatMap( + new ArrayFlatMap(transform), TypeInformation.of(SeaTunnelRow.class)); + } + return stream.transform( String.format("%s-Transform", transform.getPluginName()), TypeInformation.of(SeaTunnelRow.class), @@ -151,4 +159,22 @@ protected DataStream flinkTransform( ((SeaTunnelTransform) transform) .map(row)))); } + + public static class ArrayFlatMap implements FlatMapFunction { + + private SeaTunnelTransform transform; + + public ArrayFlatMap(SeaTunnelTransform transform) { + this.transform = transform; + } + + @Override + public void flatMap(SeaTunnelRow row, Collector collector) { + List rows = + ((SeaTunnelMultiRowTransform) transform).MultiRowMap(row); + for (SeaTunnelRow rowResult : rows) { + collector.collect(rowResult); + } + } + } } 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 3736576817d..d4505df6300 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 @@ -26,6 +26,7 @@ import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform; import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.core.starter.execution.PluginUtil; @@ -45,7 +46,6 @@ import lombok.extern.slf4j.Slf4j; -import java.io.Serializable; import java.net.URL; import java.util.ArrayList; import java.util.Collections; @@ -164,33 +164,25 @@ private Dataset sparkTransform(SeaTunnelTransform transform, DatasetTableIn SeaTunnelRowConverter inputRowConverter = new SeaTunnelRowConverter(inputDataType); SeaTunnelRowConverter outputRowConverter = new SeaTunnelRowConverter(outputDataTYpe); ExpressionEncoder encoder = RowEncoder.apply(outputSchema); + return stream.mapPartitions( - (MapPartitionsFunction) - (Iterator rowIterator) -> - new TransformIterator( - rowIterator, - transform, - outputSchema, - inputRowConverter, - outputRowConverter), + new TransformMapPartitionsFunction( + transform, outputSchema, inputRowConverter, outputRowConverter), encoder) .filter(Objects::nonNull); } - private static class TransformIterator implements Iterator, Serializable { - private Iterator sourceIterator; + private static class TransformMapPartitionsFunction implements MapPartitionsFunction { private SeaTunnelTransform transform; private StructType structType; private SeaTunnelRowConverter inputRowConverter; private SeaTunnelRowConverter outputRowConverter; - public TransformIterator( - Iterator sourceIterator, + public TransformMapPartitionsFunction( SeaTunnelTransform transform, StructType structType, SeaTunnelRowConverter inputRowConverter, SeaTunnelRowConverter outputRowConverter) { - this.sourceIterator = sourceIterator; this.transform = transform; this.structType = structType; this.inputRowConverter = inputRowConverter; @@ -198,23 +190,28 @@ public TransformIterator( } @Override - public boolean hasNext() { - return sourceIterator.hasNext(); - } - - @Override - public Row next() { - try { + public Iterator call(Iterator sourceIterator) throws Exception { + List rows = new ArrayList<>(); + while (sourceIterator.hasNext()) { Row row = sourceIterator.next(); SeaTunnelRow seaTunnelRow = inputRowConverter.unpack((GenericRowWithSchema) row); - seaTunnelRow = (SeaTunnelRow) transform.map(seaTunnelRow); - if (seaTunnelRow == null) { - return null; + if (transform instanceof SeaTunnelMultiRowTransform) { + List seaTunnelRows = + ((SeaTunnelMultiRowTransform) transform) + .MultiRowMap(seaTunnelRow); + if (!seaTunnelRows.isEmpty()) { + for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) { + rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); + } + } + } else { + SeaTunnelRow seaTunnelRowTransform = transform.map(seaTunnelRow); + if (seaTunnelRow != null) { + rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); + } } - return outputRowConverter.parcel(seaTunnelRow); - } catch (Exception e) { - throw new TaskExecuteException("Row convert failed, caused: " + e.getMessage(), e); } + return rows.iterator(); } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java new file mode 100644 index 00000000000..34ba67a5b70 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java @@ -0,0 +1,34 @@ +/* + * 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.e2e.transform; + +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import java.io.IOException; + +public class TestExplodeIT extends TestSuiteBase { + + @TestTemplate + public void testExplode(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/explode_transform.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } +} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf new file mode 100644 index 00000000000..59d3c9b9bd3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf @@ -0,0 +1,92 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" + parallelism = 1 +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + pk_id = string + name = string + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = ["id001;id001", "zhangsan,zhangsan"] + }, + { + kind = INSERT + fields = ["id001", "zhangsan,zhangsan"] + }, + { + kind = INSERT + fields = ["id001;id001", "zhangsan"] + } + ] + } +} + +transform { + Explode { + source_table_name = "fake" + result_table_name = "fake1" + explode_fields = {"pk_id":";","name":","} + } +} + +sink{ + assert { + rules = + { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 8 + }, + { + rule_type = MIN_ROW + rule_value = 8 + } + ], + field_rules = [ + { + field_name = pk_id + field_type = string + field_value = [{equals_to = id001}] + }, + { + field_name = name + field_type = string + field_value = [{equals_to = zhangsan}] + } + ] + } + } +} 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 0447513b5ff..53f08eae0a8 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 @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.table.type.Record; import org.apache.seatunnel.api.transform.Collector; +import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform; import org.apache.seatunnel.api.transform.SeaTunnelTransform; import org.apache.seatunnel.engine.core.dag.actions.TransformChainAction; import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey; @@ -30,6 +31,7 @@ import lombok.extern.slf4j.Slf4j; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -89,22 +91,59 @@ public void received(Record record) { return; } T inputData = (T) record.getData(); - T outputData = inputData; - for (SeaTunnelTransform t : transform) { - outputData = t.map(inputData); - log.debug("Transform[{}] input row {} and output row {}", t, inputData, outputData); - if (outputData == null) { - log.trace("Transform[{}] filtered data row {}", t, inputData); - break; + List outputDataList = transform(inputData); + if (!outputDataList.isEmpty()) { + // todo log metrics + for (T outputData : outputDataList) { + collector.collect(new Record<>(outputData)); } - - inputData = outputData; } - if (outputData != null) { - // todo log metrics - collector.collect(new Record<>(outputData)); + } + } + + public List transform(T inputData) { + if (transform.isEmpty()) { + return Collections.singletonList(inputData); + } + + List dataList = new ArrayList<>(); + dataList.add(inputData); + + for (SeaTunnelTransform transformer : transform) { + log.info("transform test: {}", transformer.getPluginName()); + List nextInputDataList = new ArrayList<>(); + if (transformer instanceof SeaTunnelMultiRowTransform) { + SeaTunnelMultiRowTransform transformDecorator = + (SeaTunnelMultiRowTransform) transformer; + for (T data : dataList) { + List outputDataArray = transformDecorator.MultiRowMap(data); + log.debug( + "Transform[{}] input row {} and output row {}", + transformer, + data, + outputDataArray); + nextInputDataList.addAll(outputDataArray); + } + } else { + for (T data : dataList) { + T outputData = transformer.map(data); + log.debug( + "Transform[{}] input row {} and output row {}", + transformer, + data, + outputData); + if (outputData == null) { + log.trace("Transform[{}] filtered data row {}", transformer, data); + continue; + } + nextInputDataList.add(outputData); + } } + + dataList = nextInputDataList; } + + return dataList; } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java new file mode 100644 index 00000000000..109944eb3a2 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java @@ -0,0 +1,104 @@ +/* + * 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.transform.common; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform; +import org.apache.seatunnel.transform.exception.ErrorDataTransformException; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.util.List; + +@Slf4j +public abstract class AbstractCatalogMultiRowTransform + implements SeaTunnelMultiRowTransform { + protected final ErrorHandleWay rowErrorHandleWay; + protected CatalogTable inputCatalogTable; + + protected volatile CatalogTable outputCatalogTable; + + public AbstractCatalogMultiRowTransform(@NonNull CatalogTable inputCatalogTable) { + this(inputCatalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + } + + public AbstractCatalogMultiRowTransform( + @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { + this.inputCatalogTable = inputCatalogTable; + this.rowErrorHandleWay = rowErrorHandleWay; + } + + @Override + public List MultiRowMap(SeaTunnelRow row) { + try { + return transformRow(row); + } catch (ErrorDataTransformException e) { + if (e.getErrorHandleWay() != null) { + ErrorHandleWay errorHandleWay = e.getErrorHandleWay(); + if (errorHandleWay.allowSkipThisRow()) { + log.debug("Skip row due to error", e); + return null; + } + throw e; + } + if (rowErrorHandleWay.allowSkip()) { + log.debug("Skip row due to error", e); + return null; + } + throw e; + } + } + + /** + * Outputs transformed row data. + * + * @param inputRow upstream input row data + */ + protected abstract List transformRow(SeaTunnelRow inputRow); + + @Override + public CatalogTable getProducedCatalogTable() { + if (outputCatalogTable == null) { + synchronized (this) { + if (outputCatalogTable == null) { + outputCatalogTable = transformCatalogTable(); + } + } + } + + return outputCatalogTable; + } + + private CatalogTable transformCatalogTable() { + TableIdentifier tableIdentifier = transformTableIdentifier(); + TableSchema tableSchema = transformTableSchema(); + return CatalogTable.of( + tableIdentifier, + tableSchema, + inputCatalogTable.getOptions(), + inputCatalogTable.getPartitionKeys(), + inputCatalogTable.getComment()); + } + + protected abstract TableSchema transformTableSchema(); + + protected abstract TableIdentifier transformTableIdentifier(); +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java new file mode 100644 index 00000000000..4e613a4244d --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.transform.explode; + +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; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.transform.common.AbstractCatalogMultiRowTransform; +import org.apache.seatunnel.transform.common.CommonOptions; +import org.apache.seatunnel.transform.exception.TransformCommonError; +import org.apache.seatunnel.transform.replace.ReplaceTransformConfig; + +import org.apache.commons.collections4.CollectionUtils; + +import com.google.common.collect.Lists; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +@Slf4j +public class ExplodeTransform extends AbstractCatalogMultiRowTransform { + protected CatalogTable inputCatalogTable; + public static final String PLUGIN_NAME = "Explode"; + protected Map explodeFields; + protected SeaTunnelRowType seaTunnelRowType; + private int[] fieldsIndex; + private ReadonlyConfig config; + + public ExplodeTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { + super(catalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + this.config = config; + this.explodeFields = config.get(ExplodeTransformConfig.EXPLODE_FIELDS); + this.seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); + this.inputCatalogTable = catalogTable; + } + + /** + * Outputs transformed row data. + * + * @param inputRow upstream input row data + */ + protected List transformRow(SeaTunnelRow inputRow) { + List rows = Lists.newArrayList(inputRow); + for (Map.Entry entry : explodeFields.entrySet()) { + List next = new ArrayList<>(); + for (SeaTunnelRow row : rows) { + + String field = entry.getKey(); + int fieldIndex = seaTunnelRowType.indexOf(field); + Object splitFieldValue = inputRow.getField(fieldIndex); + if (splitFieldValue == null) { + continue; + } + String separator = entry.getValue(); + String[] splitFieldValues = splitFieldValue.toString().split(separator); + for (String fieldValue : splitFieldValues) { + SeaTunnelRow outputRow = row.copy(); + outputRow.setField(fieldIndex, fieldValue); + next.add(outputRow); + } + } + rows = next; + } + return rows; + } + + @Override + public String getPluginName() { + return PLUGIN_NAME; + } + + @Override + protected TableSchema transformTableSchema() { + Column[] outputColumns = getOutputColumns(); + + List copiedConstraintKeys = + inputCatalogTable.getTableSchema().getConstraintKeys().stream() + .map(ConstraintKey::copy) + .collect(Collectors.toList()); + + TableSchema.Builder builder = TableSchema.builder(); + if (inputCatalogTable.getTableSchema().getPrimaryKey() != null) { + builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy()); + } + builder.constraintKey(copiedConstraintKeys); + List columns = + inputCatalogTable.getTableSchema().getColumns().stream() + .map(Column::copy) + .collect(Collectors.toList()); + + int addFieldCount = 0; + this.fieldsIndex = new int[outputColumns.length]; + for (int i = 0; i < outputColumns.length; i++) { + Column outputColumn = outputColumns[i]; + Optional optional = + columns.stream() + .filter(c -> c.getName().equals(outputColumn.getName())) + .findFirst(); + if (optional.isPresent()) { + Column originalColumn = optional.get(); + int originalColumnIndex = columns.indexOf(originalColumn); + if (!originalColumn.getDataType().equals(outputColumn.getDataType())) { + columns.set( + originalColumnIndex, originalColumn.copy(outputColumn.getDataType())); + } + fieldsIndex[i] = originalColumnIndex; + } else { + addFieldCount++; + columns.add(outputColumn); + fieldsIndex[i] = columns.indexOf(outputColumn); + } + } + + TableSchema outputTableSchema = builder.columns(columns).build(); + log.info( + "Changed input table schema: {} to output table schema: {}", + inputCatalogTable.getTableSchema(), + outputTableSchema); + + return outputTableSchema; + } + + @Override + protected TableIdentifier transformTableIdentifier() { + return inputCatalogTable.getTableId().copy(); + } + + protected Column[] getOutputColumns() { + List columns = inputCatalogTable.getTableSchema().getColumns(); + + List collect = + columns.stream() + .filter(column -> explodeFields.keySet().contains(column.getName())) + .collect(Collectors.toList()); + if (CollectionUtils.isEmpty(collect)) { + throw TransformCommonError.cannotFindInputFieldError( + getPluginName(), config.get(ReplaceTransformConfig.KEY_REPLACE_FIELD)); + } + return collect.toArray(new Column[0]); + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java new file mode 100644 index 00000000000..395db47c569 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.transform.explode; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import java.io.Serializable; +import java.util.Map; + +public class ExplodeTransformConfig implements Serializable { + public static final Option> EXPLODE_FIELDS = + Options.key("explode_fields") + .mapType() + .noDefaultValue() + .withDescription("The map of fields that need to be explode."); +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java new file mode 100644 index 00000000000..6d7407a5245 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.transform.explode; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableTransform; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableTransformFactory; +import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; + +import com.google.auto.service.AutoService; + +import static org.apache.seatunnel.transform.explode.ExplodeTransform.PLUGIN_NAME; + +@AutoService(Factory.class) +public class ExplodeTransformFactory implements TableTransformFactory { + + @Override + public String factoryIdentifier() { + return PLUGIN_NAME; + } + + @Override + public OptionRule optionRule() { + + return OptionRule.builder().required(ExplodeTransformConfig.EXPLODE_FIELDS).build(); + } + + @Override + public TableTransform createTransform(TableTransformFactoryContext context) { + CatalogTable catalogTable = context.getCatalogTables().get(0); + return () -> new ExplodeTransform(context.getOptions(), catalogTable); + } +} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java new file mode 100644 index 00000000000..17e2c240b50 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.seatunnel.transform; + +import org.apache.seatunnel.transform.explode.ExplodeTransformFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class ExplodeTransformFactoryTest { + @Test + public void testOptionRule() throws Exception { + ExplodeTransformFactory explodeTransformFactory = new ExplodeTransformFactory(); + Assertions.assertNotNull(explodeTransformFactory.optionRule()); + } +} diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java new file mode 100644 index 00000000000..2b8555caac2 --- /dev/null +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java @@ -0,0 +1,188 @@ +/* + * 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.transform.explode; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ExplodeTransformTest { + + static Map explodeFields = new HashMap<>(); + static CatalogTable catalogTable; + static Object[] values; + + @BeforeAll + static void setUp() { + + explodeFields.put("key1", ","); + explodeFields.put("key2", ";"); + + catalogTable = + CatalogTable.of( + TableIdentifier.of("catalog", TablePath.DEFAULT), + TableSchema.builder() + .column( + PhysicalColumn.of( + "key1", + BasicType.STRING_TYPE, + 1L, + Boolean.FALSE, + null, + null)) + .column( + PhysicalColumn.of( + "key2", + BasicType.STRING_TYPE, + 1L, + Boolean.FALSE, + null, + null)) + .column( + PhysicalColumn.of( + "key3", + BasicType.STRING_TYPE, + 1L, + Boolean.FALSE, + null, + null)) + .column( + PhysicalColumn.of( + "key4", + BasicType.STRING_TYPE, + 1L, + Boolean.FALSE, + null, + null)) + .column( + PhysicalColumn.of( + "key5", + BasicType.STRING_TYPE, + 1L, + Boolean.FALSE, + null, + null)) + .build(), + new HashMap<>(), + new ArrayList<>(), + "comment"); + values = new Object[] {"value1,value2", "value3;value4", "value5", "value6", "value7"}; + } + + @Test + void testConfig() { + // test both not set + try { + new ExplodeTransform(ReadonlyConfig.fromMap(new HashMap<>()), catalogTable); + } catch (Exception e) { + assertEquals( + "ErrorCode:[API-02], ErrorDescription:[Option item validate failed] - There are unconfigured options, these options('include_fields', 'exclude_fields') are mutually exclusive, allowing only one set(\"[] for a set\") of options to be configured.", + e.getMessage()); + } + + // test both include and exclude set + try { + new ExplodeTransform( + ReadonlyConfig.fromMap( + new HashMap() { + { + put(ExplodeTransformConfig.EXPLODE_FIELDS.key(), explodeFields); + } + }), + catalogTable); + } catch (Exception e) { + assertEquals( + "ErrorCode:[API-02], ErrorDescription:[Option item validate failed] - These options('include_fields', 'exclude_fields') are mutually exclusive, allowing only one set(\"[] for a set\") of options to be configured.", + e.getMessage()); + } + + // not exception should be thrown now + new ExplodeTransform( + ReadonlyConfig.fromMap( + new HashMap() { + { + put(ExplodeTransformConfig.EXPLODE_FIELDS.key(), explodeFields); + } + }), + catalogTable); + } + + @Test + void testExplode() { + // default include + Map configMap = new HashMap<>(); + configMap.put(ExplodeTransformConfig.EXPLODE_FIELDS.key(), explodeFields); + + ExplodeTransform explodeTransform = + new ExplodeTransform(ReadonlyConfig.fromMap(configMap), catalogTable); + + // test output schema + TableSchema resultSchema = explodeTransform.transformTableSchema(); + Assertions.assertNotNull(resultSchema); + + // test output row + SeaTunnelRow input = new SeaTunnelRow(values); + List output = explodeTransform.transformRow(input); + Assertions.assertNotNull(output); + List result = + Lists.newArrayList( + new Object[] {"value1", "value3", "value5", "value6", "value7"}, + new Object[] {"value1", "value4", "value5", "value6", "value7"}, + new Object[] {"value2", "value3", "value5", "value6", "value7"}, + new Object[] {"value2", "value4", "value5", "value6", "value7"}); + + List outputValues = + output.stream().map(SeaTunnelRow::getFields).collect(Collectors.toList()); + assertEquals(outputValues.size(), result.size()); + for (int i = 0; i < result.size(); i++) { + assertTrue(arraysEqual(result.get(i), outputValues.get(i))); + } + } + + private boolean arraysEqual(Object[] array1, Object[] array2) { + if (array1.length != array2.length) { + return false; + } + for (int i = 0; i < array1.length; i++) { + if (!array1[i].equals(array2[i])) { + return false; + } + } + return true; + } +} From d83c2aa169b4a7082404da4006ce447dc94eebb4 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Mon, 28 Oct 2024 14:36:48 +0800 Subject: [PATCH 02/29] [Feature][transform] transform support explode --- .../seatunnel/api/transform/SeaTunnelMultiRowTransform.java | 2 +- .../core/starter/flink/execution/TransformExecuteProcessor.java | 2 +- .../core/starter/spark/execution/TransformExecuteProcessor.java | 2 +- .../engine/server/task/flow/TransformFlowLifeCycle.java | 2 +- .../transform/common/AbstractCatalogMultiRowTransform.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java index 2f4e1026ad1..27e5e9def8d 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java @@ -27,7 +27,7 @@ public interface SeaTunnelMultiRowTransform extends SeaTunnelTransform { * @param row the data need be transformed. * @return transformed data. */ - List MultiRowMap(T row); + List multiRowMap(T row); default T map(T row) { throw new UnsupportedOperationException("Heads-up conversion is not supported"); 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 2e5cdfb6039..297c5dc15ed 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 @@ -171,7 +171,7 @@ public ArrayFlatMap(SeaTunnelTransform transform) { @Override public void flatMap(SeaTunnelRow row, Collector collector) { List rows = - ((SeaTunnelMultiRowTransform) transform).MultiRowMap(row); + ((SeaTunnelMultiRowTransform) transform).multiRowMap(row); for (SeaTunnelRow rowResult : rows) { collector.collect(rowResult); } 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 d4505df6300..b0985e83845 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 @@ -198,7 +198,7 @@ public Iterator call(Iterator sourceIterator) throws Exception { if (transform instanceof SeaTunnelMultiRowTransform) { List seaTunnelRows = ((SeaTunnelMultiRowTransform) transform) - .MultiRowMap(seaTunnelRow); + .multiRowMap(seaTunnelRow); if (!seaTunnelRows.isEmpty()) { for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) { rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); 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 53f08eae0a8..1634cc444db 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 @@ -116,7 +116,7 @@ public List transform(T inputData) { SeaTunnelMultiRowTransform transformDecorator = (SeaTunnelMultiRowTransform) transformer; for (T data : dataList) { - List outputDataArray = transformDecorator.MultiRowMap(data); + List outputDataArray = transformDecorator.multiRowMap(data); log.debug( "Transform[{}] input row {} and output row {}", transformer, diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java index 109944eb3a2..82d8d49fe37 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java @@ -47,7 +47,7 @@ public AbstractCatalogMultiRowTransform( } @Override - public List MultiRowMap(SeaTunnelRow row) { + public List multiRowMap(SeaTunnelRow row) { try { return transformRow(row); } catch (ErrorDataTransformException e) { From 6f7f52ddb74e316817ae9d904e3222cebcf073a9 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 29 Oct 2024 11:07:43 +0800 Subject: [PATCH 03/29] [Feature][transform] transform support explode --- docs/en/transform-v2/explode.md | 67 +++++++++++++++++++ docs/zh/transform-v2/explode.md | 67 +++++++++++++++++++ .../transform/SeaTunnelMultiRowTransform.java | 2 +- .../execution/TransformExecuteProcessor.java | 2 +- .../execution/SparkRuntimeEnvironment.java | 1 + .../execution/TransformExecuteProcessor.java | 4 +- .../task/flow/TransformFlowLifeCycle.java | 2 +- .../AbstractCatalogMultiRowTransform.java | 2 +- 8 files changed, 141 insertions(+), 6 deletions(-) create mode 100644 docs/en/transform-v2/explode.md create mode 100644 docs/zh/transform-v2/explode.md diff --git a/docs/en/transform-v2/explode.md b/docs/en/transform-v2/explode.md new file mode 100644 index 00000000000..ea87f14d958 --- /dev/null +++ b/docs/en/transform-v2/explode.md @@ -0,0 +1,67 @@ +# Explode + +> Explode transform plugin + +## Description + +Split a row of data into multiple data according to the specified delimiter. + +## Options + +| name | type | required | Description | +|----------------|------|----------|---------------| +| explode_fields | Map | yes | Explode field | + + +### explode_fields [Map] + +The fields that need to be separated by the specified delimiter +The map is composed of field as key and separator as value. + +### common options [string] + +Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details + +## Example + +The data read from source is a table like this: + +| name | age | card | +|-------------------|-----|---------| +| Joy Ding,May Ding | 20 | 123;234 | +| Kin Dom,Joy Dom | 20 | 123;345 | + +we want to explode name and card, we can add a `Explode` Transform like below: + +``` +transform { + Explode { + source_table_name = "fake" + result_table_name = "fake1" + explode_fields = {"name":",","card":";"} + } +} +``` + + +It is useful when you want to explode row. + +Then the data in result table `fake1` will like this + +| name | age | card | +|----------|-----|------| +| Joy Ding | 20 | 123 | +| Joy Ding | 20 | 234 | +| May Ding | 20 | 123 | +| May Ding | 20 | 234 | +| Kin Dom | 20 | 123 | +| Kin Dom | 20 | 345 | +| Joy Dom | 20 | 123 | +| Joy Dom | 20 | 345 | + +## Changelog + +### new version + +- Add Explode Transform Connector + diff --git a/docs/zh/transform-v2/explode.md b/docs/zh/transform-v2/explode.md new file mode 100644 index 00000000000..85f03009e6f --- /dev/null +++ b/docs/zh/transform-v2/explode.md @@ -0,0 +1,67 @@ +# Explode + +> Explode transform plugin + +## Description + +将一行数据按照指定的分割符拆分成多条数据。 + +## Options + +| name | type | required | +|----------------|------|----------| +| explode_fields | Map | yes | + + +### explode_fields [Map] + +需要按照指定分隔符分割的字段。 +map是由field作为key,分隔符作为value。 + + +### common options [config] + +转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 + +## 示例 + +源端数据读取的表格如下: + +The data read from source is a table like this: + +| name | age | card | +|-------------------|-----|---------| +| Joy Ding,May Ding | 20 | 123;234 | +| Kin Dom,Joy Dom | 20 | 123;345 | + +当我们想根据name和card进行数据拆分: + +``` +transform { + Explode { + source_table_name = "fake" + result_table_name = "fake1" + explode_fields = {"name":",","card":";"} + } +} +``` + +那么结果表 `fake1` 中的数据将会像这样: + +| name | age | card | +|----------|-----|------| +| Joy Ding | 20 | 123 | +| Joy Ding | 20 | 234 | +| May Ding | 20 | 123 | +| May Ding | 20 | 234 | +| Kin Dom | 20 | 123 | +| Kin Dom | 20 | 345 | +| Joy Dom | 20 | 123 | +| Joy Dom | 20 | 345 | + +## Changelog + +### new version + +- Add Explode Transform Connector + diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java index 27e5e9def8d..1f78e8be486 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/transform/SeaTunnelMultiRowTransform.java @@ -27,7 +27,7 @@ public interface SeaTunnelMultiRowTransform extends SeaTunnelTransform { * @param row the data need be transformed. * @return transformed data. */ - List multiRowMap(T row); + List flatMap(T row); default T map(T row) { throw new UnsupportedOperationException("Heads-up conversion is not supported"); 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 297c5dc15ed..46ade083ed2 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 @@ -171,7 +171,7 @@ public ArrayFlatMap(SeaTunnelTransform transform) { @Override public void flatMap(SeaTunnelRow row, Collector collector) { List rows = - ((SeaTunnelMultiRowTransform) transform).multiRowMap(row); + ((SeaTunnelMultiRowTransform) transform).flatMap(row); for (SeaTunnelRow rowResult : rows) { collector.collect(rowResult); } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java index 7e31ca463bc..50af1f59661 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java @@ -137,6 +137,7 @@ private SparkConf createSparkConf() { entry.getKey(), String.valueOf(entry.getValue().unwrapped()))); sparkConf.setAppName(jobName); + sparkConf.setMaster("local[*]"); return sparkConf; } 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 b0985e83845..e69a9f8d27b 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 @@ -198,7 +198,7 @@ public Iterator call(Iterator sourceIterator) throws Exception { if (transform instanceof SeaTunnelMultiRowTransform) { List seaTunnelRows = ((SeaTunnelMultiRowTransform) transform) - .multiRowMap(seaTunnelRow); + .flatMap(seaTunnelRow); if (!seaTunnelRows.isEmpty()) { for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) { rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); @@ -206,7 +206,7 @@ public Iterator call(Iterator sourceIterator) throws Exception { } } else { SeaTunnelRow seaTunnelRowTransform = transform.map(seaTunnelRow); - if (seaTunnelRow != null) { + if (seaTunnelRowTransform != null) { rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); } } 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 1634cc444db..f34a3118137 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 @@ -116,7 +116,7 @@ public List transform(T inputData) { SeaTunnelMultiRowTransform transformDecorator = (SeaTunnelMultiRowTransform) transformer; for (T data : dataList) { - List outputDataArray = transformDecorator.multiRowMap(data); + List outputDataArray = transformDecorator.flatMap(data); log.debug( "Transform[{}] input row {} and output row {}", transformer, diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java index 82d8d49fe37..c4d983476ae 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java @@ -47,7 +47,7 @@ public AbstractCatalogMultiRowTransform( } @Override - public List multiRowMap(SeaTunnelRow row) { + public List flatMap(SeaTunnelRow row) { try { return transformRow(row); } catch (ErrorDataTransformException e) { From 6b80660ff2a44618d308879e6c495115446d0b35 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 29 Oct 2024 16:22:40 +0800 Subject: [PATCH 04/29] [Feature][transform] transform support explode --- .../execution/SparkRuntimeEnvironment.java | 1 - .../execution/TransformExecuteProcessor.java | 41 ++++++++----------- 2 files changed, 18 insertions(+), 24 deletions(-) diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java index 50af1f59661..7e31ca463bc 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java @@ -137,7 +137,6 @@ private SparkConf createSparkConf() { entry.getKey(), String.valueOf(entry.getValue().unwrapped()))); sparkConf.setAppName(jobName); - sparkConf.setMaster("local[*]"); return sparkConf; } 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 e69a9f8d27b..8b98b7e75d1 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 @@ -36,7 +36,7 @@ import org.apache.seatunnel.translation.spark.serialization.SeaTunnelRowConverter; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; -import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; @@ -165,51 +165,46 @@ private Dataset sparkTransform(SeaTunnelTransform transform, DatasetTableIn SeaTunnelRowConverter outputRowConverter = new SeaTunnelRowConverter(outputDataTYpe); ExpressionEncoder encoder = RowEncoder.apply(outputSchema); - return stream.mapPartitions( + return stream.flatMap( new TransformMapPartitionsFunction( - transform, outputSchema, inputRowConverter, outputRowConverter), + transform, inputRowConverter, outputRowConverter), encoder) .filter(Objects::nonNull); } - private static class TransformMapPartitionsFunction implements MapPartitionsFunction { + private static class TransformMapPartitionsFunction implements FlatMapFunction { private SeaTunnelTransform transform; - private StructType structType; private SeaTunnelRowConverter inputRowConverter; private SeaTunnelRowConverter outputRowConverter; public TransformMapPartitionsFunction( SeaTunnelTransform transform, - StructType structType, SeaTunnelRowConverter inputRowConverter, SeaTunnelRowConverter outputRowConverter) { this.transform = transform; - this.structType = structType; this.inputRowConverter = inputRowConverter; this.outputRowConverter = outputRowConverter; } @Override - public Iterator call(Iterator sourceIterator) throws Exception { + public Iterator call(Row row) throws Exception { List rows = new ArrayList<>(); - while (sourceIterator.hasNext()) { - Row row = sourceIterator.next(); - SeaTunnelRow seaTunnelRow = inputRowConverter.unpack((GenericRowWithSchema) row); - if (transform instanceof SeaTunnelMultiRowTransform) { - List seaTunnelRows = - ((SeaTunnelMultiRowTransform) transform) - .flatMap(seaTunnelRow); - if (!seaTunnelRows.isEmpty()) { - for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) { - rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); - } - } - } else { - SeaTunnelRow seaTunnelRowTransform = transform.map(seaTunnelRow); - if (seaTunnelRowTransform != null) { + + SeaTunnelRow seaTunnelRow = inputRowConverter.unpack((GenericRowWithSchema) row); + if (transform instanceof SeaTunnelMultiRowTransform) { + List seaTunnelRows = + ((SeaTunnelMultiRowTransform) transform) + .flatMap(seaTunnelRow); + if (!seaTunnelRows.isEmpty()) { + for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) { rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); } } + } else { + SeaTunnelRow seaTunnelRowTransform = transform.map(seaTunnelRow); + if (seaTunnelRowTransform != null) { + rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); + } } return rows.iterator(); } From d30be82cabfaf4235d03d9b8fa857f860a05a11e Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 30 Oct 2024 17:42:38 +0800 Subject: [PATCH 05/29] [Feature][transform] explode transform support spilt list --- .../src/test/resources/explode_transform.conf | 16 ++-- .../task/flow/TransformFlowLifeCycle.java | 1 - .../transform/explode/ExplodeTransform.java | 85 ++++++++++++++----- .../explode/ExplodeTransformConfig.java | 13 ++- .../explode/ExplodeTransformFactory.java | 2 +- .../explode/ExplodeTransformTest.java | 39 +++++++-- 6 files changed, 118 insertions(+), 38 deletions(-) diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf index 59d3c9b9bd3..6d16378f2bc 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf @@ -30,6 +30,7 @@ source { fields { pk_id = string name = string + age = array } primaryKey { name = "pk_id" @@ -39,15 +40,15 @@ source { rows = [ { kind = INSERT - fields = ["id001;id001", "zhangsan,zhangsan"] + fields = ["id001;id001", "zhangsan,zhangsan",["1","1"]] }, { kind = INSERT - fields = ["id001", "zhangsan,zhangsan"] + fields = ["id001", "zhangsan,zhangsan",["1"]] }, { kind = INSERT - fields = ["id001;id001", "zhangsan"] + fields = ["id001;id001", "zhangsan",["1"]] } ] } @@ -68,11 +69,11 @@ sink{ row_rules = [ { rule_type = MAX_ROW - rule_value = 8 + rule_value = 12 }, { rule_type = MIN_ROW - rule_value = 8 + rule_value = 12 } ], field_rules = [ @@ -85,6 +86,11 @@ sink{ field_name = name field_type = string field_value = [{equals_to = zhangsan}] + }, + { + field_name = age + field_type = string + field_value = [{equals_to = 1}] } ] } 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 f34a3118137..52d21afda75 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 @@ -110,7 +110,6 @@ public List transform(T inputData) { dataList.add(inputData); for (SeaTunnelTransform transformer : transform) { - log.info("transform test: {}", transformer.getPluginName()); List nextInputDataList = new ArrayList<>(); if (transformer instanceof SeaTunnelMultiRowTransform) { SeaTunnelMultiRowTransform transformDecorator = diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java index 4e613a4244d..a9279fa6086 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java @@ -20,8 +20,10 @@ import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.Column; import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.transform.common.AbstractCatalogMultiRowTransform; @@ -30,6 +32,7 @@ import org.apache.seatunnel.transform.replace.ReplaceTransformConfig; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; import com.google.common.collect.Lists; import lombok.NonNull; @@ -45,7 +48,8 @@ public class ExplodeTransform extends AbstractCatalogMultiRowTransform { protected CatalogTable inputCatalogTable; public static final String PLUGIN_NAME = "Explode"; - protected Map explodeFields; + protected Map explodeStringFields; + protected List explodeListFields; protected SeaTunnelRowType seaTunnelRowType; private int[] fieldsIndex; private ReadonlyConfig config; @@ -53,7 +57,8 @@ public class ExplodeTransform extends AbstractCatalogMultiRowTransform { public ExplodeTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { super(catalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); this.config = config; - this.explodeFields = config.get(ExplodeTransformConfig.EXPLODE_FIELDS); + this.explodeStringFields = config.get(ExplodeTransformConfig.EXPLODE_STRING_FIELDS); + this.explodeListFields = config.get(ExplodeTransformConfig.EXPLODE_LIST_FIELDS); this.seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); this.inputCatalogTable = catalogTable; } @@ -65,26 +70,50 @@ public ExplodeTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable ca */ protected List transformRow(SeaTunnelRow inputRow) { List rows = Lists.newArrayList(inputRow); - for (Map.Entry entry : explodeFields.entrySet()) { - List next = new ArrayList<>(); - for (SeaTunnelRow row : rows) { - - String field = entry.getKey(); - int fieldIndex = seaTunnelRowType.indexOf(field); - Object splitFieldValue = inputRow.getField(fieldIndex); - if (splitFieldValue == null) { - continue; + if (MapUtils.isNotEmpty(explodeStringFields)) { + for (Map.Entry entry : explodeStringFields.entrySet()) { + List next = new ArrayList<>(); + for (SeaTunnelRow row : rows) { + + String field = entry.getKey(); + int fieldIndex = seaTunnelRowType.indexOf(field); + Object splitFieldValue = inputRow.getField(fieldIndex); + if (splitFieldValue == null) { + continue; + } + String separator = entry.getValue(); + String[] splitFieldValues = splitFieldValue.toString().split(separator); + for (String fieldValue : splitFieldValues) { + SeaTunnelRow outputRow = row.copy(); + outputRow.setField(fieldIndex, fieldValue); + next.add(outputRow); + } } - String separator = entry.getValue(); - String[] splitFieldValues = splitFieldValue.toString().split(separator); - for (String fieldValue : splitFieldValues) { - SeaTunnelRow outputRow = row.copy(); - outputRow.setField(fieldIndex, fieldValue); - next.add(outputRow); + rows = next; + } + } + if (!CollectionUtils.isEmpty(explodeListFields)) { + for (String field : explodeListFields) { + List next = new ArrayList<>(); + for (SeaTunnelRow row : rows) { + int fieldIndex = seaTunnelRowType.indexOf(field); + Object splitFieldValue = inputRow.getField(fieldIndex); + if (splitFieldValue == null) { + continue; + } + if (splitFieldValue instanceof Object[]) { + Object[] rowList = (Object[]) splitFieldValue; + for (Object fieldValue : rowList) { + SeaTunnelRow outputRow = row.copy(); + outputRow.setField(fieldIndex, fieldValue); + next.add(outputRow); + } + } } + rows = next; } - rows = next; } + return rows; } @@ -154,8 +183,26 @@ protected Column[] getOutputColumns() { List collect = columns.stream() - .filter(column -> explodeFields.keySet().contains(column.getName())) + .filter( + column -> + explodeStringFields.containsKey(column.getName()) + || explodeListFields.contains(column.getName())) + .map( + column -> { + if (explodeListFields.contains(column.getName())) { + ArrayType arrayType = (ArrayType) column.getDataType(); + return PhysicalColumn.of( + column.getName(), + arrayType.getElementType(), + 200, + true, + "", + ""); + } + return column; + }) .collect(Collectors.toList()); + if (CollectionUtils.isEmpty(collect)) { throw TransformCommonError.cannotFindInputFieldError( getPluginName(), config.get(ReplaceTransformConfig.KEY_REPLACE_FIELD)); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java index 395db47c569..f3844b3d6df 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java @@ -20,12 +20,19 @@ import org.apache.seatunnel.api.configuration.Options; import java.io.Serializable; +import java.util.List; import java.util.Map; public class ExplodeTransformConfig implements Serializable { - public static final Option> EXPLODE_FIELDS = - Options.key("explode_fields") + public static final Option> EXPLODE_STRING_FIELDS = + Options.key("explode_string_fields") .mapType() .noDefaultValue() - .withDescription("The map of fields that need to be explode."); + .withDescription("The map of string fields that need to be explode."); + + public static final Option> EXPLODE_LIST_FIELDS = + Options.key("explode_list_fields") + .listType() + .noDefaultValue() + .withDescription("The map of list fields that need to be explode."); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java index 6d7407a5245..653f5601c37 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java @@ -38,7 +38,7 @@ public String factoryIdentifier() { @Override public OptionRule optionRule() { - return OptionRule.builder().required(ExplodeTransformConfig.EXPLODE_FIELDS).build(); + return OptionRule.builder().build(); } @Override diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java index 2b8555caac2..2568810a00f 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -44,6 +45,7 @@ class ExplodeTransformTest { static Map explodeFields = new HashMap<>(); + static List explodeListFields = new ArrayList<>(); static CatalogTable catalogTable; static Object[] values; @@ -52,6 +54,7 @@ static void setUp() { explodeFields.put("key1", ","); explodeFields.put("key2", ";"); + explodeListFields.add("key3"); catalogTable = CatalogTable.of( @@ -76,7 +79,7 @@ static void setUp() { .column( PhysicalColumn.of( "key3", - BasicType.STRING_TYPE, + ArrayType.STRING_ARRAY_TYPE, 1L, Boolean.FALSE, null, @@ -101,7 +104,10 @@ static void setUp() { new HashMap<>(), new ArrayList<>(), "comment"); - values = new Object[] {"value1,value2", "value3;value4", "value5", "value6", "value7"}; + String[] key3 = new String[2]; + key3[0] = "value5"; + key3[1] = "value6"; + values = new Object[] {"value1,value2", "value3;value4", key3, "value7", "value8"}; } @Test @@ -121,7 +127,12 @@ void testConfig() { ReadonlyConfig.fromMap( new HashMap() { { - put(ExplodeTransformConfig.EXPLODE_FIELDS.key(), explodeFields); + put( + ExplodeTransformConfig.EXPLODE_STRING_FIELDS.key(), + explodeFields); + put( + ExplodeTransformConfig.EXPLODE_LIST_FIELDS.key(), + explodeListFields); } }), catalogTable); @@ -136,7 +147,12 @@ void testConfig() { ReadonlyConfig.fromMap( new HashMap() { { - put(ExplodeTransformConfig.EXPLODE_FIELDS.key(), explodeFields); + put( + ExplodeTransformConfig.EXPLODE_STRING_FIELDS.key(), + explodeFields); + put( + ExplodeTransformConfig.EXPLODE_LIST_FIELDS.key(), + explodeListFields); } }), catalogTable); @@ -146,7 +162,8 @@ void testConfig() { void testExplode() { // default include Map configMap = new HashMap<>(); - configMap.put(ExplodeTransformConfig.EXPLODE_FIELDS.key(), explodeFields); + configMap.put(ExplodeTransformConfig.EXPLODE_STRING_FIELDS.key(), explodeFields); + configMap.put(ExplodeTransformConfig.EXPLODE_LIST_FIELDS.key(), explodeListFields); ExplodeTransform explodeTransform = new ExplodeTransform(ReadonlyConfig.fromMap(configMap), catalogTable); @@ -161,10 +178,14 @@ void testExplode() { Assertions.assertNotNull(output); List result = Lists.newArrayList( - new Object[] {"value1", "value3", "value5", "value6", "value7"}, - new Object[] {"value1", "value4", "value5", "value6", "value7"}, - new Object[] {"value2", "value3", "value5", "value6", "value7"}, - new Object[] {"value2", "value4", "value5", "value6", "value7"}); + new Object[] {"value1", "value3", "value5", "value7", "value8"}, + new Object[] {"value1", "value3", "value6", "value7", "value8"}, + new Object[] {"value1", "value4", "value5", "value7", "value8"}, + new Object[] {"value1", "value4", "value6", "value7", "value8"}, + new Object[] {"value2", "value3", "value5", "value7", "value8"}, + new Object[] {"value2", "value3", "value6", "value7", "value8"}, + new Object[] {"value2", "value4", "value5", "value7", "value8"}, + new Object[] {"value2", "value4", "value6", "value7", "value8"}); List outputValues = output.stream().map(SeaTunnelRow::getFields).collect(Collectors.toList()); From 48adff18b6e1baebf17751024b53d07152710941 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 30 Oct 2024 17:48:32 +0800 Subject: [PATCH 06/29] [Feature][transform] explode transform support spilt list --- docs/en/transform-v2/explode.md | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/docs/en/transform-v2/explode.md b/docs/en/transform-v2/explode.md index ea87f14d958..1e20aa9d878 100644 --- a/docs/en/transform-v2/explode.md +++ b/docs/en/transform-v2/explode.md @@ -8,16 +8,22 @@ Split a row of data into multiple data according to the specified delimiter. ## Options -| name | type | required | Description | -|----------------|------|----------|---------------| -| explode_fields | Map | yes | Explode field | +| name | type | required | Description | +|-----------------------|------|----------|----------------------| +| explode_string_fields | Map | yes | Explode String field | +| explode_list_fields | List | yes | Explode List field | -### explode_fields [Map] +### explode_string_fields [Map] The fields that need to be separated by the specified delimiter The map is composed of field as key and separator as value. +### explode_list_fields [List] + +The List fields that need to be separated by the specified delimiter + + ### common options [string] Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details @@ -26,10 +32,10 @@ Transform plugin common parameters, please refer to [Transform Plugin](common-op The data read from source is a table like this: -| name | age | card | -|-------------------|-----|---------| -| Joy Ding,May Ding | 20 | 123;234 | -| Kin Dom,Joy Dom | 20 | 123;345 | +| name | age | card | +|-------------------|-----|-----------| +| Joy Ding,May Ding | 20 | [123,234] | +| Kin Dom,Joy Dom | 20 | [123,345] | we want to explode name and card, we can add a `Explode` Transform like below: @@ -38,7 +44,8 @@ transform { Explode { source_table_name = "fake" result_table_name = "fake1" - explode_fields = {"name":",","card":";"} + explode_string_fields = {"name":","} + explode_list_fields = ["card"] } } ``` From 3a55991db2fae0471724cce13c47260105550a15 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 30 Oct 2024 17:48:43 +0800 Subject: [PATCH 07/29] [Feature][transform] explode transform support spilt list --- docs/zh/transform-v2/explode.md | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/docs/zh/transform-v2/explode.md b/docs/zh/transform-v2/explode.md index 85f03009e6f..71509ae196e 100644 --- a/docs/zh/transform-v2/explode.md +++ b/docs/zh/transform-v2/explode.md @@ -10,15 +10,22 @@ | name | type | required | |----------------|------|----------| -| explode_fields | Map | yes | +| explode_string_fields | Map | yes | +| explode_list_fields | List | yes | -### explode_fields [Map] +### explode_string_fields [Map] 需要按照指定分隔符分割的字段。 map是由field作为key,分隔符作为value。 +### explode_list_fields [List] + +需要按照指定分隔符分割的字段。 +List为需要切分的字段。 + + ### common options [config] 转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 @@ -31,8 +38,8 @@ The data read from source is a table like this: | name | age | card | |-------------------|-----|---------| -| Joy Ding,May Ding | 20 | 123;234 | -| Kin Dom,Joy Dom | 20 | 123;345 | +| Joy Ding,May Ding | 20 | [123,234] | +| Kin Dom,Joy Dom | 20 | [123,345] | 当我们想根据name和card进行数据拆分: @@ -41,7 +48,8 @@ transform { Explode { source_table_name = "fake" result_table_name = "fake1" - explode_fields = {"name":",","card":";"} + explode_string_fields = {"name":","} + explode_list_fields = ["card"] } } ``` From 87f3befaf0f58622ea3f3e6e5887b64996ccd829 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 1 Nov 2024 11:57:14 +0800 Subject: [PATCH 08/29] [Feature][transform] sql transform support lateral view explode --- pom.xml | 2 +- .../config/sql/SqlConfigBuilder.java | 5 +- .../SqlToPaimonPredicateConverter.java | 15 +- .../execution/TransformExecuteProcessor.java | 7 +- .../execution/TransformExecuteProcessor.java | 3 +- .../e2e/transform/TestExplodeIT.java | 3 + .../src/test/resources/explode_transform.conf | 3 +- .../test/resources/explode_transform_sql.conf | 98 ++++++++++ .../resources/sql_transform/func_system.conf | 6 +- .../task/flow/TransformFlowLifeCycle.java | 6 +- .../seatunnel/transform/sql/SQLEngine.java | 2 +- .../seatunnel/transform/sql/SQLTransform.java | 12 +- .../transform/sql/zeta/ZetaSQLEngine.java | 53 +++--- .../transform/sql/zeta/ZetaSQLFilter.java | 5 +- .../transform/sql/zeta/ZetaSQLFunction.java | 180 +++++++++++++++++- .../transform/sql/zeta/ZetaSQLType.java | 11 +- .../sql/zeta/functions/SystemFunction.java | 1 + .../transform/sql/SQLTransformTest.java | 13 +- .../sql/zeta/DateTimeFunctionTest.java | 4 +- 19 files changed, 358 insertions(+), 71 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf diff --git a/pom.xml b/pom.xml index f9d59c79ab0..16317b88a30 100644 --- a/pom.xml +++ b/pom.xml @@ -135,7 +135,7 @@ 2.0.0 1.17.6 2.29.0 - 4.5 + 4.9 2.7.0 4.0.16 9.4.56.v20240826 diff --git a/seatunnel-config/seatunnel-config-sql/src/main/java/org/apache/seatunnel/config/sql/SqlConfigBuilder.java b/seatunnel-config/seatunnel-config-sql/src/main/java/org/apache/seatunnel/config/sql/SqlConfigBuilder.java index f0d68e089b5..00f6a40a436 100644 --- a/seatunnel-config/seatunnel-config-sql/src/main/java/org/apache/seatunnel/config/sql/SqlConfigBuilder.java +++ b/seatunnel-config/seatunnel-config-sql/src/main/java/org/apache/seatunnel/config/sql/SqlConfigBuilder.java @@ -43,7 +43,6 @@ import net.sf.jsqlparser.statement.insert.Insert; import net.sf.jsqlparser.statement.select.PlainSelect; import net.sf.jsqlparser.statement.select.Select; -import net.sf.jsqlparser.statement.select.SelectExpressionItem; import net.sf.jsqlparser.statement.select.SelectItem; import java.nio.file.Files; @@ -360,12 +359,12 @@ private static void parseInsertSql( String sourceTableName; String resultTableName; if (plainSelect.getFromItem() == null) { - List selectItems = plainSelect.getSelectItems(); + List> selectItems = plainSelect.getSelectItems(); if (selectItems.size() != 1) { throw new ParserException( "Source table must be specified in SQL: " + insertSql); } - SelectExpressionItem selectItem = (SelectExpressionItem) selectItems.get(0); + SelectItem selectItem = selectItems.get(0); Column column = (Column) selectItem.getExpression(); sourceTableName = column.getColumnName(); resultTableName = sourceTableName; diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/source/converter/SqlToPaimonPredicateConverter.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/source/converter/SqlToPaimonPredicateConverter.java index 212bfd6e8b8..0bf47b13105 100644 --- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/source/converter/SqlToPaimonPredicateConverter.java +++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/source/converter/SqlToPaimonPredicateConverter.java @@ -54,8 +54,6 @@ import net.sf.jsqlparser.statement.select.AllColumns; import net.sf.jsqlparser.statement.select.PlainSelect; import net.sf.jsqlparser.statement.select.Select; -import net.sf.jsqlparser.statement.select.SelectBody; -import net.sf.jsqlparser.statement.select.SelectExpressionItem; import net.sf.jsqlparser.statement.select.SelectItem; import java.math.BigDecimal; @@ -83,7 +81,7 @@ public static PlainSelect convertToPlainSelect(String query) { throw new IllegalArgumentException("Only SELECT statements are supported."); } Select select = (Select) statement; - SelectBody selectBody = select.getSelectBody(); + Select selectBody = select.getSelectBody(); if (!(selectBody instanceof PlainSelect)) { throw new IllegalArgumentException("Only simple SELECT statements are supported."); } @@ -101,18 +99,15 @@ public static PlainSelect convertToPlainSelect(String query) { public static int[] convertSqlSelectToPaimonProjectionIndex( String[] fieldNames, PlainSelect plainSelect) { int[] projectionIndex = null; - List selectItems = plainSelect.getSelectItems(); + List> selectItems = plainSelect.getSelectItems(); List columnNames = new ArrayList<>(); for (SelectItem selectItem : selectItems) { - if (selectItem instanceof AllColumns) { + if (selectItem.getExpression() instanceof AllColumns) { return null; - } else if (selectItem instanceof SelectExpressionItem) { - SelectExpressionItem selectExpressionItem = (SelectExpressionItem) selectItem; - String columnName = selectExpressionItem.getExpression().toString(); - columnNames.add(columnName); } else { - throw new IllegalArgumentException("Error encountered parsing query fields."); + String columnName = ((Column) selectItem.getExpression()).getColumnName(); + columnNames.add(columnName); } } 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 46ade083ed2..982179dd2e0 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 @@ -32,6 +32,7 @@ import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; @@ -172,8 +173,10 @@ public ArrayFlatMap(SeaTunnelTransform transform) { public void flatMap(SeaTunnelRow row, Collector collector) { List rows = ((SeaTunnelMultiRowTransform) transform).flatMap(row); - for (SeaTunnelRow rowResult : rows) { - collector.collect(rowResult); + if (CollectionUtils.isNotEmpty(rows)) { + for (SeaTunnelRow rowResult : rows) { + collector.collect(rowResult); + } } } } 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 8b98b7e75d1..44a60195ee5 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 @@ -36,6 +36,7 @@ import org.apache.seatunnel.translation.spark.serialization.SeaTunnelRowConverter; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; +import org.apache.commons.collections.CollectionUtils; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -195,7 +196,7 @@ public Iterator call(Row row) throws Exception { List seaTunnelRows = ((SeaTunnelMultiRowTransform) transform) .flatMap(seaTunnelRow); - if (!seaTunnelRows.isEmpty()) { + if (CollectionUtils.isNotEmpty(seaTunnelRows)) { for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) { rows.add(outputRowConverter.parcel(seaTunnelRowTransform)); } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java index 34ba67a5b70..4658603b514 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java @@ -30,5 +30,8 @@ public class TestExplodeIT extends TestSuiteBase { public void testExplode(TestContainer container) throws IOException, InterruptedException { Container.ExecResult execResult = container.executeJob("/explode_transform.conf"); Assertions.assertEquals(0, execResult.getExitCode()); + + Container.ExecResult execResultBySql = container.executeJob("/explode_transform_sql.conf"); + Assertions.assertEquals(0, execResultBySql.getExitCode()); } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf index 6d16378f2bc..c1fbf6b0ca3 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf @@ -58,7 +58,8 @@ transform { Explode { source_table_name = "fake" result_table_name = "fake1" - explode_fields = {"pk_id":";","name":","} + explode_string_fields = {"pk_id":";","name":","} + explode_list_fields = ["age"] } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf new file mode 100644 index 00000000000..3215a50b82b --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf @@ -0,0 +1,98 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" + parallelism = 1 +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + pk_id = string + name = string + age = array + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = ["id001;id001", "zhangsan,zhangsan",["1","1"]] + }, + { + kind = INSERT + fields = ["id001", "zhangsan,zhangsan",["1"]] + }, + { + kind = INSERT + fields = ["id001;id001", "zhangsan",["1"]] + } + ] + } +} + +transform { + Sql { + source_table_name = "fake" + result_table_name = "fake1" + query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPILT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPILT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age" + } +} + +sink{ + assert { + rules = + { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 12 + }, + { + rule_type = MIN_ROW + rule_value = 12 + } + ], + field_rules = [ + { + field_name = pk_id + field_type = string + field_value = [{equals_to = id001}] + }, + { + field_name = name + field_type = string + field_value = [{equals_to = zhangsan}] + }, + { + field_name = age + field_type = string + field_value = [{equals_to = 1}] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf index a189c7c2ddc..8afb18e6d8c 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf @@ -49,7 +49,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as bytes) as c7, name as `apply` from fake" + query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as BINARY) as c7, name as apply from fake" } } @@ -76,7 +76,7 @@ sink { field_name = "id3" field_type = "double" field_value = [ - {equals_to = 1} + {equals_to = 1.0} ] }, { @@ -175,4 +175,4 @@ sink { ] } } -} \ No newline at end of file +} 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 52d21afda75..093ac1cca68 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 @@ -28,6 +28,8 @@ import org.apache.seatunnel.engine.server.task.SeaTunnelTask; import org.apache.seatunnel.engine.server.task.record.Barrier; +import org.apache.commons.collections4.CollectionUtils; + import lombok.extern.slf4j.Slf4j; import java.io.IOException; @@ -121,7 +123,9 @@ public List transform(T inputData) { transformer, data, outputDataArray); - nextInputDataList.addAll(outputDataArray); + if (CollectionUtils.isNotEmpty(outputDataArray)) { + nextInputDataList.addAll(outputDataArray); + } } } else { for (T data : dataList) { diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLEngine.java index 6dfaddca00a..62c25be374b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/SQLEngine.java @@ -31,7 +31,7 @@ void init( SeaTunnelRowType typeMapping(List inputColumnsMapping); - SeaTunnelRow transformBySQL(SeaTunnelRow inputRow); + List transformBySQL(SeaTunnelRow inputRow, SeaTunnelRowType outputRowType); default void close() {} } 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 00316bba8e7..71f448500c5 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 @@ -30,7 +30,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.transform.common.AbstractCatalogSupportTransform; +import org.apache.seatunnel.transform.common.AbstractCatalogMultiRowTransform; import org.apache.seatunnel.transform.sql.SQLEngineFactory.EngineType; import lombok.NonNull; @@ -44,7 +44,7 @@ import static org.apache.seatunnel.transform.sql.SQLEngineFactory.EngineType.ZETA; @Slf4j -public class SQLTransform extends AbstractCatalogSupportTransform { +public class SQLTransform extends AbstractCatalogMultiRowTransform { public static final String PLUGIN_NAME = "Sql"; public static final Option KEY_QUERY = @@ -60,6 +60,8 @@ public class SQLTransform extends AbstractCatalogSupportTransform { private final EngineType engineType; + private SeaTunnelRowType outRowType; + private transient SQLEngine sqlEngine; private final String inputTableName; @@ -103,16 +105,16 @@ private void tryOpen() { } @Override - protected SeaTunnelRow transformRow(SeaTunnelRow inputRow) { + protected List transformRow(SeaTunnelRow inputRow) { tryOpen(); - return sqlEngine.transformBySQL(inputRow); + return sqlEngine.transformBySQL(inputRow, outRowType); } @Override protected TableSchema transformTableSchema() { tryOpen(); List inputColumnsMapping = new ArrayList<>(); - SeaTunnelRowType outRowType = sqlEngine.typeMapping(inputColumnsMapping); + outRowType = sqlEngine.typeMapping(inputColumnsMapping); List outputColumns = Arrays.asList(outRowType.getFieldNames()); TableSchema.Builder builder = TableSchema.builder(); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java index 993b4e0a3c2..b72d239b7d2 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java @@ -24,9 +24,12 @@ import org.apache.seatunnel.transform.exception.TransformException; import org.apache.seatunnel.transform.sql.SQLEngine; +import org.apache.commons.collections4.CollectionUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.collect.Lists; import net.sf.jsqlparser.JSQLParserException; import net.sf.jsqlparser.expression.Expression; import net.sf.jsqlparser.parser.CCJSqlParserUtil; @@ -35,9 +38,9 @@ import net.sf.jsqlparser.statement.Statement; import net.sf.jsqlparser.statement.select.AllColumns; import net.sf.jsqlparser.statement.select.FromItem; +import net.sf.jsqlparser.statement.select.LateralView; import net.sf.jsqlparser.statement.select.PlainSelect; import net.sf.jsqlparser.statement.select.Select; -import net.sf.jsqlparser.statement.select.SelectExpressionItem; import net.sf.jsqlparser.statement.select.SelectItem; import javax.annotation.Nullable; @@ -150,12 +153,6 @@ private void validateSQL(Statement statement) { if (selectBody.getLimit() != null || selectBody.getOffset() != null) { throw new IllegalArgumentException("Unsupported LIMIT,OFFSET syntax"); } - - // for (SelectItem selectItem : selectBody.getSelectItems()) { - // if (selectItem instanceof AllColumns) { - // throw new IllegalArgumentException("Unsupported all columns select syntax"); - // } - // } } catch (Exception e) { throw new TransformException( CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, @@ -165,7 +162,7 @@ private void validateSQL(Statement statement) { @Override public SeaTunnelRowType typeMapping(List inputColumnsMapping) { - List selectItems = selectBody.getSelectItems(); + List> selectItems = selectBody.getSelectItems(); // count number of all columns int columnsSize = countColumnsSize(selectItems); @@ -183,7 +180,7 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { int idx = 0; for (SelectItem selectItem : selectItems) { - if (selectItem instanceof AllColumns) { + if (selectItem.getExpression() instanceof AllColumns) { for (int i = 0; i < inputRowType.getFieldNames().length; i++) { fieldNames[idx] = inputRowType.getFieldName(i); seaTunnelDataTypes[idx] = inputRowType.getFieldType(i); @@ -192,11 +189,10 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { } idx++; } - } else if (selectItem instanceof SelectExpressionItem) { - SelectExpressionItem expressionItem = (SelectExpressionItem) selectItem; - Expression expression = expressionItem.getExpression(); - if (expressionItem.getAlias() != null) { - String aliasName = expressionItem.getAlias().getName(); + } else { + Expression expression = selectItem.getExpression(); + if (selectItem.getAlias() != null) { + String aliasName = selectItem.getAlias().getName(); if (aliasName.startsWith(ESCAPE_IDENTIFIER) && aliasName.endsWith(ESCAPE_IDENTIFIER)) { aliasName = aliasName.substring(1, aliasName.length() - 1); @@ -218,15 +214,14 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { seaTunnelDataTypes[idx] = zetaSQLType.getExpressionType(expression); idx++; - } else { - idx++; } } - return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); + return zetaSQLFunction.lateralViewMapping( + fieldNames, seaTunnelDataTypes, selectBody.getLateralViews()); } @Override - public SeaTunnelRow transformBySQL(SeaTunnelRow inputRow) { + public List transformBySQL(SeaTunnelRow inputRow, SeaTunnelRowType outRowType) { // ------Physical Query Plan Execution------ // Scan Table Object[] inputFields = scanTable(inputRow); @@ -243,7 +238,12 @@ public SeaTunnelRow transformBySQL(SeaTunnelRow inputRow) { SeaTunnelRow seaTunnelRow = new SeaTunnelRow(outputFields); seaTunnelRow.setRowKind(inputRow.getRowKind()); seaTunnelRow.setTableId(inputRow.getTableId()); - return seaTunnelRow; + List lateralViews = selectBody.getLateralViews(); + if (CollectionUtils.isEmpty(lateralViews)) { + return Lists.newArrayList(seaTunnelRow); + } + return zetaSQLFunction.lateralView( + Lists.newArrayList(seaTunnelRow), lateralViews, outRowType); } private Object[] scanTable(SeaTunnelRow inputRow) { @@ -252,7 +252,7 @@ private Object[] scanTable(SeaTunnelRow inputRow) { } private Object[] project(Object[] inputFields) { - List selectItems = selectBody.getSelectItems(); + List> selectItems = selectBody.getSelectItems(); int columnsSize = countColumnsSize(selectItems); @@ -260,30 +260,27 @@ private Object[] project(Object[] inputFields) { int idx = 0; for (SelectItem selectItem : selectItems) { - if (selectItem instanceof AllColumns) { + if (selectItem.getExpression() instanceof AllColumns) { for (Object inputField : inputFields) { fields[idx] = inputField; idx++; } - } else if (selectItem instanceof SelectExpressionItem) { - SelectExpressionItem expressionItem = (SelectExpressionItem) selectItem; - Expression expression = expressionItem.getExpression(); - fields[idx] = zetaSQLFunction.computeForValue(expression, inputFields); - idx++; } else { + Expression expression = selectItem.getExpression(); + fields[idx] = zetaSQLFunction.computeForValue(expression, inputFields); idx++; } } return fields; } - private int countColumnsSize(List selectItems) { + private int countColumnsSize(List> selectItems) { if (allColumnsCount != null) { return allColumnsCount; } int allColumnsCnt = 0; for (SelectItem selectItem : selectItems) { - if (selectItem instanceof AllColumns) { + if (selectItem.getExpression() instanceof AllColumns) { allColumnsCnt++; } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFilter.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFilter.java index b3542663ebf..7e84093bf4e 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFilter.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFilter.java @@ -30,7 +30,6 @@ import net.sf.jsqlparser.expression.operators.conditional.OrExpression; import net.sf.jsqlparser.expression.operators.relational.ComparisonOperator; import net.sf.jsqlparser.expression.operators.relational.EqualsTo; -import net.sf.jsqlparser.expression.operators.relational.ExpressionList; import net.sf.jsqlparser.expression.operators.relational.GreaterThan; import net.sf.jsqlparser.expression.operators.relational.GreaterThanEquals; import net.sf.jsqlparser.expression.operators.relational.InExpression; @@ -39,6 +38,7 @@ import net.sf.jsqlparser.expression.operators.relational.MinorThan; import net.sf.jsqlparser.expression.operators.relational.MinorThanEquals; import net.sf.jsqlparser.expression.operators.relational.NotEqualsTo; +import net.sf.jsqlparser.expression.operators.relational.ParenthesedExpressionList; import java.time.LocalDate; import java.time.LocalDateTime; @@ -139,7 +139,8 @@ private boolean isNullExpr(IsNullExpression isNullExpression, Object[] inputFiel private boolean inExpr(InExpression inExpression, Object[] inputFields) { Expression leftExpr = inExpression.getLeftExpression(); - ExpressionList itemsList = (ExpressionList) inExpression.getRightItemsList(); + ParenthesedExpressionList itemsList = + (ParenthesedExpressionList) inExpression.getRightExpression(); Object leftValue = zetaSQLFunction.computeForValue(leftExpr, inputFields); for (Expression exprItem : itemsList.getExpressions()) { Object rightValue = zetaSQLFunction.computeForValue(exprItem, inputFields); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index ce02832712c..79efbcfb506 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.transform.sql.zeta; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.type.ArrayType; import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -24,12 +26,15 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.api.table.type.SqlType; import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.transform.exception.TransformException; import org.apache.seatunnel.transform.sql.zeta.functions.DateTimeFunction; import org.apache.seatunnel.transform.sql.zeta.functions.NumericFunction; import org.apache.seatunnel.transform.sql.zeta.functions.StringFunction; import org.apache.seatunnel.transform.sql.zeta.functions.SystemFunction; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.tuple.Pair; import net.sf.jsqlparser.expression.BinaryExpression; @@ -45,6 +50,7 @@ import net.sf.jsqlparser.expression.SignedExpression; import net.sf.jsqlparser.expression.StringValue; import net.sf.jsqlparser.expression.TimeKeyExpression; +import net.sf.jsqlparser.expression.TrimFunction; import net.sf.jsqlparser.expression.WhenClause; import net.sf.jsqlparser.expression.operators.arithmetic.Addition; import net.sf.jsqlparser.expression.operators.arithmetic.Concat; @@ -54,14 +60,17 @@ import net.sf.jsqlparser.expression.operators.arithmetic.Subtraction; import net.sf.jsqlparser.expression.operators.relational.ExpressionList; import net.sf.jsqlparser.schema.Column; +import net.sf.jsqlparser.statement.select.LateralView; import java.math.BigDecimal; import java.math.RoundingMode; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import static java.util.UUID.randomUUID; +import static org.apache.seatunnel.transform.exception.TransformCommonErrorCode.INPUT_FIELDS_NOT_FOUND; public class ZetaSQLFunction { // ============================internal functions===================== @@ -81,6 +90,8 @@ public class ZetaSQLFunction { public static final String INSERT = "INSERT"; public static final String LOWER = "LOWER"; public static final String LCASE = "LCASE"; + public static final String BINARY = "BINARY"; + public static final String BYTE = "BYTE"; public static final String UPPER = "UPPER"; public static final String UCASE = "UCASE"; public static final String LEFT = "LEFT"; @@ -168,6 +179,9 @@ public class ZetaSQLFunction { public static final String YEAR = "YEAR"; public static final String FROM_UNIXTIME = "FROM_UNIXTIME"; + public static final String EXPLODE = "EXPLODE"; + public static final String SPILT = "SPILT"; + // -------------------------system functions---------------------------- public static final String COALESCE = "COALESCE"; public static final String IFNULL = "IFNULL"; @@ -193,6 +207,16 @@ public Object computeForValue(Expression expression, Object[] inputFields) { if (expression instanceof NullValue) { return null; } + if (expression instanceof TrimFunction) { + TrimFunction function = (TrimFunction) expression; + Column column = (Column) function.getExpression(); + List functionArgs = new ArrayList<>(); + if (column != null) { + functionArgs.add(computeForValue(column, inputFields)); + functionArgs.add(((StringValue) function.getFromExpression()).getValue()); + } + return executeFunctionExpr(TRIM, functionArgs); + } if (expression instanceof SignedExpression) { SignedExpression signedExpression = (SignedExpression) expression; if (signedExpression.getSign() == '-') { @@ -280,7 +304,8 @@ public Object computeForValue(Expression expression, Object[] inputFields) { } if (expression instanceof Function) { Function function = (Function) expression; - ExpressionList expressionList = function.getParameters(); + ExpressionList expressionList = + (ExpressionList) function.getParameters(); List functionArgs = new ArrayList<>(); if (expressionList != null) { for (Expression funcArgExpression : expressionList.getExpressions()) { @@ -551,12 +576,12 @@ public Object executeTimeKeyExpr(String timeKeyExpr) { } public Object executeCastExpr(CastExpression castExpression, Object arg) { - String dataType = castExpression.getType().getDataType(); + String dataType = castExpression.getColDataType().getDataType(); List args = new ArrayList<>(2); args.add(arg); args.add(dataType.toUpperCase()); if (dataType.equalsIgnoreCase("DECIMAL")) { - List ps = castExpression.getType().getArgumentsStringList(); + List ps = castExpression.getColDataType().getArgumentsStringList(); args.add(Integer.parseInt(ps.get(0))); args.add(Integer.parseInt(ps.get(1))); } @@ -665,4 +690,153 @@ private Object executeBinaryExpr(BinaryExpression binaryExpression, Object[] inp CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, String.format("Unsupported SQL Expression: %s ", binaryExpression)); } + + public List lateralView( + List seaTunnelRows, + List lateralViews, + SeaTunnelRowType outRowType) { + for (LateralView lateralView : lateralViews) { + Function function = lateralView.getGeneratorFunction(); + boolean isUsingOuter = lateralView.isUsingOuter(); + String functionName = function.getName(); + if (EXPLODE.equalsIgnoreCase(functionName)) { + seaTunnelRows = explode(seaTunnelRows, function, outRowType, isUsingOuter); + } else { + throw new SeaTunnelRuntimeException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Transform config error! UnSupport function:" + functionName); + } + } + + return seaTunnelRows; + } + + private List explode( + List seaTunnelRows, + Function lateralViewFunction, + SeaTunnelRowType outRowType, + boolean isUsingOuter) { + ExpressionList expressions = lateralViewFunction.getParameters(); + for (Expression expression : expressions) { + if (expression instanceof Column) { + String column = ((Column) expression).getColumnName(); + List next = new ArrayList<>(); + for (SeaTunnelRow row : seaTunnelRows) { + int fieldIndex = outRowType.indexOf(column); + Object splitFieldValue = row.getField(fieldIndex); + if (splitFieldValue == null) { + continue; + } + if (splitFieldValue instanceof Object[]) { + Object[] rowList = (Object[]) splitFieldValue; + if (ArrayUtils.isEmpty(rowList) && isUsingOuter) { + SeaTunnelRow outputRow = row.copy(); + outputRow.setField(fieldIndex, null); + next.add(outputRow); + } else { + for (Object fieldValue : rowList) { + SeaTunnelRow outputRow = row.copy(); + outputRow.setField(fieldIndex, fieldValue); + next.add(outputRow); + } + } + } + } + seaTunnelRows = next; + } + if (expression instanceof Function) { + Function function = (Function) expression; + String functionName = function.getName(); + if (SPILT.equalsIgnoreCase(functionName)) { + ExpressionList expressionList = function.getParameters(); + String column = ((Column) expressionList.get(0)).getColumnName(); + String delimiter = ((StringValue) expressionList.get(1)).getValue(); + List next = new ArrayList<>(); + for (SeaTunnelRow row : seaTunnelRows) { + + int fieldIndex = outRowType.indexOf(column); + Object splitFieldValue = row.getField(fieldIndex); + if (splitFieldValue == null) { + continue; + } + String[] splitFieldValues = splitFieldValue.toString().split(delimiter); + for (String fieldValue : splitFieldValues) { + SeaTunnelRow outputRow = row.copy(); + outputRow.setField(fieldIndex, fieldValue); + next.add(outputRow); + } + } + seaTunnelRows = next; + + } else { + throw new SeaTunnelRuntimeException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Transform config error! UnSupport function:" + functionName); + } + } + } + return seaTunnelRows; + } + + public SeaTunnelRowType lateralViewMapping( + String[] fieldNames, + SeaTunnelDataType[] seaTunnelDataTypes, + List lateralViews) { + if (CollectionUtils.isEmpty(lateralViews)) { + return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); + } + for (LateralView lateralView : lateralViews) { + Function function = lateralView.getGeneratorFunction(); + String functionName = function.getName(); + if (EXPLODE.equalsIgnoreCase(functionName)) { + explodeTypeMapping(fieldNames, seaTunnelDataTypes, function); + } else { + throw new SeaTunnelRuntimeException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Transform config error! UnSupport function:" + functionName); + } + } + + return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); + } + + private void explodeTypeMapping( + String[] fieldNames, + SeaTunnelDataType[] seaTunnelDataTypes, + Function lateralViewFunction) { + ExpressionList expressions = lateralViewFunction.getParameters(); + for (Expression expression : expressions) { + if (expression instanceof Column) { + String column = ((Column) expression).getColumnName(); + int columnIndex = Arrays.asList(fieldNames).indexOf(column); + if (columnIndex == -1) { + throw new TransformException( + INPUT_FIELDS_NOT_FOUND, + "Lateral view field must be in select item:" + fieldNames); + } + ArrayType arrayType = (ArrayType) seaTunnelDataTypes[columnIndex]; + seaTunnelDataTypes[columnIndex] = + PhysicalColumn.of(column, arrayType.getElementType(), 200, true, "", "") + .getDataType(); + } + if (expression instanceof Function) { + Function function = (Function) expression; + String functionName = function.getName(); + if (SPILT.equalsIgnoreCase(functionName)) { + ExpressionList expressionList = function.getParameters(); + String column = ((Column) expressionList.get(0)).getColumnName(); + int columnIndex = Arrays.asList(fieldNames).indexOf(column); + if (columnIndex == -1) { + throw new TransformException( + INPUT_FIELDS_NOT_FOUND, + "Lateral view field must be in select item:" + fieldNames); + } + } else { + throw new SeaTunnelRuntimeException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Transform config error! UnSupport function:" + functionName); + } + } + } + } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java index 3d9715561c7..f07f27ebbb9 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLType.java @@ -43,6 +43,7 @@ import net.sf.jsqlparser.expression.SignedExpression; import net.sf.jsqlparser.expression.StringValue; import net.sf.jsqlparser.expression.TimeKeyExpression; +import net.sf.jsqlparser.expression.TrimFunction; import net.sf.jsqlparser.expression.WhenClause; import net.sf.jsqlparser.expression.operators.arithmetic.Concat; import net.sf.jsqlparser.expression.operators.conditional.AndExpression; @@ -71,6 +72,7 @@ public class ZetaSQLType { public static final String LONG = "LONG"; public static final String BYTE = "BYTE"; public static final String BYTES = "BYTES"; + public static final String BINARY = "BINARY"; public static final String DOUBLE = "DOUBLE"; public static final String FLOAT = "FLOAT"; public static final String TIMESTAMP = "TIMESTAMP"; @@ -161,6 +163,9 @@ public SeaTunnelDataType getExpressionType(Expression expression) { if (expression instanceof Function) { return getFunctionType((Function) expression); } + if (expression instanceof TrimFunction) { + return BasicType.STRING_TYPE; + } if (expression instanceof TimeKeyExpression) { return getTimeKeyExprType((TimeKeyExpression) expression); } @@ -191,6 +196,7 @@ public SeaTunnelDataType getExpressionType(Expression expression) { if (expression instanceof CastExpression) { return getCastType((CastExpression) expression); } + if (expression instanceof BinaryExpression) { BinaryExpression binaryExpression = (BinaryExpression) expression; SeaTunnelDataType leftType = getExpressionType(binaryExpression.getLeftExpression()); @@ -314,10 +320,10 @@ private SeaTunnelDataType getCaseType(CaseExpression caseExpression) { } private SeaTunnelDataType getCastType(CastExpression castExpression) { - String dataType = castExpression.getType().getDataType(); + String dataType = castExpression.getColDataType().getDataType(); switch (dataType.toUpperCase()) { case DECIMAL: - List ps = castExpression.getType().getArgumentsStringList(); + List ps = castExpression.getColDataType().getArgumentsStringList(); return new DecimalType(Integer.parseInt(ps.get(0)), Integer.parseInt(ps.get(1))); case VARCHAR: case STRING: @@ -331,6 +337,7 @@ private SeaTunnelDataType getCastType(CastExpression castExpression) { case BYTE: return BasicType.BYTE_TYPE; case BYTES: + case BINARY: return PrimitiveByteArrayType.INSTANCE; case DOUBLE: return BasicType.DOUBLE_TYPE; diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java index 0b616b0fbe8..75584a92dea 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java @@ -100,6 +100,7 @@ public static Object castAs(List args) { case "BYTE": return Byte.parseByte(v1.toString()); case "BYTES": + case "BINARY": return v1.toString().getBytes(StandardCharsets.UTF_8); case "DOUBLE": return Double.parseDouble(v1.toString()); diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java index ff253eac210..fcf14cc7b9d 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java @@ -36,6 +36,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Objects; public class SQLTransformTest { @@ -168,11 +169,11 @@ public void testEscapeIdentifier() { "select id, trim(`apply`) as `apply` from test where `apply` = 'a'")); SQLTransform sqlTransform = new SQLTransform(config, table); TableSchema tableSchema = sqlTransform.transformTableSchema(); - SeaTunnelRow result = + List result = sqlTransform.transformRow( new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); - Assertions.assertEquals("a", result.getField(1)); + Assertions.assertEquals("a", result.get(0).getField(1)); result = sqlTransform.transformRow( new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("b")})); @@ -191,7 +192,7 @@ public void testEscapeIdentifier() { Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); Assertions.assertEquals( BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); - Assertions.assertEquals("a", result.getField(1)); + Assertions.assertEquals("a", result.get(0).getField(1)); table = CatalogTableUtil.getCatalogTable( @@ -211,7 +212,7 @@ public void testEscapeIdentifier() { new SeaTunnelRow(new Object[] {Integer.valueOf(1), Long.valueOf(1)})); Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); Assertions.assertEquals(BasicType.LONG_TYPE, tableSchema.getColumns().get(1).getDataType()); - Assertions.assertEquals(Long.valueOf(2), result.getField(1)); + Assertions.assertEquals(Long.valueOf(2), result.get(0).getField(1)); result = sqlTransform.transformRow( new SeaTunnelRow(new Object[] {Integer.valueOf(1), Long.valueOf(0)})); @@ -243,7 +244,7 @@ public void testEscapeIdentifier() { Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); Assertions.assertEquals( BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); - Assertions.assertEquals("a", result.getField(1)); + Assertions.assertEquals("a", result.get(0).getField(1)); result = sqlTransform.transformRow( new SeaTunnelRow( @@ -278,6 +279,6 @@ public void testEscapeIdentifier() { Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); Assertions.assertEquals( BasicType.STRING_TYPE, tableSchema.getColumns().get(1).getDataType()); - Assertions.assertEquals("a", result.getField(1)); + Assertions.assertEquals("a", result.get(0).getField(1)); } } diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/zeta/DateTimeFunctionTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/zeta/DateTimeFunctionTest.java index 70b744a04c1..c0621646084 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/zeta/DateTimeFunctionTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/zeta/DateTimeFunctionTest.java @@ -48,7 +48,7 @@ public void testFromUnixtimeFunction() { null, rowType, "select from_unixtime(unixtime,'yyyy-MM-dd') as ts from test"); - SeaTunnelRow outRow = sqlEngine.transformBySQL(inputRow); + SeaTunnelRow outRow = sqlEngine.transformBySQL(inputRow, rowType).get(0); Object field = outRow.getField(0); Assertions.assertNotNull(field.toString()); @@ -58,7 +58,7 @@ public void testFromUnixtimeFunction() { null, rowType, "select from_unixtime(unixtime,'yyyy-MM-dd HH:mm:ss','UTC+6') as ts from test"); - SeaTunnelRow outRow1 = sqlEngine.transformBySQL(inputRow); + SeaTunnelRow outRow1 = sqlEngine.transformBySQL(inputRow, rowType).get(0); Object field1 = outRow1.getField(0); Assertions.assertEquals("2023-01-01 10:00:00", field1.toString()); } From 445ccc176af02acc313f911e955de30180011809 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 1 Nov 2024 13:38:39 +0800 Subject: [PATCH 09/29] [Feature][transform] sql transform support lateral view explode --- tools/dependencies/known-dependencies.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 1f49332ef92..b2012936bd2 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -30,7 +30,7 @@ seatunnel-jackson-2.3.9-SNAPSHOT-optional.jar seatunnel-guava-2.3.9-SNAPSHOT-optional.jar seatunnel-hazelcast-shade-2.3.9-SNAPSHOT-optional.jar slf4j-api-1.7.25.jar -jsqlparser-4.5.jar +jsqlparser-4.9.jar animal-sniffer-annotations-1.17.jar checker-qual-3.10.0.jar error_prone_annotations-2.2.0.jar From 83f2ca746c056464d703b594e01a3f623fbb54b2 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 1 Nov 2024 15:10:53 +0800 Subject: [PATCH 10/29] [Feature][transform] sql transform support lateral view explode --- .../org/apache/seatunnel/transform/sql/SQLTransformTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java index fcf14cc7b9d..aa6823f0733 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java @@ -166,12 +166,12 @@ public void testEscapeIdentifier() { ReadonlyConfig.fromMap( Collections.singletonMap( "query", - "select id, trim(`apply`) as `apply` from test where `apply` = 'a'")); + "select id, trim(`apply`,'*') as `apply` from test where `apply` = 'a*'")); SQLTransform sqlTransform = new SQLTransform(config, table); TableSchema tableSchema = sqlTransform.transformTableSchema(); List result = sqlTransform.transformRow( - new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a*")})); Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); Assertions.assertEquals("a", result.get(0).getField(1)); result = From 5a1d277391d3ab8b48d896e6ef9d7d624fa94a81 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 5 Nov 2024 09:36:49 +0800 Subject: [PATCH 11/29] [Feature][transform] sql transform support lateral view explode --- docs/en/transform-v2/explode.md | 74 ------ docs/en/transform-v2/sql-functions.md | 13 ++ docs/zh/transform-v2/explode.md | 75 ------- docs/zh/transform-v2/sql-functions.md | 12 + plugin-mapping.properties | 1 - .../e2e/transform/TestExplodeIT.java | 37 --- .../seatunnel/e2e/transform/TestSQLIT.java | 4 + .../sql_transform}/explode_transform.conf | 0 .../transform/explode/ExplodeTransform.java | 212 ------------------ .../explode/ExplodeTransformConfig.java | 38 ---- .../explode/ExplodeTransformFactory.java | 49 ---- 11 files changed, 29 insertions(+), 486 deletions(-) delete mode 100644 docs/en/transform-v2/explode.md delete mode 100644 docs/zh/transform-v2/explode.md delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java rename seatunnel-e2e/seatunnel-transforms-v2-e2e/{seatunnel-transforms-v2-e2e-part-1/src/test/resources => seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform}/explode_transform.conf (100%) delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java delete mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java diff --git a/docs/en/transform-v2/explode.md b/docs/en/transform-v2/explode.md deleted file mode 100644 index 1e20aa9d878..00000000000 --- a/docs/en/transform-v2/explode.md +++ /dev/null @@ -1,74 +0,0 @@ -# Explode - -> Explode transform plugin - -## Description - -Split a row of data into multiple data according to the specified delimiter. - -## Options - -| name | type | required | Description | -|-----------------------|------|----------|----------------------| -| explode_string_fields | Map | yes | Explode String field | -| explode_list_fields | List | yes | Explode List field | - - -### explode_string_fields [Map] - -The fields that need to be separated by the specified delimiter -The map is composed of field as key and separator as value. - -### explode_list_fields [List] - -The List fields that need to be separated by the specified delimiter - - -### common options [string] - -Transform plugin common parameters, please refer to [Transform Plugin](common-options.md) for details - -## Example - -The data read from source is a table like this: - -| name | age | card | -|-------------------|-----|-----------| -| Joy Ding,May Ding | 20 | [123,234] | -| Kin Dom,Joy Dom | 20 | [123,345] | - -we want to explode name and card, we can add a `Explode` Transform like below: - -``` -transform { - Explode { - source_table_name = "fake" - result_table_name = "fake1" - explode_string_fields = {"name":","} - explode_list_fields = ["card"] - } -} -``` - - -It is useful when you want to explode row. - -Then the data in result table `fake1` will like this - -| name | age | card | -|----------|-----|------| -| Joy Ding | 20 | 123 | -| Joy Ding | 20 | 234 | -| May Ding | 20 | 123 | -| May Ding | 20 | 234 | -| Kin Dom | 20 | 123 | -| Kin Dom | 20 | 345 | -| Joy Dom | 20 | 123 | -| Joy Dom | 20 | 345 | - -## Changelog - -### new version - -- Add Explode Transform Connector - diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index ce01df937fc..ed22213dafb 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -984,3 +984,16 @@ Example: select UUID() as seatunnel_uuid + +### LATERAL VIEW +#### EXPLODE + +explode array column to rows. +OUTER EXPLODE will return NULL, while array is NULL or empty + +``` +SELECT * FROM fake + LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME + LATERAL VIEW EXPLODE ( SPILT ( pk_id, ';' ) ) AS pk_id + LATERAL VIEW OUTER EXPLODE ( age ) AS age +``` diff --git a/docs/zh/transform-v2/explode.md b/docs/zh/transform-v2/explode.md deleted file mode 100644 index 71509ae196e..00000000000 --- a/docs/zh/transform-v2/explode.md +++ /dev/null @@ -1,75 +0,0 @@ -# Explode - -> Explode transform plugin - -## Description - -将一行数据按照指定的分割符拆分成多条数据。 - -## Options - -| name | type | required | -|----------------|------|----------| -| explode_string_fields | Map | yes | -| explode_list_fields | List | yes | - - -### explode_string_fields [Map] - -需要按照指定分隔符分割的字段。 -map是由field作为key,分隔符作为value。 - - -### explode_list_fields [List] - -需要按照指定分隔符分割的字段。 -List为需要切分的字段。 - - -### common options [config] - -转换插件的常见参数, 请参考 [Transform Plugin](common-options.md) 了解详情 - -## 示例 - -源端数据读取的表格如下: - -The data read from source is a table like this: - -| name | age | card | -|-------------------|-----|---------| -| Joy Ding,May Ding | 20 | [123,234] | -| Kin Dom,Joy Dom | 20 | [123,345] | - -当我们想根据name和card进行数据拆分: - -``` -transform { - Explode { - source_table_name = "fake" - result_table_name = "fake1" - explode_string_fields = {"name":","} - explode_list_fields = ["card"] - } -} -``` - -那么结果表 `fake1` 中的数据将会像这样: - -| name | age | card | -|----------|-----|------| -| Joy Ding | 20 | 123 | -| Joy Ding | 20 | 234 | -| May Ding | 20 | 123 | -| May Ding | 20 | 234 | -| Kin Dom | 20 | 123 | -| Kin Dom | 20 | 345 | -| Joy Dom | 20 | 123 | -| Joy Dom | 20 | 345 | - -## Changelog - -### new version - -- Add Explode Transform Connector - diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index 13dc3a9bc54..24f2feb3e83 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -975,3 +975,15 @@ case when c_string in ('c_string') then 1 else 0 end select UUID() as seatunnel_uuid +### LATERAL VIEW +#### EXPLODE + +将 array 列展开成多行。 +OUTER EXPLODE 当 array 为NULL或者为空时,返回NULL + +``` +SELECT * FROM fake + LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME + LATERAL VIEW EXPLODE ( SPILT ( pk_id, ';' ) ) AS pk_id + LATERAL VIEW OUTER EXPLODE ( age ) AS age +``` diff --git a/plugin-mapping.properties b/plugin-mapping.properties index ef10c340243..82c941b70f6 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -154,4 +154,3 @@ seatunnel.transform.DynamicCompile = seatunnel-transforms-v2 seatunnel.transform.LLM = seatunnel-transforms-v2 seatunnel.transform.Embedding = seatunnel-transforms-v2 seatunnel.transform.RowKindExtractor = seatunnel-transforms-v2 -seatunnel.transform.Explode = seatunnel-transforms-v2 diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java deleted file mode 100644 index 4658603b514..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/java/org/apache/seatunnel/e2e/transform/TestExplodeIT.java +++ /dev/null @@ -1,37 +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.e2e.transform; - -import org.apache.seatunnel.e2e.common.container.TestContainer; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.TestTemplate; -import org.testcontainers.containers.Container; - -import java.io.IOException; - -public class TestExplodeIT extends TestSuiteBase { - - @TestTemplate - public void testExplode(TestContainer container) throws IOException, InterruptedException { - Container.ExecResult execResult = container.executeJob("/explode_transform.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - - Container.ExecResult execResultBySql = container.executeJob("/explode_transform_sql.conf"); - Assertions.assertEquals(0, execResultBySql.getExitCode()); - } -} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java index df404a28525..0a8deb6ccfc 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java @@ -59,6 +59,10 @@ public void testSQLTransform(TestContainer container) throws IOException, Interr Assertions.assertEquals(0, sqlAllColumns.getExitCode()); Container.ExecResult caseWhenSql = container.executeJob("/sql_transform/case_when.conf"); Assertions.assertEquals(0, caseWhenSql.getExitCode()); + + Container.ExecResult execResultBySql = + container.executeJob("/sql_transform/explode_transform_sql.conf"); + Assertions.assertEquals(0, execResultBySql.getExitCode()); } @TestTemplate diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf similarity index 100% rename from seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform.conf rename to seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java deleted file mode 100644 index a9279fa6086..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransform.java +++ /dev/null @@ -1,212 +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.transform.explode; - -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; -import org.apache.seatunnel.api.table.catalog.PhysicalColumn; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; -import org.apache.seatunnel.api.table.catalog.TableSchema; -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.transform.common.AbstractCatalogMultiRowTransform; -import org.apache.seatunnel.transform.common.CommonOptions; -import org.apache.seatunnel.transform.exception.TransformCommonError; -import org.apache.seatunnel.transform.replace.ReplaceTransformConfig; - -import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.collections4.MapUtils; - -import com.google.common.collect.Lists; -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - -@Slf4j -public class ExplodeTransform extends AbstractCatalogMultiRowTransform { - protected CatalogTable inputCatalogTable; - public static final String PLUGIN_NAME = "Explode"; - protected Map explodeStringFields; - protected List explodeListFields; - protected SeaTunnelRowType seaTunnelRowType; - private int[] fieldsIndex; - private ReadonlyConfig config; - - public ExplodeTransform(@NonNull ReadonlyConfig config, @NonNull CatalogTable catalogTable) { - super(catalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); - this.config = config; - this.explodeStringFields = config.get(ExplodeTransformConfig.EXPLODE_STRING_FIELDS); - this.explodeListFields = config.get(ExplodeTransformConfig.EXPLODE_LIST_FIELDS); - this.seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); - this.inputCatalogTable = catalogTable; - } - - /** - * Outputs transformed row data. - * - * @param inputRow upstream input row data - */ - protected List transformRow(SeaTunnelRow inputRow) { - List rows = Lists.newArrayList(inputRow); - if (MapUtils.isNotEmpty(explodeStringFields)) { - for (Map.Entry entry : explodeStringFields.entrySet()) { - List next = new ArrayList<>(); - for (SeaTunnelRow row : rows) { - - String field = entry.getKey(); - int fieldIndex = seaTunnelRowType.indexOf(field); - Object splitFieldValue = inputRow.getField(fieldIndex); - if (splitFieldValue == null) { - continue; - } - String separator = entry.getValue(); - String[] splitFieldValues = splitFieldValue.toString().split(separator); - for (String fieldValue : splitFieldValues) { - SeaTunnelRow outputRow = row.copy(); - outputRow.setField(fieldIndex, fieldValue); - next.add(outputRow); - } - } - rows = next; - } - } - if (!CollectionUtils.isEmpty(explodeListFields)) { - for (String field : explodeListFields) { - List next = new ArrayList<>(); - for (SeaTunnelRow row : rows) { - int fieldIndex = seaTunnelRowType.indexOf(field); - Object splitFieldValue = inputRow.getField(fieldIndex); - if (splitFieldValue == null) { - continue; - } - if (splitFieldValue instanceof Object[]) { - Object[] rowList = (Object[]) splitFieldValue; - for (Object fieldValue : rowList) { - SeaTunnelRow outputRow = row.copy(); - outputRow.setField(fieldIndex, fieldValue); - next.add(outputRow); - } - } - } - rows = next; - } - } - - return rows; - } - - @Override - public String getPluginName() { - return PLUGIN_NAME; - } - - @Override - protected TableSchema transformTableSchema() { - Column[] outputColumns = getOutputColumns(); - - List copiedConstraintKeys = - inputCatalogTable.getTableSchema().getConstraintKeys().stream() - .map(ConstraintKey::copy) - .collect(Collectors.toList()); - - TableSchema.Builder builder = TableSchema.builder(); - if (inputCatalogTable.getTableSchema().getPrimaryKey() != null) { - builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy()); - } - builder.constraintKey(copiedConstraintKeys); - List columns = - inputCatalogTable.getTableSchema().getColumns().stream() - .map(Column::copy) - .collect(Collectors.toList()); - - int addFieldCount = 0; - this.fieldsIndex = new int[outputColumns.length]; - for (int i = 0; i < outputColumns.length; i++) { - Column outputColumn = outputColumns[i]; - Optional optional = - columns.stream() - .filter(c -> c.getName().equals(outputColumn.getName())) - .findFirst(); - if (optional.isPresent()) { - Column originalColumn = optional.get(); - int originalColumnIndex = columns.indexOf(originalColumn); - if (!originalColumn.getDataType().equals(outputColumn.getDataType())) { - columns.set( - originalColumnIndex, originalColumn.copy(outputColumn.getDataType())); - } - fieldsIndex[i] = originalColumnIndex; - } else { - addFieldCount++; - columns.add(outputColumn); - fieldsIndex[i] = columns.indexOf(outputColumn); - } - } - - TableSchema outputTableSchema = builder.columns(columns).build(); - log.info( - "Changed input table schema: {} to output table schema: {}", - inputCatalogTable.getTableSchema(), - outputTableSchema); - - return outputTableSchema; - } - - @Override - protected TableIdentifier transformTableIdentifier() { - return inputCatalogTable.getTableId().copy(); - } - - protected Column[] getOutputColumns() { - List columns = inputCatalogTable.getTableSchema().getColumns(); - - List collect = - columns.stream() - .filter( - column -> - explodeStringFields.containsKey(column.getName()) - || explodeListFields.contains(column.getName())) - .map( - column -> { - if (explodeListFields.contains(column.getName())) { - ArrayType arrayType = (ArrayType) column.getDataType(); - return PhysicalColumn.of( - column.getName(), - arrayType.getElementType(), - 200, - true, - "", - ""); - } - return column; - }) - .collect(Collectors.toList()); - - if (CollectionUtils.isEmpty(collect)) { - throw TransformCommonError.cannotFindInputFieldError( - getPluginName(), config.get(ReplaceTransformConfig.KEY_REPLACE_FIELD)); - } - return collect.toArray(new Column[0]); - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java deleted file mode 100644 index f3844b3d6df..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformConfig.java +++ /dev/null @@ -1,38 +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.transform.explode; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; - -public class ExplodeTransformConfig implements Serializable { - public static final Option> EXPLODE_STRING_FIELDS = - Options.key("explode_string_fields") - .mapType() - .noDefaultValue() - .withDescription("The map of string fields that need to be explode."); - - public static final Option> EXPLODE_LIST_FIELDS = - Options.key("explode_list_fields") - .listType() - .noDefaultValue() - .withDescription("The map of list fields that need to be explode."); -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java deleted file mode 100644 index 653f5601c37..00000000000 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/explode/ExplodeTransformFactory.java +++ /dev/null @@ -1,49 +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.transform.explode; - -import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.connector.TableTransform; -import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableTransformFactory; -import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext; - -import com.google.auto.service.AutoService; - -import static org.apache.seatunnel.transform.explode.ExplodeTransform.PLUGIN_NAME; - -@AutoService(Factory.class) -public class ExplodeTransformFactory implements TableTransformFactory { - - @Override - public String factoryIdentifier() { - return PLUGIN_NAME; - } - - @Override - public OptionRule optionRule() { - - return OptionRule.builder().build(); - } - - @Override - public TableTransform createTransform(TableTransformFactoryContext context) { - CatalogTable catalogTable = context.getCatalogTables().get(0); - return () -> new ExplodeTransform(context.getOptions(), catalogTable); - } -} From db7b403446a7f2a171df571584ae31d047720e51 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 5 Nov 2024 09:41:00 +0800 Subject: [PATCH 12/29] [Feature][transform] sql transform support lateral view explode --- .../explode/ExplodeTransformTest.java | 209 ------------------ 1 file changed, 209 deletions(-) delete mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java deleted file mode 100644 index 2568810a00f..00000000000 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/explode/ExplodeTransformTest.java +++ /dev/null @@ -1,209 +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.transform.explode; - -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.PhysicalColumn; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; -import org.apache.seatunnel.api.table.catalog.TablePath; -import org.apache.seatunnel.api.table.catalog.TableSchema; -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import com.google.common.collect.Lists; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; - -class ExplodeTransformTest { - - static Map explodeFields = new HashMap<>(); - static List explodeListFields = new ArrayList<>(); - static CatalogTable catalogTable; - static Object[] values; - - @BeforeAll - static void setUp() { - - explodeFields.put("key1", ","); - explodeFields.put("key2", ";"); - explodeListFields.add("key3"); - - catalogTable = - CatalogTable.of( - TableIdentifier.of("catalog", TablePath.DEFAULT), - TableSchema.builder() - .column( - PhysicalColumn.of( - "key1", - BasicType.STRING_TYPE, - 1L, - Boolean.FALSE, - null, - null)) - .column( - PhysicalColumn.of( - "key2", - BasicType.STRING_TYPE, - 1L, - Boolean.FALSE, - null, - null)) - .column( - PhysicalColumn.of( - "key3", - ArrayType.STRING_ARRAY_TYPE, - 1L, - Boolean.FALSE, - null, - null)) - .column( - PhysicalColumn.of( - "key4", - BasicType.STRING_TYPE, - 1L, - Boolean.FALSE, - null, - null)) - .column( - PhysicalColumn.of( - "key5", - BasicType.STRING_TYPE, - 1L, - Boolean.FALSE, - null, - null)) - .build(), - new HashMap<>(), - new ArrayList<>(), - "comment"); - String[] key3 = new String[2]; - key3[0] = "value5"; - key3[1] = "value6"; - values = new Object[] {"value1,value2", "value3;value4", key3, "value7", "value8"}; - } - - @Test - void testConfig() { - // test both not set - try { - new ExplodeTransform(ReadonlyConfig.fromMap(new HashMap<>()), catalogTable); - } catch (Exception e) { - assertEquals( - "ErrorCode:[API-02], ErrorDescription:[Option item validate failed] - There are unconfigured options, these options('include_fields', 'exclude_fields') are mutually exclusive, allowing only one set(\"[] for a set\") of options to be configured.", - e.getMessage()); - } - - // test both include and exclude set - try { - new ExplodeTransform( - ReadonlyConfig.fromMap( - new HashMap() { - { - put( - ExplodeTransformConfig.EXPLODE_STRING_FIELDS.key(), - explodeFields); - put( - ExplodeTransformConfig.EXPLODE_LIST_FIELDS.key(), - explodeListFields); - } - }), - catalogTable); - } catch (Exception e) { - assertEquals( - "ErrorCode:[API-02], ErrorDescription:[Option item validate failed] - These options('include_fields', 'exclude_fields') are mutually exclusive, allowing only one set(\"[] for a set\") of options to be configured.", - e.getMessage()); - } - - // not exception should be thrown now - new ExplodeTransform( - ReadonlyConfig.fromMap( - new HashMap() { - { - put( - ExplodeTransformConfig.EXPLODE_STRING_FIELDS.key(), - explodeFields); - put( - ExplodeTransformConfig.EXPLODE_LIST_FIELDS.key(), - explodeListFields); - } - }), - catalogTable); - } - - @Test - void testExplode() { - // default include - Map configMap = new HashMap<>(); - configMap.put(ExplodeTransformConfig.EXPLODE_STRING_FIELDS.key(), explodeFields); - configMap.put(ExplodeTransformConfig.EXPLODE_LIST_FIELDS.key(), explodeListFields); - - ExplodeTransform explodeTransform = - new ExplodeTransform(ReadonlyConfig.fromMap(configMap), catalogTable); - - // test output schema - TableSchema resultSchema = explodeTransform.transformTableSchema(); - Assertions.assertNotNull(resultSchema); - - // test output row - SeaTunnelRow input = new SeaTunnelRow(values); - List output = explodeTransform.transformRow(input); - Assertions.assertNotNull(output); - List result = - Lists.newArrayList( - new Object[] {"value1", "value3", "value5", "value7", "value8"}, - new Object[] {"value1", "value3", "value6", "value7", "value8"}, - new Object[] {"value1", "value4", "value5", "value7", "value8"}, - new Object[] {"value1", "value4", "value6", "value7", "value8"}, - new Object[] {"value2", "value3", "value5", "value7", "value8"}, - new Object[] {"value2", "value3", "value6", "value7", "value8"}, - new Object[] {"value2", "value4", "value5", "value7", "value8"}, - new Object[] {"value2", "value4", "value6", "value7", "value8"}); - - List outputValues = - output.stream().map(SeaTunnelRow::getFields).collect(Collectors.toList()); - assertEquals(outputValues.size(), result.size()); - for (int i = 0; i < result.size(); i++) { - assertTrue(arraysEqual(result.get(i), outputValues.get(i))); - } - } - - private boolean arraysEqual(Object[] array1, Object[] array2) { - if (array1.length != array2.length) { - return false; - } - for (int i = 0; i < array1.length; i++) { - if (!array1[i].equals(array2[i])) { - return false; - } - } - return true; - } -} From caea9700a5aeecf4ad3f646a33722c877396932f Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 5 Nov 2024 09:54:34 +0800 Subject: [PATCH 13/29] [Feature][transform] sql transform support lateral view explode --- .../ExplodeTransformFactoryTest.java | 30 ------------------- 1 file changed, 30 deletions(-) delete mode 100644 seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java deleted file mode 100644 index 17e2c240b50..00000000000 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/ExplodeTransformFactoryTest.java +++ /dev/null @@ -1,30 +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.transform; - -import org.apache.seatunnel.transform.explode.ExplodeTransformFactory; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class ExplodeTransformFactoryTest { - @Test - public void testOptionRule() throws Exception { - ExplodeTransformFactory explodeTransformFactory = new ExplodeTransformFactory(); - Assertions.assertNotNull(explodeTransformFactory.optionRule()); - } -} From cb0f56d516c300d89cdccee80157d0928c267836 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 5 Nov 2024 13:43:19 +0800 Subject: [PATCH 14/29] [Feature][transform] sql transform support lateral view explode --- .../test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java index 0a8deb6ccfc..beec110e0ac 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java @@ -61,7 +61,7 @@ public void testSQLTransform(TestContainer container) throws IOException, Interr Assertions.assertEquals(0, caseWhenSql.getExitCode()); Container.ExecResult execResultBySql = - container.executeJob("/sql_transform/explode_transform_sql.conf"); + container.executeJob("/sql_transform/explode_transform.conf"); Assertions.assertEquals(0, execResultBySql.getExitCode()); } From 4bfa386d79283d5eb2470a3f66d11ecfad5a8c97 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Tue, 5 Nov 2024 13:43:36 +0800 Subject: [PATCH 15/29] [Feature][transform] sql transform support lateral view explode --- .../src/test/resources/sql_transform/explode_transform.conf | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf index c1fbf6b0ca3..3215a50b82b 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf @@ -55,11 +55,10 @@ source { } transform { - Explode { + Sql { source_table_name = "fake" result_table_name = "fake1" - explode_string_fields = {"pk_id":";","name":","} - explode_list_fields = ["age"] + query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPILT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPILT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age" } } From 46125fed515f52d13e40d36f3f3aa0bf9efed35d Mon Sep 17 00:00:00 2001 From: CosmosNi <627165587@qq.com> Date: Tue, 5 Nov 2024 21:21:15 +0800 Subject: [PATCH 16/29] [Feature][transform] fix cr --- docs/en/transform-v2/sql-functions.md | 1 + docs/zh/transform-v2/sql-functions.md | 1 + .../test/resources/explode_transform_sql.conf | 98 ------------------- .../transform/sql/zeta/ZetaSQLFunction.java | 4 - 4 files changed, 2 insertions(+), 102 deletions(-) delete mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index ed22213dafb..e29defc5459 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -990,6 +990,7 @@ select UUID() as seatunnel_uuid explode array column to rows. OUTER EXPLODE will return NULL, while array is NULL or empty +EXPLODE(SPILT(FIELD_NAME,separator))Used to split string type. The first parameter of SPILT function is the field name, the second parameter is the separator ``` SELECT * FROM fake diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index 24f2feb3e83..0a4ca5a5b2b 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -980,6 +980,7 @@ select UUID() as seatunnel_uuid 将 array 列展开成多行。 OUTER EXPLODE 当 array 为NULL或者为空时,返回NULL +EXPLODE(SPILT(FIELD_NAME,separator))用来切分字符串类型,SPILT 第一个参数是字段名,第二个参数是分隔符 ``` SELECT * FROM fake diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf deleted file mode 100644 index 3215a50b82b..00000000000 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/explode_transform_sql.conf +++ /dev/null @@ -1,98 +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. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - job.mode = "BATCH" - parallelism = 1 -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - pk_id = string - name = string - age = array - } - primaryKey { - name = "pk_id" - columnNames = [pk_id] - } - } - rows = [ - { - kind = INSERT - fields = ["id001;id001", "zhangsan,zhangsan",["1","1"]] - }, - { - kind = INSERT - fields = ["id001", "zhangsan,zhangsan",["1"]] - }, - { - kind = INSERT - fields = ["id001;id001", "zhangsan",["1"]] - } - ] - } -} - -transform { - Sql { - source_table_name = "fake" - result_table_name = "fake1" - query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPILT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPILT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age" - } -} - -sink{ - assert { - rules = - { - row_rules = [ - { - rule_type = MAX_ROW - rule_value = 12 - }, - { - rule_type = MIN_ROW - rule_value = 12 - } - ], - field_rules = [ - { - field_name = pk_id - field_type = string - field_value = [{equals_to = id001}] - }, - { - field_name = name - field_type = string - field_value = [{equals_to = zhangsan}] - }, - { - field_name = age - field_type = string - field_value = [{equals_to = 1}] - } - ] - } - } -} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 79efbcfb506..80e34f55621 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -33,7 +33,6 @@ import org.apache.seatunnel.transform.sql.zeta.functions.StringFunction; import org.apache.seatunnel.transform.sql.zeta.functions.SystemFunction; -import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.tuple.Pair; @@ -782,9 +781,6 @@ public SeaTunnelRowType lateralViewMapping( String[] fieldNames, SeaTunnelDataType[] seaTunnelDataTypes, List lateralViews) { - if (CollectionUtils.isEmpty(lateralViews)) { - return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); - } for (LateralView lateralView : lateralViews) { Function function = lateralView.getGeneratorFunction(); String functionName = function.getName(); From 2c440d57f4e5e4fa0edcc8140e127eb2b49bc7fd Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 09:16:51 +0800 Subject: [PATCH 17/29] [Feature][transform] sql transform support lateral view explode --- .../org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java index b72d239b7d2..8fcef473b9a 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java @@ -216,6 +216,9 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { idx++; } } + if (CollectionUtils.isEmpty(selectBody.getLateralViews())) { + return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); + } return zetaSQLFunction.lateralViewMapping( fieldNames, seaTunnelDataTypes, selectBody.getLateralViews()); } From 88d49ca64bf2590fadacea60cbd33a730dcefac0 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 09:21:05 +0800 Subject: [PATCH 18/29] [Feature][transform] sql transform support lateral view explode --- .../apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java index 8fcef473b9a..339793cb27f 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java @@ -216,11 +216,11 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { idx++; } } - if (CollectionUtils.isEmpty(selectBody.getLateralViews())) { + List lateralViews = selectBody.getLateralViews(); + if (CollectionUtils.isEmpty(lateralViews)) { return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); } - return zetaSQLFunction.lateralViewMapping( - fieldNames, seaTunnelDataTypes, selectBody.getLateralViews()); + return zetaSQLFunction.lateralViewMapping(fieldNames, seaTunnelDataTypes, lateralViews); } @Override From 873a95154e6e086aa9da42b6aeb421258b151d10 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 16:25:34 +0800 Subject: [PATCH 19/29] [Feature][transform] sql transform support lateral view explode --- docs/en/transform-v2/sql-functions.md | 3 +- docs/zh/transform-v2/sql-functions.md | 2 +- .../sql_transform/explode_transform.conf | 11 +- .../resources/sql_transform/func_system.conf | 2 +- .../transform/sql/zeta/ZetaSQLEngine.java | 3 +- .../transform/sql/zeta/ZetaSQLFunction.java | 172 ++++++++++-------- .../sql/zeta/functions/StringFunction.java | 12 ++ .../sql/zeta/functions/SystemFunction.java | 11 ++ 8 files changed, 130 insertions(+), 86 deletions(-) diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index e29defc5459..2dcc1e421c0 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -991,10 +991,11 @@ select UUID() as seatunnel_uuid explode array column to rows. OUTER EXPLODE will return NULL, while array is NULL or empty EXPLODE(SPILT(FIELD_NAME,separator))Used to split string type. The first parameter of SPILT function is the field name, the second parameter is the separator - +EXPLODE(ARRAY(value1,value2)) Used to custom array type. ``` SELECT * FROM fake LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME LATERAL VIEW EXPLODE ( SPILT ( pk_id, ';' ) ) AS pk_id LATERAL VIEW OUTER EXPLODE ( age ) AS age + LATERAL VIEW OUTER EXPLODE ( ARRAY(1,1) ) AS num ``` diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index 0a4ca5a5b2b..2cc2cc6d749 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -981,7 +981,7 @@ select UUID() as seatunnel_uuid 将 array 列展开成多行。 OUTER EXPLODE 当 array 为NULL或者为空时,返回NULL EXPLODE(SPILT(FIELD_NAME,separator))用来切分字符串类型,SPILT 第一个参数是字段名,第二个参数是分隔符 - +EXPLODE(ARRAY(value1,value2)) 用于自定义数组切分,在原有基础上生成一个新的字段。 ``` SELECT * FROM fake LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf index 3215a50b82b..9f0614ede13 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf @@ -58,7 +58,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPILT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPILT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age" + query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPILT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPILT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age LATERAL VIEW EXPLODE(ARRAY(1,1)) as num" } } @@ -69,11 +69,11 @@ sink{ row_rules = [ { rule_type = MAX_ROW - rule_value = 12 + rule_value = 24 }, { rule_type = MIN_ROW - rule_value = 12 + rule_value = 24 } ], field_rules = [ @@ -91,6 +91,11 @@ sink{ field_name = age field_type = string field_value = [{equals_to = 1}] + }, + { + field_name = num + field_type = string + field_value = [{equals_to = 1}] } ] } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf index 8afb18e6d8c..c3c97f7f10a 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/func_system.conf @@ -49,7 +49,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as BINARY) as c7, name as apply from fake" + query = "select cast(id as STRING) as id, cast(id as INT) as id2, cast(id as DOUBLE) as id3 , cast(c1 as double) as c1_1, cast(c1 as DECIMAL(10,2)) as c1_2, cast(c2 as DATE) as c2_1, coalesce(c3,'Unknown') c3_1, ifnull(c3,'Unknown') c3_2, ifnull(nullif(name,'Joy Ding'),'NULL') name1, nullif(name,'Joy Ding_') name2, cast(c4 as timestamp) as c4_1, cast(c4 as decimal(17,4)) as c4_2, cast(c5 as date) as c5, cast(c6 as time) as c6, cast(name as BINARY) as c7, name as `apply` from fake" } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java index 339793cb27f..e701e5bcbc9 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLEngine.java @@ -220,7 +220,8 @@ public SeaTunnelRowType typeMapping(List inputColumnsMapping) { if (CollectionUtils.isEmpty(lateralViews)) { return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); } - return zetaSQLFunction.lateralViewMapping(fieldNames, seaTunnelDataTypes, lateralViews); + return zetaSQLFunction.lateralViewMapping( + fieldNames, seaTunnelDataTypes, lateralViews, inputColumnsMapping); } @Override diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 80e34f55621..e6005e7f296 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.table.catalog.PhysicalColumn; import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.MapType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -114,6 +115,7 @@ public class ZetaSQLFunction { public static final String SUBSTR = "SUBSTR"; public static final String TO_CHAR = "TO_CHAR"; public static final String TRANSLATE = "TRANSLATE"; + public static final String SPILT = "SPILT"; // -------------------------numeric functions---------------------------- public static final String ABS = "ABS"; @@ -178,8 +180,9 @@ public class ZetaSQLFunction { public static final String YEAR = "YEAR"; public static final String FROM_UNIXTIME = "FROM_UNIXTIME"; + // -------------------------lateralView functions---------------------------- public static final String EXPLODE = "EXPLODE"; - public static final String SPILT = "SPILT"; + public static final String ARRAY = "ARRAY"; // -------------------------system functions---------------------------- public static final String COALESCE = "COALESCE"; @@ -436,6 +439,8 @@ public Object executeFunctionExpr(String functionName, List args) { return StringFunction.toChar(args); case TRANSLATE: return StringFunction.translate(args); + case SPILT: + return StringFunction.spilt(args); case ABS: return NumericFunction.abs(args); case ACOS: @@ -543,6 +548,8 @@ public Object executeFunctionExpr(String functionName, List args) { return SystemFunction.ifnull(args); case NULLIF: return SystemFunction.nullif(args); + case ARRAY: + return SystemFunction.array(args); case UUID: return randomUUID().toString(); default: @@ -698,8 +705,9 @@ public List lateralView( Function function = lateralView.getGeneratorFunction(); boolean isUsingOuter = lateralView.isUsingOuter(); String functionName = function.getName(); + String alias = lateralView.getColumnAlias().getName(); if (EXPLODE.equalsIgnoreCase(functionName)) { - seaTunnelRows = explode(seaTunnelRows, function, outRowType, isUsingOuter); + seaTunnelRows = explode(seaTunnelRows, function, outRowType, isUsingOuter, alias); } else { throw new SeaTunnelRuntimeException( CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, @@ -714,8 +722,10 @@ private List explode( List seaTunnelRows, Function lateralViewFunction, SeaTunnelRowType outRowType, - boolean isUsingOuter) { + boolean isUsingOuter, + String alias) { ExpressionList expressions = lateralViewFunction.getParameters(); + int aliasFieldIndex = outRowType.indexOf(alias); for (Expression expression : expressions) { if (expression instanceof Column) { String column = ((Column) expression).getColumnName(); @@ -729,110 +739,114 @@ private List explode( if (splitFieldValue instanceof Object[]) { Object[] rowList = (Object[]) splitFieldValue; if (ArrayUtils.isEmpty(rowList) && isUsingOuter) { - SeaTunnelRow outputRow = row.copy(); - outputRow.setField(fieldIndex, null); - next.add(outputRow); + next.add(copy(outRowType.getTotalFields(), row, aliasFieldIndex, null)); } else { for (Object fieldValue : rowList) { - SeaTunnelRow outputRow = row.copy(); - outputRow.setField(fieldIndex, fieldValue); - next.add(outputRow); + next.add( + copy( + outRowType.getTotalFields(), + row, + aliasFieldIndex, + fieldValue)); } } } } seaTunnelRows = next; - } - if (expression instanceof Function) { - Function function = (Function) expression; - String functionName = function.getName(); - if (SPILT.equalsIgnoreCase(functionName)) { - ExpressionList expressionList = function.getParameters(); - String column = ((Column) expressionList.get(0)).getColumnName(); - String delimiter = ((StringValue) expressionList.get(1)).getValue(); - List next = new ArrayList<>(); - for (SeaTunnelRow row : seaTunnelRows) { - - int fieldIndex = outRowType.indexOf(column); - Object splitFieldValue = row.getField(fieldIndex); - if (splitFieldValue == null) { - continue; - } - String[] splitFieldValues = splitFieldValue.toString().split(delimiter); - for (String fieldValue : splitFieldValues) { - SeaTunnelRow outputRow = row.copy(); - outputRow.setField(fieldIndex, fieldValue); - next.add(outputRow); + } else if (expression instanceof Function) { + List next = new ArrayList<>(); + for (SeaTunnelRow row : seaTunnelRows) { + Object values = computeForValue(expression, row.getFields()); + if (values.getClass().isArray()) { + for (Object fieldValue : (Object[]) values) { + next.add( + copy( + outRowType.getTotalFields(), + row, + aliasFieldIndex, + fieldValue.toString())); } + } else { + throw new SeaTunnelRuntimeException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Transform config error! UnSupport explode function:" + + ((Function) expression).getName()); } - seaTunnelRows = next; - - } else { - throw new SeaTunnelRuntimeException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "Transform config error! UnSupport function:" + functionName); } + seaTunnelRows = next; } } return seaTunnelRows; } + private SeaTunnelRow copy(int length, SeaTunnelRow row, int fieldIndex, Object fieldValue) { + Object[] fields = new Object[length]; + System.arraycopy(row.getFields(), 0, fields, 0, row.getFields().length); + SeaTunnelRow outputRow = new SeaTunnelRow(fields); + outputRow.setRowKind(row.getRowKind()); + outputRow.setTableId(row.getTableId()); + outputRow.setField(fieldIndex, fieldValue); + return outputRow; + } + public SeaTunnelRowType lateralViewMapping( String[] fieldNames, SeaTunnelDataType[] seaTunnelDataTypes, - List lateralViews) { + List lateralViews, + List inputColumnsMapping) { for (LateralView lateralView : lateralViews) { Function function = lateralView.getGeneratorFunction(); String functionName = function.getName(); + String alias = lateralView.getColumnAlias().getName(); if (EXPLODE.equalsIgnoreCase(functionName)) { - explodeTypeMapping(fieldNames, seaTunnelDataTypes, function); + ExpressionList expressions = function.getParameters(); + int aliasIndex = Arrays.asList(fieldNames).indexOf(alias); + for (Expression expression : expressions) { + if (expression instanceof Column) { + String column = ((Column) expression).getColumnName(); + int columnIndex = Arrays.asList(fieldNames).indexOf(column); + if (columnIndex == -1) { + throw new TransformException( + INPUT_FIELDS_NOT_FOUND, + "Lateral view field must be in select item:" + fieldNames); + } + ArrayType arrayType = (ArrayType) seaTunnelDataTypes[columnIndex]; + SeaTunnelDataType seaTunnelDataType = + PhysicalColumn.of( + column, + arrayType.getElementType(), + 200, + true, + "", + "") + .getDataType(); + if (aliasIndex == -1) { + fieldNames = ArrayUtils.add(fieldNames, alias); + seaTunnelDataTypes = + ArrayUtils.add(seaTunnelDataTypes, seaTunnelDataType); + inputColumnsMapping.add(alias); + } else { + seaTunnelDataTypes[columnIndex] = seaTunnelDataType; + } + } else { + // default string type + SeaTunnelDataType seaTunnelDataType = + PhysicalColumn.of(alias, BasicType.STRING_TYPE, 10L, true, "", "") + .getDataType(); + if (aliasIndex == -1) { + fieldNames = ArrayUtils.add(fieldNames, alias); + seaTunnelDataTypes = + ArrayUtils.add(seaTunnelDataTypes, seaTunnelDataType); + inputColumnsMapping.add(alias); + } + } + } } else { throw new SeaTunnelRuntimeException( CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, "Transform config error! UnSupport function:" + functionName); } } - return new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); } - - private void explodeTypeMapping( - String[] fieldNames, - SeaTunnelDataType[] seaTunnelDataTypes, - Function lateralViewFunction) { - ExpressionList expressions = lateralViewFunction.getParameters(); - for (Expression expression : expressions) { - if (expression instanceof Column) { - String column = ((Column) expression).getColumnName(); - int columnIndex = Arrays.asList(fieldNames).indexOf(column); - if (columnIndex == -1) { - throw new TransformException( - INPUT_FIELDS_NOT_FOUND, - "Lateral view field must be in select item:" + fieldNames); - } - ArrayType arrayType = (ArrayType) seaTunnelDataTypes[columnIndex]; - seaTunnelDataTypes[columnIndex] = - PhysicalColumn.of(column, arrayType.getElementType(), 200, true, "", "") - .getDataType(); - } - if (expression instanceof Function) { - Function function = (Function) expression; - String functionName = function.getName(); - if (SPILT.equalsIgnoreCase(functionName)) { - ExpressionList expressionList = function.getParameters(); - String column = ((Column) expressionList.get(0)).getColumnName(); - int columnIndex = Arrays.asList(fieldNames).indexOf(column); - if (columnIndex == -1) { - throw new TransformException( - INPUT_FIELDS_NOT_FOUND, - "Lateral view field must be in select item:" + fieldNames); - } - } else { - throw new SeaTunnelRuntimeException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "Transform config error! UnSupport function:" + functionName); - } - } - } - } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java index d665117bfd0..124f8267b27 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java @@ -332,6 +332,18 @@ public static String trim(List args) { return trim(arg, true, true, sp); } + public static String[] spilt(List args) { + String arg = (String) args.get(0); + if (arg == null) { + return null; + } + String delimiter = ""; + if (args.size() >= 2) { + delimiter = (String) args.get(1); + } + return arg.split(delimiter); + } + public static String trim(String s, boolean leading, boolean trailing, String sp) { char space = sp == null || sp.isEmpty() ? ' ' : sp.charAt(0); int begin = 0, end = s.length(); diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java index 75584a92dea..3968fbf2e76 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/SystemFunction.java @@ -22,6 +22,10 @@ import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; import org.apache.seatunnel.transform.exception.TransformException; +import org.apache.commons.collections4.CollectionUtils; + +import com.google.common.collect.Lists; + import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.charset.StandardCharsets; @@ -66,6 +70,13 @@ public static Object nullif(List args) { return v1; } + public static Object array(List args) { + if (CollectionUtils.isNotEmpty(args)) { + return args.toArray(); + } + return Lists.newArrayList(); + } + public static Object castAs(Object arg, SeaTunnelDataType type) { final ArrayList args = new ArrayList<>(4); args.add(arg); From c5c0399ebaac95c2d561216d11ea255565e81fc7 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 16:30:01 +0800 Subject: [PATCH 20/29] [Feature][transform] sql transform support lateral view explode --- .../org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index e6005e7f296..4ddf91cf07d 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -330,6 +330,7 @@ public Object computeForValue(Expression expression, Object[] inputFields) { Parenthesis parenthesis = (Parenthesis) expression; return computeForValue(parenthesis.getExpression(), inputFields); } + // bytes not supported at the moment,use BINARY instead. if (expression instanceof CaseExpression) { CaseExpression caseExpression = (CaseExpression) expression; final Object value = executeCaseExpr(caseExpression, inputFields); From 97ea9146f18634155933d8da7dc6c7658c607e01 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 16:32:20 +0800 Subject: [PATCH 21/29] [Feature][transform] sql transform support lateral view explode --- docs/zh/transform-v2/sql-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index 2cc2cc6d749..fbc28180f23 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -987,4 +987,5 @@ SELECT * FROM fake LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME LATERAL VIEW EXPLODE ( SPILT ( pk_id, ';' ) ) AS pk_id LATERAL VIEW OUTER EXPLODE ( age ) AS age + LATERAL VIEW OUTER EXPLODE ( ARRAY(1,1) ) AS num ``` From 27f320de19bee29dd369e2675ff3634a845fdae5 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 17:24:58 +0800 Subject: [PATCH 22/29] [Feature][transform] sql transform support lateral view explode --- .../transform/sql/zeta/ZetaSQLFunction.java | 85 +++++++++++-------- 1 file changed, 50 insertions(+), 35 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 4ddf91cf07d..af29548937b 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -734,45 +734,28 @@ private List explode( for (SeaTunnelRow row : seaTunnelRows) { int fieldIndex = outRowType.indexOf(column); Object splitFieldValue = row.getField(fieldIndex); - if (splitFieldValue == null) { - continue; - } - if (splitFieldValue instanceof Object[]) { - Object[] rowList = (Object[]) splitFieldValue; - if (ArrayUtils.isEmpty(rowList) && isUsingOuter) { - next.add(copy(outRowType.getTotalFields(), row, aliasFieldIndex, null)); - } else { - for (Object fieldValue : rowList) { - next.add( - copy( - outRowType.getTotalFields(), - row, - aliasFieldIndex, - fieldValue)); - } - } - } + transformExplodeValue( + splitFieldValue, + outRowType, + isUsingOuter, + next, + aliasFieldIndex, + row, + expression); } seaTunnelRows = next; } else if (expression instanceof Function) { List next = new ArrayList<>(); for (SeaTunnelRow row : seaTunnelRows) { - Object values = computeForValue(expression, row.getFields()); - if (values.getClass().isArray()) { - for (Object fieldValue : (Object[]) values) { - next.add( - copy( - outRowType.getTotalFields(), - row, - aliasFieldIndex, - fieldValue.toString())); - } - } else { - throw new SeaTunnelRuntimeException( - CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, - "Transform config error! UnSupport explode function:" - + ((Function) expression).getName()); - } + Object splitFieldValue = computeForValue(expression, row.getFields()); + transformExplodeValue( + splitFieldValue, + outRowType, + isUsingOuter, + next, + aliasFieldIndex, + row, + expression); } seaTunnelRows = next; } @@ -780,7 +763,39 @@ private List explode( return seaTunnelRows; } - private SeaTunnelRow copy(int length, SeaTunnelRow row, int fieldIndex, Object fieldValue) { + private void transformExplodeValue( + Object splitFieldValue, + SeaTunnelRowType outRowType, + boolean isUsingOuter, + List next, + int aliasFieldIndex, + SeaTunnelRow row, + Expression expression) { + if (splitFieldValue == null) { + if (isUsingOuter) { + next.add(copySeaTunnelRow(outRowType.getTotalFields(), row, aliasFieldIndex, null)); + } + return; + } + if (splitFieldValue.getClass().isArray()) { + for (Object fieldValue : (Object[]) splitFieldValue) { + next.add( + copySeaTunnelRow( + outRowType.getTotalFields(), + row, + aliasFieldIndex, + fieldValue.toString())); + } + } else { + throw new SeaTunnelRuntimeException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + "Transform config error! UnSupport explode function:" + + ((Function) expression).getName()); + } + } + + private SeaTunnelRow copySeaTunnelRow( + int length, SeaTunnelRow row, int fieldIndex, Object fieldValue) { Object[] fields = new Object[length]; System.arraycopy(row.getFields(), 0, fields, 0, row.getFields().length); SeaTunnelRow outputRow = new SeaTunnelRow(fields); From e1c60e3b9825d123ed78c82010aabeb66ecc3118 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 17:45:00 +0800 Subject: [PATCH 23/29] [Feature][transform] handle null --- .../seatunnel/transform/sql/zeta/ZetaSQLFunction.java | 8 ++++++++ .../transform/sql/zeta/functions/StringFunction.java | 4 +++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index af29548937b..b29a6bf1e77 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -778,6 +778,14 @@ private void transformExplodeValue( return; } if (splitFieldValue.getClass().isArray()) { + if (ArrayUtils.isEmpty((Object[]) splitFieldValue)) { + if (isUsingOuter) { + next.add( + copySeaTunnelRow( + outRowType.getTotalFields(), row, aliasFieldIndex, null)); + } + return; + } for (Object fieldValue : (Object[]) splitFieldValue) { next.add( copySeaTunnelRow( diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java index 124f8267b27..4ab3705f272 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java @@ -21,6 +21,8 @@ import org.apache.seatunnel.transform.exception.TransformException; import org.apache.seatunnel.transform.sql.zeta.ZetaSQLFunction; +import org.apache.groovy.parser.antlr4.util.StringUtils; + import java.nio.charset.StandardCharsets; import java.time.temporal.Temporal; import java.util.Arrays; @@ -334,7 +336,7 @@ public static String trim(List args) { public static String[] spilt(List args) { String arg = (String) args.get(0); - if (arg == null) { + if (StringUtils.isEmpty(arg)) { return null; } String delimiter = ""; From 5f809300c408abd0e1adcfb8a474f421c034c188 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Wed, 6 Nov 2024 18:38:02 +0800 Subject: [PATCH 24/29] [Feature][transform] fix array type --- .../transform/sql/zeta/ZetaSQLFunction.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index b29a6bf1e77..70fc73aec03 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -741,7 +741,8 @@ private List explode( next, aliasFieldIndex, row, - expression); + expression, + true); } seaTunnelRows = next; } else if (expression instanceof Function) { @@ -755,7 +756,8 @@ private List explode( next, aliasFieldIndex, row, - expression); + expression, + false); } seaTunnelRows = next; } @@ -770,7 +772,8 @@ private void transformExplodeValue( List next, int aliasFieldIndex, SeaTunnelRow row, - Expression expression) { + Expression expression, + boolean keepValueType) { if (splitFieldValue == null) { if (isUsingOuter) { next.add(copySeaTunnelRow(outRowType.getTotalFields(), row, aliasFieldIndex, null)); @@ -787,12 +790,9 @@ private void transformExplodeValue( return; } for (Object fieldValue : (Object[]) splitFieldValue) { + Object value = keepValueType ? fieldValue : String.valueOf(fieldValue); next.add( - copySeaTunnelRow( - outRowType.getTotalFields(), - row, - aliasFieldIndex, - fieldValue.toString())); + copySeaTunnelRow(outRowType.getTotalFields(), row, aliasFieldIndex, value)); } } else { throw new SeaTunnelRuntimeException( From fd715b9ca470f9a3399861755c64f0b836de866b Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 8 Nov 2024 09:41:12 +0800 Subject: [PATCH 25/29] [Feature][transform] fix error word --- docs/en/transform-v2/sql-functions.md | 21 ++++++++++++++--- docs/zh/transform-v2/sql-functions.md | 23 ++++++++++++++++--- .../jdbc/source/FixedChunkSplitter.java | 4 ++-- .../sql_transform/explode_transform.conf | 2 +- .../transform/sql/zeta/ZetaSQLFunction.java | 10 ++++---- .../sql/zeta/functions/StringFunction.java | 2 +- .../transform/sql/SQLTransformTest.java | 4 ++-- 7 files changed, 50 insertions(+), 16 deletions(-) diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index 2dcc1e421c0..3161abfc405 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -984,18 +984,33 @@ Example: select UUID() as seatunnel_uuid +### ARRAY + +Generate an array. + +Example: + +select ARRAY('test1','test2','test3') as arrays + +### SPLIT + +Split a string into an array. + +Example: + +select SPLIT(test,';') as arrays ### LATERAL VIEW #### EXPLODE explode array column to rows. OUTER EXPLODE will return NULL, while array is NULL or empty -EXPLODE(SPILT(FIELD_NAME,separator))Used to split string type. The first parameter of SPILT function is the field name, the second parameter is the separator +EXPLODE(SPLIT(FIELD_NAME,separator))Used to split string type. The first parameter of SPLIT function is the field name, the second parameter is the separator EXPLODE(ARRAY(value1,value2)) Used to custom array type. ``` SELECT * FROM fake - LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME - LATERAL VIEW EXPLODE ( SPILT ( pk_id, ';' ) ) AS pk_id + LATERAL VIEW EXPLODE ( SPLIT ( NAME, ',' ) ) AS NAME + LATERAL VIEW EXPLODE ( SPLIT ( pk_id, ';' ) ) AS pk_id LATERAL VIEW OUTER EXPLODE ( age ) AS age LATERAL VIEW OUTER EXPLODE ( ARRAY(1,1) ) AS num ``` diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index fbc28180f23..b2f2f822fa6 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -975,17 +975,34 @@ case when c_string in ('c_string') then 1 else 0 end select UUID() as seatunnel_uuid + +### ARRAY + +生成一个数组。 + +示例: + +select ARRAY('test1','test2','test3') as arrays + +### SPLIT + +将字符串切分成数组。 + +示例: + +select SPLIT(test,';') as arrays + ### LATERAL VIEW #### EXPLODE 将 array 列展开成多行。 OUTER EXPLODE 当 array 为NULL或者为空时,返回NULL -EXPLODE(SPILT(FIELD_NAME,separator))用来切分字符串类型,SPILT 第一个参数是字段名,第二个参数是分隔符 +EXPLODE(SPLIT(FIELD_NAME,separator))用来切分字符串类型,SPLIT 第一个参数是字段名,第二个参数是分隔符 EXPLODE(ARRAY(value1,value2)) 用于自定义数组切分,在原有基础上生成一个新的字段。 ``` SELECT * FROM fake - LATERAL VIEW EXPLODE ( SPILT ( NAME, ',' ) ) AS NAME - LATERAL VIEW EXPLODE ( SPILT ( pk_id, ';' ) ) AS pk_id + LATERAL VIEW EXPLODE ( SPLIT ( NAME, ',' ) ) AS NAME + LATERAL VIEW EXPLODE ( SPLIT ( pk_id, ';' ) ) AS pk_id LATERAL VIEW OUTER EXPLODE ( age ) AS age LATERAL VIEW OUTER EXPLODE ( ARRAY(1,1) ) AS num ``` diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/FixedChunkSplitter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/FixedChunkSplitter.java index edeef96f0a2..72a4e061ac5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/FixedChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/FixedChunkSplitter.java @@ -82,8 +82,8 @@ protected Collection createSplits( partitionEnd = range.getRight(); } if (partitionStart == null || partitionEnd == null) { - JdbcSourceSplit spilt = createSingleSplit(table); - return Collections.singletonList(spilt); + JdbcSourceSplit split = createSingleSplit(table); + return Collections.singletonList(split); } return createNumberColumnSplits( diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf index 9f0614ede13..8df8bc5076e 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform.conf @@ -58,7 +58,7 @@ transform { Sql { source_table_name = "fake" result_table_name = "fake1" - query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPILT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPILT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age LATERAL VIEW EXPLODE(ARRAY(1,1)) as num" + query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(SPLIT(name, ',')) as name LATERAL VIEW OUTER EXPLODE(SPLIT(pk_id, ';')) as pk_id LATERAL VIEW OUTER EXPLODE(age) as age LATERAL VIEW EXPLODE(ARRAY(1,1)) as num" } } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 70fc73aec03..b9c506f73cc 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -115,7 +115,7 @@ public class ZetaSQLFunction { public static final String SUBSTR = "SUBSTR"; public static final String TO_CHAR = "TO_CHAR"; public static final String TRANSLATE = "TRANSLATE"; - public static final String SPILT = "SPILT"; + public static final String SPLIT = "SPLIT"; // -------------------------numeric functions---------------------------- public static final String ABS = "ABS"; @@ -215,7 +215,9 @@ public Object computeForValue(Expression expression, Object[] inputFields) { List functionArgs = new ArrayList<>(); if (column != null) { functionArgs.add(computeForValue(column, inputFields)); - functionArgs.add(((StringValue) function.getFromExpression()).getValue()); + if (function.getFromExpression() != null) { + functionArgs.add(((StringValue) function.getFromExpression()).getValue()); + } } return executeFunctionExpr(TRIM, functionArgs); } @@ -440,8 +442,8 @@ public Object executeFunctionExpr(String functionName, List args) { return StringFunction.toChar(args); case TRANSLATE: return StringFunction.translate(args); - case SPILT: - return StringFunction.spilt(args); + case SPLIT: + return StringFunction.split(args); case ABS: return NumericFunction.abs(args); case ACOS: diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java index 4ab3705f272..2bf71e3b606 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/StringFunction.java @@ -334,7 +334,7 @@ public static String trim(List args) { return trim(arg, true, true, sp); } - public static String[] spilt(List args) { + public static String[] split(List args) { String arg = (String) args.get(0); if (StringUtils.isEmpty(arg)) { return null; diff --git a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java index aa6823f0733..fcf14cc7b9d 100644 --- a/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java +++ b/seatunnel-transforms-v2/src/test/java/org/apache/seatunnel/transform/sql/SQLTransformTest.java @@ -166,12 +166,12 @@ public void testEscapeIdentifier() { ReadonlyConfig.fromMap( Collections.singletonMap( "query", - "select id, trim(`apply`,'*') as `apply` from test where `apply` = 'a*'")); + "select id, trim(`apply`) as `apply` from test where `apply` = 'a'")); SQLTransform sqlTransform = new SQLTransform(config, table); TableSchema tableSchema = sqlTransform.transformTableSchema(); List result = sqlTransform.transformRow( - new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a*")})); + new SeaTunnelRow(new Object[] {Integer.valueOf(1), String.valueOf("a")})); Assertions.assertEquals("apply", tableSchema.getFieldNames()[1]); Assertions.assertEquals("a", result.get(0).getField(1)); result = From 013c2847f539edafc0218e1082d7bbaed629176c Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 8 Nov 2024 17:47:31 +0800 Subject: [PATCH 26/29] [Feature][transform] transform support explode --- .../AbstractCatalogMultiRowTransform.java | 66 +----------- .../AbstractCatalogSupportTransform.java | 70 ++---------- .../common/AbstractSeaTunnelTransform.java | 100 ++++++++++++++++++ 3 files changed, 110 insertions(+), 126 deletions(-) create mode 100644 seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java index c4d983476ae..bf7dc2ab961 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogMultiRowTransform.java @@ -17,11 +17,8 @@ package org.apache.seatunnel.transform.common; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; -import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform; -import org.apache.seatunnel.transform.exception.ErrorDataTransformException; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @@ -30,75 +27,20 @@ @Slf4j public abstract class AbstractCatalogMultiRowTransform + extends AbstractSeaTunnelTransform> implements SeaTunnelMultiRowTransform { - protected final ErrorHandleWay rowErrorHandleWay; - protected CatalogTable inputCatalogTable; - - protected volatile CatalogTable outputCatalogTable; public AbstractCatalogMultiRowTransform(@NonNull CatalogTable inputCatalogTable) { - this(inputCatalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + super(inputCatalogTable); } public AbstractCatalogMultiRowTransform( @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { - this.inputCatalogTable = inputCatalogTable; - this.rowErrorHandleWay = rowErrorHandleWay; + super(inputCatalogTable, rowErrorHandleWay); } @Override public List flatMap(SeaTunnelRow row) { - try { - return transformRow(row); - } catch (ErrorDataTransformException e) { - if (e.getErrorHandleWay() != null) { - ErrorHandleWay errorHandleWay = e.getErrorHandleWay(); - if (errorHandleWay.allowSkipThisRow()) { - log.debug("Skip row due to error", e); - return null; - } - throw e; - } - if (rowErrorHandleWay.allowSkip()) { - log.debug("Skip row due to error", e); - return null; - } - throw e; - } - } - - /** - * Outputs transformed row data. - * - * @param inputRow upstream input row data - */ - protected abstract List transformRow(SeaTunnelRow inputRow); - - @Override - public CatalogTable getProducedCatalogTable() { - if (outputCatalogTable == null) { - synchronized (this) { - if (outputCatalogTable == null) { - outputCatalogTable = transformCatalogTable(); - } - } - } - - return outputCatalogTable; - } - - private CatalogTable transformCatalogTable() { - TableIdentifier tableIdentifier = transformTableIdentifier(); - TableSchema tableSchema = transformTableSchema(); - return CatalogTable.of( - tableIdentifier, - tableSchema, - inputCatalogTable.getOptions(), - inputCatalogTable.getPartitionKeys(), - inputCatalogTable.getComment()); + return transform(row); } - - protected abstract TableSchema transformTableSchema(); - - protected abstract TableIdentifier transformTableIdentifier(); } diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java index a0fa464af7a..358bcd42982 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractCatalogSupportTransform.java @@ -18,85 +18,27 @@ package org.apache.seatunnel.transform.common; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.catalog.TableIdentifier; -import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.transform.SeaTunnelTransform; -import org.apache.seatunnel.transform.exception.ErrorDataTransformException; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; @Slf4j -public abstract class AbstractCatalogSupportTransform implements SeaTunnelTransform { - protected final ErrorHandleWay rowErrorHandleWay; - protected CatalogTable inputCatalogTable; - - protected volatile CatalogTable outputCatalogTable; - +public abstract class AbstractCatalogSupportTransform + extends AbstractSeaTunnelTransform + implements SeaTunnelTransform { public AbstractCatalogSupportTransform(@NonNull CatalogTable inputCatalogTable) { - this(inputCatalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + super(inputCatalogTable); } public AbstractCatalogSupportTransform( @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { - this.inputCatalogTable = inputCatalogTable; - this.rowErrorHandleWay = rowErrorHandleWay; + super(inputCatalogTable, rowErrorHandleWay); } @Override public SeaTunnelRow map(SeaTunnelRow row) { - try { - return transformRow(row); - } catch (ErrorDataTransformException e) { - if (e.getErrorHandleWay() != null) { - ErrorHandleWay errorHandleWay = e.getErrorHandleWay(); - if (errorHandleWay.allowSkipThisRow()) { - log.debug("Skip row due to error", e); - return null; - } - throw e; - } - if (rowErrorHandleWay.allowSkip()) { - log.debug("Skip row due to error", e); - return null; - } - throw e; - } + return transform(row); } - - /** - * Outputs transformed row data. - * - * @param inputRow upstream input row data - */ - protected abstract SeaTunnelRow transformRow(SeaTunnelRow inputRow); - - @Override - public CatalogTable getProducedCatalogTable() { - if (outputCatalogTable == null) { - synchronized (this) { - if (outputCatalogTable == null) { - outputCatalogTable = transformCatalogTable(); - } - } - } - - return outputCatalogTable; - } - - private CatalogTable transformCatalogTable() { - TableIdentifier tableIdentifier = transformTableIdentifier(); - TableSchema tableSchema = transformTableSchema(); - return CatalogTable.of( - tableIdentifier, - tableSchema, - inputCatalogTable.getOptions(), - inputCatalogTable.getPartitionKeys(), - inputCatalogTable.getComment()); - } - - protected abstract TableSchema transformTableSchema(); - - protected abstract TableIdentifier transformTableIdentifier(); } 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 new file mode 100644 index 00000000000..a6dc217adb3 --- /dev/null +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/AbstractSeaTunnelTransform.java @@ -0,0 +1,100 @@ +/* + * 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.transform.common; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.transform.SeaTunnelTransform; +import org.apache.seatunnel.transform.exception.ErrorDataTransformException; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class AbstractSeaTunnelTransform implements SeaTunnelTransform { + + protected final ErrorHandleWay rowErrorHandleWay; + protected CatalogTable inputCatalogTable; + + protected volatile CatalogTable outputCatalogTable; + + public AbstractSeaTunnelTransform(@NonNull CatalogTable inputCatalogTable) { + this(inputCatalogTable, CommonOptions.ROW_ERROR_HANDLE_WAY_OPTION.defaultValue()); + } + + public AbstractSeaTunnelTransform( + @NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) { + this.inputCatalogTable = inputCatalogTable; + this.rowErrorHandleWay = rowErrorHandleWay; + } + + public CatalogTable getProducedCatalogTable() { + if (outputCatalogTable == null) { + synchronized (this) { + if (outputCatalogTable == null) { + outputCatalogTable = transformCatalogTable(); + } + } + } + + return outputCatalogTable; + } + + private CatalogTable transformCatalogTable() { + TableIdentifier tableIdentifier = transformTableIdentifier(); + TableSchema tableSchema = transformTableSchema(); + return CatalogTable.of( + tableIdentifier, + tableSchema, + inputCatalogTable.getOptions(), + inputCatalogTable.getPartitionKeys(), + inputCatalogTable.getComment()); + } + + public R transform(SeaTunnelRow row) { + try { + return transformRow(row); + } catch (ErrorDataTransformException e) { + if (e.getErrorHandleWay() != null) { + ErrorHandleWay errorHandleWay = e.getErrorHandleWay(); + if (errorHandleWay.allowSkipThisRow()) { + log.debug("Skip row due to error", e); + return null; + } + throw e; + } + if (rowErrorHandleWay.allowSkip()) { + log.debug("Skip row due to error", e); + return null; + } + throw e; + } + } + + /** + * Outputs transformed row data. + * + * @param inputRow upstream input row data + */ + protected abstract R transformRow(SeaTunnelRow inputRow); + + protected abstract TableSchema transformTableSchema(); + + protected abstract TableIdentifier transformTableIdentifier(); +} From 619cd2f4ada351b80e1ad88419fcc23280ef6020 Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 8 Nov 2024 18:16:09 +0800 Subject: [PATCH 27/29] [Feature][RestAPI] Support submit job with seatunnel style hocon format config --- docs/en/transform-v2/sql-functions.md | 15 ++++++++------- docs/zh/transform-v2/sql-functions.md | 17 +++++++++-------- .../transform/sql/zeta/ZetaSQLFunction.java | 6 +++++- 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/docs/en/transform-v2/sql-functions.md b/docs/en/transform-v2/sql-functions.md index 3161abfc405..31a33989375 100644 --- a/docs/en/transform-v2/sql-functions.md +++ b/docs/en/transform-v2/sql-functions.md @@ -302,6 +302,14 @@ Example: REPLACE(NAME, ' ') +### SPLIT + +Split a string into an array. + +Example: + +select SPLIT(test,';') as arrays + ### SOUNDEX ```SOUNDEX(string)``` @@ -992,13 +1000,6 @@ Example: select ARRAY('test1','test2','test3') as arrays -### SPLIT - -Split a string into an array. - -Example: - -select SPLIT(test,';') as arrays ### LATERAL VIEW #### EXPLODE diff --git a/docs/zh/transform-v2/sql-functions.md b/docs/zh/transform-v2/sql-functions.md index b2f2f822fa6..7e3f8454e1d 100644 --- a/docs/zh/transform-v2/sql-functions.md +++ b/docs/zh/transform-v2/sql-functions.md @@ -302,6 +302,15 @@ REPEAT(NAME || ' ', 10) REPLACE(NAME, ' ') + +### SPLIT + +将字符串切分成数组。 + +示例: + +select SPLIT(test,';') as arrays + ### SOUNDEX ```SOUNDEX(string)``` @@ -984,14 +993,6 @@ select UUID() as seatunnel_uuid select ARRAY('test1','test2','test3') as arrays -### SPLIT - -将字符串切分成数组。 - -示例: - -select SPLIT(test,';') as arrays - ### LATERAL VIEW #### EXPLODE diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index b9c506f73cc..66d0f041d96 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -792,7 +792,11 @@ private void transformExplodeValue( return; } for (Object fieldValue : (Object[]) splitFieldValue) { - Object value = keepValueType ? fieldValue : String.valueOf(fieldValue); + Object value = + fieldValue == null + ? null + : (keepValueType ? fieldValue : String.valueOf(fieldValue)); + next.add( copySeaTunnelRow(outRowType.getTotalFields(), row, aliasFieldIndex, value)); } From 46f0857aa32a1f73636a4d6cb7c8deb04359a82b Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 8 Nov 2024 18:21:11 +0800 Subject: [PATCH 28/29] [Feature][transform] transform support explode --- .../seatunnel/transform/sql/zeta/ZetaSQLFunction.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 66d0f041d96..70dbde546bf 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -778,7 +778,9 @@ private void transformExplodeValue( boolean keepValueType) { if (splitFieldValue == null) { if (isUsingOuter) { - next.add(copySeaTunnelRow(outRowType.getTotalFields(), row, aliasFieldIndex, null)); + next.add( + copySeaTunnelRowWithNewValue( + outRowType.getTotalFields(), row, aliasFieldIndex, null)); } return; } @@ -786,7 +788,7 @@ private void transformExplodeValue( if (ArrayUtils.isEmpty((Object[]) splitFieldValue)) { if (isUsingOuter) { next.add( - copySeaTunnelRow( + copySeaTunnelRowWithNewValue( outRowType.getTotalFields(), row, aliasFieldIndex, null)); } return; @@ -798,7 +800,8 @@ private void transformExplodeValue( : (keepValueType ? fieldValue : String.valueOf(fieldValue)); next.add( - copySeaTunnelRow(outRowType.getTotalFields(), row, aliasFieldIndex, value)); + copySeaTunnelRowWithNewValue( + outRowType.getTotalFields(), row, aliasFieldIndex, value)); } } else { throw new SeaTunnelRuntimeException( @@ -808,7 +811,7 @@ private void transformExplodeValue( } } - private SeaTunnelRow copySeaTunnelRow( + private SeaTunnelRow copySeaTunnelRowWithNewValue( int length, SeaTunnelRow row, int fieldIndex, Object fieldValue) { Object[] fields = new Object[length]; System.arraycopy(row.getFields(), 0, fields, 0, row.getFields().length); From b337b5040f36c173a0dbe985cb7c23ab9677cd8a Mon Sep 17 00:00:00 2001 From: njh_cmss Date: Fri, 8 Nov 2024 22:39:06 +0800 Subject: [PATCH 29/29] [Feature][transform] transform support explode --- .../seatunnel/e2e/transform/TestSQLIT.java | 8 ++ .../explode_transform_with_outer.conf | 99 +++++++++++++++++++ .../explode_transform_without_outer.conf | 95 ++++++++++++++++++ .../transform/sql/zeta/ZetaSQLFunction.java | 5 +- 4 files changed, 206 insertions(+), 1 deletion(-) create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_with_outer.conf create mode 100644 seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_without_outer.conf diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java index beec110e0ac..a12eabe7ef6 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/java/org/apache/seatunnel/e2e/transform/TestSQLIT.java @@ -63,6 +63,14 @@ public void testSQLTransform(TestContainer container) throws IOException, Interr Container.ExecResult execResultBySql = container.executeJob("/sql_transform/explode_transform.conf"); Assertions.assertEquals(0, execResultBySql.getExitCode()); + + Container.ExecResult execResultBySqlWithoutOuter = + container.executeJob("/sql_transform/explode_transform_without_outer.conf"); + Assertions.assertEquals(0, execResultBySqlWithoutOuter.getExitCode()); + + Container.ExecResult execResultBySqlWithOuter = + container.executeJob("/sql_transform/explode_transform_with_outer.conf"); + Assertions.assertEquals(0, execResultBySqlWithOuter.getExitCode()); } @TestTemplate diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_with_outer.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_with_outer.conf new file mode 100644 index 00000000000..177cc381f08 --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_with_outer.conf @@ -0,0 +1,99 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" + parallelism = 1 +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + pk_id = string + name = string + age = array + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = ["id001", "zhangsan",[null,null]] + } + ] + } +} + +transform { + Sql { + source_table_name = "fake" + result_table_name = "fake1" + query = "SELECT * FROM fake LATERAL VIEW OUTER EXPLODE(age) as age LATERAL VIEW OUTER EXPLODE(ARRAY(null,null)) as num" + } +} + +sink{ + assert { + rules = + { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 4 + }, + { + rule_type = MIN_ROW + rule_value = 4 + } + ], + field_rules = [ + { + field_name = pk_id + field_type = string + field_value = [{equals_to = id001}] + }, + { + field_name = name + field_type = string + field_value = [{equals_to = zhangsan}] + }, + { + field_name = age + field_type = "null" + field_value = [ + {rule_type = NULL} + ] + }, + { + field_name = num + field_type = "null" + field_value = [ + {rule_type = NULL} + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_without_outer.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_without_outer.conf new file mode 100644 index 00000000000..1e35a2cc50e --- /dev/null +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform/explode_transform_without_outer.conf @@ -0,0 +1,95 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + job.mode = "BATCH" + parallelism = 1 +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + pk_id = string + name = string + age = array + } + primaryKey { + name = "pk_id" + columnNames = [pk_id] + } + } + rows = [ + { + kind = INSERT + fields = ["id001", "zhangsan",[1,null]] + } + ] + } +} + +transform { + Sql { + source_table_name = "fake" + result_table_name = "fake1" + query = "SELECT * FROM fake LATERAL VIEW EXPLODE(age) as age LATERAL VIEW EXPLODE(ARRAY(1,1,null)) as num" + } +} + +sink{ + assert { + rules = + { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 2 + }, + { + rule_type = MIN_ROW + rule_value = 2 + } + ], + field_rules = [ + { + field_name = pk_id + field_type = string + field_value = [{equals_to = id001}] + }, + { + field_name = name + field_type = string + field_value = [{equals_to = zhangsan}] + }, + { + field_name = age + field_type = "string" + field_value = [{equals_to = 1}] + }, + { + field_name = num + field_type = "string" + field_value = [{equals_to = 1}] + } + ] + } + } +} diff --git a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java index 70dbde546bf..8cbc3ed86a6 100644 --- a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java +++ b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/ZetaSQLFunction.java @@ -794,11 +794,14 @@ private void transformExplodeValue( return; } for (Object fieldValue : (Object[]) splitFieldValue) { + + if (!isUsingOuter && fieldValue == null) { + continue; + } Object value = fieldValue == null ? null : (keepValueType ? fieldValue : String.valueOf(fieldValue)); - next.add( copySeaTunnelRowWithNewValue( outRowType.getTotalFields(), row, aliasFieldIndex, value));