This is an automated email from the ASF dual-hosted git repository.
gaojun2048 pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/seatunnel.git
The following commit(s) were added to refs/heads/dev by this push:
new 50fe637511 [Improve] Code clean for SeaTunnel transform (#5810)
50fe637511 is described below
commit 50fe637511bc920a1f87cc2dd37b037a413250b3
Author: Jia Fan <[email protected]>
AuthorDate: Fri Nov 10 15:10:42 2023 +0800
[Improve] Code clean for SeaTunnel transform (#5810)
---
.../common/AbstractCatalogSupportTransform.java | 14 +--
.../common/MultipleFieldOutputTransform.java | 4 +-
.../transform/common/SeaTunnelRowTypeEditor.java | 132 ---------------------
.../common/SingleFieldOutputTransform.java | 4 +-
.../transform/copy/CopyFieldTransform.java | 31 ++---
.../fieldmapper/FieldMapperTransform.java | 2 +-
.../transform/filter/FilterFieldTransform.java | 2 +-
.../filterrowkind/FilterRowKindTransform.java | 6 +-
.../transform/replace/ReplaceTransform.java | 7 +-
.../seatunnel/transform/split/SplitTransform.java | 21 ++--
.../seatunnel/transform/sql/SQLTransform.java | 8 +-
.../transform/sql/zeta/ZetaSQLEngine.java | 3 -
.../sql/zeta/functions/NumericFunction.java | 2 +-
.../transform/sql/zeta/functions/udf/DESUtil.java | 7 +-
14 files changed, 46 insertions(+), 197 deletions(-)
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 4a438f0242..78fe02094f 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
@@ -50,14 +50,12 @@ public abstract class AbstractCatalogSupportTransform
extends AbstractSeaTunnelT
private CatalogTable transformCatalogTable() {
TableIdentifier tableIdentifier = transformTableIdentifier();
TableSchema tableSchema = transformTableSchema();
- CatalogTable catalogTable =
- CatalogTable.of(
- tableIdentifier,
- tableSchema,
- inputCatalogTable.getOptions(),
- inputCatalogTable.getPartitionKeys(),
- inputCatalogTable.getComment());
- return catalogTable;
+ return CatalogTable.of(
+ tableIdentifier,
+ tableSchema,
+ inputCatalogTable.getOptions(),
+ inputCatalogTable.getPartitionKeys(),
+ inputCatalogTable.getComment());
}
protected abstract TableSchema transformTableSchema();
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java
index a003794eeb..bfe6336182 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/MultipleFieldOutputTransform.java
@@ -78,9 +78,9 @@ public abstract class MultipleFieldOutputTransform extends
AbstractCatalogSuppor
TableSchema.Builder builder = TableSchema.builder();
if (inputCatalogTable.getTableSchema().getPrimaryKey() != null) {
- builder =
builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy());
+
builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy());
}
- builder = builder.constraintKey(copiedConstraintKeys);
+ builder.constraintKey(copiedConstraintKeys);
List<Column> columns =
inputCatalogTable.getTableSchema().getColumns().stream()
.map(Column::copy)
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowTypeEditor.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowTypeEditor.java
deleted file mode 100644
index 1ae51f21b0..0000000000
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SeaTunnelRowTypeEditor.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.transform.common;
-
-import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
-import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-
-import lombok.AllArgsConstructor;
-import lombok.EqualsAndHashCode;
-import lombok.Getter;
-import lombok.Setter;
-import lombok.ToString;
-import lombok.extern.slf4j.Slf4j;
-
-import java.io.Serializable;
-import java.util.LinkedList;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-@Slf4j
-public class SeaTunnelRowTypeEditor implements Serializable {
- private final LinkedList<Field> fields;
-
- public SeaTunnelRowTypeEditor(SeaTunnelRowType rowType) {
- log.info("Input row type: {}", rowType);
- this.fields =
- IntStream.range(0, rowType.getTotalFields())
- .boxed()
- .map(
- index ->
- new Field(
- rowType.getFieldName(index),
- rowType.getFieldType(index)))
- .collect(Collectors.toCollection(LinkedList::new));
- }
-
- public SeaTunnelRowTypeEditor remove(String... fieldNames) {
- for (String fieldName : fieldNames) {
- int index = fields.indexOf(Field.name(fieldName));
- fields.remove(index);
- }
- return this;
- }
-
- public SeaTunnelRowTypeEditor addLast(String fieldName, SeaTunnelDataType
dataType) {
- fields.addLast(new Field(fieldName, dataType));
- return this;
- }
-
- public SeaTunnelRowTypeEditor addFirst(String fieldName, SeaTunnelDataType
dataType) {
- fields.addFirst(new Field(fieldName, dataType));
- return this;
- }
-
- public SeaTunnelRowTypeEditor add(int index, String fieldName,
SeaTunnelDataType dataType) {
- fields.add(index, new Field(fieldName, dataType));
- return this;
- }
-
- public SeaTunnelRowTypeEditor update(String fieldName, SeaTunnelDataType
dataType) {
- int index = fields.indexOf(Field.name(fieldName));
- if (index == -1) {
- throw new IllegalArgumentException("");
- }
-
- Field field = fields.get(index);
- field.setDataType(dataType);
- return this;
- }
-
- public SeaTunnelRowTypeEditor moveBefore(String srcFieldName, String
destFieldName) {
- int srcFieldIndex = fields.indexOf(Field.name(srcFieldName));
- int destFieldIndex = fields.indexOf(Field.name(destFieldName));
-
- Field srcField = fields.get(srcFieldIndex);
- fields.remove(srcField);
- fields.add(destFieldIndex, srcField);
- return this;
- }
-
- public SeaTunnelRowTypeEditor moveAfter(String srcFieldName, String
destFieldName) {
- int srcFieldIndex = fields.indexOf(Field.name(srcFieldName));
- int destFieldIndex = fields.indexOf(Field.name(destFieldName));
-
- Field srcField = fields.get(srcFieldIndex);
- fields.remove(srcField);
- fields.add(destFieldIndex + 1, srcField);
- return this;
- }
-
- public SeaTunnelRowType build() {
- String[] fieldNames = new String[fields.size()];
- SeaTunnelDataType[] fieldDataTypes = new
SeaTunnelDataType[fields.size()];
- for (int i = 0; i < fields.size(); i++) {
- Field field = fields.get(i);
- fieldNames[i] = field.getName();
- fieldDataTypes[i] = field.getDataType();
- }
-
- SeaTunnelRowType rowType = new SeaTunnelRowType(fieldNames,
fieldDataTypes);
- log.info("Output row type: {}", rowType);
-
- return rowType;
- }
-
- @AllArgsConstructor
- @EqualsAndHashCode(of = "name")
- @ToString
- private static class Field implements Serializable {
- @Getter private String name;
- @Setter @Getter private SeaTunnelDataType dataType;
-
- static Field name(String name) {
- return new Field(name, null);
- }
- }
-}
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java
index 43ea4bf683..c0ba3d1ca9 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/common/SingleFieldOutputTransform.java
@@ -67,9 +67,9 @@ public abstract class SingleFieldOutputTransform extends
AbstractCatalogSupportT
TableSchema.Builder builder = TableSchema.builder();
if (inputCatalogTable.getTableSchema().getPrimaryKey() != null) {
- builder =
builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy());
+
builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy());
}
- builder = builder.constraintKey(copiedConstraintKeys);
+ builder.constraintKey(copiedConstraintKeys);
List<Column> columns =
inputCatalogTable.getTableSchema().getColumns().stream()
.map(Column::copy)
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java
index e15df5b7f4..3455b8e8b4 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/copy/CopyFieldTransform.java
@@ -39,10 +39,10 @@ import java.util.stream.Collectors;
public class CopyFieldTransform extends MultipleFieldOutputTransform {
public static final String PLUGIN_NAME = "Copy";
- private CopyTransformConfig config;
+ private final CopyTransformConfig config;
private List<String> fieldNames;
- private List<Integer> fieldOriginalIndexs;
- private List<SeaTunnelDataType> fieldTypes;
+ private List<Integer> fieldOriginalIndexes;
+ private List<SeaTunnelDataType<?>> fieldTypes;
public CopyFieldTransform(CopyTransformConfig copyTransformConfig,
CatalogTable catalogTable) {
super(catalogTable);
@@ -59,8 +59,8 @@ public class CopyFieldTransform extends
MultipleFieldOutputTransform {
private void initOutputFields(
SeaTunnelRowType inputRowType, LinkedHashMap<String, String>
fields) {
List<String> fieldNames = new ArrayList<>();
- List<Integer> fieldOriginalIndexs = new ArrayList<>();
- List<SeaTunnelDataType> fieldsType = new ArrayList<>();
+ List<Integer> fieldOriginalIndexes = new ArrayList<>();
+ List<SeaTunnelDataType<?>> fieldsType = new ArrayList<>();
for (Map.Entry<String, String> field : fields.entrySet()) {
String srcField = field.getValue();
int srcFieldIndex = inputRowType.indexOf(srcField);
@@ -69,11 +69,11 @@ public class CopyFieldTransform extends
MultipleFieldOutputTransform {
"Cannot find [" + srcField + "] field in input row
type");
}
fieldNames.add(field.getKey());
- fieldOriginalIndexs.add(srcFieldIndex);
+ fieldOriginalIndexes.add(srcFieldIndex);
fieldsType.add(inputRowType.getFieldType(srcFieldIndex));
}
this.fieldNames = fieldNames;
- this.fieldOriginalIndexs = fieldOriginalIndexs;
+ this.fieldOriginalIndexes = fieldOriginalIndexes;
this.fieldTypes = fieldsType;
}
@@ -111,14 +111,14 @@ public class CopyFieldTransform extends
MultipleFieldOutputTransform {
@Override
protected Object[] getOutputFieldValues(SeaTunnelRowAccessor inputRow) {
Object[] fieldValues = new Object[fieldNames.size()];
- for (int i = 0; i < fieldOriginalIndexs.size(); i++) {
+ for (int i = 0; i < fieldOriginalIndexes.size(); i++) {
fieldValues[i] =
- clone(fieldTypes.get(i),
inputRow.getField(fieldOriginalIndexs.get(i)));
+ clone(fieldTypes.get(i),
inputRow.getField(fieldOriginalIndexes.get(i)));
}
return fieldValues;
}
- private Object clone(SeaTunnelDataType dataType, Object value) {
+ private Object clone(SeaTunnelDataType<?> dataType, Object value) {
if (value == null) {
return null;
}
@@ -138,18 +138,12 @@ public class CopyFieldTransform extends
MultipleFieldOutputTransform {
return value;
case BYTES:
byte[] bytes = (byte[]) value;
- if (bytes == null) {
- return null;
- }
byte[] newBytes = new byte[bytes.length];
System.arraycopy(bytes, 0, newBytes, 0, bytes.length);
return newBytes;
case ARRAY:
ArrayType arrayType = (ArrayType) dataType;
Object[] array = (Object[]) value;
- if (array == null) {
- return null;
- }
Object newArray =
Array.newInstance(arrayType.getElementType().getTypeClass(), array.length);
for (int i = 0; i < array.length; i++) {
@@ -159,7 +153,7 @@ public class CopyFieldTransform extends
MultipleFieldOutputTransform {
case MAP:
MapType mapType = (MapType) dataType;
Map map = (Map) value;
- Map newMap = new HashMap();
+ Map<Object, Object> newMap = new HashMap<>();
for (Object key : map.keySet()) {
newMap.put(
clone(mapType.getKeyType(), key),
@@ -169,9 +163,6 @@ public class CopyFieldTransform extends
MultipleFieldOutputTransform {
case ROW:
SeaTunnelRowType rowType = (SeaTunnelRowType) dataType;
SeaTunnelRow row = (SeaTunnelRow) value;
- if (row == null) {
- return null;
- }
Object[] newFields = new Object[rowType.getTotalFields()];
for (int i = 0; i < rowType.getTotalFields(); i++) {
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java
index f90c498219..6d4e312f20 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/fieldmapper/FieldMapperTransform.java
@@ -45,7 +45,7 @@ import java.util.stream.Collectors;
@Slf4j
public class FieldMapperTransform extends AbstractCatalogSupportTransform {
public static String PLUGIN_NAME = "FieldMapper";
- private FieldMapperTransformConfig config;
+ private final FieldMapperTransformConfig config;
private List<Integer> needReaderColIndex;
public FieldMapperTransform(
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java
index 011713a982..0105149036 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filter/FilterFieldTransform.java
@@ -44,7 +44,7 @@ import java.util.stream.Collectors;
public class FilterFieldTransform extends AbstractCatalogSupportTransform {
public static final String PLUGIN_NAME = "Filter";
private int[] inputValueIndex;
- private String[] fields;
+ private final String[] fields;
public FilterFieldTransform(
@NonNull ReadonlyConfig config, @NonNull CatalogTable
catalogTable) {
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java
index de9ef3a466..9df88d5061 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/filterrowkind/FilterRowKindTransform.java
@@ -52,11 +52,9 @@ public class FilterRowKindTransform extends
FilterRowTransform {
private void initConfig(ReadonlyConfig config) {
if (config.get(FilterRowKinkTransformConfig.INCLUDE_KINDS) == null) {
- excludeKinds =
- new
HashSet<RowKind>(config.get(FilterRowKinkTransformConfig.EXCLUDE_KINDS));
+ excludeKinds = new
HashSet<>(config.get(FilterRowKinkTransformConfig.EXCLUDE_KINDS));
} else {
- includeKinds =
- new
HashSet<RowKind>(config.get(FilterRowKinkTransformConfig.INCLUDE_KINDS));
+ includeKinds = new
HashSet<>(config.get(FilterRowKinkTransformConfig.INCLUDE_KINDS));
}
if ((includeKinds.isEmpty() && excludeKinds.isEmpty())
|| (!includeKinds.isEmpty() && !excludeKinds.isEmpty())) {
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java
index b72ad39c87..a99aab49b0 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/replace/ReplaceTransform.java
@@ -32,7 +32,7 @@ import java.util.List;
import java.util.stream.Collectors;
public class ReplaceTransform extends SingleFieldOutputTransform {
- private ReadonlyConfig config;
+ private final ReadonlyConfig config;
private int inputFieldIndex;
public ReplaceTransform(
@@ -65,9 +65,8 @@ public class ReplaceTransform extends
SingleFieldOutputTransform {
}
boolean isRegex =
- config.get(ReplaceTransformConfig.KEY_IS_REGEX) == null
- ? false
- : config.get(ReplaceTransformConfig.KEY_IS_REGEX);
+ config.get(ReplaceTransformConfig.KEY_IS_REGEX) != null
+ && config.get(ReplaceTransformConfig.KEY_IS_REGEX);
if (isRegex) {
if (config.get(ReplaceTransformConfig.KEY_REPLACE_FIRST)) {
return inputFieldValue
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java
index 7132388f8e..46c38639fd 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/split/SplitTransform.java
@@ -28,12 +28,10 @@ import
org.apache.seatunnel.transform.common.SeaTunnelRowAccessor;
import lombok.NonNull;
import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
public class SplitTransform extends MultipleFieldOutputTransform {
- private SplitTransformConfig splitTransformConfig;
- private int splitFieldIndex;
+ private final SplitTransformConfig splitTransformConfig;
+ private final int splitFieldIndex;
public SplitTransform(
@NonNull SplitTransformConfig splitTransformConfig,
@@ -79,14 +77,11 @@ public class SplitTransform extends
MultipleFieldOutputTransform {
@Override
protected Column[] getOutputColumns() {
- List<PhysicalColumn> collect =
- Arrays.stream(splitTransformConfig.getOutputFields())
- .map(
- fieldName -> {
- return PhysicalColumn.of(
- fieldName, BasicType.STRING_TYPE,
200, true, "", "");
- })
- .collect(Collectors.toList());
- return collect.toArray(new Column[0]);
+ return Arrays.stream(splitTransformConfig.getOutputFields())
+ .map(
+ fieldName ->
+ PhysicalColumn.of(
+ fieldName, BasicType.STRING_TYPE, 200,
true, "", ""))
+ .toArray(Column[]::new);
}
}
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 2fcbaa05ee..bddb1c64f2 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
@@ -56,9 +56,9 @@ public class SQLTransform extends
AbstractCatalogSupportTransform {
.defaultValue(ZETA.name())
.withDescription("The SQL engine type");
- private String query;
+ private final String query;
- private EngineType engineType;
+ private final EngineType engineType;
private transient SQLEngine sqlEngine;
@@ -126,7 +126,7 @@ public class SQLTransform extends
AbstractCatalogSupportTransform {
if (inputCatalogTable.getTableSchema().getPrimaryKey() != null
&& outputColumns.containsAll(
inputCatalogTable.getTableSchema().getPrimaryKey().getColumnNames())) {
- builder =
builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy());
+
builder.primaryKey(inputCatalogTable.getTableSchema().getPrimaryKey().copy());
}
List<ConstraintKey> outputConstraintKeys =
@@ -144,7 +144,7 @@ public class SQLTransform extends
AbstractCatalogSupportTransform {
.map(ConstraintKey::copy)
.collect(Collectors.toList());
- builder = builder.constraintKey(outputConstraintKeys);
+ builder.constraintKey(outputConstraintKeys);
String[] fieldNames = outRowType.getFieldNames();
SeaTunnelDataType<?>[] fieldTypes = outRowType.getFieldTypes();
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 7f1130ba27..b34dcb2c2a 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
@@ -244,9 +244,6 @@ public class ZetaSQLEngine implements SQLEngine {
int columnsSize = countColumnsSize(selectItems);
Object[] fields = new Object[columnsSize];
- for (int i = 0; i < columnsSize; i++) {
- fields[i] = null;
- }
int idx = 0;
for (SelectItem selectItem : selectItems) {
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/NumericFunction.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/NumericFunction.java
index 4c14b57392..055116430c 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/NumericFunction.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/NumericFunction.java
@@ -382,7 +382,7 @@ public class NumericFunction {
public static Double random(List<Object> args) {
Random random = new Random();
- if (args.size() >= 1) {
+ if (!args.isEmpty()) {
Number v1 = (Number) args.get(0);
if (v1 != null) {
random.setSeed(v1.intValue());
diff --git
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/udf/DESUtil.java
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/udf/DESUtil.java
index c698abb496..c6cd5d2f05 100644
---
a/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/udf/DESUtil.java
+++
b/seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sql/zeta/functions/udf/DESUtil.java
@@ -17,6 +17,8 @@
package org.apache.seatunnel.transform.sql.zeta.functions.udf;
+import lombok.extern.slf4j.Slf4j;
+
import javax.crypto.Cipher;
import javax.crypto.SecretKeyFactory;
import javax.crypto.spec.DESKeySpec;
@@ -25,6 +27,7 @@ import javax.crypto.spec.IvParameterSpec;
import java.security.Key;
import java.util.Base64;
+@Slf4j
public class DESUtil {
private static final String IV_PARAMETER = "12345678";
@@ -56,7 +59,7 @@ public class DESUtil {
return new String(Base64.getEncoder().encode(bytes));
} catch (Exception e) {
- e.printStackTrace();
+ log.error("Encrypt failed", e);
return data;
}
}
@@ -74,7 +77,7 @@ public class DESUtil {
return new String(
cipher.doFinal(Base64.getDecoder().decode(data.getBytes(CHARSET))), CHARSET);
} catch (Exception e) {
- e.printStackTrace();
+ log.error("Decrypt failed", e);
return data;
}
}