From 1b62fdd49edd3577732a8f8b7fec0bdbedb09b2e Mon Sep 17 00:00:00 2001 From: kamianlaida <165994047+wohainilaodou@users.noreply.github.com> Date: Thu, 29 Aug 2024 16:29:45 +0800 Subject: [PATCH 01/30] [INLONG-10947][Dashboard] Modify field description missing for English locale (#10948) --- inlong-dashboard/src/ui/locales/en.json | 1 + 1 file changed, 1 insertion(+) diff --git a/inlong-dashboard/src/ui/locales/en.json b/inlong-dashboard/src/ui/locales/en.json index c281478532..ebf2bbfcb1 100644 --- a/inlong-dashboard/src/ui/locales/en.json +++ b/inlong-dashboard/src/ui/locales/en.json @@ -838,6 +838,7 @@ "pages.Clusters.Node.Status": "Status", "pages.Clusters.Node.Status.Normal": "Normal", "pages.Clusters.Node.Status.Timeout": "Timeout", + "pages.Clusters.Node.LastModifier": "Last modifier", "pages.Clusters.Node.Creator": "Creator", "pages.Clusters.Node.Create": "Create", "pages.Clusters.Node.IpRule": "Please enter the IP address correctly", From a9719038f14245ea92c65446c28b5fa442757b0a Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Thu, 29 Aug 2024 20:19:28 +0800 Subject: [PATCH 02/30] [INLONG-10946][SDK] Transform SQL supports SPACE function (#10949) --- .../process/function/SpaceFunction.java | 59 +++++++++++++++++++ ...TestTransformStringFunctionsProcessor.java | 39 ++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SpaceFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SpaceFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SpaceFunction.java new file mode 100644 index 0000000000..30c300488a --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SpaceFunction.java @@ -0,0 +1,59 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Function; + +/** + * SpaceFunction + * description: SPACE(N) + * - return NULL if N is NULL. + * - return "" if N is less than or equal to 0 + * - return a string consisting of N space characters + */ +@TransformFunction(names = {"space"}) +public class SpaceFunction implements ValueParser { + + private final ValueParser cntParser; + + public SpaceFunction(Function expr) { + cntParser = OperatorTools.buildParser(expr.getParameters().getExpressions().get(0)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object cntObj = cntParser.parse(sourceData, rowIndex, context); + if (cntObj == null) { + return null; + } + int cnt = OperatorTools.parseBigDecimal(cntObj).intValue(); + if (cnt <= 0) { + return ""; + } + StringBuilder builder = new StringBuilder(cnt); + for (int i = 0; i < cnt; i++) { + builder.append(" "); + } + return builder.toString(); + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java index 2a47615958..d8576d12fc 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java @@ -175,6 +175,44 @@ public void testLocateFunction() throws Exception { Assert.assertEquals(1, output5.size()); Assert.assertEquals(output5.get(0), "result=null"); } + + @Test + public void testSpaceFunction() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + transformSql = "select space(numeric1) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case1: space(5) + data = "hello world|banana|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result= ", output.get(0)); + + // case2: space(-1) + data = "hello world|banana|cloud|-1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=", output.get(0)); + + // case3: space(null) + transformSql = "select space(xxd) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "hello world|banana|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + } + @Test public void testReplicateFunction() throws Exception { String transformSql1 = "select replicate(string1, numeric1) from source"; @@ -310,6 +348,7 @@ public void testLengthFunction() throws Exception { Assert.assertEquals(1, output1.size()); Assert.assertEquals("result=null", output1.get(0)); } + @Test public void testReplaceFunction() throws Exception { String transformSql = "select replace(string1, string2, string3) from source"; From f35ab6f619004fac58a06f79b98e681e778622f0 Mon Sep 17 00:00:00 2001 From: Xincheng Huang <60057611+ying-hua@users.noreply.github.com> Date: Thu, 29 Aug 2024 20:19:44 +0800 Subject: [PATCH 03/30] [INLONG-10934][SDK] Transform support LocalDate function (#10951) --- .../process/function/LocalDateFunction.java | 56 +++++++++++++++++++ ...stTransformTemporalFunctionsProcessor.java | 42 ++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LocalDateFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LocalDateFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LocalDateFunction.java new file mode 100644 index 0000000000..ba2d909d95 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LocalDateFunction.java @@ -0,0 +1,56 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Function; + +import java.time.LocalDate; +import java.time.ZoneId; + +/** + * LocalDateFunction + * description: + * localDate([string1]) returns the current date in the specified time zone. + * (by default: the current date in the system time zone) + */ +@TransformFunction(names = {"localdate", "currentdate", "current_date", "curdate"}) +public class LocalDateFunction implements ValueParser { + + private ValueParser stringParser; + + public LocalDateFunction(Function expr) { + if (expr.getParameters() != null) { + stringParser = OperatorTools.buildParser(expr.getParameters().getExpressions().get(0)); + } + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + if (stringParser != null) { + String zoneString = OperatorTools.parseString(stringParser.parse(sourceData, rowIndex, context)); + return LocalDate.now(ZoneId.of(zoneString)); + } else { + return LocalDate.now(ZoneId.systemDefault()); + } + } +} \ No newline at end of file diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java index e0830dd72e..354b8b7f18 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java @@ -29,6 +29,7 @@ import org.junit.Test; import java.time.Duration; +import java.time.LocalDate; import java.time.LocalTime; import java.time.ZoneId; import java.time.format.DateTimeFormatter; @@ -421,6 +422,47 @@ public void testLocalTimeFunction() throws Exception { Assert.assertTrue(duration3.getSeconds() < 1); } + @Test + public void testLocalDateFunction() throws Exception { + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + + // case1: localDate() - default system time zone + String transformSql1 = "select localdate() from source"; + TransformConfig config1 = new TransformConfig(transformSql1); + TransformProcessor processor1 = TransformProcessor + .create(config1, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor1.transform("", new HashMap<>()); + LocalDate expectedDate1 = LocalDate.now(ZoneId.systemDefault()); + LocalDate actualDate1 = LocalDate.parse(output1.get(0).split("=")[1], formatter); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(expectedDate1, actualDate1); + + // case2: localDate("UTC") + String transformSql2 = "select localdate('UTC') from source"; + TransformConfig config2 = new TransformConfig(transformSql2); + TransformProcessor processor2 = TransformProcessor + .create(config2, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output2 = processor2.transform("", new HashMap<>()); + LocalDate expectedDate2 = LocalDate.now(ZoneId.of("UTC")); + LocalDate actualDate2 = LocalDate.parse(output2.get(0).split("=")[1], formatter); + Assert.assertEquals(1, output2.size()); + Assert.assertEquals(expectedDate2, actualDate2); + + // case3: localDate("UTC-12") + String transformSql3 = "select localdate('UTC-12') from source"; + TransformConfig config3 = new TransformConfig(transformSql3); + TransformProcessor processor3 = TransformProcessor + .create(config3, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor3.transform("", new HashMap<>()); + LocalDate expectedDate3 = LocalDate.now(ZoneId.of("UTC-12")); + LocalDate actualDate3 = LocalDate.parse(output3.get(0).split("=")[1], formatter); + Assert.assertEquals(1, output3.size()); + Assert.assertEquals(expectedDate3, actualDate3); + } + @Test public void testTimestampAdd() throws Exception { String transformSql1 = "select timestamp_add('day',string2,string1) from source"; From 7cfbf8a74bb9570daa5171ed9300420fb37e1489 Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Fri, 30 Aug 2024 09:53:02 +0800 Subject: [PATCH 04/30] [INLONG-10939][SDK] Transform SQL supports STRCMP function (#10941) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../process/function/StrcmpFunction.java | 67 +++++++++++++++++++ ...TestTransformStringFunctionsProcessor.java | 42 ++++++++++++ 2 files changed, 109 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/StrcmpFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/StrcmpFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/StrcmpFunction.java new file mode 100644 index 0000000000..ebbcc94ae9 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/StrcmpFunction.java @@ -0,0 +1,67 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.util.List; + +/** + * StrcmpFunction + * description: strcmp(s1,s2) + * return NULL if either argument is NULL + * return 0 if the strings are the same + * return -1 if the first argument is smaller than the second according to the current sort order + * return 1 otherwise + */ +@TransformFunction(names = {"strcmp"}) +public class StrcmpFunction implements ValueParser { + + private final ValueParser leftStringParser; + private final ValueParser rightStringParser; + + public StrcmpFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + leftStringParser = OperatorTools.buildParser(expressions.get(0)); + rightStringParser = OperatorTools.buildParser(expressions.get(1)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object leftStringObj = leftStringParser.parse(sourceData, rowIndex, context); + Object rightStringObj = rightStringParser.parse(sourceData, rowIndex, context); + if (leftStringObj == null || rightStringObj == null) { + return null; + } + String leftString = OperatorTools.parseString(leftStringObj); + String rightString = OperatorTools.parseString(rightStringObj); + int cmp = OperatorTools.compareValue(leftString, rightString); + if (cmp > 0) { + return 1; + } else if (cmp < 0) { + return -1; + } + return 0; + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java index d8576d12fc..10cfa740c9 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java @@ -386,6 +386,48 @@ public void testReplaceFunction() throws Exception { Assert.assertEquals(output7.get(0), "result=da"); } + @Test + public void testStrcmpFunction() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + transformSql = "select strcmp(string1,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case1: strcmp('hello world','banana') + data = "hello world|banana|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=1", output.get(0)); + + // case2: strcmp('hello world','hello world') + data = "hello world|hello world|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=0", output.get(0)); + + // case3: strcmp('hello world','zzzzz') + data = "hello world|zzzzz|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=-1", output.get(0)); + + // case4: strcmp('hello world',null) + transformSql = "select strcmp(string1,xxd) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "hello world|zzzzz|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + @Test public void testRightFunction() throws Exception { String transformSql = "select right(string1,numeric1) from source"; From 88f0d4e1fbe9364157ed4b6642d51f77d141b5d7 Mon Sep 17 00:00:00 2001 From: fuweng11 <76141879+fuweng11@users.noreply.github.com> Date: Fri, 30 Aug 2024 17:42:41 +0800 Subject: [PATCH 05/30] [INLONG-10954][Manager] Support fields of timestamptz type (#10955) --- .../apache/inlong/manager/pojo/sort/util/FieldInfoUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sort/util/FieldInfoUtils.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sort/util/FieldInfoUtils.java index 05a541770e..0893caf853 100644 --- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sort/util/FieldInfoUtils.java +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sort/util/FieldInfoUtils.java @@ -298,7 +298,6 @@ public static FormatInfo convertFieldFormat(String type, String format) { formatInfo = new TimeFormatInfo(); } break; - case TIMESTAMPTZ: case TIMESTAMP: case DATETIME: if (StringUtils.isNotBlank(format)) { @@ -307,6 +306,7 @@ public static FormatInfo convertFieldFormat(String type, String format) { formatInfo = new TimestampFormatInfo(); } break; + case TIMESTAMPTZ: case LOCAL_ZONE_TIMESTAMP: if (StringUtils.isNotBlank(format)) { formatInfo = new LocalZonedTimestampFormatInfo(convertTimestampOrDataFormat(format), 2); From 681e88da886c31f6d368a0cda3ff14d251099c63 Mon Sep 17 00:00:00 2001 From: PeterZh6 Date: Sat, 31 Aug 2024 16:46:53 +0800 Subject: [PATCH 06/30] [INLONG-7056][Sort] Adjust sort resources according to data scale (#10916) --- .../audit/consts}/OpenApiConstants.java | 6 +- .../inlong/audit/cache/AbstractCache.java | 8 +- .../inlong/audit/cache/RealTimeQuery.java | 4 +- .../inlong/audit/service/ApiService.java | 102 ++++---- .../flink/FlinkParallelismOptimizer.java | 218 ++++++++++++++++++ .../manager/plugin/flink/FlinkService.java | 22 +- .../manager/plugin/flink/dto/FlinkConfig.java | 5 + .../manager/plugin/flink/enums/Constants.java | 4 + .../util/ApplicationContextProvider.java | 41 ++++ .../manager/plugin/util/FlinkUtils.java | 3 + .../resources/flink-sort-plugin.properties | 4 + 11 files changed, 356 insertions(+), 61 deletions(-) rename inlong-audit/{audit-service/src/main/java/org/apache/inlong/audit/config => audit-common/src/main/java/org/apache/inlong/audit/consts}/OpenApiConstants.java (95%) create mode 100644 inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkParallelismOptimizer.java create mode 100644 inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/ApplicationContextProvider.java diff --git a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/config/OpenApiConstants.java b/inlong-audit/audit-common/src/main/java/org/apache/inlong/audit/consts/OpenApiConstants.java similarity index 95% rename from inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/config/OpenApiConstants.java rename to inlong-audit/audit-common/src/main/java/org/apache/inlong/audit/consts/OpenApiConstants.java index a727eba46b..cdb2a05fda 100644 --- a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/config/OpenApiConstants.java +++ b/inlong-audit/audit-common/src/main/java/org/apache/inlong/audit/consts/OpenApiConstants.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.audit.config; +package org.apache.inlong.audit.consts; /** * Open api constants @@ -54,8 +54,8 @@ public class OpenApiConstants { public static final String PARAMS_END_TIME = "endTime"; public static final String PARAMS_AUDIT_ID = "auditId"; public static final String PARAMS_AUDIT_TAG = "auditTag"; - public static final String PARAMS_INLONG_GROUP_Id = "inlongGroupId"; - public static final String PARAMS_INLONG_STREAM_Id = "inlongStreamId"; + public static final String PARAMS_INLONG_GROUP_ID = "inlongGroupId"; + public static final String PARAMS_INLONG_STREAM_ID = "inlongStreamId"; public static final String PARAMS_IP = "ip"; public static final String PARAMS_AUDIT_CYCLE = "auditCycle"; public static final String KEY_HTTP_BODY_SUCCESS = "success"; diff --git a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/AbstractCache.java b/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/AbstractCache.java index 766f64f70d..8463501ffd 100644 --- a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/AbstractCache.java +++ b/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/AbstractCache.java @@ -37,11 +37,11 @@ import java.util.concurrent.TimeUnit; import static org.apache.inlong.audit.config.ConfigConstants.DATE_FORMAT; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_CACHE_EXPIRED_HOURS; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_CACHE_MAX_SIZE; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_CACHE_EXPIRED_HOURS; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_CACHE_MAX_SIZE; import static org.apache.inlong.audit.consts.ConfigConstants.DEFAULT_AUDIT_TAG; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_CACHE_EXPIRED_HOURS; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_CACHE_MAX_SIZE; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_CACHE_EXPIRED_HOURS; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_CACHE_MAX_SIZE; /** * Abstract cache. diff --git a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/RealTimeQuery.java b/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/RealTimeQuery.java index 45984e203b..21991ed564 100644 --- a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/RealTimeQuery.java +++ b/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/cache/RealTimeQuery.java @@ -51,14 +51,14 @@ import static org.apache.inlong.audit.config.ConfigConstants.KEY_DATASOURCE_MAX_IDLE_CONNECTIONS; import static org.apache.inlong.audit.config.ConfigConstants.KEY_DATASOURCE_MAX_TOTAL_CONNECTIONS; import static org.apache.inlong.audit.config.ConfigConstants.KEY_DATASOURCE_MIN_IDLE_CONNECTIONS; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_THREAD_POOL_SIZE; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_THREAD_POOL_SIZE; import static org.apache.inlong.audit.config.SqlConstants.DEFAULT_SOURCE_QUERY_IDS_SQL; import static org.apache.inlong.audit.config.SqlConstants.DEFAULT_SOURCE_QUERY_IPS_SQL; import static org.apache.inlong.audit.config.SqlConstants.DEFAULT_SOURCE_QUERY_MINUTE_SQL; import static org.apache.inlong.audit.config.SqlConstants.KEY_SOURCE_QUERY_IDS_SQL; import static org.apache.inlong.audit.config.SqlConstants.KEY_SOURCE_QUERY_IPS_SQL; import static org.apache.inlong.audit.config.SqlConstants.KEY_SOURCE_QUERY_MINUTE_SQL; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_THREAD_POOL_SIZE; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_THREAD_POOL_SIZE; /** * Real time query data from audit source. diff --git a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/service/ApiService.java b/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/service/ApiService.java index fa1c56ab23..3fc095f643 100644 --- a/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/service/ApiService.java +++ b/inlong-audit/audit-service/src/main/java/org/apache/inlong/audit/service/ApiService.java @@ -48,42 +48,42 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_BACKLOG_SIZE; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_DAY_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_GET_AUDIT_PROXY_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_GET_IDS_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_GET_IPS_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_HOUR_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_MINUTES_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_REAL_LIMITER_QPS; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_API_THREAD_POOL_SIZE; -import static org.apache.inlong.audit.config.OpenApiConstants.DEFAULT_HTTP_SERVER_BIND_PORT; -import static org.apache.inlong.audit.config.OpenApiConstants.HTTP_RESPOND_CODE; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_BACKLOG_SIZE; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_DAY_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_GET_AUDIT_PROXY_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_GET_IDS_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_GET_IPS_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_HOUR_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_MINUTES_PATH; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_REAL_LIMITER_QPS; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_API_THREAD_POOL_SIZE; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_HTTP_BODY_ERR_DATA; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_HTTP_BODY_ERR_MSG; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_HTTP_BODY_SUCCESS; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_HTTP_HEADER_CONTENT_TYPE; -import static org.apache.inlong.audit.config.OpenApiConstants.KEY_HTTP_SERVER_BIND_PORT; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_AUDIT_COMPONENT; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_AUDIT_CYCLE; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_AUDIT_ID; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_AUDIT_TAG; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_END_TIME; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_INLONG_GROUP_Id; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_INLONG_STREAM_Id; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_IP; -import static org.apache.inlong.audit.config.OpenApiConstants.PARAMS_START_TIME; -import static org.apache.inlong.audit.config.OpenApiConstants.VALUE_HTTP_HEADER_CONTENT_TYPE; import static org.apache.inlong.audit.consts.ConfigConstants.DEFAULT_AUDIT_TAG; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_BACKLOG_SIZE; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_DAY_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_GET_AUDIT_PROXY_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_GET_IDS_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_GET_IPS_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_HOUR_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_MINUTES_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_REAL_LIMITER_QPS; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_API_THREAD_POOL_SIZE; +import static org.apache.inlong.audit.consts.OpenApiConstants.DEFAULT_HTTP_SERVER_BIND_PORT; +import static org.apache.inlong.audit.consts.OpenApiConstants.HTTP_RESPOND_CODE; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_BACKLOG_SIZE; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_DAY_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_GET_AUDIT_PROXY_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_GET_IDS_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_GET_IPS_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_HOUR_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_MINUTES_PATH; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_REAL_LIMITER_QPS; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_API_THREAD_POOL_SIZE; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_HTTP_BODY_ERR_DATA; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_HTTP_BODY_ERR_MSG; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_HTTP_BODY_SUCCESS; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_HTTP_HEADER_CONTENT_TYPE; +import static org.apache.inlong.audit.consts.OpenApiConstants.KEY_HTTP_SERVER_BIND_PORT; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_AUDIT_COMPONENT; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_AUDIT_CYCLE; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_AUDIT_ID; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_AUDIT_TAG; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_END_TIME; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_INLONG_GROUP_ID; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_INLONG_STREAM_ID; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_IP; +import static org.apache.inlong.audit.consts.OpenApiConstants.PARAMS_START_TIME; +import static org.apache.inlong.audit.consts.OpenApiConstants.VALUE_HTTP_HEADER_CONTENT_TYPE; import static org.apache.inlong.audit.entities.ApiType.DAY; import static org.apache.inlong.audit.entities.ApiType.GET_AUDIT_PROXY; import static org.apache.inlong.audit.entities.ApiType.GET_IDS; @@ -208,14 +208,14 @@ private boolean checkNecessaryParams(Map params) { return params.containsKey(PARAMS_START_TIME) && params.containsKey(PARAMS_END_TIME) && params.containsKey(PARAMS_AUDIT_ID) - && params.containsKey(PARAMS_INLONG_GROUP_Id) - && params.containsKey(PARAMS_INLONG_STREAM_Id); + && params.containsKey(PARAMS_INLONG_GROUP_ID) + && params.containsKey(PARAMS_INLONG_STREAM_ID); case MINUTES: return params.containsKey(PARAMS_START_TIME) && params.containsKey(PARAMS_END_TIME) && params.containsKey(PARAMS_AUDIT_ID) - && params.containsKey(PARAMS_INLONG_GROUP_Id) - && params.containsKey(PARAMS_INLONG_STREAM_Id) + && params.containsKey(PARAMS_INLONG_GROUP_ID) + && params.containsKey(PARAMS_INLONG_STREAM_ID) && params.containsKey(PARAMS_AUDIT_CYCLE); case GET_IDS: return params.containsKey(PARAMS_START_TIME) @@ -249,8 +249,8 @@ private void handleLegalParams(JsonObject responseJson, Map para case HOUR: statData = HourCache.getInstance().getData(params.get(PARAMS_START_TIME), params.get(PARAMS_END_TIME), - params.get(PARAMS_INLONG_GROUP_Id), - params.get(PARAMS_INLONG_STREAM_Id), + params.get(PARAMS_INLONG_GROUP_ID), + params.get(PARAMS_INLONG_STREAM_ID), params.get(PARAMS_AUDIT_ID), params.get(PARAMS_AUDIT_TAG)); responseJson.add(KEY_HTTP_BODY_ERR_DATA, gson.toJsonTree(statData)); @@ -259,8 +259,8 @@ private void handleLegalParams(JsonObject responseJson, Map para statData = DayCache.getInstance().getData( params.get(PARAMS_START_TIME), params.get(PARAMS_END_TIME), - params.get(PARAMS_INLONG_GROUP_Id), - params.get(PARAMS_INLONG_STREAM_Id), + params.get(PARAMS_INLONG_GROUP_ID), + params.get(PARAMS_INLONG_STREAM_ID), params.get(PARAMS_AUDIT_ID)); responseJson.add(KEY_HTTP_BODY_ERR_DATA, gson.toJsonTree(statData)); break; @@ -276,8 +276,8 @@ private void handleLegalParams(JsonObject responseJson, Map para statData = RealTimeQuery.getInstance().queryIpsById( params.get(PARAMS_START_TIME), params.get(PARAMS_END_TIME), - params.get(PARAMS_INLONG_GROUP_Id), - params.get(PARAMS_INLONG_STREAM_Id), + params.get(PARAMS_INLONG_GROUP_ID), + params.get(PARAMS_INLONG_STREAM_ID), params.get(PARAMS_AUDIT_ID)); responseJson.add(KEY_HTTP_BODY_ERR_DATA, gson.toJsonTree(statData)); break; @@ -303,22 +303,22 @@ private List handleMinutesApi(Map params) { case MINUTE: statData = RealTimeQuery.getInstance().queryLogTs(params.get(PARAMS_START_TIME), params.get(PARAMS_END_TIME), - params.get(PARAMS_INLONG_GROUP_Id), - params.get(PARAMS_INLONG_STREAM_Id), + params.get(PARAMS_INLONG_GROUP_ID), + params.get(PARAMS_INLONG_STREAM_ID), params.get(PARAMS_AUDIT_ID)); break; case MINUTE_10: statData = TenMinutesCache.getInstance().getData(params.get(PARAMS_START_TIME), params.get(PARAMS_END_TIME), - params.get(PARAMS_INLONG_GROUP_Id), - params.get(PARAMS_INLONG_STREAM_Id), params.get(PARAMS_AUDIT_ID), + params.get(PARAMS_INLONG_GROUP_ID), + params.get(PARAMS_INLONG_STREAM_ID), params.get(PARAMS_AUDIT_ID), params.get(PARAMS_AUDIT_TAG)); break; case MINUTE_30: statData = HalfHourCache.getInstance().getData(params.get(PARAMS_START_TIME), params.get(PARAMS_END_TIME), - params.get(PARAMS_INLONG_GROUP_Id), - params.get(PARAMS_INLONG_STREAM_Id), params.get(PARAMS_AUDIT_ID), + params.get(PARAMS_INLONG_GROUP_ID), + params.get(PARAMS_INLONG_STREAM_ID), params.get(PARAMS_AUDIT_ID), params.get(PARAMS_AUDIT_TAG)); break; default: diff --git a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkParallelismOptimizer.java b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkParallelismOptimizer.java new file mode 100644 index 0000000000..121a489655 --- /dev/null +++ b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkParallelismOptimizer.java @@ -0,0 +1,218 @@ +/* + * 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.inlong.manager.plugin.flink; + +import org.apache.inlong.audit.AuditIdEnum; +import org.apache.inlong.audit.entity.FlowType; +import org.apache.inlong.manager.pojo.audit.AuditInfo; +import org.apache.inlong.manager.pojo.stream.InlongStreamInfo; + +import com.google.gson.Gson; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import lombok.extern.slf4j.Slf4j; +import org.apache.http.HttpEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Component; + +import java.io.IOException; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.StringJoiner; + +import static org.apache.inlong.audit.consts.OpenApiConstants.*; +import static org.apache.inlong.manager.common.consts.InlongConstants.*; + +/** + * This class is used to calculate the recommended parallelism based on the maximum message per second per core. + * The data volume is calculated based on the average data count per hour. + * The data count is retrieved from the inlong audit API. + */ +@Slf4j +@Component +public class FlinkParallelismOptimizer { + + @Value("${audit.query.url:http://127.0.0.1:10080}") + public String auditQueryUrl; + + private static final int MAX_PARALLELISM = 2048; + private long maximumMessagePerSecondPerCore = 1000L; + private static final long DEFAULT_ERROR_DATA_VOLUME = 0L; + private static final FlowType DEFAULT_FLOWTYPE = FlowType.OUTPUT; + private static final String DEFAULT_AUDIT_TYPE = "DataProxy"; + private static final String AUDIT_CYCLE_REALTIME = "1"; + // maxmimum data scale counting range in hours + private static final int DATA_SCALE_COUNTING_RANGE_IN_HOURS = 1; + // sample time format: 2024-08-23T22:47:38.866 + private static final String AUDIT_QUERY_DATE_TIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS"; + + private static final String LOGTS_DATE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss"; + private static final String TIMEZONE_REGEX = "([+-])(\\d):"; + + /** + * Calculate recommended parallelism based on maximum message per second per core + * + * @return Recommended parallelism + */ + public int calculateRecommendedParallelism(List streamInfos) { + long averageDataVolume; + InlongStreamInfo streamInfo = streamInfos.get(0); + try { + averageDataVolume = getAverageDataVolume(streamInfo); + log.info("Retrieved data volume: {}", averageDataVolume); + } catch (Exception e) { + log.error("Error retrieving data volume: {}", e.getMessage(), e); + averageDataVolume = DEFAULT_ERROR_DATA_VOLUME; + } + int newParallelism = (int) (averageDataVolume / maximumMessagePerSecondPerCore); + // Ensure parallelism is at most MAX_PARALLELISM + newParallelism = Math.min(newParallelism, MAX_PARALLELISM); + log.info("Calculated parallelism: {} for data volume: {}", newParallelism, averageDataVolume); + return newParallelism; + } + + /** + * Initialize maximum message per second per core based on configuration + * + * @param maximumMessagePerSecondPerCore The maximum messages per second per core + */ + public void setMaximumMessagePerSecondPerCore(Integer maximumMessagePerSecondPerCore) { + if (maximumMessagePerSecondPerCore == null || maximumMessagePerSecondPerCore <= 0) { + log.error( + "Illegal flink.max.msg.rate.percore property, must be nonnull and positive, using default value: {}", + this.maximumMessagePerSecondPerCore); + } else { + this.maximumMessagePerSecondPerCore = maximumMessagePerSecondPerCore; + } + } + + /** + * Get average data volume on the scale specified by DATA_SCALE_COUNTING_RANGE_IN_HOURS + * + * @param streamInfo inlong stream info + * @return The average data count per hour + */ + private long getAverageDataVolume(InlongStreamInfo streamInfo) { + // Since the audit module uses local time, we need to use ZonedDateTime to get the current time + String dataTimeZone = streamInfo.getSourceList().get(0).getDataTimeZone(); + + // This regex pattern matches a time zone offset in the format of "GMT+/-X:00" + // where X is a single digit (e.g., "GMT+8:00"). The pattern captures the "+" or "-" sign + // and the single digit, then it replaces the single digit with two digits by adding a "0" in front of it. + // For example, "GMT+8:00" becomes "GMT+08:00" in order to match standard offset-based ZoneId. + dataTimeZone = dataTimeZone.replaceAll(TIMEZONE_REGEX, "$10$2:"); + ZoneId dataZone = ZoneId.of(dataTimeZone); + + ZonedDateTime endTime = ZonedDateTime.now(dataZone); + ZonedDateTime startTime = endTime.minusHours(DATA_SCALE_COUNTING_RANGE_IN_HOURS); + DateTimeFormatter formatter = DateTimeFormatter.ofPattern(AUDIT_QUERY_DATE_TIME_FORMAT); + + // counting data volume on with DATA_PROXY_OUTPUT auditId + int auditId = AuditIdEnum.getAuditId(DEFAULT_AUDIT_TYPE, DEFAULT_FLOWTYPE).getValue(); + StringJoiner urlParameters = new StringJoiner(AMPERSAND) + .add(PARAMS_START_TIME + EQUAL + startTime.format(formatter)) + .add(PARAMS_END_TIME + EQUAL + endTime.format(formatter)) + .add(PARAMS_INLONG_GROUP_ID + EQUAL + streamInfo.getInlongGroupId()) + .add(PARAMS_INLONG_STREAM_ID + EQUAL + streamInfo.getInlongStreamId()) + .add(PARAMS_AUDIT_ID + EQUAL + auditId) + .add(PARAMS_AUDIT_CYCLE + EQUAL + AUDIT_CYCLE_REALTIME); + + String url = auditQueryUrl + DEFAULT_API_MINUTES_PATH + QUESTION_MARK + urlParameters; + + return getAverageDataVolumeFromAuditInfo(url); + } + + /** + * Request audit data from inlong audit API, parse the response and return the total count in the given time range. + * + * @param url The URL to request data from + * @return The total count of the audit data + */ + private long getAverageDataVolumeFromAuditInfo(String url) { + log.debug("Requesting audit data from URL: {}", url); + try (CloseableHttpClient httpClient = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet(url); + try (CloseableHttpResponse response = httpClient.execute(httpGet)) { + return parseResponseAndCalculateAverageDataVolume(response); + } catch (IOException e) { + log.error("Error executing HTTP request to audit API: {}", url, e); + } + } catch (IOException e) { + log.error("Error creating or closing HTTP client: {}", url, e); + } + return DEFAULT_ERROR_DATA_VOLUME; + } + + /** + * Parse the HTTP response and calculate the total count from the audit data. + * + * @param response The HTTP response + * @return The total count of the audit data + * @throws IOException If an I/O error occurs + */ + private long parseResponseAndCalculateAverageDataVolume(CloseableHttpResponse response) throws IOException { + HttpEntity entity = response.getEntity(); + if (entity == null) { + log.warn("Empty response entity from audit API, returning default count."); + return DEFAULT_ERROR_DATA_VOLUME; + } + + String responseString = EntityUtils.toString(entity); + log.debug("Flink dynamic parallelism optimizer got response from audit API: {}", responseString); + + JsonObject jsonObject = JsonParser.parseString(responseString).getAsJsonObject(); + AuditInfo[] auditInfoArray = new Gson().fromJson(jsonObject.getAsJsonArray("data"), AuditInfo[].class); + + ZonedDateTime minLogTs = null; + ZonedDateTime maxLogTs = null; + DateTimeFormatter logTsFormatter = + DateTimeFormatter.ofPattern(LOGTS_DATE_TIME_FORMAT).withZone(ZoneId.systemDefault()); + long totalCount = 0L; + for (AuditInfo auditData : auditInfoArray) { + if (auditData != null) { + ZonedDateTime logTs = ZonedDateTime.parse(auditData.getLogTs(), logTsFormatter); + if (minLogTs == null || logTs.isBefore(minLogTs)) { + minLogTs = logTs; + } + if (maxLogTs == null || logTs.isAfter(maxLogTs)) { + maxLogTs = logTs; + } + log.debug("parsed AuditInfo, Count: {}, Size: {}", auditData.getCount(), auditData.getSize()); + totalCount += auditData.getCount(); + } else { + log.error("Null AuditInfo found in response data."); + } + } + + if (minLogTs != null && maxLogTs != null) { + long timeDifferenceInSeconds = maxLogTs.toEpochSecond() - minLogTs.toEpochSecond(); + log.info("Time difference in seconds: {}", timeDifferenceInSeconds); + if (timeDifferenceInSeconds > 0) { + return totalCount / timeDifferenceInSeconds; + } + } + return DEFAULT_ERROR_DATA_VOLUME; + } +} diff --git a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkService.java b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkService.java index f3809ce8ae..b9d10acf5b 100644 --- a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkService.java +++ b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/FlinkService.java @@ -23,6 +23,7 @@ import org.apache.inlong.manager.plugin.flink.dto.FlinkInfo; import org.apache.inlong.manager.plugin.flink.dto.StopWithSavepointRequest; import org.apache.inlong.manager.plugin.flink.enums.Constants; +import org.apache.inlong.manager.plugin.util.ApplicationContextProvider; import org.apache.inlong.manager.plugin.util.FlinkUtils; import lombok.extern.slf4j.Slf4j; @@ -62,13 +63,15 @@ public class FlinkService { private static final Pattern IP_PORT_PATTERN = Pattern.compile("(\\d+\\.\\d+\\.\\d+\\.\\d+):(\\d+)"); private final FlinkConfig flinkConfig; - private final Integer parallelism; + private Integer parallelism; private final String savepointDirectory; // map endpoint to Configuration private final Map configurations = new HashMap<>(); // map Configuration to FlinkClientService private final Map flinkClientServices = new HashMap<>(); + private final FlinkParallelismOptimizer flinkParallelismOptimizer; + /** * Constructor of FlinkService. */ @@ -76,6 +79,8 @@ public FlinkService() throws Exception { flinkConfig = FlinkUtils.getFlinkConfigFromFile(); parallelism = flinkConfig.getParallelism(); savepointDirectory = flinkConfig.getSavepointDirectory(); + // let spring inject the bean + flinkParallelismOptimizer = ApplicationContextProvider.getContext().getBean(FlinkParallelismOptimizer.class); } private static class FlinkServiceHolder { @@ -213,6 +218,21 @@ private String submitJobBySavepoint(FlinkInfo flinkInfo, SavepointRestoreSetting }).filter(Objects::nonNull).collect(Collectors.toList()); Configuration configuration = getFlinkConfiguration(flinkInfo.getEndpoint()); + log.debug("flink info: {}", flinkInfo); + if (flinkConfig.getDynamicParallelismEnable()) { + flinkParallelismOptimizer.setMaximumMessagePerSecondPerCore(flinkConfig.getMaxMsgRatePerCore()); + // get stream info list for auditing + int recommendedParallelism = + flinkParallelismOptimizer.calculateRecommendedParallelism(flinkInfo.getInlongStreamInfoList()); + // Ensure parallelism is at least the default value + recommendedParallelism = recommendedParallelism < parallelism ? parallelism : recommendedParallelism; + + if (recommendedParallelism != parallelism) { + log.info("switched to recommended parallelism: {}", recommendedParallelism); + parallelism = recommendedParallelism; + } + } + log.info("current parallelism: {}", parallelism); PackagedProgram program = PackagedProgram.newBuilder() .setConfiguration(configuration) diff --git a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/dto/FlinkConfig.java b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/dto/FlinkConfig.java index 7d2948510c..0e16977b70 100644 --- a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/dto/FlinkConfig.java +++ b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/dto/FlinkConfig.java @@ -40,4 +40,9 @@ public class FlinkConfig { // flink version private String version; + // max msg rate per core + private Integer maxMsgRatePerCore; + + // whether to enable dynamic parallelism + private Boolean dynamicParallelismEnable; } diff --git a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/enums/Constants.java b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/enums/Constants.java index b5628e8664..af6d7b7ca6 100644 --- a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/enums/Constants.java +++ b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/flink/enums/Constants.java @@ -43,6 +43,10 @@ public class Constants { public static final String FLINK_VERSION = "flink.version"; + public static final String FLINK_MAX_MSG_RATE_PERCORE = "flink.max.msg.rate.percore"; + + public static final String FLINK_DYNAMIC_PARALLELISM_ENABLE = "flink.dynamic.parallelism.enable"; + // dataflow public static final String SOURCE_INFO = "source_info"; diff --git a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/ApplicationContextProvider.java b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/ApplicationContextProvider.java new file mode 100644 index 0000000000..fe6afddb8c --- /dev/null +++ b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/ApplicationContextProvider.java @@ -0,0 +1,41 @@ +/* + * 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.inlong.manager.plugin.util; + +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; +import org.springframework.stereotype.Component; + +/** + * Get the Spring ApplicationContext + * instantiate class with Spring in non-Spring managed environment + */ +@Component +public class ApplicationContextProvider implements ApplicationContextAware { + + private static ApplicationContext context; + + @Override + public void setApplicationContext(ApplicationContext applicationContext) { + context = applicationContext; + } + + public static ApplicationContext getContext() { + return context; + } +} diff --git a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/FlinkUtils.java b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/FlinkUtils.java index 1110ef45d8..3797e7dbd6 100644 --- a/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/FlinkUtils.java +++ b/inlong-manager/manager-plugins/base/src/main/java/org/apache/inlong/manager/plugin/util/FlinkUtils.java @@ -218,6 +218,9 @@ public static FlinkConfig getFlinkConfigFromFile() throws Exception { flinkConfig.setJobManagerPort(Integer.valueOf(properties.getProperty(JOB_MANAGER_PORT))); flinkConfig.setDrain(Boolean.parseBoolean(properties.getProperty(DRAIN))); flinkConfig.setVersion(properties.getProperty(FLINK_VERSION)); + flinkConfig.setDynamicParallelismEnable(Boolean.parseBoolean(properties.getProperty( + Constants.FLINK_DYNAMIC_PARALLELISM_ENABLE))); + flinkConfig.setMaxMsgRatePerCore(Integer.valueOf(properties.getProperty(Constants.FLINK_MAX_MSG_RATE_PERCORE))); return flinkConfig; } diff --git a/inlong-manager/manager-plugins/base/src/main/resources/flink-sort-plugin.properties b/inlong-manager/manager-plugins/base/src/main/resources/flink-sort-plugin.properties index ff34f913f8..bc8b126311 100644 --- a/inlong-manager/manager-plugins/base/src/main/resources/flink-sort-plugin.properties +++ b/inlong-manager/manager-plugins/base/src/main/resources/flink-sort-plugin.properties @@ -35,3 +35,7 @@ flink.savepoint.directory=file:///data/inlong-sort/savepoints flink.parallelism=1 # flink stop request drain flink.drain=false +# max msg rate per core +flink.max.msg.rate.percore=1000 +# switch on or off dynamic parallelism based on data scale +flink.dynamic.parallelism.enable=true From 7aa0247228a4439123e5096dc6ac669ec2167f3f Mon Sep 17 00:00:00 2001 From: emptyOVO <118812562+emptyOVO@users.noreply.github.com> Date: Sat, 31 Aug 2024 18:32:45 +0800 Subject: [PATCH 07/30] [INLONG-10902][SDK] Transform support HEX(numeric or string) function (#10904) * [INLONG-10902][SDK] Transform support HEX(numeric or string) function * fix: add functions to check data type * fix: test data comment * fix: add anotation * fix: code conflicts * fix: add isBigDecimal function to check whether it can be parsed to BigDecimal --------- Co-authored-by: AloysZhang --- .../process/function/HexFunction.java | 108 ++++++++++++++++++ ...TransformArithmeticFunctionsProcessor.java | 29 +++++ 2 files changed, 137 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java new file mode 100644 index 0000000000..c641209162 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Function; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.Optional; +import java.util.regex.Pattern; + +/** + * HexFunction + * description: + * - If the input argument is a numeric value (such as an integer), the HEX function converts the value to the corresponding hexadecimal string. + * - If the input argument is a string, the HEX function converts each character in the string to its corresponding hexadecimal ASCII encoding and returns the hexadecimal representation of the entire string. + */ +@TransformFunction(names = {"hex"}) +class HexFunction implements ValueParser { + + private static final Pattern BIG_DECIMAL_PATTERN = Pattern.compile("^[-+]?\\d+(\\.\\d+)?([eE][-+]?\\d+)?$"); + + private ValueParser valueParser; + + public HexFunction(Function expr) { + valueParser = OperatorTools.buildParser(expr.getParameters().getExpressions().get(0)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object valueObj = valueParser.parse(sourceData, rowIndex, context); + if (isBigDecimal(valueObj)) { + return hex(OperatorTools.parseBigDecimal(valueObj)).toUpperCase(); + } + return hex(OperatorTools.parseString(valueObj)).toUpperCase(); + } + + private boolean isBigDecimal(Object valueObj) { + if (valueObj instanceof BigDecimal) { + return true; + } + if (valueObj instanceof String) { + String str = (String) valueObj; + return BIG_DECIMAL_PATTERN.matcher(str).matches(); + } + return false; + } + + // Handle Integer type + private String hex(int number) { + return Integer.toHexString(number).toUpperCase(); + } + + // Handle long type + private String hex(long number) { + return Long.toHexString(number).toUpperCase(); + } + + // Handle String type + private String hex(String input) { + StringBuilder hexString = new StringBuilder(); + for (char c : input.toCharArray()) { + hexString.append(Integer.toHexString((int) c).toUpperCase()); + } + return hexString.toString(); + } + + // Handle BigDecimal type + private String hex(BigDecimal number) { + // keep the integer part + BigDecimal integerValue = number.setScale(0, RoundingMode.DOWN); + return tryConvert(integerValue, BigDecimal::intValueExact, this::hex) + // If it cannot convert to integer, try converting to long + .orElseGet(() -> tryConvert(integerValue, BigDecimal::longValueExact, this::hex) + .orElseThrow(() -> new IllegalArgumentException("Number out of range"))); + } + + // Common conversion and processing methods + private Optional tryConvert(BigDecimal number, java.util.function.Function converter, + java.util.function.Function handler) { + try { + T value = converter.apply(number); + return Optional.ofNullable(handler.apply(value)); + } catch (ArithmeticException e) { + return Optional.empty(); + } + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java index 1467636e65..87ce145df6 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java @@ -619,4 +619,33 @@ public void testRandFunction() throws Exception { Assert.assertTrue(result >= 0.0 && result < 1.0); } + @Test + public void testHexFunction() throws Exception { + String transformSql1 = "select hex(numeric1) from source"; + TransformConfig config1 = new TransformConfig(transformSql1); + TransformProcessor processor1 = TransformProcessor + .create(config1, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case: hex(1007) + List output1 = processor1.transform("1007|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=3EF"); + // case: hex(3.14) + List output2 = processor1.transform("3.14|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output2.size()); + Assert.assertEquals(output2.get(0), "result=3"); + // case: hex(3.141592653589793) + List output3 = processor1.transform("3.141592653589793|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output3.size()); + Assert.assertEquals(output3.get(0), "result=3"); + // case: hex(-9223372036854775808) + List output4 = processor1.transform("-9223372036854775808|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output4.size()); + Assert.assertEquals(output4.get(0), "result=8000000000000000"); + // case: hex(abc) + List output5 = processor1.transform("abc|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output5.size()); + Assert.assertEquals(output5.get(0), "result=616263"); + } + } From d8a09abac25615e66f7893e90858b726a6c0fa61 Mon Sep 17 00:00:00 2001 From: emptyOVO <118812562+emptyOVO@users.noreply.github.com> Date: Sat, 31 Aug 2024 18:45:36 +0800 Subject: [PATCH 08/30] [INLONG-10944][SDK] Support Inlong Transform parser annotation (#10945) * [INLONG-10944][SDK] Support Inlong Transform parser annotation * fix: naming notations * fix: remove the check of class==null * fix: code standard check * fix: naming Convention --- .../process/operator/OperatorTools.java | 55 +------------ .../process/parser/AdditionParser.java | 1 + .../process/parser/ColumnParser.java | 1 + .../transform/process/parser/DateParser.java | 1 + .../process/parser/DivisionParser.java | 1 + .../process/parser/DoubleParser.java | 1 + .../transform/process/parser/LongParser.java | 1 + .../process/parser/ModuloParser.java | 1 + .../process/parser/MultiplicationParser.java | 1 + .../process/parser/ParenthesisParser.java | 1 + .../transform/process/parser/ParserTools.java | 79 +++++++++++++++++++ .../transform/process/parser/SignParser.java | 1 + .../process/parser/StringParser.java | 1 + .../process/parser/SubtractionParser.java | 1 + .../process/parser/TimestampParser.java | 1 + .../process/parser/TransformParser.java | 33 ++++++++ 16 files changed, 128 insertions(+), 52 deletions(-) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParserTools.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TransformParser.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java index 90405d687c..dbe12ff814 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java @@ -18,36 +18,14 @@ package org.apache.inlong.sdk.transform.process.operator; import org.apache.inlong.sdk.transform.process.function.FunctionTools; -import org.apache.inlong.sdk.transform.process.parser.AdditionParser; import org.apache.inlong.sdk.transform.process.parser.ColumnParser; -import org.apache.inlong.sdk.transform.process.parser.DateParser; -import org.apache.inlong.sdk.transform.process.parser.DivisionParser; -import org.apache.inlong.sdk.transform.process.parser.DoubleParser; -import org.apache.inlong.sdk.transform.process.parser.LongParser; -import org.apache.inlong.sdk.transform.process.parser.ModuloParser; -import org.apache.inlong.sdk.transform.process.parser.MultiplicationParser; -import org.apache.inlong.sdk.transform.process.parser.ParenthesisParser; -import org.apache.inlong.sdk.transform.process.parser.SignParser; -import org.apache.inlong.sdk.transform.process.parser.StringParser; -import org.apache.inlong.sdk.transform.process.parser.SubtractionParser; -import org.apache.inlong.sdk.transform.process.parser.TimestampParser; +import org.apache.inlong.sdk.transform.process.parser.ParserTools; import org.apache.inlong.sdk.transform.process.parser.ValueParser; -import net.sf.jsqlparser.expression.DateValue; -import net.sf.jsqlparser.expression.DoubleValue; import net.sf.jsqlparser.expression.Expression; import net.sf.jsqlparser.expression.Function; -import net.sf.jsqlparser.expression.LongValue; import net.sf.jsqlparser.expression.NotExpression; import net.sf.jsqlparser.expression.Parenthesis; -import net.sf.jsqlparser.expression.SignedExpression; -import net.sf.jsqlparser.expression.StringValue; -import net.sf.jsqlparser.expression.TimestampValue; -import net.sf.jsqlparser.expression.operators.arithmetic.Addition; -import net.sf.jsqlparser.expression.operators.arithmetic.Division; -import net.sf.jsqlparser.expression.operators.arithmetic.Modulo; -import net.sf.jsqlparser.expression.operators.arithmetic.Multiplication; -import net.sf.jsqlparser.expression.operators.arithmetic.Subtraction; import net.sf.jsqlparser.expression.operators.conditional.AndExpression; import net.sf.jsqlparser.expression.operators.conditional.OrExpression; import net.sf.jsqlparser.expression.operators.relational.EqualsTo; @@ -56,7 +34,6 @@ 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.schema.Column; import org.apache.commons.lang.ObjectUtils; import java.math.BigDecimal; @@ -99,33 +76,7 @@ public static ExpressionOperator buildOperator(Expression expr) { } public static ValueParser buildParser(Expression expr) { - if (expr instanceof Column) { - return new ColumnParser((Column) expr); - } else if (expr instanceof StringValue) { - return new StringParser((StringValue) expr); - } else if (expr instanceof LongValue) { - return new LongParser((LongValue) expr); - } else if (expr instanceof DoubleValue) { - return new DoubleParser((DoubleValue) expr); - } else if (expr instanceof SignedExpression) { - return new SignParser((SignedExpression) expr); - } else if (expr instanceof Parenthesis) { - return new ParenthesisParser((Parenthesis) expr); - } else if (expr instanceof Addition) { - return new AdditionParser((Addition) expr); - } else if (expr instanceof Subtraction) { - return new SubtractionParser((Subtraction) expr); - } else if (expr instanceof Multiplication) { - return new MultiplicationParser((Multiplication) expr); - } else if (expr instanceof Division) { - return new DivisionParser((Division) expr); - } else if (expr instanceof Modulo) { - return new ModuloParser((Modulo) expr); - } else if (expr instanceof DateValue) { - return new DateParser((DateValue) expr); - } else if (expr instanceof TimestampValue) { - return new TimestampParser((TimestampValue) expr); - } else if (expr instanceof Function) { + if (expr instanceof Function) { String exprString = expr.toString(); if (exprString.startsWith(ROOT_KEY) || exprString.startsWith(CHILD_KEY)) { return new ColumnParser((Function) expr); @@ -133,7 +84,7 @@ public static ValueParser buildParser(Expression expr) { return FunctionTools.getTransformFunction((Function) expr); } } - return null; + return ParserTools.getTransformParser(expr); } /** diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/AdditionParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/AdditionParser.java index 08474fe81f..13536be30d 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/AdditionParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/AdditionParser.java @@ -29,6 +29,7 @@ * AdditionParser * */ +@TransformParser(values = Addition.class) public class AdditionParser implements ValueParser { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ColumnParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ColumnParser.java index 3a5000a57f..ebec767375 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ColumnParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ColumnParser.java @@ -27,6 +27,7 @@ * ColumnParser * */ +@TransformParser(values = Column.class) public class ColumnParser implements ValueParser { private final String fieldName; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DateParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DateParser.java index 0f0da4345d..4ded1cafd9 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DateParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DateParser.java @@ -28,6 +28,7 @@ * DateParser * description: parse the sql expression to a java.sql.Date object */ +@TransformParser(values = DateValue.class) public class DateParser implements ValueParser { private final Date dateValue; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DivisionParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DivisionParser.java index 61cf1bb82f..343481372e 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DivisionParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DivisionParser.java @@ -29,6 +29,7 @@ * DivisionParser * */ +@TransformParser(values = Division.class) public class DivisionParser implements ValueParser { private ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java index 2f50b5311b..ad39558a11 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java @@ -25,6 +25,7 @@ /** * LongParser */ +@TransformParser(values = DoubleValue.class) public class DoubleParser implements ValueParser { private final Double value; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/LongParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/LongParser.java index 7abb8af77c..acad3cf91c 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/LongParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/LongParser.java @@ -26,6 +26,7 @@ * LongParser * */ +@TransformParser(values = LongValue.class) public class LongParser implements ValueParser { private final Long value; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ModuloParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ModuloParser.java index ca0a841170..e995a6535c 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ModuloParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ModuloParser.java @@ -29,6 +29,7 @@ * ModuloParser * description: analyze the % expression */ +@TransformParser(values = Modulo.class) public class ModuloParser implements ValueParser { private ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/MultiplicationParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/MultiplicationParser.java index f7299dcf8c..af71b8b240 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/MultiplicationParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/MultiplicationParser.java @@ -29,6 +29,7 @@ * MultiplicationParser * */ +@TransformParser(values = Multiplication.class) public class MultiplicationParser implements ValueParser { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParenthesisParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParenthesisParser.java index 1899017087..bb046aff5f 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParenthesisParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParenthesisParser.java @@ -27,6 +27,7 @@ * ParenthesisParser * */ +@TransformParser(values = Parenthesis.class) public class ParenthesisParser implements ValueParser { private final ValueParser node; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParserTools.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParserTools.java new file mode 100644 index 0000000000..a68259e3c9 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/ParserTools.java @@ -0,0 +1,79 @@ +/* + * 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.inlong.sdk.transform.process.parser; + +import com.google.common.collect.Maps; +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.schema.Column; +import org.reflections.Reflections; +import org.reflections.scanners.Scanners; + +import java.lang.reflect.Constructor; +import java.util.Map; +import java.util.Set; + +@Slf4j +public class ParserTools { + + private static final String PARSER_PATH = "org.apache.inlong.sdk.transform.process.parser"; + private final static Map, Class> parserMap = Maps.newConcurrentMap(); + + static { + init(); + } + private static void init() { + Reflections reflections = new Reflections(PARSER_PATH, Scanners.TypesAnnotated); + Set> clazzSet = reflections.getTypesAnnotatedWith(TransformParser.class); + for (Class clazz : clazzSet) { + if (ValueParser.class.isAssignableFrom(clazz)) { + TransformParser annotation = clazz.getAnnotation(TransformParser.class); + if (annotation == null) { + continue; + } + Class[] values = annotation.values(); + for (Class value : values) { + parserMap.compute(value, (key, former) -> { + if (former != null) { + log.warn("find a conflict for parser class [{}], the former one is [{}], new one is [{}]", + key, former.getName(), clazz.getName()); + } + return clazz; + }); + } + } + } + } + + public static ValueParser getTransformParser(Expression expr) { + Class clazz = parserMap.get(expr.getClass()); + if (clazz == null) { + return new ColumnParser((Column) expr); + } + try { + Constructor constructor = clazz.getDeclaredConstructor(expr.getClass()); + return (ValueParser) constructor.newInstance(expr); + } catch (NoSuchMethodException e) { + log.error("transform parser {} needs one constructor that accept one params whose type is {}", + clazz.getName(), expr.getClass().getName(), e); + throw new RuntimeException(e); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java index 360fc0d0f6..ff97aadfdb 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java @@ -29,6 +29,7 @@ * SignParser * */ +@TransformParser(values = SignedExpression.class) public class SignParser implements ValueParser { private final Integer sign; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/StringParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/StringParser.java index 9ce0646cc9..934d99c42e 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/StringParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/StringParser.java @@ -26,6 +26,7 @@ * StringParser * */ +@TransformParser(values = StringValue.class) public class StringParser implements ValueParser { private final String stringValue; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SubtractionParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SubtractionParser.java index 15d534d50f..140a1dc995 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SubtractionParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SubtractionParser.java @@ -29,6 +29,7 @@ * SubtractionParser * */ +@TransformParser(values = Subtraction.class) public class SubtractionParser implements ValueParser { private ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TimestampParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TimestampParser.java index 1971ba722a..3bb22e076a 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TimestampParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TimestampParser.java @@ -28,6 +28,7 @@ * DateParser * description: parse the sql expression to a java.sql.Timestamp object */ +@TransformParser(values = TimestampValue.class) public class TimestampParser implements ValueParser { private final Timestamp timestampValue; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TransformParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TransformParser.java new file mode 100644 index 0000000000..4af20d8e56 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/TransformParser.java @@ -0,0 +1,33 @@ +/* + * 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.inlong.sdk.transform.process.parser; + +import net.sf.jsqlparser.expression.Expression; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.TYPE; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target(TYPE) +public @interface TransformParser { + + Class[] values(); +} From a34d877a3d9e69443f9497a27b4cf3b13a7e9b78 Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Mon, 2 Sep 2024 09:42:41 +0800 Subject: [PATCH 09/30] [INLONG-10927][SDK] Transform supports padding of left and right strings (#10928) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../process/function/LpadFunction.java | 81 ++++++++++ .../process/function/RpadFunction.java | 81 ++++++++++ .../process/operator/OperatorTools.java | 2 +- ...TestTransformStringFunctionsProcessor.java | 140 ++++++++++++++++++ 4 files changed, 303 insertions(+), 1 deletion(-) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LpadFunction.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RpadFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LpadFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LpadFunction.java new file mode 100644 index 0000000000..cce51ba1e4 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/LpadFunction.java @@ -0,0 +1,81 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.util.List; + +/** + * LpadFunction + * description: Lpad(s1,len,s2) Fill string s2 at the beginning of string s1 to make the string length len + * - return null if any of the three parameters is null or len is less than 0 + * - return the substring of s1 with subscripts in the range of [0, len) if len is less than or equal to the length of s1 + * - if s2 is "" + * - return "" if len is longer than the length of s1 + * - if s2 is not "" + * - return the filled string + */ +@TransformFunction(names = {"lpad"}) +public class LpadFunction implements ValueParser { + + private final ValueParser leftStringParser; + private final ValueParser lengthParser; + private final ValueParser rightStringParser; + + public LpadFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + leftStringParser = OperatorTools.buildParser(expressions.get(0)); + lengthParser = OperatorTools.buildParser(expressions.get(1)); + rightStringParser = OperatorTools.buildParser(expressions.get(2)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object leftStringObj = leftStringParser.parse(sourceData, rowIndex, context); + Object lengthObj = lengthParser.parse(sourceData, rowIndex, context); + Object rightStringObj = rightStringParser.parse(sourceData, rowIndex, context); + if (leftStringObj == null || lengthObj == null || rightStringObj == null) { + return null; + } + int len = Integer.parseInt(OperatorTools.parseString(lengthObj)); + if (len < 0) { + return null; + } + String leftStr = OperatorTools.parseString(leftStringObj); + if (len <= leftStr.length()) { + return leftStr.substring(0, len); + } + String rightStr = OperatorTools.parseString(rightStringObj); + if (rightStr.isEmpty()) { + return ""; + } + int padLen = len - leftStr.length(); + StringBuilder builder = new StringBuilder(padLen); + while (builder.length() < padLen) { + builder.append(rightStr); + } + return builder.substring(0, padLen).concat(leftStr); + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RpadFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RpadFunction.java new file mode 100644 index 0000000000..c3357c7b55 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RpadFunction.java @@ -0,0 +1,81 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.util.List; + +/** + * RpadFunction + * description: RPAD(s1,len,s2) : Fill string s2 at the end of string s1 to make the length of the string len + * - return null if any of the three parameters is null or len is less than 0 + * - return the substring of s1 with subscripts in the range of [0, len) if len is less than or equal to the length of s1 + * - if s2 is "" + * - return "" if len is longer than the length of s1 + * - if s2 is not "" + * - return the filled string + */ +@TransformFunction(names = {"rpad"}) +public class RpadFunction implements ValueParser { + + private final ValueParser leftStringParser; + private final ValueParser lengthParser; + private final ValueParser rightStringParser; + + public RpadFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + leftStringParser = OperatorTools.buildParser(expressions.get(0)); + lengthParser = OperatorTools.buildParser(expressions.get(1)); + rightStringParser = OperatorTools.buildParser(expressions.get(2)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object leftStringObj = leftStringParser.parse(sourceData, rowIndex, context); + Object lengthObj = lengthParser.parse(sourceData, rowIndex, context); + Object rightStringObj = rightStringParser.parse(sourceData, rowIndex, context); + if (leftStringObj == null || lengthObj == null || rightStringObj == null) { + return null; + } + int len = Integer.parseInt(OperatorTools.parseString(lengthObj)); + if (len < 0) { + return null; + } + String leftStr = OperatorTools.parseString(leftStringObj); + if (len <= leftStr.length()) { + return leftStr.substring(0, len); + } + String rightStr = OperatorTools.parseString(rightStringObj); + if (rightStr.isEmpty()) { + return ""; + } + StringBuilder builder = new StringBuilder(len); + builder.append(leftStr); + while (builder.length() < len) { + builder.append(rightStr); + } + return builder.substring(0, len); + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java index dbe12ff814..5e69984564 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java @@ -42,7 +42,7 @@ /** * OperatorTools - * + * */ public class OperatorTools { diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java index 10cfa740c9..1451194607 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java @@ -428,6 +428,146 @@ public void testStrcmpFunction() throws Exception { Assert.assertEquals("result=null", output.get(0)); } + @Test + public void testRpadFunction() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + transformSql = "select rpad(string1,numeric1,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case1: rpad('he',7,'xxd') + data = "he|xxd|cloud|7|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=hexxdxx", output.get(0)); + + // case2: rpad('he',1,'xxd') + data = "he|xxd|cloud|1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=h", output.get(0)); + + // case3: rpad('he',1,'') + data = "he||cloud|1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=h", output.get(0)); + + // case4: rpad('he',-1,'xxd') + data = "he|xxd|cloud|-1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case5: rpad(null,5,'xxd') + transformSql = "select rpad(xxd,numeric1,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "he|xxd|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case6: rpad('he',null,'xxd') + transformSql = "select rpad(string1,xxd,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "he|xxd|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case7: rpad('he',5,null) + transformSql = "select rpad(string1,numeric1,xxd) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "he|xxd|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + + @Test + public void testLpadFunction() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + transformSql = "select lpad(string1,numeric1,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case1: lpad('he',7,'xxd') + data = "he|xxd|cloud|7|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=xxdxxhe", output.get(0)); + + // case2: lpad('he',1,'xxd') + data = "he|xxd|cloud|1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=h", output.get(0)); + + // case3: lpad('he',1,'') + data = "he||cloud|1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=h", output.get(0)); + + // case4: lpad('he',-1,'xxd') + data = "he|xxd|cloud|-1|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case5: lpad(null,5,'xxd') + transformSql = "select lpad(xxd,numeric1,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "he|xxd|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case6: lpad('he',null,'xxd') + transformSql = "select lpad(string1,xxd,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "he|xxd|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case7: lpad('he',5,null) + transformSql = "select lpad(string1,numeric1,xxd) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "he|xxd|cloud|5|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + @Test public void testRightFunction() throws Exception { String transformSql = "select right(string1,numeric1) from source"; From e7878063ff0d90fd2f1a1992e738c19256123387 Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Mon, 2 Sep 2024 09:43:14 +0800 Subject: [PATCH 10/30] [INLONG-10897][SDK] Transform support DATEDIFF function (#10925) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../process/function/DateDiffFunction.java | 89 +++++++++++++++++++ ...stTransformTemporalFunctionsProcessor.java | 48 ++++++++++ 2 files changed, 137 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateDiffFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateDiffFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateDiffFunction.java new file mode 100644 index 0000000000..77b2c93dfc --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateDiffFunction.java @@ -0,0 +1,89 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.temporal.ChronoUnit; +import java.util.List; + +/** + * DateDiffFunction + * description: DATEDIFF(d1, d2) + * - return null if one of the two parameters is null or "" + * - return null if one of the two parameters has an incorrect date format + * - return the number of days between the dates d1->d2. + */ +@TransformFunction(names = {"datediff", "date_diff"}) +public class DateDiffFunction implements ValueParser { + + private final ValueParser leftDateParser; + private final ValueParser rightDateParser; + private static final DateTimeFormatter DEFAULT_FORMAT_DATE_TIME = + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + private static final DateTimeFormatter DEFAULT_FORMAT_DATE = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + + public DateDiffFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + leftDateParser = OperatorTools.buildParser(expressions.get(0)); + rightDateParser = OperatorTools.buildParser(expressions.get(1)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object leftDateObj = leftDateParser.parse(sourceData, rowIndex, context); + Object rightDateObj = rightDateParser.parse(sourceData, rowIndex, context); + if (leftDateObj == null || rightDateObj == null) { + return null; + } + String leftDate = OperatorTools.parseString(leftDateObj); + String rightDate = OperatorTools.parseString(rightDateObj); + if (leftDate.isEmpty() || rightDate.isEmpty()) { + return null; + } + try { + LocalDate left = getLocalDate(leftDate); + LocalDate right = getLocalDate(rightDate); + return ChronoUnit.DAYS.between(right, left); + } catch (Exception e) { + return null; + } + } + + public LocalDate getLocalDate(String dateString) { + DateTimeFormatter formatter = null; + LocalDate dateTime = null; + if (dateString.indexOf(' ') != -1) { + formatter = DEFAULT_FORMAT_DATE_TIME; + dateTime = LocalDateTime.parse(dateString, formatter).toLocalDate(); + } else { + formatter = DEFAULT_FORMAT_DATE; + dateTime = LocalDate.parse(dateString, formatter); + } + return dateTime; + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java index 354b8b7f18..f2c73ee3d9 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java @@ -379,6 +379,54 @@ public void testToTimestampFunction() throws Exception { Assert.assertEquals(output4.get(0), "result=1970-01-01 00:00:00.0"); } + @Test + public void testDateDiffFunction() throws Exception { + String transformSql = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + transformSql = "select datediff(string1,string2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case1: datediff('1970-01-01','1970-01-02') + output = processor.transform("1970-01-01|1970-01-02", new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=-1", output.get(0)); + + // case2: datediff('1970-01-02','1970-01-01') + output = processor.transform("1970-01-02|1970-01-01", new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=1", output.get(0)); + + // case3: datediff('2018-12-10 12:30:00', '2018-12-09 13:30:00') + output = processor.transform("2018-12-10 12:30:00|2018-12-09 13:30:00", new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=1", output.get(0)); + + // case4: datediff('2018-12-10 12:30:00', '') + output = processor.transform("2018-12-10 12:30:00|", new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case5: datediff('2018-12', '2018-12-12') + output = processor.transform("2018-12|2018-12-12", new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case6: datediff('1970-01-01',null) + transformSql = "select datediff(string1,xxd) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + output = processor.transform("1970-01-01|1970-01-02", new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + @Test public void testLocalTimeFunction() throws Exception { String transformSql1 = "select localtime() from source"; From de98601ce75ec4b05e20d57d5d4307a9079fd079 Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Mon, 2 Sep 2024 10:48:41 +0800 Subject: [PATCH 11/30] [INLONG-10959][SDK] Transform support IFNULL function (#10960) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../process/function/IfNullFunction.java | 72 +++++++++++++++++++ ...TransformArithmeticFunctionsProcessor.java | 63 ++++++++++++++++ 2 files changed, 135 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfNullFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfNullFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfNullFunction.java new file mode 100644 index 0000000000..f19b291d7e --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfNullFunction.java @@ -0,0 +1,72 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.util.List; + +/** + * IfNullFunction + * description: IFNULL(expr1,expr2) + * - return expr1 if expr1 is not NULL + * - return expr2 otherwise + */ +@Slf4j +@TransformFunction(names = {"ifnull", "if_null"}) +public class IfNullFunction implements ValueParser { + + private final ValueParser firstExprParser; + private final ValueParser secondExprParser; + + public IfNullFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + firstExprParser = OperatorTools.buildParser(expressions.get(0)); + secondExprParser = OperatorTools.buildParser(expressions.get(1)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + try { + Object firstExprObj = firstExprParser.parse(sourceData, rowIndex, context); + if (firstExprObj == null) { + return parseSecondExpr(sourceData, rowIndex, context); + } + return firstExprObj; + } catch (Exception e) { + log.error("Value parsing failed", e); + return parseSecondExpr(sourceData, rowIndex, context); + } + } + + private Object parseSecondExpr(SourceData sourceData, int rowIndex, Context context) { + try { + return secondExprParser.parse(sourceData, rowIndex, context); + } catch (Exception e) { + log.error("Value parsing failed", e); + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java index 87ce145df6..1e0f14025f 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java @@ -224,6 +224,69 @@ public void testModuloFunction() throws Exception { } + @Test + public void testIfNullFunction() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: ifnull(5, 3) + transformSql = "select ifnull(numeric1,numeric2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "5|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=5", output.get(0)); + + // case2: ifnull(null,3) + transformSql = "select ifnull(xxd,numeric2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "5|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=3", output.get(0)); + + // case3: ifnull(6 / 3,'YES') + transformSql = "select ifnull(numeric1 / numeric2,'YES') from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "6|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=2", output.get(0)); + + // case4: ifnull(6 / 0,'YES') + transformSql = "select ifnull(numeric1 / numeric2,'YES') from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "6|0|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=YES", output.get(0)); + + // case5: ifnull(6 / 0,3 / 0) + transformSql = "select ifnull(numeric1 / numeric2,numeric3 / numeric2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "6|0|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + @Test public void testMd5Function() throws Exception { String transformSql = "select md5(numeric1) from source"; From 9f9acbd7f524a73cd531b6770ce188a2851e1032 Mon Sep 17 00:00:00 2001 From: Xincheng Huang <60057611+ying-hua@users.noreply.github.com> Date: Mon, 2 Sep 2024 10:49:08 +0800 Subject: [PATCH 12/30] [INLONG-10957][SDK] Improve some code structures (#10958) --- .../sdk/transform/decode/JsonSourceData.java | 24 ++++---- .../transform/decode/JsonSourceDecoder.java | 55 ++++++++++--------- 2 files changed, 41 insertions(+), 38 deletions(-) diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceData.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceData.java index 5f92c34965..539f4b06dc 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceData.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceData.java @@ -111,20 +111,22 @@ public String getField(int rowNum, String fieldName) { // error data return ""; } + // node is not array if (!node.isArray()) { current = newElement; - } else { - if (!newElement.isJsonArray()) { - // error data - return ""; - } - JsonArray newArray = newElement.getAsJsonArray(); - if (node.getArrayIndex() >= newArray.size()) { - // error data - return ""; - } - current = newArray.get(node.getArrayIndex()); + continue; + } + // node is an array + if (!newElement.isJsonArray()) { + // error data + return ""; + } + JsonArray newArray = newElement.getAsJsonArray(); + if (node.getArrayIndex() >= newArray.size()) { + // error data + return ""; } + current = newArray.get(node.getArrayIndex()); } return current.getAsString(); } catch (Exception e) { diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceDecoder.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceDecoder.java index 2d16d92bc1..426ae167a2 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceDecoder.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/JsonSourceDecoder.java @@ -85,39 +85,40 @@ public SourceData decode(byte[] srcBytes, Context context) { public SourceData decode(String srcString, Context context) { JsonObject root = gson.fromJson(srcString, JsonObject.class); JsonArray childRoot = null; - if (CollectionUtils.isNotEmpty(childNodes)) { - JsonElement current = root; - for (JsonNode node : childNodes) { - if (!current.isJsonObject()) { + if (CollectionUtils.isEmpty(childNodes)) { + return new JsonSourceData(root, null); + } + JsonElement current = root; + for (JsonNode node : childNodes) { + if (!current.isJsonObject()) { + // error data + return new JsonSourceData(root, null); + } + JsonElement newElement = current.getAsJsonObject().get(node.getName()); + if (newElement == null) { + // error data + return new JsonSourceData(root, null); + } + if (!node.isArray()) { + current = newElement; + } else { + if (!newElement.isJsonArray()) { // error data - return new JsonSourceData(root, childRoot); + return new JsonSourceData(root, null); } - JsonElement newElement = current.getAsJsonObject().get(node.getName()); - if (newElement == null) { + JsonArray newArray = newElement.getAsJsonArray(); + if (node.getArrayIndex() >= newArray.size()) { // error data - return new JsonSourceData(root, childRoot); + return new JsonSourceData(root, null); } - if (!node.isArray()) { - current = newElement; - } else { - if (!newElement.isJsonArray()) { - // error data - return new JsonSourceData(root, childRoot); - } - JsonArray newArray = newElement.getAsJsonArray(); - if (node.getArrayIndex() >= newArray.size()) { - // error data - return new JsonSourceData(root, childRoot); - } - current = newArray.get(node.getArrayIndex()); - } - } - if (!current.isJsonArray()) { - // error data - return new JsonSourceData(root, childRoot); + current = newArray.get(node.getArrayIndex()); } - childRoot = current.getAsJsonArray(); } + if (!current.isJsonArray()) { + // error data + return new JsonSourceData(root, null); + } + childRoot = current.getAsJsonArray(); return new JsonSourceData(root, childRoot); } } From 6234b9ea01391b536928c0c0126023a4f5e2ab49 Mon Sep 17 00:00:00 2001 From: Xincheng Huang <60057611+ying-hua@users.noreply.github.com> Date: Mon, 2 Sep 2024 10:50:14 +0800 Subject: [PATCH 13/30] [INLONG-10935][SDK] Transform support DAYNAME function (#10937) * [INLONG-10935][SDK] Transform support DAYNAME function * fix the style * fix the failed UT --- .../process/function/DateExtractFunction.java | 14 +++++++++++++- .../transform/process/operator/OperatorTools.java | 1 - .../TestTransformTemporalFunctionsProcessor.java | 10 ++++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java index ff0eb6b01e..cac2273619 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java @@ -42,6 +42,7 @@ * - dayofyear(date)--returns the day of a year (an integer between 1 and 366) from SQL date * - dayofmonth(date)--returns the day of a month (an integer between 1 and 31) from SQL date * - dayofweek(date)--returns the day of a week (an integer between 1(Sunday) and 7(Saturday)) from SQL date + * - dayname(date)--returns the name of the day of the week from SQL date */ public abstract class DateExtractFunction implements ValueParser { @@ -50,7 +51,7 @@ public abstract class DateExtractFunction implements ValueParser { private static final TemporalField weekOfYearField = WeekFields.of(Locale.getDefault()).weekOfWeekBasedYear(); public enum DateExtractFunctionType { - YEAR, QUARTER, MONTH, WEEK, DAY_OF_YEAR, DAY_OF_MONTH, DAY_OF_WEEK + YEAR, QUARTER, MONTH, WEEK, DAY_OF_YEAR, DAY_OF_MONTH, DAY_OF_WEEK, DAY_NAME } @TransformFunction(names = {"year"}) @@ -109,6 +110,14 @@ public DayOfWeekExtractFunction(Function expr) { } } + @TransformFunction(names = {"day_name", "dayname"}) + public static class DayNameExtractFunction extends DateExtractFunction { + + public DayNameExtractFunction(Function expr) { + super(DateExtractFunctionType.DAY_NAME, expr); + } + } + public DateExtractFunction(DateExtractFunctionType type, Function expr) { this.type = type; List expressions = expr.getParameters().getExpressions(); @@ -142,6 +151,9 @@ public Object parse(SourceData sourceData, int rowIndex, Context context) { // dayofweek(between 1 and 7) case DAY_OF_WEEK: return localDate.getDayOfWeek().getValue() % 7 + 1; + // dayname(between Sunday and Saturday) + case DAY_NAME: + return localDate.getDayOfWeek().name(); default: return null; } diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java index 5e69984564..02b24cdb6b 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java @@ -49,7 +49,6 @@ public class OperatorTools { public static final String ROOT_KEY = "$root"; public static final String CHILD_KEY = "$child"; - public static ExpressionOperator buildOperator(Expression expr) { if (expr instanceof AndExpression) { return new AndOperator((AndExpression) expr); diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java index f2c73ee3d9..14c1fab04b 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformTemporalFunctionsProcessor.java @@ -216,6 +216,16 @@ public void testDateExtractFunction() throws Exception { List output7 = processor7.transform("2024-02-29", new HashMap<>()); Assert.assertEquals(1, output7.size()); Assert.assertEquals(output7.get(0), "result=5"); + + String transformSql8 = "select dayname(string1) from source"; + TransformConfig config8 = new TransformConfig(transformSql8); + TransformProcessor processor8 = TransformProcessor + .create(config8, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case8: dayname(2024-02-29) (Thursday) + List output8 = processor8.transform("2024-02-29", new HashMap<>()); + Assert.assertEquals(1, output8.size()); + Assert.assertEquals(output8.get(0), "result=THURSDAY"); } @Test From 5fd7cc0923e6f3fb1b0bd6da3ecfe84908e3c672 Mon Sep 17 00:00:00 2001 From: XiaoYou201 Date: Mon, 2 Sep 2024 13:08:44 +0800 Subject: [PATCH 14/30] [INLONG-10725][CI] Add UT test workflow for flink 1.18 connectors. (#10726) --- .github/workflows/ci_ut.yml | 2 +- .github/workflows/ci_ut_flink18.yml | 96 +++++++ inlong-sort/sort-core/pom.xml | 54 +++- inlong-sort/sort-dist/pom.xml | 75 ++++-- inlong-sort/sort-end-to-end-tests/pom.xml | 9 + .../sort-end-to-end-tests-v1.18/pom.xml | 209 +++++++++++++++ .../tests/utils/FlinkContainerTestEnv.java | 241 ++++++++++++++++++ .../utils/FlinkContainerTestEnvJRE11.java | 55 ++++ .../utils/FlinkContainerTestEnvJRE8.java | 55 ++++ .../sort/tests/utils/PlaceholderResolver.java | 150 +++++++++++ .../inlong/sort/tests/utils/TestUtils.java | 124 +++++++++ .../src/test/resources/log4j2-test.properties | 47 ++++ inlong-sort/sort-formats/pom.xml | 3 +- 13 files changed, 1081 insertions(+), 39 deletions(-) create mode 100644 .github/workflows/ci_ut_flink18.yml create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnv.java create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE11.java create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE8.java create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/PlaceholderResolver.java create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/TestUtils.java create mode 100644 inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/log4j2-test.properties diff --git a/.github/workflows/ci_ut.yml b/.github/workflows/ci_ut.yml index 5c433872bf..0d67d46320 100644 --- a/.github/workflows/ci_ut.yml +++ b/.github/workflows/ci_ut.yml @@ -101,7 +101,7 @@ jobs: CI: false - name: Unit test with Maven - run: mvn --batch-mode --update-snapshots -e -V test -pl !:sort-end-to-end-tests-v1.15,!:sort-end-to-end-tests-v1.13 + run: mvn --batch-mode --update-snapshots -e -V test -pl !:sort-end-to-end-tests-v1.15,!:sort-end-to-end-tests-v1.13,!:sort-end-to-end-tests-v1.18 env: CI: false diff --git a/.github/workflows/ci_ut_flink18.yml b/.github/workflows/ci_ut_flink18.yml new file mode 100644 index 0000000000..5c2f2709f2 --- /dev/null +++ b/.github/workflows/ci_ut_flink18.yml @@ -0,0 +1,96 @@ +# +# 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. +# + +name: + InLong Unit Test For Flink 1.18 + +on: + push: + paths: + - '.github/workflows/ci_ut_flink18.yml' + - 'inlong-sort/**' + - '!**.md' + + pull_request: + paths: + - '.github/workflows/ci_ut_flink18.yml' + - 'inlong-sort/**' + - '!**.md' + +jobs: + unit-test: + name: Unit Test + runs-on: ubuntu-22.04 + steps: + - name: Checkout + uses: actions/checkout@v4 + + # Release space size + - name: Remove unnecessary packages + run: | + echo "=== Before pruning ===" + df -h + sudo rm -rf /usr/share/dotnet + sudo rm -rf /usr/local/lib/android + sudo rm -rf /opt/ghc + sudo rm -rf /opt/hostedtoolcache + echo "=== After pruning ===" + df -h + + - name: Set up JDK + uses: actions/setup-java@v4 + with: + java-version: 8 + distribution: adopt + + - name: Cache Maven packages + uses: actions/cache@v4 + with: + path: | + ~/.m2/repository/*/*/* + !~/.m2/repository/org/apache/inlong + key: ${{ runner.os }}-inlong-flink18-${{ hashFiles('**/pom.xml') }} + restore-keys: ${{ runner.os }}-inlong-flink18 + + - name: Build for Flink 1.18 with Maven + run: mvn --update-snapshots -e -V clean install -U -pl :sort-end-to-end-tests-v1.18 -am -Pv1.18 -DskipTests -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.httpconnectionManager.ttlSeconds=120 -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 + env: + CI: false + + - name: Unit test for Flink 1.18 with Maven + run: mvn --update-snapshots -e -V verify -pl :sort-end-to-end-tests-v1.18 -am -Pv1.18 + env: + CI: false + + - name: Upload unit test results + if: ${{ failure() }} + uses: actions/upload-artifact@v4 + with: + name: surefire-reports + path: ./**/target/surefire-reports/ + if-no-files-found: ignore + + - name: Upload integration test results + if: ${{ failure() }} + uses: actions/upload-artifact@v4 + with: + name: failsafe-reports + path: ./**/target/failsafe-reports/ + if-no-files-found: ignore + + - name: Clean up build packages + run: mvn clean \ No newline at end of file diff --git a/inlong-sort/sort-core/pom.xml b/inlong-sort/sort-core/pom.xml index 2ef9506520..e4881e6043 100644 --- a/inlong-sort/sort-core/pom.xml +++ b/inlong-sort/sort-core/pom.xml @@ -72,18 +72,6 @@ ${mysql.jdbc.version} provided - - org.apache.inlong - sort-format-inlongmsg-base - ${project.version} - test - - - org.apache.inlong - sort-format-csv - ${project.version} - test - org.apache.hadoop hadoop-common @@ -106,6 +94,18 @@ flink-table-common provided + + org.apache.inlong + sort-format-csv + ${project.version} + test + + + org.apache.inlong + sort-format-inlongmsg-base + ${project.version} + test + org.apache.inlong sort-flink-dependencies-${sort.flink.version} @@ -251,6 +251,18 @@ flink-table-common provided + + org.apache.inlong + sort-format-csv + ${project.version} + test + + + org.apache.inlong + sort-format-inlongmsg-base + ${project.version} + test + org.apache.inlong sort-flink-dependencies-${sort.flink.version} @@ -372,6 +384,24 @@ ${project.version} test + + org.apache.inlong + sort-connector-jdbc-v1.18 + ${project.version} + test + + + org.apache.inlong + sort-connector-elasticsearch6-v1.18 + ${project.version} + test + + + org.apache.inlong + sort-connector-elasticsearch7-v1.18 + ${project.version} + test + diff --git a/inlong-sort/sort-dist/pom.xml b/inlong-sort/sort-dist/pom.xml index 0f28f495bf..4541bb37ea 100644 --- a/inlong-sort/sort-dist/pom.xml +++ b/inlong-sort/sort-dist/pom.xml @@ -55,31 +55,6 @@ sort-format-common ${project.version} - - org.apache.inlong - sort-format-base - ${project.version} - - - org.apache.inlong - sort-format-csv - ${project.version} - - - org.apache.inlong - sort-format-inlongmsg-base - ${project.version} - - - org.apache.inlong - sort-format-inlongmsg-csv - ${project.version} - - - org.apache.inlong - sort-format-inlongmsg-kv - ${project.version} - org.apache.inlong sort-format-rowdata-kv @@ -134,6 +109,31 @@ sort-format-json-v1.13 ${project.version} + + org.apache.inlong + sort-format-csv + ${project.version} + + + org.apache.inlong + sort-format-inlongmsg-base + ${project.version} + + + org.apache.inlong + sort-format-inlongmsg-csv + ${project.version} + + + org.apache.inlong + sort-format-inlongmsg-kv + ${project.version} + + + org.apache.inlong + sort-format-base + ${project.version} + org.apache.flink flink-sql-parquet_${scala.binary.version} @@ -172,6 +172,31 @@ sort-format-json-v1.15 ${project.version} + + org.apache.inlong + sort-format-csv + ${project.version} + + + org.apache.inlong + sort-format-inlongmsg-base + ${project.version} + + + org.apache.inlong + sort-format-inlongmsg-csv + ${project.version} + + + org.apache.inlong + sort-format-inlongmsg-kv + ${project.version} + + + org.apache.inlong + sort-format-base + ${project.version} + org.apache.flink flink-sql-parquet diff --git a/inlong-sort/sort-end-to-end-tests/pom.xml b/inlong-sort/sort-end-to-end-tests/pom.xml index 04b87c0282..6c6319cd4e 100644 --- a/inlong-sort/sort-end-to-end-tests/pom.xml +++ b/inlong-sort/sort-end-to-end-tests/pom.xml @@ -52,6 +52,15 @@ sort-end-to-end-tests-v1.15 + + v1.18 + + true + + + sort-end-to-end-tests-v1.18 + + diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml new file mode 100644 index 0000000000..22c8e6fc6b --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml @@ -0,0 +1,209 @@ + + + + 4.0.0 + + org.apache.inlong + sort-end-to-end-tests + 1.14.0-SNAPSHOT + + + sort-end-to-end-tests-v1.18 + Apache InLong - Sort End to End Tests v1.18 + + + ${project.parent.parent.parent.basedir} + 1.18.1 + 6.8.17 + 2.15.3-18.0 + + + + + org.apache.inlong + sort-dist + ${project.version} + test + + + org.testcontainers + testcontainers + + + org.testcontainers + postgresql + ${testcontainers.version} + + + org.postgresql + postgresql + test + + + org.testcontainers + elasticsearch + ${testcontainers.version} + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + org.elasticsearch.client + elasticsearch-rest-client + ${elasticsearch.version} + + + + org.apache.flink + flink-shaded-jackson + ${flink.shaded.jackson.version} + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + org.apache.logging.log4j + log4j-slf4j-impl + + + org.apache.logging.log4j + log4j-core + + + + + org.apache.inlong + sort-flink-dependencies-v1.18 + ${project.version} + test + + + org.apache.flink + flink-core + ${flink.version} + test + + + org.apache.flink + flink-json + ${flink.version} + test + + + org.apache.flink + flink-avro + ${flink.version} + test + + + org.apache.flink + flink-csv + ${flink.version} + test + + + org.apache.flink + flink-sql-avro + ${flink.version} + test + + + org.apache.flink + flink-runtime + ${flink.version} + test + + + org.apache.flink + flink-table-common + ${flink.version} + test + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.apache.inlong + sort-dist + ${project.version} + sort-dist.jar + jar + ${project.build.directory}/dependencies + + + + + + copy-jars + + copy + + validate + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + end-to-end-tests-v1.18 + integration-test + + + **/*.* + + 1 + + ${project.basedir} + + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${plugin.surefire.version} + + + + diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnv.java b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnv.java new file mode 100644 index 0000000000..de6166442e --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnv.java @@ -0,0 +1,241 @@ +/* + * 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.inlong.sort.tests.utils; + +import org.apache.commons.io.IOUtils; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.client.deployment.StandaloneClusterId; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container.ExecResult; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.images.builder.Transferable; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; +import java.util.jar.JarOutputStream; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * End to end base test environment for test sort-connectors. + * Every link : MySQL -> Xxx (Test connector) -> MySQL + */ +public abstract class FlinkContainerTestEnv extends TestLogger { + + static final Logger JM_LOG = LoggerFactory.getLogger(JobMaster.class); + static final Logger TM_LOG = LoggerFactory.getLogger(TaskExecutor.class); + static final Logger LOG = LoggerFactory.getLogger(FlinkContainerTestEnv.class); + + private static final Path SORT_DIST_JAR = TestUtils.getResource("sort-dist.jar"); + // ------------------------------------------------------------------------------------------ + // Flink Variables + // ------------------------------------------------------------------------------------------ + static final int JOB_MANAGER_REST_PORT = 8081; + static final int DEBUG_PORT = 20000; + static final String FLINK_BIN = "bin"; + static final String INTER_CONTAINER_JM_ALIAS = "jobmanager"; + static final String INTER_CONTAINER_TM_ALIAS = "taskmanager"; + static final String FLINK_PROPERTIES = String.join("\n", Arrays.asList( + "jobmanager.rpc.address: jobmanager", + "taskmanager.numberOfTaskSlots: 10", + "parallelism.default: 4", + "env.java.opts.jobmanager: -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=20000", + "env.java.opts.taskmanager: -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=20000", + // this is needed for oracle-cdc tests. + // see https://stackoverflow.com/a/47062742/4915129 + "env.java.opts: -Doracle.jdbc.timezoneAsRegion=false")); + + @ClassRule + public static final Network NETWORK = Network.newNetwork(); + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Nullable + private static RestClusterClient restClusterClient; + + static GenericContainer jobManager; + static GenericContainer taskManager; + + @AfterClass + public static void after() { + if (restClusterClient != null) { + restClusterClient.close(); + } + if (jobManager != null) { + jobManager.stop(); + } + if (taskManager != null) { + taskManager.stop(); + } + } + + /** + * Submits a SQL job to the running cluster. + * + *

NOTE: You should not use {@code '\t'}. + */ + public void submitSQLJob(String sqlFile, Path... jars) + throws IOException, InterruptedException { + final List commands = new ArrayList<>(); + String containerSqlFile = copyToContainerTmpPath(jobManager, sqlFile); + commands.add(FLINK_BIN + "/flink run -d"); + commands.add("-c org.apache.inlong.sort.Entrance"); + commands.add(copyToContainerTmpPath(jobManager, constructDistJar(jars))); + commands.add("--sql.script.file"); + commands.add(containerSqlFile); + + ExecResult execResult = + jobManager.execInContainer("bash", "-c", String.join(" ", commands)); + LOG.info(execResult.getStdout()); + if (execResult.getExitCode() != 0) { + LOG.error(execResult.getStderr()); + throw new AssertionError("Failed when submitting the SQL job."); + } + } + + /** + * Get {@link RestClusterClient} connected to this FlinkContainer. + * + *

This method lazily initializes the REST client on-demand. + */ + public RestClusterClient getRestClusterClient() { + checkState( + jobManager.isRunning(), + "Cluster client should only be retrieved for a running cluster"); + try { + final Configuration clientConfiguration = new Configuration(); + clientConfiguration.set(RestOptions.ADDRESS, jobManager.getHost()); + clientConfiguration.set( + RestOptions.PORT, jobManager.getMappedPort(JOB_MANAGER_REST_PORT)); + this.restClusterClient = + new RestClusterClient<>(clientConfiguration, StandaloneClusterId.getInstance()); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to create client for Flink container cluster", e); + } + return restClusterClient; + } + + /** + * Polling to detect task status until the task successfully into {@link JobStatus.RUNNING} + * + * @param timeout + */ + public void waitUntilJobRunning(Duration timeout) { + RestClusterClient clusterClient = getRestClusterClient(); + Deadline deadline = Deadline.fromNow(timeout); + while (deadline.hasTimeLeft()) { + Collection jobStatusMessages; + try { + jobStatusMessages = clusterClient.listJobs().get(10, TimeUnit.SECONDS); + } catch (Exception e) { + LOG.warn("Error when fetching job status.", e); + continue; + } + if (jobStatusMessages != null && !jobStatusMessages.isEmpty()) { + JobStatusMessage message = jobStatusMessages.iterator().next(); + JobStatus jobStatus = message.getJobState(); + if (jobStatus.isTerminalState()) { + throw new ValidationException( + String.format( + "Job has been terminated! JobName: %s, JobID: %s, Status: %s", + message.getJobName(), + message.getJobId(), + message.getJobState())); + } else if (jobStatus == JobStatus.RUNNING) { + return; + } + } + } + } + + /** + * Copy all other dependencies into user jar 'lib/' entry. + * Flink per-job mode only support upload one jar to cluster. + */ + private String constructDistJar(Path... jars) throws IOException { + + File newJar = temporaryFolder.newFile("sort-dist.jar"); + try ( + JarFile jarFile = new JarFile(SORT_DIST_JAR.toFile()); + JarOutputStream jos = new JarOutputStream(new FileOutputStream(newJar))) { + jarFile.stream().forEach(entry -> { + try (InputStream is = jarFile.getInputStream(entry)) { + jos.putNextEntry(entry); + jos.write(IOUtils.toByteArray(is)); + jos.closeEntry(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + for (Path jar : jars) { + try (InputStream is = new FileInputStream(jar.toFile())) { + jos.putNextEntry(new JarEntry("lib/" + jar.getFileName().toString())); + jos.write(IOUtils.toByteArray(is)); + jos.closeEntry(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + } + return newJar.getAbsolutePath(); + } + + // Should not a big file, all file data will load into memory, then copy to container. + private String copyToContainerTmpPath(GenericContainer container, String filePath) throws IOException { + Path path = Paths.get(filePath); + byte[] fileData = Files.readAllBytes(path); + String containerPath = "/tmp/" + path.getFileName(); + container.copyFileToContainer(Transferable.of(fileData), containerPath); + return containerPath; + } +} diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE11.java b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE11.java new file mode 100644 index 0000000000..9033740822 --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE11.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.tests.utils; + +import org.junit.BeforeClass; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.util.stream.Stream; + +public abstract class FlinkContainerTestEnvJRE11 extends FlinkContainerTestEnv { + + @BeforeClass + public static void before() { + LOG.info("Starting containers..."); + jobManager = + new GenericContainer<>("flink:1.18.1-scala_2.12") + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_JM_ALIAS) + .withExposedPorts(JOB_MANAGER_REST_PORT, DEBUG_PORT) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withExposedPorts(JOB_MANAGER_REST_PORT) + .withLogConsumer(new Slf4jLogConsumer(JM_LOG)); + taskManager = + new GenericContainer<>("flink:1.18.1-scala_2.12") + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + .withExposedPorts(DEBUG_PORT) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .dependsOn(jobManager) + .withLogConsumer(new Slf4jLogConsumer(TM_LOG)); + + Startables.deepStart(Stream.of(jobManager)).join(); + Startables.deepStart(Stream.of(taskManager)).join(); + LOG.info("Containers are started."); + } +} diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE8.java b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE8.java new file mode 100644 index 0000000000..de982da4ba --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/FlinkContainerTestEnvJRE8.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.tests.utils; + +import org.junit.BeforeClass; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.util.stream.Stream; + +public abstract class FlinkContainerTestEnvJRE8 extends FlinkContainerTestEnv { + + @BeforeClass + public static void before() { + LOG.info("Starting containers..."); + jobManager = + new GenericContainer<>("flink:1.18.1-scala_2.12-java8") + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_JM_ALIAS) + .withExposedPorts(JOB_MANAGER_REST_PORT, DEBUG_PORT) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .withExposedPorts(JOB_MANAGER_REST_PORT) + .withLogConsumer(new Slf4jLogConsumer(JM_LOG)); + taskManager = + new GenericContainer<>("flink:1.18.1-scala_2.12-java8") + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_TM_ALIAS) + .withExposedPorts(DEBUG_PORT) + .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) + .dependsOn(jobManager) + .withLogConsumer(new Slf4jLogConsumer(TM_LOG)); + + Startables.deepStart(Stream.of(jobManager)).join(); + Startables.deepStart(Stream.of(taskManager)).join(); + LOG.info("Containers are started."); + } +} diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/PlaceholderResolver.java b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/PlaceholderResolver.java new file mode 100644 index 0000000000..0c28333699 --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/PlaceholderResolver.java @@ -0,0 +1,150 @@ +/* + * 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.inlong.sort.tests.utils; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * A file placeholder replacement tool. + */ +public class PlaceholderResolver { + + /** + * Default placeholder prefix + */ + public static final String DEFAULT_PLACEHOLDER_PREFIX = "${"; + + /** + * Default placeholder suffix + */ + public static final String DEFAULT_PLACEHOLDER_SUFFIX = "}"; + + /** + * Default singleton resolver + */ + private static PlaceholderResolver defaultResolver = new PlaceholderResolver(); + + /** + * Placeholder prefix + */ + private String placeholderPrefix = DEFAULT_PLACEHOLDER_PREFIX; + + /** + * Placeholder suffix + */ + private String placeholderSuffix = DEFAULT_PLACEHOLDER_SUFFIX; + + private PlaceholderResolver() { + + } + + private PlaceholderResolver(String placeholderPrefix, String placeholderSuffix) { + this.placeholderPrefix = placeholderPrefix; + this.placeholderSuffix = placeholderSuffix; + } + + public static PlaceholderResolver getDefaultResolver() { + return defaultResolver; + } + + public static PlaceholderResolver getResolver(String placeholderPrefix, String placeholderSuffix) { + return new PlaceholderResolver(placeholderPrefix, placeholderSuffix); + } + + /** + * Replace template string with special placeholder according to replace function. + * @param content template string with special placeholder + * @param rule placeholder replacement rule + * @return new replaced string + */ + public String resolveByRule(String content, Function rule) { + int start = content.indexOf(this.placeholderPrefix); + if (start == -1) { + return content; + } + StringBuilder result = new StringBuilder(content); + while (start != -1) { + int end = result.indexOf(this.placeholderSuffix, start); + // get placeholder actual value (e.g. ${id}, get the value represent id) + String placeholder = result.substring(start + this.placeholderPrefix.length(), end); + // replace placeholder value + String replaceContent = placeholder.trim().isEmpty() ? "" : rule.apply(placeholder); + result.replace(start, end + this.placeholderSuffix.length(), replaceContent); + start = result.indexOf(this.placeholderPrefix, start + replaceContent.length()); + } + return result.toString(); + } + + /** + * Replace template string with special placeholder according to replace function. + * @param file template file with special placeholder + * @param rule placeholder replacement rule + * @return new replaced string + */ + public Path resolveByRule(Path file, Function rule) { + try { + List newContents = Files.readAllLines(file, StandardCharsets.UTF_8) + .stream() + .map(content -> resolveByRule(content, rule)) + .collect(Collectors.toList()); + Path newPath = Paths.get(file.getParent().toString(), file.getFileName() + "$"); + Files.write(newPath, String.join(System.lineSeparator(), newContents).getBytes(StandardCharsets.UTF_8)); + return newPath; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Replace template string with special placeholder according to properties file. + * Key is the content of the placeholder

+ * e.g: content = product:${id}:detail:${did}
+ * valueMap = id -> 1; pid -> 2
+ * return: product:1:detail:2
+ * + * @param content template string with special placeholder + * @param valueMap placeholder replacement map + * @return new replaced string + */ + public String resolveByMap(String content, final Map valueMap) { + return resolveByRule(content, placeholderValue -> String.valueOf(valueMap.get(placeholderValue))); + } + + /** + * Replace template string with special placeholder according to properties file. + * Key is the content of the placeholder

+ * e.g: content = product:${id}:detail:${did}
+ * valueMap = id -> 1; pid -> 2
+ * return: product:1:detail:2
+ * + * @param file template string with special placeholder + * @param valueMap placeholder replacement map + * @return new replaced string + */ + public Path resolveByMap(Path file, final Map valueMap) { + return resolveByRule(file, placeholderValue -> String.valueOf(valueMap.get(placeholderValue))); + } +} diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/TestUtils.java b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/TestUtils.java new file mode 100644 index 0000000000..8daff533da --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/utils/TestUtils.java @@ -0,0 +1,124 @@ +/* + * 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.inlong.sort.tests.utils; + +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.Assert.assertEquals; + +/** + * Test util for test container. + */ +public class TestUtils { + + private static final ParameterProperty MODULE_DIRECTORY = + new ParameterProperty<>("moduleDir", Paths::get); + + /** + * Searches for a resource file matching the given regex in the given directory. This method is + * primarily intended to be used for the initialization of static {@link Path} fields for + * resource file(i.e. jar, config file) that reside in the modules {@code target} directory. + * + * @param resourceNameRegex regex pattern to match against + * @return Path pointing to the matching jar + * @throws RuntimeException if none or multiple resource files could be found + */ + public static Path getResource(final String resourceNameRegex) { + // if the property is not set then we are most likely running in the IDE, where the working + // directory is the + // module of the test that is currently running, which is exactly what we want + Path moduleDirectory = MODULE_DIRECTORY.get(Paths.get("").toAbsolutePath()); + + try (Stream dependencyResources = Files.walk(moduleDirectory)) { + final List matchingResources = + dependencyResources + .filter( + jar -> Pattern.compile(resourceNameRegex) + .matcher(jar.toAbsolutePath().toString()) + .find()) + .collect(Collectors.toList()); + switch (matchingResources.size()) { + case 0: + throw new RuntimeException( + new FileNotFoundException( + String.format( + "No resource file could be found that matches the pattern %s. " + + "This could mean that the test module must be rebuilt via maven.", + resourceNameRegex))); + case 1: + return matchingResources.get(0); + default: + throw new RuntimeException( + new IOException( + String.format( + "Multiple resource files were found matching the pattern %s. Matches=%s", + resourceNameRegex, matchingResources))); + } + } catch (final IOException ioe) { + throw new RuntimeException("Could not search for resource resource files.", ioe); + } + } + + /** + * A simple system properties value getter with default value when could not find the system property. + * @param + */ + static class ParameterProperty { + + private final String propertyName; + private final Function converter; + + public ParameterProperty(final String propertyName, final Function converter) { + this.propertyName = propertyName; + this.converter = converter; + } + + /** + * Retrieves the value of this property, or the given default if no value was set. + * + * @return the value of this property, or the given default if no value was set + */ + public V get(final V defaultValue) { + final String value = System.getProperty(propertyName); + return value == null ? defaultValue : converter.apply(value); + } + } + + @Test + public void testReplaceholder() { + String before = "today is ${date}, today weather is ${weather}"; + Map maps = new HashMap<>(); + maps.put("date", "2024.07.15"); + maps.put("weather", "song"); + String after = PlaceholderResolver.getDefaultResolver().resolveByMap(before, maps); + assertEquals(after, "today is 2024.07.15, today weather is song"); + } +} diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/log4j2-test.properties b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000000..3e95477751 --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/log4j2-test.properties @@ -0,0 +1,47 @@ +# +# 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. +# +rootLogger=INFO, STDOUT + +appender.console.type=Console +appender.console.name=STDOUT +appender.console.layout.type=PatternLayout +appender.console.layout.pattern=%-4r [%t] %-5p %c %x - %m%n + +appender.jm.type = File +appender.jm.name = jobmanager +appender.jm.fileName = target/logs/jobmanager.log +appender.jm.layout.type = PatternLayout +appender.jm.layout.pattern = - %m%n + +appender.tm.type = File +appender.tm.name = taskmanager +appender.tm.fileName = target/logs/taskmanager.log +appender.tm.layout.type = PatternLayout +appender.tm.layout.pattern = - %m%n + +logger.jm=INFO, jobmanager +logger.jm.name=org.apache.flink.runtime.jobmaster.JobMaster +logger.jm.additivity=false + +logger.tm=INFO, taskmanager +logger.tm.name=org.apache.flink.runtime.taskexecutor.TaskExecutor +logger.tm.additivity=false + + + diff --git a/inlong-sort/sort-formats/pom.xml b/inlong-sort/sort-formats/pom.xml index e36378306f..3c392001f3 100644 --- a/inlong-sort/sort-formats/pom.xml +++ b/inlong-sort/sort-formats/pom.xml @@ -249,7 +249,8 @@ v1.18 format-common - format-row/format-json-v1.18 + format-row + format-rowdata From dfe9ab590b8b319957372511bdcb12692b6604f0 Mon Sep 17 00:00:00 2001 From: fuweng11 <76141879+fuweng11@users.noreply.github.com> Date: Mon, 2 Sep 2024 14:15:43 +0800 Subject: [PATCH 15/30] [INLONG-10977][Manager] Data preview supports escaping for KV data type (#10978) --- .../service/datatype/KvDataTypeOperator.java | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/datatype/KvDataTypeOperator.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/datatype/KvDataTypeOperator.java index 1eb73888cd..8b0a40af37 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/datatype/KvDataTypeOperator.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/datatype/KvDataTypeOperator.java @@ -23,12 +23,14 @@ import org.apache.inlong.manager.common.util.CommonBeanUtils; import org.apache.inlong.manager.pojo.consume.BriefMQMessage.FieldInfo; import org.apache.inlong.manager.pojo.stream.InlongStreamInfo; +import org.apache.inlong.sdk.transform.decode.KvUtils; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.springframework.stereotype.Service; import java.util.List; +import java.util.Map; @Slf4j @Service @@ -51,18 +53,20 @@ public List parseFields(String str, InlongStreamInfo streamInfo) thro if (StringUtils.isNotBlank(streamInfo.getKvSeparator())) { kvSeparator = (char) Integer.parseInt(streamInfo.getKvSeparator()); } - String[] bodys = StringUtils.split(str, separator); - if (bodys.length != fields.size()) { - log.warn( - "The number of reported fields does not match the number of stream fields for groupId={}, streamId={}, reported field size ={}, stream field size ={}", - streamInfo.getInlongGroupId(), streamInfo.getInlongStreamId(), bodys.length, fields.size()); - return fields; + Character escapeChar = null; + if (StringUtils.isNotBlank(streamInfo.getDataEscapeChar())) { + escapeChar = streamInfo.getDataEscapeChar().charAt(0); } - for (int i = 0; i < bodys.length; i++) { - String body = bodys[i]; - String[] values = StringUtils.split(body, kvSeparator); - fields.get(i).setFieldName(values[0]); - fields.get(i).setFieldValue(values[1]); + Character lineSeparator = null; + if (StringUtils.isNotBlank(streamInfo.getLineSeparator())) { + lineSeparator = (char) Integer.parseInt(streamInfo.getLineSeparator()); + } + List> rowValues = + KvUtils.splitKv(str, separator, kvSeparator, escapeChar, '\"', lineSeparator); + for (Map row : rowValues) { + for (FieldInfo fieldInfo : fields) { + fieldInfo.setFieldValue(row.get(fieldInfo.getFieldName())); + } } } catch (Exception e) { log.warn("parse fields failed for groupId = {}, streamId = {}", streamInfo.getInlongGroupId(), From a3dd2a9a8ae2dcbbda476ffb3a1154da9b45ee1d Mon Sep 17 00:00:00 2001 From: fuweng11 <76141879+fuweng11@users.noreply.github.com> Date: Mon, 2 Sep 2024 14:15:54 +0800 Subject: [PATCH 16/30] [INLONG-10975][Manager] Fix the problem of when saving the group, only the existence of the groupid under the current tenant was verified (#10976) --- .../inlong/manager/service/group/InlongGroupServiceImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/group/InlongGroupServiceImpl.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/group/InlongGroupServiceImpl.java index 5440768191..3ee94a23ce 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/group/InlongGroupServiceImpl.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/group/InlongGroupServiceImpl.java @@ -205,7 +205,7 @@ public String save(InlongGroupRequest request, String operator) { Preconditions.expectNotNull(request, "inlong group request cannot be empty"); String groupId = request.getInlongGroupId(); - InlongGroupEntity entity = groupMapper.selectByGroupId(groupId); + InlongGroupEntity entity = groupMapper.selectByGroupIdWithoutTenant(groupId); if (entity != null) { LOGGER.error("groupId={} has already exists", groupId); throw new BusinessException(ErrorCodeEnum.GROUP_DUPLICATE); @@ -278,7 +278,7 @@ public List batchSave(List groupRequestList, St @Override public Boolean exist(String groupId) { Preconditions.expectNotNull(groupId, ErrorCodeEnum.GROUP_ID_IS_EMPTY.getMessage()); - InlongGroupEntity entity = groupMapper.selectByGroupId(groupId); + InlongGroupEntity entity = groupMapper.selectByGroupIdWithoutTenant(groupId); LOGGER.debug("success to check inlong group {}, exist? {}", groupId, entity != null); return entity != null; } From 7ecff9cd253bcb94755cc988860fbc8ce46381bc Mon Sep 17 00:00:00 2001 From: kamianlaida <165994047+wohainilaodou@users.noreply.github.com> Date: Mon, 2 Sep 2024 14:16:13 +0800 Subject: [PATCH 17/30] [INLONG-10947][Dashboard] A wrong error occurs in data preview (#10974) --- .../src/ui/pages/GroupDetail/DataStream/PreviewModal.tsx | 6 +++--- .../src/ui/pages/ModuleAudit/AuditModule/config.tsx | 7 +++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/inlong-dashboard/src/ui/pages/GroupDetail/DataStream/PreviewModal.tsx b/inlong-dashboard/src/ui/pages/GroupDetail/DataStream/PreviewModal.tsx index fa9edff445..6593c59e6c 100644 --- a/inlong-dashboard/src/ui/pages/GroupDetail/DataStream/PreviewModal.tsx +++ b/inlong-dashboard/src/ui/pages/GroupDetail/DataStream/PreviewModal.tsx @@ -69,9 +69,9 @@ const Comp: React.FC = ({ inlongGroupId, inlongStreamId, ...modalProps }) exitsId = true; } const width = - (cur['fieldName'].length > cur['fieldValue'].length - ? cur['fieldName'].length - : cur['fieldValue'].length) * 10; + (cur['fieldName']?.length > cur['fieldValue']?.length + ? cur['fieldName']?.length + : cur['fieldValue']?.length) * 10; acc.push({ title: cur['fieldName'], key: cur['fieldName'], diff --git a/inlong-dashboard/src/ui/pages/ModuleAudit/AuditModule/config.tsx b/inlong-dashboard/src/ui/pages/ModuleAudit/AuditModule/config.tsx index 81d05bd15d..2ef04c5bd5 100644 --- a/inlong-dashboard/src/ui/pages/ModuleAudit/AuditModule/config.tsx +++ b/inlong-dashboard/src/ui/pages/ModuleAudit/AuditModule/config.tsx @@ -350,13 +350,12 @@ export const getTableColumns = (source, dim) => { title: item.auditName, dataIndex: item.auditId, render: text => { - let color = 'black'; if (text?.includes('+')) { - color = 'red'; + return {text}; } else if (text?.includes('-')) { - color = 'green'; + return {text}; } - return {text}; + return {text}; }, })); return [ From 5c44cde51727cd3fa918358e624b751050530d01 Mon Sep 17 00:00:00 2001 From: emptyOVO <118812562+emptyOVO@users.noreply.github.com> Date: Mon, 2 Sep 2024 14:26:22 +0800 Subject: [PATCH 18/30] [INLONG-10966][SDK] Fix HexFunction set the Java keyword 'public' (#10967) --- .../inlong/sdk/transform/process/function/HexFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java index c641209162..0d33056c72 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/HexFunction.java @@ -36,7 +36,7 @@ * - If the input argument is a string, the HEX function converts each character in the string to its corresponding hexadecimal ASCII encoding and returns the hexadecimal representation of the entire string. */ @TransformFunction(names = {"hex"}) -class HexFunction implements ValueParser { +public class HexFunction implements ValueParser { private static final Pattern BIG_DECIMAL_PATTERN = Pattern.compile("^[-+]?\\d+(\\.\\d+)?([eE][-+]?\\d+)?$"); From 4de972e9324f443ab3c71f88b5c43f93c9825ee6 Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Mon, 2 Sep 2024 14:28:32 +0800 Subject: [PATCH 19/30] [INLONG-10942][SDK] Add official function names for all Transform functions (#10943) --- .../sdk/transform/process/function/DateExtractFunction.java | 6 +++--- .../transform/process/function/FromUnixTimeFunction.java | 2 +- .../sdk/transform/process/function/SubstringFunction.java | 4 +++- .../transform/process/function/TimestampAddFunction.java | 2 +- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java index cac2273619..a1ab7e2dfa 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/DateExtractFunction.java @@ -86,7 +86,7 @@ public WeekExtractFunction(Function expr) { } } - @TransformFunction(names = {"day_of_year"}) + @TransformFunction(names = {"day_of_year", "dayofyear"}) public static class DayOfYearExtractFunction extends DateExtractFunction { public DayOfYearExtractFunction(Function expr) { @@ -94,7 +94,7 @@ public DayOfYearExtractFunction(Function expr) { } } - @TransformFunction(names = {"day_of_month"}) + @TransformFunction(names = {"day_of_month", "dayofmonth"}) public static class DayOfMonthExtractFunction extends DateExtractFunction { public DayOfMonthExtractFunction(Function expr) { @@ -102,7 +102,7 @@ public DayOfMonthExtractFunction(Function expr) { } } - @TransformFunction(names = {"day_of_week"}) + @TransformFunction(names = {"day_of_week", "dayofweek"}) public static class DayOfWeekExtractFunction extends DateExtractFunction { public DayOfWeekExtractFunction(Function expr) { diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/FromUnixTimeFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/FromUnixTimeFunction.java index b7b68a0e73..6a897ee3e0 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/FromUnixTimeFunction.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/FromUnixTimeFunction.java @@ -41,7 +41,7 @@ * format(default is ‘yyyy-MM-dd HH:mm:ss’). numeric is an internal timestamp value representing seconds * since ‘1970-01-01 00:00:00’ UTC, such as produced by the UNIX_TIMESTAMP() function. */ -@TransformFunction(names = {"from_unix_time"}) +@TransformFunction(names = {"from_unix_time", "form_unixtime"}) public class FromUnixTimeFunction implements ValueParser { private ValueParser numericParser; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SubstringFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SubstringFunction.java index 772dc2950c..727c0484b4 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SubstringFunction.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/SubstringFunction.java @@ -32,7 +32,7 @@ * description: substring(string FROM INT1 [ FOR INT2 ])--returns a substring of STRING starting from position INT1 with * length INT2 (to the end by default) */ -@TransformFunction(names = {"substring"}) +@TransformFunction(names = {"substring", "substr"}) public class SubstringFunction implements ValueParser { private ValueParser stringParser; @@ -41,6 +41,7 @@ public class SubstringFunction implements ValueParser { /** * Constructor + * * @param expr */ public SubstringFunction(Function expr) { @@ -55,6 +56,7 @@ public SubstringFunction(Function expr) { /** * parse + * * @param sourceData * @param rowIndex * @return diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/TimestampAddFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/TimestampAddFunction.java index 9bcbb8c8e4..29b4636d81 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/TimestampAddFunction.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/TimestampAddFunction.java @@ -36,7 +36,7 @@ * The unit of the time interval is specified by the unit parameter, which should be one of the following values: * FRAC_SECOND, SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR. */ -@TransformFunction(names = {"timestamp_add"}) +@TransformFunction(names = {"timestamp_add", "timestampadd"}) public class TimestampAddFunction implements ValueParser { private ValueParser intervalParser; From c4bf54bbb1bb0d83429c1aa9b93ef721bf376e90 Mon Sep 17 00:00:00 2001 From: fuweng11 <76141879+fuweng11@users.noreply.github.com> Date: Mon, 2 Sep 2024 20:24:25 +0800 Subject: [PATCH 20/30] [INLONG-10988][Manager] Data preview filters data in tubes based on streamId (#10989) --- .../manager/service/resource/queue/tubemq/TubeMQOperator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/tubemq/TubeMQOperator.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/tubemq/TubeMQOperator.java index 050de07805..66d1fcf59a 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/tubemq/TubeMQOperator.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/tubemq/TubeMQOperator.java @@ -71,6 +71,7 @@ public class TubeMQOperator { private static final String CREATE_USER = "&createUser="; private static final String CONF_MOD_AUTH_TOKEN = "&confModAuthToken="; private static final String MSG_COUNT = "&msgCount="; + private static final String FILTER_CONDS = "&filterConds="; private static final String QUERY_TOPIC_PATH = "/webapi.htm?method=admin_query_cluster_topic_view"; private static final String QUERY_BROKER_PATH = "/webapi.htm?method=admin_query_broker_run_status"; @@ -288,7 +289,7 @@ public List queryLastMessage(TubeClusterInfo tubeCluster, String } String url = "http://" + brokerUrl + QUERY_MESSAGE_PATH + TOPIC_NAME + topicName + MSG_COUNT - + request.getMessageCount(); + + request.getMessageCount() + FILTER_CONDS + streamInfo.getInlongStreamId(); TubeMessageResponse response = HttpUtils.request(restTemplate, url, HttpMethod.GET, null, new HttpHeaders(), TubeMessageResponse.class); if (response.getErrCode() != SUCCESS_CODE && response.getErrCode() != 200) { From c08549eff5113d23b03f4436ca02a77949f4a49d Mon Sep 17 00:00:00 2001 From: emptyOVO <118812562+emptyOVO@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:23:03 +0800 Subject: [PATCH 21/30] [INLONG-10984][SDK] Transform support RADIANS(x) function (#10987) * [INLONG-10984][SDK] Transform support RADIANS(x) function * fix: add description --- .../process/function/RadiansFunction.java | 49 +++++++++++++++++++ ...TransformArithmeticFunctionsProcessor.java | 17 +++++++ 2 files changed, 66 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RadiansFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RadiansFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RadiansFunction.java new file mode 100644 index 0000000000..635d18b9b8 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/RadiansFunction.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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Function; + +/** + * RadiansFunction + * description: + * - RADIANS(x)--returns radians of x, Convert degrees to radians + */ +@TransformFunction(names = {"radians"}) +public class RadiansFunction implements ValueParser { + + private ValueParser degreeParser; + + public RadiansFunction(Function expr) { + degreeParser = OperatorTools.buildParser(expr.getParameters().getExpressions().get(0)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object degreeObj = degreeParser.parse(sourceData, rowIndex, context); + if (degreeObj == null) { + return null; + } + return Math.toRadians(OperatorTools.parseBigDecimal(degreeObj).doubleValue()); + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java index 1e0f14025f..c36835e579 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java @@ -422,6 +422,23 @@ public void testLnFunction() throws Exception { Assert.assertEquals(output2.get(0), "result=2.302585092994046"); } + @Test + public void testRadiansFunction() throws Exception { + String transformSql = "select radians(numeric1) from source"; + TransformConfig config = new TransformConfig(transformSql); + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case1: radians(10) + List output1 = processor.transform("10|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=0.17453292519943295"); + // case2: radians(18.97) + List output2 = processor.transform("18.97|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output2.size()); + Assert.assertEquals(output2.get(0), "result=0.33108895910332425"); + } + @Test public void testLog10Function() throws Exception { String transformSql = "select log10(numeric1) from source"; From bc6e9610bb84efe3e794cda5d79a7dda784b71e8 Mon Sep 17 00:00:00 2001 From: Xincheng Huang <60057611+ying-hua@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:23:19 +0800 Subject: [PATCH 22/30] [INLONG-10982][SDK] Improve the test code structure (#10985) --- .../process/TestTransformProcessor.java | 67 +++++-------------- 1 file changed, 16 insertions(+), 51 deletions(-) diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java index 20af097de6..3413f1aca3 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java @@ -44,13 +44,7 @@ public class TestTransformProcessor { @Test public void testCsv2Kv() throws Exception { - List fields = new ArrayList<>(); - FieldInfo ftime = new FieldInfo(); - ftime.setName("ftime"); - fields.add(ftime); - FieldInfo extinfo = new FieldInfo(); - extinfo.setName("extinfo"); - fields.add(extinfo); + List fields = this.getTestFieldList("ftime", "extinfo"); CsvSourceInfo csvSource = new CsvSourceInfo("UTF-8", '|', '\\', fields); KvSinkInfo kvSink = new KvSinkInfo("UTF-8", fields); String transformSql = "select ftime,extinfo from source where extinfo='ok'"; @@ -97,13 +91,7 @@ public void testCsv2KvNoField() throws Exception { @Test public void testKv2Csv() throws Exception { - List fields = new ArrayList<>(); - FieldInfo ftime = new FieldInfo(); - ftime.setName("ftime"); - fields.add(ftime); - FieldInfo extinfo = new FieldInfo(); - extinfo.setName("extinfo"); - fields.add(extinfo); + List fields = this.getTestFieldList("ftime", "extinfo"); KvSourceInfo kvSource = new KvSourceInfo("UTF-8", fields); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); String transformSql = "select ftime,extinfo from source where extinfo='ok'"; @@ -148,7 +136,7 @@ public void testKv2CsvNoField() throws Exception { @Test public void testJson2Csv() throws Exception { - List fields1 = this.getTestFieldList(); + List fields1 = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); JsonSourceInfo jsonSource1 = new JsonSourceInfo("UTF-8", "msgs"); CsvSinkInfo csvSink1 = new CsvSinkInfo("UTF-8", '|', '\\', fields1); String transformSql1 = "select $root.sid,$root.packageID,$child.msgTime,$child.msg from source"; @@ -170,7 +158,7 @@ public void testJson2Csv() throws Exception { Assert.assertEquals(output1.get(0), "value1|value2|1713243918000|value4"); Assert.assertEquals(output1.get(1), "value1|value2|1713243918000|v4"); // case2 - List fields2 = this.getTestFieldList2(); + List fields2 = this.getTestFieldList("id", "itemId", "subItemId", "msg"); JsonSourceInfo jsonSource2 = new JsonSourceInfo("UTF-8", "items"); CsvSinkInfo csvSink2 = new CsvSinkInfo("UTF-8", '|', '\\', fields2); String transformSql2 = @@ -205,7 +193,7 @@ public void testJson2Csv() throws Exception { @Test public void testJson2CsvForOne() throws Exception { - List fields = this.getTestFieldList(); + List fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); JsonSourceInfo jsonSource = new JsonSourceInfo("UTF-8", ""); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); String transformSql = "select $root.sid,$root.packageID,$root.msgs(1).msgTime,$root.msgs(0).msg from source"; @@ -229,7 +217,7 @@ public void testJson2CsvForOne() throws Exception { @Test public void testPb2Csv() throws Exception { - List fields = this.getTestFieldList(); + List fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); String transformBase64 = this.getPbTestDescription(); PbSourceInfo pbSource = new PbSourceInfo("UTF-8", transformBase64, "SdkDataRequest", "msgs"); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); @@ -246,36 +234,13 @@ public void testPb2Csv() throws Exception { Assert.assertEquals(output.get(1), "sid|1|1713243918002|msgValue42"); } - private List getTestFieldList() { - List fields = new ArrayList<>(); - FieldInfo sid = new FieldInfo(); - sid.setName("sid"); - fields.add(sid); - FieldInfo packageID = new FieldInfo(); - packageID.setName("packageID"); - fields.add(packageID); - FieldInfo msgTime = new FieldInfo(); - msgTime.setName("msgTime"); - fields.add(msgTime); - FieldInfo msg = new FieldInfo(); - msg.setName("msg"); - fields.add(msg); - return fields; - } - private List getTestFieldList2() { + private List getTestFieldList(String... fieldNames) { List fields = new ArrayList<>(); - FieldInfo id = new FieldInfo(); - id.setName("id"); - fields.add(id); - FieldInfo itemId = new FieldInfo(); - itemId.setName("itemId"); - fields.add(itemId); - FieldInfo subItemId = new FieldInfo(); - subItemId.setName("subItemId"); - fields.add(subItemId); - FieldInfo msg = new FieldInfo(); - msg.setName("msg"); - fields.add(msg); + for (String fieldName : fieldNames) { + FieldInfo field = new FieldInfo(); + field.setName(fieldName); + fields.add(field); + } return fields; } @@ -309,7 +274,7 @@ private String getPbTestDescription() { @Test public void testPb2CsvForOne() throws Exception { - List fields = this.getTestFieldList(); + List fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); String transformBase64 = this.getPbTestDescription(); PbSourceInfo pbSource = new PbSourceInfo("UTF-8", transformBase64, "SdkDataRequest", null); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); @@ -327,7 +292,7 @@ public void testPb2CsvForOne() throws Exception { @Test public void testPb2CsvForAdd() throws Exception { - List fields = this.getTestFieldList(); + List fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); String transformBase64 = this.getPbTestDescription(); PbSourceInfo pbSource = new PbSourceInfo("UTF-8", transformBase64, "SdkDataRequest", null); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); @@ -351,7 +316,7 @@ public void testPb2CsvForAdd() throws Exception { @Test public void testPb2CsvForConcat() throws Exception { - List fields = this.getTestFieldList(); + List fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); String transformBase64 = this.getPbTestDescription(); PbSourceInfo pbSource = new PbSourceInfo("UTF-8", transformBase64, "SdkDataRequest", "msgs"); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); @@ -371,7 +336,7 @@ public void testPb2CsvForConcat() throws Exception { @Test public void testPb2CsvForNow() throws Exception { - List fields = this.getTestFieldList(); + List fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg"); String transformBase64 = this.getPbTestDescription(); PbSourceInfo pbSource = new PbSourceInfo("UTF-8", transformBase64, "SdkDataRequest", "msgs"); CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', fields); From dfd42dc88d7f5bb95d66165ae5cfcf6c92921bd7 Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:24:08 +0800 Subject: [PATCH 23/30] [INLONG-10809][SDK] Improvements to TypeConverter field types and CompareValue in OperatorTools (#10817) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../sdk/transform/decode/CsvSourceData.java | 10 +- .../transform/decode/CsvSourceDecoder.java | 8 +- .../sdk/transform/decode/SourceData.java | 2 +- .../inlong/sdk/transform/pojo/FieldInfo.java | 2 +- .../process/converter/DoubleConverter.java | 26 ++ .../process/converter/LongConverter.java | 26 ++ .../process/function/IfFunction.java | 55 +++ .../process/operator/OperatorTools.java | 21 +- .../process/parser/DoubleParser.java | 1 + ...TransformExpressionOperatorsProcessor.java | 354 ++++++++++++++++++ 10 files changed, 485 insertions(+), 20 deletions(-) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/DoubleConverter.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/LongConverter.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfFunction.java create mode 100644 inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformExpressionOperatorsProcessor.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceData.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceData.java index d4492b4b85..e0bd9f794c 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceData.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceData.java @@ -28,14 +28,14 @@ */ public class CsvSourceData implements SourceData { - private List> rows = new ArrayList<>(); + private List> rows = new ArrayList<>(); - private Map currentRow; + private Map currentRow; public CsvSourceData() { } - public void putField(String fieldName, String fieldValue) { + public void putField(String fieldName, Object fieldValue) { this.currentRow.put(fieldName, fieldValue); } @@ -50,11 +50,11 @@ public int getRowCount() { } @Override - public String getField(int rowNum, String fieldName) { + public Object getField(int rowNum, String fieldName) { if (rowNum >= this.rows.size()) { return null; } - Map targetRow = this.rows.get(rowNum); + Map targetRow = this.rows.get(rowNum); return targetRow.get(fieldName); } } diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceDecoder.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceDecoder.java index fb95dadc43..7b3dedb637 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceDecoder.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/CsvSourceDecoder.java @@ -76,9 +76,13 @@ public SourceData decode(String srcString, Context context) { int fieldIndex = 0; for (FieldInfo field : fields) { String fieldName = field.getName(); - String fieldValue = null; + Object fieldValue = null; if (fieldIndex < fieldValues.length) { - fieldValue = fieldValues[fieldIndex]; + try { + fieldValue = field.getConverter().convert(fieldValues[fieldIndex]); + } catch (Exception e) { + throw new RuntimeException(e); + } } sourceData.putField(fieldName, fieldValue); fieldIndex++; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/SourceData.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/SourceData.java index 2c39948f2d..cf5f9c0fbe 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/SourceData.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/decode/SourceData.java @@ -26,5 +26,5 @@ public interface SourceData { int getRowCount(); - String getField(int rowNum, String fieldName); + Object getField(int rowNum, String fieldName); } diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/pojo/FieldInfo.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/pojo/FieldInfo.java index 1027dad944..2a7834112a 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/pojo/FieldInfo.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/pojo/FieldInfo.java @@ -28,7 +28,7 @@ public class FieldInfo { private String name; - private TypeConverter converter; + private TypeConverter converter = TypeConverter.DefaultTypeConverter(); public FieldInfo() { diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/DoubleConverter.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/DoubleConverter.java new file mode 100644 index 0000000000..52afbda16b --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/DoubleConverter.java @@ -0,0 +1,26 @@ +/* + * 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.inlong.sdk.transform.process.converter; + +public class DoubleConverter implements TypeConverter { + + @Override + public Object convert(String value) throws Exception { + return Double.parseDouble(value); + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/LongConverter.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/LongConverter.java new file mode 100644 index 0000000000..5a18f8ee13 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/converter/LongConverter.java @@ -0,0 +1,26 @@ +/* + * 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.inlong.sdk.transform.process.converter; + +public class LongConverter implements TypeConverter { + + @Override + public Object convert(String value) throws Exception { + return Long.parseLong(value); + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfFunction.java new file mode 100644 index 0000000000..bda7b9301c --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/IfFunction.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.ExpressionOperator; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.util.List; + +/** + * IfFunction + * description: if(expr,r1,r2) -- expr is an expression, if it holds, return r1; otherwise, return r2 + */ +@TransformFunction(names = {"if"}) +public class IfFunction implements ValueParser { + + private final ExpressionOperator expressionOperator; + private final ValueParser tureValueParser; + private final ValueParser falseValueParser; + + public IfFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + expressionOperator = OperatorTools.buildOperator(expressions.get(0)); + tureValueParser = OperatorTools.buildParser(expressions.get(1)); + falseValueParser = OperatorTools.buildParser(expressions.get(2)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + boolean condition = expressionOperator.check(sourceData, rowIndex, context); + return condition ? tureValueParser.parse(sourceData, rowIndex, context) + : falseValueParser.parse(sourceData, rowIndex, context); + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java index 02b24cdb6b..bb35bb4490 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java @@ -133,19 +133,18 @@ public static int compareValue(Comparable left, Comparable right) { if (right == null) { return 1; } - if (left instanceof String) { - if (right instanceof String) { - return ObjectUtils.compare(left, right); - } else { - BigDecimal leftValue = parseBigDecimal(left); - return ObjectUtils.compare(leftValue, right); - } + + if (((Object) left).getClass() == ((Object) right).getClass()) { + return ObjectUtils.compare(left, right); } else { - if (right instanceof String) { + try { + BigDecimal leftValue = parseBigDecimal(left); BigDecimal rightValue = parseBigDecimal(right); - return ObjectUtils.compare(left, rightValue); - } else { - return ObjectUtils.compare(left, right); + return ObjectUtils.compare(leftValue, rightValue); + } catch (Exception e) { + String leftValue = parseString(left); + String rightValue = parseString(right); + return ObjectUtils.compare(leftValue, rightValue); } } } diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java index ad39558a11..a88b17f6ba 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/DoubleParser.java @@ -24,6 +24,7 @@ /** * LongParser + * */ @TransformParser(values = DoubleValue.class) public class DoubleParser implements ValueParser { diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformExpressionOperatorsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformExpressionOperatorsProcessor.java new file mode 100644 index 0000000000..67e4e331d4 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformExpressionOperatorsProcessor.java @@ -0,0 +1,354 @@ +/* + * 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.inlong.sdk.transform.process; + +import org.apache.inlong.sdk.transform.decode.SourceDecoderFactory; +import org.apache.inlong.sdk.transform.encode.SinkEncoderFactory; +import org.apache.inlong.sdk.transform.pojo.CsvSourceInfo; +import org.apache.inlong.sdk.transform.pojo.FieldInfo; +import org.apache.inlong.sdk.transform.pojo.KvSinkInfo; +import org.apache.inlong.sdk.transform.pojo.TransformConfig; +import org.apache.inlong.sdk.transform.process.converter.DoubleConverter; +import org.apache.inlong.sdk.transform.process.converter.LongConverter; +import org.apache.inlong.sdk.transform.process.converter.TypeConverter; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +/** + * TestArithmeticFunctionsTransformProcessor + * description: test the arithmetic functions in transform processor + */ +public class TestTransformExpressionOperatorsProcessor { + + private static final List srcFields = new ArrayList<>(); + private static final List dstFields = new ArrayList<>(); + private static final CsvSourceInfo csvSource; + private static final KvSinkInfo kvSink; + + static { + srcFields.add(new FieldInfo("numeric1", new DoubleConverter())); + srcFields.add(new FieldInfo("string2", TypeConverter.DefaultTypeConverter())); + srcFields.add(new FieldInfo("numeric3", new DoubleConverter())); + srcFields.add(new FieldInfo("numeric4", new LongConverter())); + + FieldInfo field = new FieldInfo(); + field.setName("result"); + dstFields.add(field); + csvSource = new CsvSourceInfo("UTF-8", '|', '\\', srcFields); + kvSink = new KvSinkInfo("UTF-8", dstFields); + } + + @Test + public void testEqualsToOperator() throws Exception { + String transformSql = "select if(string2 = 4,1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|4a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|4a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=0"); + // case2: "3.14159265358979323846|4|4|8" + List output2 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=1"); + + transformSql = "select if(numeric3 = 4,1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=1"); + // case4: "3.14159265358979323846|4|4.2|8" + List output4 = processor.transform("3.14159265358979323846|4|4.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=0"); + } + + @Test + public void testNotEqualsToOperator() throws Exception { + String transformSql = "select if(string2 != 4,1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|4a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|4a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=1"); + // case2: "3.14159265358979323846|4|4|8" + List output2 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=0"); + + transformSql = "select if(numeric3 != 4,1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=0"); + // case4: "3.14159265358979323846|4|4.2|8" + List output4 = processor.transform("3.14159265358979323846|4|4.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=1"); + } + + @Test + public void testGreaterThanEqualsOperator() throws Exception { + String transformSql = "select if(string2 >= 4,1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=0"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=1"); + + transformSql = "select if(numeric3 >= 4,1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=1"); + // case4: "3.14159265358979323846|4|3.2|8" + List output4 = processor.transform("3.14159265358979323846|4|3.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=0"); + } + + @Test + public void testGreaterThanOperator() throws Exception { + String transformSql = "select if(string2 > 4.1,1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=0"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=1"); + + transformSql = "select if(numeric3 > 4.1,1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=0"); + // case4: "3.14159265358979323846|4|4.2|8" + List output4 = processor.transform("3.14159265358979323846|4|4.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=1"); + } + + @Test + public void testMinorThanEqualsOperator() throws Exception { + String transformSql = "select if(string2 <= 4,1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=1"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=0"); + + transformSql = "select if(numeric3 <= 4,1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=1"); + // case4: "3.14159265358979323846|4|4.2|8" + List output4 = processor.transform("3.14159265358979323846|4|4.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=0"); + } + + @Test + public void testMinorThanOperator() throws Exception { + String transformSql = "select if(string2 < 4.1,1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=1"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=0"); + + transformSql = "select if(numeric3 < 4,1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=0"); + // case4: "3.14159265358979323846|4|3.2|8" + List output4 = processor.transform("3.14159265358979323846|4|3.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=1"); + } + + @Test + public void testNotOperator() throws Exception { + String transformSql = "select if(!(string2 < 4),1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=0"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=1"); + + transformSql = "select if(!(numeric3 < 3.9),1,0) from source"; + config = new TransformConfig(transformSql); + // case3: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output3 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=1"); + // case4: "3.14159265358979323846|4|3.2|8" + List output4 = processor.transform("3.14159265358979323846|4|3.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=0"); + } + + @Test + public void testOrOperator() throws Exception { + String transformSql = "select if((string2 < 4) or (numeric4 > 5),1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|8" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=1"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=1"); + // case3: "3.14159265358979323846|5|4|4" + List output3 = processor.transform("3.14159265358979323846|5|4|4"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=0"); + + transformSql = "select if((numeric3 < 4) or (numeric4 > 5),1,0) from source"; + config = new TransformConfig(transformSql); + // case4: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output4 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=1"); + // case5: "3.14159265358979323846|4|3.2|8" + List output5 = processor.transform("3.14159265358979323846|4|3.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output5.get(0), "result=1"); + // case6: "3.14159265358979323846|4|4.2|5" + List output6 = processor.transform("3.14159265358979323846|4|4.2|5"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output6.get(0), "result=0"); + } + + @Test + public void testAndOperator() throws Exception { + String transformSql = "select if((string2 < 4) and (numeric4 > 5),1,0) from source"; + TransformConfig config = new TransformConfig(transformSql); + // case1: "3.14159265358979323846|3a|4|4" + TransformProcessor processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor.transform("3.14159265358979323846|3a|4|4"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=0"); + // case2: "3.14159265358979323846|5|4|8" + List output2 = processor.transform("3.14159265358979323846|5|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output2.get(0), "result=0"); + // case3: "3.14159265358979323846|3|4|8" + List output3 = processor.transform("3.14159265358979323846|3|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output3.get(0), "result=1"); + + transformSql = "select if((numeric3 < 4) and (numeric4 > 5),1,0) from source"; + config = new TransformConfig(transformSql); + // case4: "3.14159265358979323846|4|4|8" + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output4 = processor.transform("3.14159265358979323846|4|4|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output4.get(0), "result=0"); + // case5: "3.14159265358979323846|4|3.2|4" + List output5 = processor.transform("3.14159265358979323846|4|3.2|4"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output5.get(0), "result=0"); + // case6: "3.14159265358979323846|4|3.2|8" + List output6 = processor.transform("3.14159265358979323846|4|3.2|8"); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output6.get(0), "result=1"); + } +} From 0a394f49ea026b2536ef0b2a90e3a9183e4a51de Mon Sep 17 00:00:00 2001 From: emptyOVO <118812562+emptyOVO@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:25:35 +0800 Subject: [PATCH 24/30] [INLONG-10979][SDK] Transform support PI() function (#10983) --- .../process/function/PiFunction.java | 40 +++++++++++++++++++ ...TransformArithmeticFunctionsProcessor.java | 13 ++++++ 2 files changed, 53 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/PiFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/PiFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/PiFunction.java new file mode 100644 index 0000000000..722955ebf5 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/PiFunction.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Function; +/** + * PiFunction + * returns the mathematical constant PI + */ +@TransformFunction(names = {"pi"}) +public class PiFunction implements ValueParser { + + public PiFunction(Function expr) { + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + return String.valueOf(Math.PI); + } + +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java index c36835e579..4007a9a876 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java @@ -728,4 +728,17 @@ public void testHexFunction() throws Exception { Assert.assertEquals(output5.get(0), "result=616263"); } + @Test + public void testPiFunction() throws Exception { + String transformSql1 = "select pi() from source"; + TransformConfig config1 = new TransformConfig(transformSql1); + TransformProcessor processor1 = TransformProcessor + .create(config1, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + // case: pi() + List output1 = processor1.transform("1007|4|6|8", new HashMap<>()); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "result=3.141592653589793"); + } + } From 3e7f81d41c0e1ab8294d93b374e54515d0b7936b Mon Sep 17 00:00:00 2001 From: Xincheng Huang <60057611+ying-hua@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:25:52 +0800 Subject: [PATCH 25/30] [INLONG-10971][SDK] Transform support INSERT function (#10972) --- .../process/function/InsertFunction.java | 101 ++++++++++++++++++ ...TestTransformStringFunctionsProcessor.java | 34 ++++++ 2 files changed, 135 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/InsertFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/InsertFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/InsertFunction.java new file mode 100644 index 0000000000..3473bc9d79 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/InsertFunction.java @@ -0,0 +1,101 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; + +import java.util.List; +/** + * InsertFunction + * + * Description: + * Returns a string where a specified substring is replaced by another string, starting at a given position and for a specified length. + * If the position is out of the string's bounds, the original string is returned. + * If the length exceeds the remaining length of the string from the given position, the replacement continues to the end of the string. + * If any argument is null, the function returns null. + * + * Arguments: + * - str: The original string. + * - pos: The position to start the replacement (1-based index). + * - len: The number of characters to replace. + * - newstr: The string to insert. + * + * Examples: + * - INSERT('12345678', 3, 4, 'word') = '12word78' + * - INSERT('12345678', -1, 4, 'word') = '12345678' + * - INSERT('12345678', 3, 100, 'word') = '12word' + */ +@TransformFunction(names = {"insert"}) +public class InsertFunction implements ValueParser { + + private ValueParser strParser; + private ValueParser posParser; + private ValueParser lenParser; + private ValueParser newStrParser; + + public InsertFunction(Function expr) { + List expressions = expr.getParameters().getExpressions(); + strParser = OperatorTools.buildParser(expressions.get(0)); + posParser = OperatorTools.buildParser(expressions.get(1)); + lenParser = OperatorTools.buildParser(expressions.get(2)); + newStrParser = OperatorTools.buildParser(expressions.get(3)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object strObject = strParser.parse(sourceData, rowIndex, context); + Object posObject = posParser.parse(sourceData, rowIndex, context); + Object lenObject = lenParser.parse(sourceData, rowIndex, context); + Object newStrObject = newStrParser.parse(sourceData, rowIndex, context); + + if (strObject == null || posObject == null || lenObject == null || newStrObject == null) { + return null; + } + + String str = OperatorTools.parseString(strObject); + int pos = OperatorTools.parseBigDecimal(posObject).intValue(); + int len = OperatorTools.parseBigDecimal(lenObject).intValue(); + String newStr = OperatorTools.parseString(newStrObject); + + if (str == null || newStr == null) { + return null; + } + + if (pos < 1 || pos > str.length()) { + return str; + } + + int startIndex = pos - 1; + int endIndex = Math.min(startIndex + len, str.length()); + + StringBuilder result = new StringBuilder(); + result.append(str, 0, startIndex); + result.append(newStr); + if (endIndex < str.length()) { + result.append(str, endIndex, str.length()); + } + + return result.toString(); + } +} \ No newline at end of file diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java index 1451194607..a3099d09f2 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformStringFunctionsProcessor.java @@ -690,4 +690,38 @@ public void testTranslateFunction() throws Exception { Assert.assertEquals(output3.get(0), "result=Apache Inlong"); } + @Test + public void testInsertFunction() throws Exception { + String transformSql1 = "select insert(string1, numeric1, numeric2, string2) from source"; + TransformConfig config1 = new TransformConfig(transformSql1); + TransformProcessor processor1 = TransformProcessor + .create(config1, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + + // case1: insert('12345678', 3, 4, 'word') -> '12word78' + List output1 = processor1.transform("12345678|word|cloud|3|4|0", new HashMap<>()); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals("result=12word78", output1.get(0)); + + // case2: insert('12345678', -1, 4, 'word') -> '12345678' + List output2 = processor1.transform("12345678|word|cloud|-1|4|0", new HashMap<>()); + Assert.assertEquals(1, output2.size()); + Assert.assertEquals("result=12345678", output2.get(0)); + + // case3: insert('12345678', 3, 100, 'word') -> '12word' + List output3 = processor1.transform("12345678|word|cloud|3|100|0", new HashMap<>()); + Assert.assertEquals(1, output3.size()); + Assert.assertEquals("result=12word", output3.get(0)); + + // case4: insert('', 3, 4, 'word') -> '' + List output4 = processor1.transform("|word|cloud|3|4|0", new HashMap<>()); + Assert.assertEquals(1, output4.size()); + Assert.assertEquals("result=", output4.get(0)); + + // case5: insert('12345678', 3, 4, '') -> '1278' + List output5 = processor1.transform("12345678||cloud|3|4|0", new HashMap<>()); + Assert.assertEquals(1, output5.size()); + Assert.assertEquals("result=1278", output5.get(0)); + } + } From cb4b61b5a889cbb035b51ad1dd48f6a02790573f Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:26:19 +0800 Subject: [PATCH 26/30] [INLONG-10965][SDK] Transform support Bitwise operation (#10970) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../process/parser/BitwiseAndParser.java | 61 ++++++ .../parser/BitwiseLeftShiftParser.java | 67 +++++++ .../process/parser/BitwiseOrParser.java | 62 +++++++ .../parser/BitwiseRightShiftParser.java | 66 +++++++ .../process/parser/BitwiseXorParser.java | 62 +++++++ .../transform/process/parser/SignParser.java | 26 ++- ...TransformArithmeticFunctionsProcessor.java | 175 ++++++++++++++++++ 7 files changed, 513 insertions(+), 6 deletions(-) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseAndParser.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseLeftShiftParser.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseOrParser.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseRightShiftParser.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseXorParser.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseAndParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseAndParser.java new file mode 100644 index 0000000000..b8920434ef --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseAndParser.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sdk.transform.process.parser; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; + +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.operators.arithmetic.BitwiseAnd; + +import java.math.BigInteger; + +/** + * BitwiseAndParser + */ +@Slf4j +@TransformParser(values = BitwiseAnd.class) +public class BitwiseAndParser implements ValueParser { + + private final ValueParser left; + + private final ValueParser right; + + public BitwiseAndParser(BitwiseAnd expr) { + this.left = OperatorTools.buildParser(expr.getLeftExpression()); + this.right = OperatorTools.buildParser(expr.getRightExpression()); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + try { + Object leftObj = this.left.parse(sourceData, rowIndex, context); + Object rightObj = this.right.parse(sourceData, rowIndex, context); + if (leftObj == null || rightObj == null) { + return null; + } + BigInteger leftValue = OperatorTools.parseBigDecimal(leftObj).toBigInteger(); + BigInteger rightValue = OperatorTools.parseBigDecimal(rightObj).toBigInteger(); + return Long.toUnsignedString(leftValue.and(rightValue).longValue()); + } catch (Exception e) { + log.error("Value parsing failed", e); + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseLeftShiftParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseLeftShiftParser.java new file mode 100644 index 0000000000..c68bc89e54 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseLeftShiftParser.java @@ -0,0 +1,67 @@ +/* + * 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.inlong.sdk.transform.process.parser; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; + +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.operators.arithmetic.BitwiseLeftShift; + +import java.math.BigInteger; + +/** + * BitwiseLeftShiftParser + * + */ +@Slf4j +@TransformParser(values = BitwiseLeftShift.class) +public class BitwiseLeftShiftParser implements ValueParser { + + private final ValueParser left; + + private final ValueParser right; + + public BitwiseLeftShiftParser(BitwiseLeftShift expr) { + this.left = OperatorTools.buildParser(expr.getLeftExpression()); + this.right = OperatorTools.buildParser(expr.getRightExpression()); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + try { + Object leftObj = this.left.parse(sourceData, rowIndex, context); + Object rightObj = this.right.parse(sourceData, rowIndex, context); + if (leftObj == null || rightObj == null) { + return null; + } + BigInteger leftValue = OperatorTools.parseBigDecimal(leftObj).toBigInteger(); + String unsignedRight = Long.toUnsignedString(OperatorTools.parseBigDecimal(rightObj).longValue()); + int cmp = new BigInteger(unsignedRight).compareTo(new BigInteger("65")); + if (cmp >= 0) { + return Long.toUnsignedString(leftValue.shiftLeft(65).longValue()); + } else { + return Long.toUnsignedString(leftValue.shiftLeft(Integer.parseInt(unsignedRight)).longValue()); + } + } catch (Exception e) { + log.error("Value parsing failed", e); + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseOrParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseOrParser.java new file mode 100644 index 0000000000..2cbf5e9d09 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseOrParser.java @@ -0,0 +1,62 @@ +/* + * 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.inlong.sdk.transform.process.parser; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; + +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.operators.arithmetic.BitwiseOr; + +import java.math.BigInteger; + +/** + * BitwiseOrParser + * + */ +@Slf4j +@TransformParser(values = BitwiseOr.class) +public class BitwiseOrParser implements ValueParser { + + private final ValueParser left; + + private final ValueParser right; + + public BitwiseOrParser(BitwiseOr expr) { + this.left = OperatorTools.buildParser(expr.getLeftExpression()); + this.right = OperatorTools.buildParser(expr.getRightExpression()); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + try { + Object leftObj = this.left.parse(sourceData, rowIndex, context); + Object rightObj = this.right.parse(sourceData, rowIndex, context); + if (leftObj == null || rightObj == null) { + return null; + } + BigInteger leftValue = OperatorTools.parseBigDecimal(leftObj).toBigInteger(); + BigInteger rightValue = OperatorTools.parseBigDecimal(rightObj).toBigInteger(); + return Long.toUnsignedString(leftValue.or(rightValue).longValue()); + } catch (Exception e) { + log.error("Value parsing failed", e); + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseRightShiftParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseRightShiftParser.java new file mode 100644 index 0000000000..7b6085725b --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseRightShiftParser.java @@ -0,0 +1,66 @@ +/* + * 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.inlong.sdk.transform.process.parser; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; + +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.operators.arithmetic.BitwiseRightShift; + +import java.math.BigInteger; + +/** + * BitwiseRightShiftParser + */ +@Slf4j +@TransformParser(values = BitwiseRightShift.class) +public class BitwiseRightShiftParser implements ValueParser { + + private final ValueParser left; + + private final ValueParser right; + + public BitwiseRightShiftParser(BitwiseRightShift expr) { + this.left = OperatorTools.buildParser(expr.getLeftExpression()); + this.right = OperatorTools.buildParser(expr.getRightExpression()); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + try { + Object leftObj = this.left.parse(sourceData, rowIndex, context); + Object rightObj = this.right.parse(sourceData, rowIndex, context); + if (leftObj == null || rightObj == null) { + return null; + } + BigInteger leftValue = OperatorTools.parseBigDecimal(leftObj).toBigInteger(); + String unsignedRight = Long.toUnsignedString(OperatorTools.parseBigDecimal(rightObj).longValue()); + int cmp = new BigInteger(unsignedRight).compareTo(new BigInteger("65")); + if (cmp >= 0) { + return Long.toUnsignedString(leftValue.shiftRight(65).longValue()); + } else { + return Long.toUnsignedString(leftValue.shiftRight(Integer.parseInt(unsignedRight)).longValue()); + } + } catch (Exception e) { + log.error("Value parsing failed", e); + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseXorParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseXorParser.java new file mode 100644 index 0000000000..3277bd91ed --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/BitwiseXorParser.java @@ -0,0 +1,62 @@ +/* + * 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.inlong.sdk.transform.process.parser; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; + +import lombok.extern.slf4j.Slf4j; +import net.sf.jsqlparser.expression.operators.arithmetic.BitwiseXor; + +import java.math.BigInteger; + +/** + * BitwiseXorParser + * + */ +@Slf4j +@TransformParser(values = BitwiseXor.class) +public class BitwiseXorParser implements ValueParser { + + private final ValueParser left; + + private final ValueParser right; + + public BitwiseXorParser(BitwiseXor expr) { + this.left = OperatorTools.buildParser(expr.getLeftExpression()); + this.right = OperatorTools.buildParser(expr.getRightExpression()); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + try { + Object leftObj = this.left.parse(sourceData, rowIndex, context); + Object rightObj = this.right.parse(sourceData, rowIndex, context); + if (leftObj == null || rightObj == null) { + return null; + } + BigInteger leftValue = OperatorTools.parseBigDecimal(leftObj).toBigInteger(); + BigInteger rightValue = OperatorTools.parseBigDecimal(rightObj).toBigInteger(); + return Long.toUnsignedString(leftValue.xor(rightValue).longValue()); + } catch (Exception e) { + log.error("Value parsing failed", e); + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java index ff97aadfdb..7a744f9015 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/parser/SignParser.java @@ -21,29 +21,43 @@ import org.apache.inlong.sdk.transform.process.Context; import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import lombok.extern.slf4j.Slf4j; import net.sf.jsqlparser.expression.SignedExpression; import java.math.BigDecimal; /** * SignParser - * */ +@Slf4j @TransformParser(values = SignedExpression.class) public class SignParser implements ValueParser { - private final Integer sign; + private final char sign; private final ValueParser number; public SignParser(SignedExpression expr) { - sign = expr.getSign() == '-' ? -1 : 1; + sign = expr.getSign(); number = OperatorTools.buildParser(expr.getExpression()); } @Override public Object parse(SourceData sourceData, int rowIndex, Context context) { - Object numberObject = number.parse(sourceData, rowIndex, context); - BigDecimal numberValue = OperatorTools.parseBigDecimal(numberObject); - return numberValue.multiply(new BigDecimal(sign)); + try { + Object numberObject = number.parse(sourceData, rowIndex, context); + if (numberObject == null) { + return null; + } + BigDecimal numberValue = OperatorTools.parseBigDecimal(numberObject); + switch (sign) { + case '-': + return numberValue.multiply(new BigDecimal(-1)); + case '~': + return Long.toUnsignedString(numberValue.toBigInteger().not().longValue()); + } + } catch (Exception e) { + log.error("Value parsing failed", e); + } + return null; } } diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java index 4007a9a876..dcdb2d61d4 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java @@ -320,6 +320,181 @@ public void testMd5Function() throws Exception { Assert.assertEquals("result=null", output4.get(0)); } + @Test + public void testBitwiseInversionOperator() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: ~-4 + transformSql = "select ~numeric1 from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "-4|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=3", output.get(0)); + + // case2: ~4 + data = "4|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551611", output.get(0)); + + // case3: ~0 + data = "0|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551615", output.get(0)); + + // case4: ~~-4 + transformSql = "select ~(~numeric1) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "-4|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551612", output.get(0)); + + } + @Test + public void testBitwiseAndOperator() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: 18446744073709551615 & -1 + transformSql = "select numeric1 & numeric2 from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "18446744073709551615|-1|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551615", output.get(0)); + + // case2: 18446744073709551615 & 0 + data = "18446744073709551615|0|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=0", output.get(0)); + } + @Test + public void testBitwiseOrOperator() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: 18446744073709551615 | -1 + transformSql = "select numeric1 | numeric2 from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "18446744073709551615|-1|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551615", output.get(0)); + + // case2: 4 | 3 + data = "4|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=7", output.get(0)); + } + @Test + public void testBitwiseRightShiftOperator() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: 4 >> -1 + transformSql = "select numeric1 >> numeric2 from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "4|-1|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=0", output.get(0)); + + // case2: 9223372036854775808 >> 2 + data = "9223372036854775808|2|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=2305843009213693952", output.get(0)); + + // case3: 9223372036854775808 >> 9223372036854775808 + data = "9223372036854775808|9223372036854775808|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=0", output.get(0)); + } + @Test + public void testBitwiseLeftShiftOperator() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: 9223372036854775807 << 1 + transformSql = "select numeric1 << numeric2 from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "9223372036854775807|1|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551614", output.get(0)); + + // case2: 18446744073709551615 << 18446744073709551615 + data = "18446744073709551615|18446744073709551615|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=0", output.get(0)); + + // case3: 9223372036854775807 << -1 + data = "9223372036854775807|-1|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=0", output.get(0)); + } + @Test + public void testBitwiseXorOperator() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: 4 ^ 3 + transformSql = "select numeric1 ^ numeric2 from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "4|3|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=7", output.get(0)); + + // case2: 4 ^ -1 + data = "4|-1|3"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=18446744073709551611", output.get(0)); + } + @Test public void testRoundFunction() throws Exception { String transformSql = "select round(numeric1) from source"; From f1285ed4b1c9ec1ff16942c6324f6fad46c6898c Mon Sep 17 00:00:00 2001 From: emptyOVO <118812562+emptyOVO@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:33:10 +0800 Subject: [PATCH 27/30] [INLONG-10968][SDK] Support Inlong Transform operator annotation (#10969) --- .../process/operator/AndOperator.java | 1 + .../process/operator/EqualsToOperator.java | 1 + .../operator/GreaterThanEqualsOperator.java | 1 + .../process/operator/GreaterThanOperator.java | 1 + .../operator/MinorThanEqualsOperator.java | 1 + .../process/operator/MinorThanOperator.java | 1 + .../process/operator/NotEqualsToOperator.java | 1 + .../process/operator/NotOperator.java | 1 + .../process/operator/OperatorTools.java | 89 ++++++++++++------- .../process/operator/OrOperator.java | 1 + .../process/operator/ParenthesisOperator.java | 1 + .../process/operator/TransformOperator.java | 31 +++++++ 12 files changed, 100 insertions(+), 30 deletions(-) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/TransformOperator.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/AndOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/AndOperator.java index a9dcd42606..f438d4295c 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/AndOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/AndOperator.java @@ -26,6 +26,7 @@ * AndOperator * */ +@TransformOperator(values = AndExpression.class) public class AndOperator implements ExpressionOperator { private final ExpressionOperator left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/EqualsToOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/EqualsToOperator.java index 709537e8a0..13010c854b 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/EqualsToOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/EqualsToOperator.java @@ -27,6 +27,7 @@ * EqualsToOperator * */ +@TransformOperator(values = EqualsTo.class) public class EqualsToOperator implements ExpressionOperator { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanEqualsOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanEqualsOperator.java index 3a53968e10..e703afdbda 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanEqualsOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanEqualsOperator.java @@ -27,6 +27,7 @@ * GreaterThanEqualsOperator * */ +@TransformOperator(values = GreaterThanEquals.class) public class GreaterThanEqualsOperator implements ExpressionOperator { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanOperator.java index a1cd8c2ea2..ba73ef4c4d 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/GreaterThanOperator.java @@ -27,6 +27,7 @@ * GreaterThanOperator * */ +@TransformOperator(values = GreaterThan.class) public class GreaterThanOperator implements ExpressionOperator { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanEqualsOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanEqualsOperator.java index 4248cf1d36..e8104e2cc3 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanEqualsOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanEqualsOperator.java @@ -27,6 +27,7 @@ * MinorThanEqualsOperator * */ +@TransformOperator(values = MinorThanEquals.class) public class MinorThanEqualsOperator implements ExpressionOperator { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanOperator.java index 21ecc0400a..ecae167c5c 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/MinorThanOperator.java @@ -27,6 +27,7 @@ * MinorThanOperator * */ +@TransformOperator(values = MinorThan.class) public class MinorThanOperator implements ExpressionOperator { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotEqualsToOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotEqualsToOperator.java index 98bf102b4f..d3fec5c8c2 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotEqualsToOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotEqualsToOperator.java @@ -27,6 +27,7 @@ * NotEqualsToOperator * */ +@TransformOperator(values = NotEqualsTo.class) public class NotEqualsToOperator implements ExpressionOperator { private final ValueParser left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotOperator.java index d8b9ff07e0..306177ffba 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/NotOperator.java @@ -26,6 +26,7 @@ * NotOperator * */ +@TransformOperator(values = NotExpression.class) public class NotOperator implements ExpressionOperator { private final ExpressionOperator node; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java index bb35bb4490..9982b37418 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OperatorTools.java @@ -22,54 +22,83 @@ import org.apache.inlong.sdk.transform.process.parser.ParserTools; import org.apache.inlong.sdk.transform.process.parser.ValueParser; +import com.google.common.collect.Maps; +import lombok.extern.slf4j.Slf4j; import net.sf.jsqlparser.expression.Expression; import net.sf.jsqlparser.expression.Function; -import net.sf.jsqlparser.expression.NotExpression; -import net.sf.jsqlparser.expression.Parenthesis; -import net.sf.jsqlparser.expression.operators.conditional.AndExpression; -import net.sf.jsqlparser.expression.operators.conditional.OrExpression; -import net.sf.jsqlparser.expression.operators.relational.EqualsTo; -import net.sf.jsqlparser.expression.operators.relational.GreaterThan; -import net.sf.jsqlparser.expression.operators.relational.GreaterThanEquals; -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 org.apache.commons.lang.ObjectUtils; +import org.reflections.Reflections; +import org.reflections.scanners.Scanners; +import java.lang.reflect.Constructor; import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; +import java.util.Map; +import java.util.Set; /** * OperatorTools * */ +@Slf4j public class OperatorTools { + private static final String OPERATOR_PATH = "org.apache.inlong.sdk.transform.process.operator"; + + private final static Map, Class> operatorMap = Maps.newConcurrentMap(); + public static final String ROOT_KEY = "$root"; public static final String CHILD_KEY = "$child"; + + static { + init(); + } + + private static void init() { + Reflections reflections = new Reflections(OPERATOR_PATH, Scanners.TypesAnnotated); + Set> clazzSet = reflections.getTypesAnnotatedWith(TransformOperator.class); + for (Class clazz : clazzSet) { + if (ExpressionOperator.class.isAssignableFrom(clazz)) { + TransformOperator annotation = clazz.getAnnotation(TransformOperator.class); + if (annotation == null) { + continue; + } + Class[] values = annotation.values(); + for (Class value : values) { + operatorMap.compute(value, (key, former) -> { + if (former != null) { + log.warn("find a conflict for parser class [{}], the former one is [{}], new one is [{}]", + key, former.getName(), clazz.getName()); + } + return clazz; + }); + } + } + } + } + + public static ExpressionOperator getTransformOperator(Expression expr) { + Class clazz = operatorMap.get(expr.getClass()); + if (clazz == null) { + return null; + } + try { + Constructor constructor = clazz.getDeclaredConstructor(expr.getClass()); + return (ExpressionOperator) constructor.newInstance(expr); + } catch (NoSuchMethodException e) { + log.error("transform operator {} needs one constructor that accept one params whose type is {}", + clazz.getName(), expr.getClass().getName(), e); + throw new RuntimeException(e); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public static ExpressionOperator buildOperator(Expression expr) { - if (expr instanceof AndExpression) { - return new AndOperator((AndExpression) expr); - } else if (expr instanceof OrExpression) { - return new OrOperator((OrExpression) expr); - } else if (expr instanceof Parenthesis) { - return new ParenthesisOperator((Parenthesis) expr); - } else if (expr instanceof NotExpression) { - return new NotOperator((NotExpression) expr); - } else if (expr instanceof EqualsTo) { - return new EqualsToOperator((EqualsTo) expr); - } else if (expr instanceof NotEqualsTo) { - return new NotEqualsToOperator((NotEqualsTo) expr); - } else if (expr instanceof GreaterThan) { - return new GreaterThanOperator((GreaterThan) expr); - } else if (expr instanceof GreaterThanEquals) { - return new GreaterThanEqualsOperator((GreaterThanEquals) expr); - } else if (expr instanceof MinorThan) { - return new MinorThanOperator((MinorThan) expr); - } else if (expr instanceof MinorThanEquals) { - return new MinorThanEqualsOperator((MinorThanEquals) expr); + if (expr != null) { + return getTransformOperator(expr); } return null; } diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OrOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OrOperator.java index b5de7f279e..9efccba3f2 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OrOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/OrOperator.java @@ -26,6 +26,7 @@ * OrOperator * */ +@TransformOperator(values = OrExpression.class) public class OrOperator implements ExpressionOperator { private final ExpressionOperator left; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/ParenthesisOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/ParenthesisOperator.java index 0ca1334fce..de71c84b4d 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/ParenthesisOperator.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/ParenthesisOperator.java @@ -26,6 +26,7 @@ * ParenthesisOperator * */ +@TransformOperator(values = Parenthesis.class) public class ParenthesisOperator implements ExpressionOperator { private final ExpressionOperator node; diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/TransformOperator.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/TransformOperator.java new file mode 100644 index 0000000000..ee9c676665 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/operator/TransformOperator.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.inlong.sdk.transform.process.operator; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.TYPE; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target(TYPE) +public @interface TransformOperator { + + Class[] values(); +} From e394d740769730f6960e7022e0bc8f4dee1e2e9b Mon Sep 17 00:00:00 2001 From: Zkplo <87751516+Zkplo@users.noreply.github.com> Date: Tue, 3 Sep 2024 10:39:27 +0800 Subject: [PATCH 28/30] [INLONG-10956][SDK] Transform SQL supports SHA encryption algorithm (#10996) Co-authored-by: ZKpLo <14148880+zkplo@user.noreply.gitee.com> --- .../process/function/Sha2Function.java | 75 ++++++++++++++++++ .../process/function/ShaFunction.java | 54 +++++++++++++ ...TransformArithmeticFunctionsProcessor.java | 78 +++++++++++++++++++ 3 files changed, 207 insertions(+) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/Sha2Function.java create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/ShaFunction.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/Sha2Function.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/Sha2Function.java new file mode 100644 index 0000000000..ad120712bf --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/Sha2Function.java @@ -0,0 +1,75 @@ +/* + * 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.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Expression; +import net.sf.jsqlparser.expression.Function; +import org.apache.commons.codec.digest.DigestUtils; + +import java.nio.charset.StandardCharsets; +import java.util.List; + +import static org.apache.commons.codec.digest.MessageDigestAlgorithms.SHA_224; + +/** + * Sha2Function + * description: SHA2(str, hash_length): Calculates the SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384, and SHA-512) + * return NULL If either argument is NULL or the hash length(224 256 384 512) is not one of the permitted values + * return a hash value containing the desired number of bits. + */ +@TransformFunction(names = {"sha2"}) +public class Sha2Function implements ValueParser { + + private final ValueParser msgParser; + private final ValueParser lenParser; + + public Sha2Function(Function expr) { + List expressions = expr.getParameters().getExpressions(); + msgParser = OperatorTools.buildParser(expressions.get(0)); + lenParser = OperatorTools.buildParser(expressions.get(1)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object msgObj = msgParser.parse(sourceData, rowIndex, context); + Object lenObj = lenParser.parse(sourceData, rowIndex, context); + if (msgObj == null || lenObj == null) { + return null; + } + String msg = msgObj.toString(); + int len = Integer.parseInt(lenObj.toString()); + switch (len) { + case 0: + case 256: + return DigestUtils.sha256Hex(msg.getBytes(StandardCharsets.UTF_8)); + case 224: + return new DigestUtils(SHA_224).digestAsHex(msg.getBytes(StandardCharsets.UTF_8)); + case 384: + return DigestUtils.sha384Hex(msg.getBytes(StandardCharsets.UTF_8)); + case 512: + return DigestUtils.sha512Hex(msg.getBytes(StandardCharsets.UTF_8)); + default: + return null; + } + } +} diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/ShaFunction.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/ShaFunction.java new file mode 100644 index 0000000000..7da5e6c3a9 --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/function/ShaFunction.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sdk.transform.process.function; + +import org.apache.inlong.sdk.transform.decode.SourceData; +import org.apache.inlong.sdk.transform.process.Context; +import org.apache.inlong.sdk.transform.process.operator.OperatorTools; +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import net.sf.jsqlparser.expression.Function; +import org.apache.commons.codec.digest.DigestUtils; + +import java.nio.charset.StandardCharsets; + +/** + * ShaFunction + * description: sha(string): Compute the SHA-1 160 bit checksum of a string. + * return NULL if the parameter is NULL + * return a string of 40 hexadecimal digits. + */ +@TransformFunction(names = {"sha"}) +public class ShaFunction implements ValueParser { + + private final ValueParser msgParser; + + public ShaFunction(Function expr) { + msgParser = OperatorTools.buildParser(expr.getParameters().getExpressions().get(0)); + } + + @Override + public Object parse(SourceData sourceData, int rowIndex, Context context) { + Object msgObj = msgParser.parse(sourceData, rowIndex, context); + if (msgObj == null) { + return null; + } + String msg = msgObj.toString(); + return DigestUtils.sha1Hex(msg.getBytes(StandardCharsets.UTF_8)); + } +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java index dcdb2d61d4..738ef2e0f6 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformArithmeticFunctionsProcessor.java @@ -287,6 +287,84 @@ public void testIfNullFunction() throws Exception { Assert.assertEquals("result=null", output.get(0)); } + @Test + public void testShaFunction() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: sha("") + transformSql = "select sha(numeric1) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=da39a3ee5e6b4b0d3255bfef95601890afd80709", output.get(0)); + + // case2: sha("5") + data = "5|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=ac3478d69a3c81fa62e60f5c3696165a4e5e6ac4", output.get(0)); + + // case3: sha(null) + transformSql = "select sha(xxd) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "3|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + + @Test + public void testSha2Function() throws Exception { + String transformSql = null, data = null; + TransformConfig config = null; + TransformProcessor processor = null; + List output = null; + + // case1: sha2("",3) + transformSql = "select sha2(numeric1,numeric2) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "|3|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + + // case2: sha2("5",224) + data = "5|224|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=b51d18b551043c1f145f22dbde6f8531faeaf68c54ed9dd79ce24d17", output.get(0)); + + // case3: sha2("5",0) + data = "5|0|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=ef2d127de37b942baad06145e54b0c619a1f22327b2ebbcfbec78f5564afe39d", output.get(0)); + + // case4: sha2(null,224) + transformSql = "select sha2(xxd,224) from source"; + config = new TransformConfig(transformSql); + processor = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + data = "3|224|3|5"; + output = processor.transform(data, new HashMap<>()); + Assert.assertEquals(1, output.size()); + Assert.assertEquals("result=null", output.get(0)); + } + @Test public void testMd5Function() throws Exception { String transformSql = "select md5(numeric1) from source"; From e1884106681a4062555d413f6f2cdc24ef741229 Mon Sep 17 00:00:00 2001 From: fuweng11 <76141879+fuweng11@users.noreply.github.com> Date: Tue, 3 Sep 2024 14:07:56 +0800 Subject: [PATCH 29/30] [INLONG-10997][Manager] Incorrect setting of transformSQL in dataflowconfig (#10998) --- .../manager/service/resource/sort/DefaultSortConfigOperator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/sort/DefaultSortConfigOperator.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/sort/DefaultSortConfigOperator.java index 116a8f0ea2..17ca4b9bad 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/sort/DefaultSortConfigOperator.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/sort/DefaultSortConfigOperator.java @@ -175,6 +175,7 @@ private DataFlowConfig getDataFlowConfig(InlongGroupInfo groupInfo, InlongStream .dataflowId(String.valueOf(sink.getId())) .sourceConfig(getSourceConfig(groupInfo, streamInfo, sink)) .auditTag(String.valueOf(sink.getId())) + .transformSql(sink.getTransformSql()) .sinkConfig(getSinkConfig(groupInfo, streamInfo, sink)) .inlongGroupId(groupInfo.getInlongGroupId()) .inlongStreamId(streamInfo.getInlongStreamId()) From edf93bd547a09df6b1d7a6d57ea3d719d9f63f4f Mon Sep 17 00:00:00 2001 From: ChunLiang Lu Date: Tue, 3 Sep 2024 21:49:46 +0800 Subject: [PATCH 30/30] [INLONG-10999][SDK] Support to return raw data by star sign in transformer SQL (#11004) * [INLONG-10999][SDK] Support to return raw data by star sign in transformer SQL * add more UT Case * fix code format problems * fix pom.xml problem --- .../sdk/transform/encode/CsvSinkEncoder.java | 6 +- .../sdk/transform/encode/KvSinkEncoder.java | 6 +- .../sdk/transform/encode/SinkEncoder.java | 2 + .../transform/process/TransformProcessor.java | 34 ++++++--- .../transform/process/ValueParserNode.java | 34 +++++++++ .../process/TestTransformProcessor.java | 76 +++++++++++++++++++ 6 files changed, 146 insertions(+), 12 deletions(-) create mode 100644 inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/ValueParserNode.java diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/CsvSinkEncoder.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/CsvSinkEncoder.java index ce47a0072c..89f6f364a0 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/CsvSinkEncoder.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/CsvSinkEncoder.java @@ -66,7 +66,11 @@ public String encode(SinkData sinkData, Context context) { } else { for (String fieldName : sinkData.keyList()) { String fieldValue = sinkData.getField(fieldName); - EscapeUtils.escapeContent(builder, delimiter, escapeChar, fieldValue); + if (StringUtils.equals(fieldName, ALL_SOURCE_FIELD_SIGN)) { + builder.append(fieldValue); + } else { + EscapeUtils.escapeContent(builder, delimiter, escapeChar, fieldValue); + } builder.append(delimiter); } } diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/KvSinkEncoder.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/KvSinkEncoder.java index 7460ec95c2..2822374c41 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/KvSinkEncoder.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/KvSinkEncoder.java @@ -63,7 +63,11 @@ public String encode(SinkData sinkData, Context context) { if (fields == null || fields.size() == 0) { for (String fieldName : sinkData.keyList()) { String fieldValue = sinkData.getField(fieldName); - builder.append(fieldName).append(kvDelimiter).append(fieldValue).append(entryDelimiter); + if (StringUtils.equals(fieldName, ALL_SOURCE_FIELD_SIGN)) { + builder.append(fieldValue).append(entryDelimiter); + } else { + builder.append(fieldName).append(kvDelimiter).append(fieldValue).append(entryDelimiter); + } } } else { for (FieldInfo field : fields) { diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/SinkEncoder.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/SinkEncoder.java index 7f845a99d6..a63f970295 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/SinkEncoder.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/encode/SinkEncoder.java @@ -27,6 +27,8 @@ */ public interface SinkEncoder { + public static final String ALL_SOURCE_FIELD_SIGN = "*"; + Output encode(SinkData sinkData, Context context); List getFields(); diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/TransformProcessor.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/TransformProcessor.java index 9944268dda..acb7e62e07 100644 --- a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/TransformProcessor.java +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/TransformProcessor.java @@ -31,23 +31,23 @@ import com.google.common.collect.ImmutableMap; import net.sf.jsqlparser.JSQLParserException; import net.sf.jsqlparser.parser.CCJSqlParserManager; +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.SelectExpressionItem; import net.sf.jsqlparser.statement.select.SelectItem; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.StringReader; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; /** * TransformProcessor - * + * */ public class TransformProcessor { @@ -61,7 +61,9 @@ public class TransformProcessor { private PlainSelect transformSelect; private ExpressionOperator where; - private Map selectItemMap; + private List selectItems; + + private boolean includeAllSourceFields = false; public static TransformProcessor create( TransformConfig config, @@ -91,7 +93,7 @@ private void initTransformSql() throws JSQLParserException { this.transformSelect = (PlainSelect) select.getSelectBody(); this.where = OperatorTools.buildOperator(this.transformSelect.getWhere()); List items = this.transformSelect.getSelectItems(); - this.selectItemMap = new HashMap<>(items.size()); + this.selectItems = new ArrayList<>(items.size()); List fields = this.encoder.getFields(); for (int i = 0; i < items.size(); i++) { SelectItem item = items.get(i); @@ -108,8 +110,12 @@ private void initTransformSql() throws JSQLParserException { fieldName = exprItem.getAlias().getName(); } } - this.selectItemMap.put(fieldName, - OperatorTools.buildParser(exprItem.getExpression())); + this.selectItems + .add(new ValueParserNode(fieldName, OperatorTools.buildParser(exprItem.getExpression()))); + } else if (item instanceof AllColumns) { + fieldName = item.toString(); + this.encoder.getFields().clear(); + this.selectItems.add(new ValueParserNode(fieldName, null)); } } } @@ -137,10 +143,18 @@ public List transform(I input, Map extParams) { // parse value SinkData sinkData = new DefaultSinkData(); - for (Entry entry : this.selectItemMap.entrySet()) { - String fieldName = entry.getKey(); + for (ValueParserNode node : this.selectItems) { + String fieldName = node.getFieldName(); + ValueParser parser = node.getParser(); + if (parser == null && StringUtils.equals(fieldName, SinkEncoder.ALL_SOURCE_FIELD_SIGN)) { + if (input instanceof String) { + sinkData.addField(fieldName, (String) input); + } else { + sinkData.addField(fieldName, ""); + } + continue; + } try { - ValueParser parser = entry.getValue(); Object fieldValue = parser.parse(sourceData, i, context); sinkData.addField(fieldName, String.valueOf(fieldValue)); } catch (Throwable t) { diff --git a/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/ValueParserNode.java b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/ValueParserNode.java new file mode 100644 index 0000000000..e36c0c9c6a --- /dev/null +++ b/inlong-sdk/transform-sdk/src/main/java/org/apache/inlong/sdk/transform/process/ValueParserNode.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.inlong.sdk.transform.process; + +import org.apache.inlong.sdk.transform.process.parser.ValueParser; + +import lombok.AllArgsConstructor; +import lombok.Data; + +/** + * ValueParserNode + */ +@AllArgsConstructor +@Data +public class ValueParserNode { + + private String fieldName; + private ValueParser parser; +} diff --git a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java index 3413f1aca3..8448260252 100644 --- a/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java +++ b/inlong-sdk/transform-sdk/src/test/java/org/apache/inlong/sdk/transform/process/TestTransformProcessor.java @@ -350,4 +350,80 @@ public void testPb2CsvForNow() throws Exception { List output = processor.transform(srcBytes, new HashMap<>()); Assert.assertEquals(2, output.size()); } + @Test + public void testCsv2Star() throws Exception { + List fields = this.getTestFieldList("ftime", "extinfo"); + CsvSourceInfo csvSource = new CsvSourceInfo("UTF-8", '|', '\\', fields); + CsvSinkInfo csvSink = new CsvSinkInfo("UTF-8", '|', '\\', new ArrayList<>()); + String transformSql = "select *"; + TransformConfig config = new TransformConfig(transformSql); + // case1 + TransformProcessor processor1 = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createCsvEncoder(csvSink)); + + List output1 = processor1.transform("2024-04-28 00:00:00|ok", new HashMap<>()); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "2024-04-28 00:00:00|ok"); + // case2 + config.setTransformSql("select * from source where extinfo!='ok'"); + TransformProcessor processor2 = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createCsvEncoder(csvSink)); + + List output2 = processor2.transform("2024-04-28 00:00:00|ok", new HashMap<>()); + Assert.assertEquals(0, output2.size()); + // case3 + config.setTransformSql("select *,extinfo,ftime from source where extinfo!='ok'"); + TransformProcessor processor3 = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSource), + SinkEncoderFactory.createCsvEncoder(csvSink)); + + List output3 = processor3.transform("2024-04-28 00:00:00|nok", new HashMap<>()); + Assert.assertEquals(1, output3.size()); + Assert.assertEquals(output3.get(0), "2024-04-28 00:00:00|nok|nok|2024-04-28 00:00:00"); + // case4 + CsvSourceInfo csvSourceNoField = new CsvSourceInfo("UTF-8", '|', '\\', new ArrayList<>()); + CsvSinkInfo csvSinkNoField = new CsvSinkInfo("UTF-8", '|', '\\', new ArrayList<>()); + config.setTransformSql("select *,$2,$1 from source where $2='nok'"); + TransformProcessor processor4 = TransformProcessor + .create(config, SourceDecoderFactory.createCsvDecoder(csvSourceNoField), + SinkEncoderFactory.createCsvEncoder(csvSinkNoField)); + + List output4 = processor4.transform("2024-04-28 00:00:00|nok", new HashMap<>()); + Assert.assertEquals(1, output4.size()); + Assert.assertEquals(output4.get(0), "2024-04-28 00:00:00|nok|nok|2024-04-28 00:00:00"); + } + + @Test + public void testKv2Star() throws Exception { + List fields = this.getTestFieldList("ftime", "extinfo"); + KvSourceInfo kvSource = new KvSourceInfo("UTF-8", fields); + KvSinkInfo kvSink = new KvSinkInfo("UTF-8", new ArrayList<>()); + String transformSql = "select *"; + TransformConfig config = new TransformConfig(transformSql); + // case1 + TransformProcessor processor1 = TransformProcessor + .create(config, SourceDecoderFactory.createKvDecoder(kvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output1 = processor1.transform("ftime=2024-04-28 00:00:00&extinfo=ok", new HashMap<>()); + Assert.assertEquals(1, output1.size()); + Assert.assertEquals(output1.get(0), "ftime=2024-04-28 00:00:00&extinfo=ok"); + // case2 + config.setTransformSql("select * from source where extinfo!='ok'"); + TransformProcessor processor2 = TransformProcessor + .create(config, SourceDecoderFactory.createKvDecoder(kvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + List output2 = processor2.transform("ftime=2024-04-28 00:00:00&extinfo=ok", new HashMap<>()); + Assert.assertEquals(0, output2.size()); + // case3 + config.setTransformSql("select *,extinfo e1,ftime f1 from source where extinfo!='ok'"); + TransformProcessor processor3 = TransformProcessor + .create(config, SourceDecoderFactory.createKvDecoder(kvSource), + SinkEncoderFactory.createKvEncoder(kvSink)); + + List output3 = processor3.transform("ftime=2024-04-28 00:00:00&extinfo=nok", new HashMap<>()); + Assert.assertEquals(1, output3.size()); + Assert.assertEquals(output3.get(0), "ftime=2024-04-28 00:00:00&extinfo=nok&e1=nok&f1=2024-04-28 00:00:00"); + } }