You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by he...@apache.org on 2022/04/24 12:28:34 UTC

[incubator-inlong] branch master updated: [INLONG-3839][Sort] Add cascade function support for transform (#3903)

This is an automated email from the ASF dual-hosted git repository.

healchow pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 72327d7e4 [INLONG-3839][Sort] Add cascade function support for transform (#3903)
72327d7e4 is described below

commit 72327d7e4a4c7e62edc9109818c0ca062180f7b9
Author: yunqingmoswu <44...@users.noreply.github.com>
AuthorDate: Sun Apr 24 20:28:29 2022 +0800

    [INLONG-3839][Sort] Add cascade function support for transform (#3903)
    
    * Add cascade function support for transform
    
    * Fix unit test error
---
 .../protocol/transformation/CascadeFunction.java   |  48 ++++++++++
 ...ceFunction.java => CascadeFunctionWrapper.java} |  52 +++++------
 .../function/RegexpReplaceFirstFunction.java       |  11 ++-
 .../function/RegexpReplaceFunction.java            |  12 ++-
 .../flink/function/CascadeFunctionWrapperTest.java | 104 +++++++++++++++++++++
 5 files changed, 195 insertions(+), 32 deletions(-)

diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/CascadeFunction.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/CascadeFunction.java
new file mode 100644
index 000000000..d33abf5b0
--- /dev/null
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/CascadeFunction.java
@@ -0,0 +1,48 @@
+/*
+ * 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.protocol.transformation;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonSubTypes;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.inlong.sort.protocol.transformation.function.RegexpReplaceFirstFunction;
+import org.apache.inlong.sort.protocol.transformation.function.RegexpReplaceFunction;
+
+/**
+ * CascadeFunction is the top-level interface abstraction for cascading function
+ */
+@JsonTypeInfo(
+        use = JsonTypeInfo.Id.NAME,
+        include = JsonTypeInfo.As.PROPERTY,
+        property = "type")
+@JsonSubTypes({
+        @JsonSubTypes.Type(value = RegexpReplaceFirstFunction.class, name = "regexpReplaceFirst"),
+        @JsonSubTypes.Type(value = RegexpReplaceFunction.class, name = "regexpReplace")
+})
+public interface CascadeFunction extends Function {
+
+    /**
+     * apply function Act on a specific cascade function
+     * It accepts the result of running a function as an input parameter
+     * and returns the result of running the function
+     *
+     * @param constantParam is a constant param
+     * @return A constant param
+     */
+    ConstantParam apply(ConstantParam constantParam);
+
+}
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/CascadeFunctionWrapper.java
similarity index 53%
copy from inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java
copy to inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/CascadeFunctionWrapper.java
index 32a2992c0..87c2c9554 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/CascadeFunctionWrapper.java
@@ -23,60 +23,56 @@ import lombok.EqualsAndHashCode;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.transformation.CascadeFunction;
+import org.apache.inlong.sort.protocol.transformation.ConstantParam;
 import org.apache.inlong.sort.protocol.transformation.Function;
 import org.apache.inlong.sort.protocol.transformation.FunctionParam;
-import org.apache.inlong.sort.protocol.transformation.StringConstantParam;
 
 import java.io.Serializable;
-import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
- * RegexpReplaceFunction class is the logic encapsulation of String replace by regexp
+ * CascadeFunctionWrapper class is a wrapper of {@link CascadeFunction}
+ * It contains a list of {@link CascadeFunction} that really will be executed
  */
-@JsonTypeName("regexpReplace")
+@JsonTypeName("cascadeFunctionWrapper")
 @EqualsAndHashCode(callSuper = false)
 @Data
-public class RegexpReplaceFunction implements Function, Serializable {
+public class CascadeFunctionWrapper implements Function, Serializable {
 
-    private static final long serialVersionUID = -2701547146694616429L;
+    private static final long serialVersionUID = 8197348412858988257L;
 
-    @JsonProperty("field")
-    private FieldInfo field;
-    @JsonProperty("regex")
-    private StringConstantParam regex;
-    @JsonProperty("replacement")
-    private StringConstantParam replacement;
+    @JsonProperty("functions")
+    private final List<CascadeFunction> functions;
 
     /**
-     * RegexpReplaceFunction constructor
+     * CascadeFunction constructor
      *
-     * @param field it is character to be replaced
-     * @param regex the regex expression of replacing
-     * @param replacement the value that to be replaced
+     * @param functions List of functions that cascade functions really need to execute
      */
     @JsonCreator
-    public RegexpReplaceFunction(@JsonProperty("field") FieldInfo field,
-            @JsonProperty("regex") StringConstantParam regex,
-            @JsonProperty("replacement") StringConstantParam replacement) {
-        this.field = Preconditions.checkNotNull(field, "field is null");
-        this.regex = Preconditions.checkNotNull(regex, "regex is null");
-        this.replacement = Preconditions.checkNotNull(replacement, "replacement is null");
+    public CascadeFunctionWrapper(@JsonProperty("functions") List<CascadeFunction> functions) {
+        this.functions = Preconditions.checkNotNull(functions, "functions is null");
+        Preconditions.checkState(!functions.isEmpty(), "functions is empty");
     }
 
     @Override
-    public String getName() {
-        return "REGEXP_REPLACE";
+    public List<FunctionParam> getParams() {
+        return new ArrayList<>(functions);
     }
 
     @Override
-    public List<FunctionParam> getParams() {
-        return Arrays.asList(field, regex, replacement);
+    public String getName() {
+        throw new UnsupportedOperationException("The method of getName is not support of CascadeFunction");
     }
 
     @Override
     public String format() {
-        return String.format("%s(%s, %s, %s)", getName(), field.format(), regex.format(), replacement.format());
+        ConstantParam s = new ConstantParam(functions.get(0).format());
+        for (int i = 1; i < functions.size(); i++) {
+            s = functions.get(i).apply(s);
+        }
+        return s.format();
     }
 }
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFirstFunction.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFirstFunction.java
index d590bac32..72d974b05 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFirstFunction.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFirstFunction.java
@@ -23,7 +23,8 @@ import lombok.EqualsAndHashCode;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.inlong.sort.protocol.transformation.Function;
+import org.apache.inlong.sort.protocol.transformation.CascadeFunction;
+import org.apache.inlong.sort.protocol.transformation.ConstantParam;
 import org.apache.inlong.sort.protocol.transformation.FunctionParam;
 import org.apache.inlong.sort.protocol.transformation.StringConstantParam;
 
@@ -38,7 +39,7 @@ import java.util.List;
 @JsonTypeName("regexpReplaceFirst")
 @EqualsAndHashCode(callSuper = false)
 @Data
-public class RegexpReplaceFirstFunction implements Function, Serializable {
+public class RegexpReplaceFirstFunction implements CascadeFunction, Serializable {
 
     private static final long serialVersionUID = -2701547146694616429L;
 
@@ -79,4 +80,10 @@ public class RegexpReplaceFirstFunction implements Function, Serializable {
     public String format() {
         return String.format("%s(%s, %s, %s)", getName(), field.format(), regex.format(), replacement.format());
     }
+
+    @Override
+    public ConstantParam apply(ConstantParam constantParam) {
+        return new ConstantParam(String.format("%s(%s, %s, %s)", getName(),
+                constantParam.format(), regex.format(), replacement.format()));
+    }
 }
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java
index 32a2992c0..afefe23e9 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/RegexpReplaceFunction.java
@@ -24,7 +24,8 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCre
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.inlong.sort.protocol.FieldInfo;
-import org.apache.inlong.sort.protocol.transformation.Function;
+import org.apache.inlong.sort.protocol.transformation.CascadeFunction;
+import org.apache.inlong.sort.protocol.transformation.ConstantParam;
 import org.apache.inlong.sort.protocol.transformation.FunctionParam;
 import org.apache.inlong.sort.protocol.transformation.StringConstantParam;
 
@@ -38,7 +39,7 @@ import java.util.List;
 @JsonTypeName("regexpReplace")
 @EqualsAndHashCode(callSuper = false)
 @Data
-public class RegexpReplaceFunction implements Function, Serializable {
+public class RegexpReplaceFunction implements CascadeFunction, Serializable {
 
     private static final long serialVersionUID = -2701547146694616429L;
 
@@ -79,4 +80,11 @@ public class RegexpReplaceFunction implements Function, Serializable {
     public String format() {
         return String.format("%s(%s, %s, %s)", getName(), field.format(), regex.format(), replacement.format());
     }
+
+    @Override
+    public ConstantParam apply(ConstantParam constantParam) {
+        return new ConstantParam(String.format("%s(%s, %s, %s)", getName(),
+                constantParam.format(), regex.format(), replacement.format()));
+    }
+
 }
diff --git a/inlong-sort/sort-single-tenant/src/test/java/org/apache/inlong/sort/singletenant/flink/function/CascadeFunctionWrapperTest.java b/inlong-sort/sort-single-tenant/src/test/java/org/apache/inlong/sort/singletenant/flink/function/CascadeFunctionWrapperTest.java
new file mode 100644
index 000000000..2989056c5
--- /dev/null
+++ b/inlong-sort/sort-single-tenant/src/test/java/org/apache/inlong/sort/singletenant/flink/function/CascadeFunctionWrapperTest.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.singletenant.flink.function;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.inlong.sort.formats.common.StringFormatInfo;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.transformation.CascadeFunction;
+import org.apache.inlong.sort.protocol.transformation.StringConstantParam;
+import org.apache.inlong.sort.protocol.transformation.function.CascadeFunctionWrapper;
+import org.apache.inlong.sort.protocol.transformation.function.RegexpReplaceFunction;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test for {@link CascadeFunctionWrapper}
+ */
+public class CascadeFunctionWrapperTest extends AbstractTestBase {
+
+    /**
+     * Test for CascadeFunctionWrapper
+     *
+     * @throws Exception The exception may throw when test CascadeFunctionWrapper
+     */
+    @Test
+    public void testCascadeFunctionWrapper() throws Exception {
+        // step 0. Initialize the execution environment
+        EnvironmentSettings settings = EnvironmentSettings
+                .newInstance()
+                .useBlinkPlanner()
+                .inStreamingMode()
+                .build();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(10000);
+        StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings);
+        // step 1. Register custom function of REGEXP_REPLACE_FIRST
+        tableEnv.createTemporarySystemFunction("REGEXP_REPLACE_FIRST",
+                RegexpReplaceFirstFunction.class);
+        // step 2. Generate test data and convert to DataStream
+        List<Row> data = new ArrayList<>();
+        data.add(Row.of("aabbccaabbcc"));
+        TypeInformation<?>[] types = {
+                BasicTypeInfo.STRING_TYPE_INFO};
+        String[] names = {"f1"};
+        RowTypeInfo typeInfo = new RowTypeInfo(types, names);
+        List<CascadeFunction> functions = new ArrayList<>();
+        functions.add(new org.apache.inlong.sort.protocol.transformation.function.RegexpReplaceFirstFunction(
+                new FieldInfo("f1", new StringFormatInfo()),
+                new StringConstantParam("aa"), new StringConstantParam("apache")));
+        functions.add(new RegexpReplaceFunction(
+                new FieldInfo("f1", new StringFormatInfo()),
+                new StringConstantParam("bb"), new StringConstantParam("inlong")));
+        functions.add(new RegexpReplaceFunction(
+                new FieldInfo("f1", new StringFormatInfo()),
+                new StringConstantParam("cc"), new StringConstantParam("etl")));
+        CascadeFunctionWrapper wrapper = new CascadeFunctionWrapper(functions);
+        DataStream<Row> ds = env.fromCollection(data).returns(typeInfo);
+        // step 3. Convert from DataStream to Table and execute the REGEXP_REPLACE_FIRST function
+        Table tempView = tableEnv.fromDataStream(ds).as("f1");
+        tableEnv.createTemporaryView("temp_view", tempView);
+        String sqlQuery = String.format("SELECT %s as a FROM temp_view", wrapper.format());
+        Table outputTable = tableEnv.sqlQuery(sqlQuery);
+        DataStream<Row> resultSet = tableEnv.toAppendStream(outputTable, Row.class);
+        // step 4. Get function execution result and parse it
+        List<String> r = new ArrayList<>();
+        for (CloseableIterator<String> it = resultSet.map(s -> s.getField(0).toString()).executeAndCollect();
+                it.hasNext(); ) {
+            String next = it.next();
+            r.add(next);
+        }
+        // step 5. Whether the comparison results are as expected
+        String expect = "apacheinlongetlaainlongetl";
+        Assert.assertEquals(expect, r.get(0));
+    }
+}