You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by zi...@apache.org on 2022/08/12 06:37:26 UTC
[inlong] branch master updated: [INLONG-5485][Sort] Support JSON UDF getter and JSON getter function (#5486)
This is an automated email from the ASF dual-hosted git repository.
zirui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/inlong.git
The following commit(s) were added to refs/heads/master by this push:
new a18aebba6 [INLONG-5485][Sort] Support JSON UDF getter and JSON getter function (#5486)
a18aebba6 is described below
commit a18aebba668c7260c67dbbadbd626879c96eed13
Author: Schnapps <zp...@connect.ust.hk>
AuthorDate: Fri Aug 12 14:37:21 2022 +0800
[INLONG-5485][Sort] Support JSON UDF getter and JSON getter function (#5486)
---
.../protocol/transformation/FunctionParam.java | 4 +-
.../function/JsonGetterFunction.java | 79 ++++++++++++++++++
.../inlong/sort/function/JsonGetterFunction.java | 51 ++++++++++++
.../inlong/sort/parser/impl/FlinkSqlParser.java | 2 +
.../sort/parser/impl/NativeFlinkSqlParser.java | 2 +
.../sort/function/JsonGetterFunctionTest.java | 97 ++++++++++++++++++++++
6 files changed, 234 insertions(+), 1 deletion(-)
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/FunctionParam.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/FunctionParam.java
index d5634aea2..8ea1b5261 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/FunctionParam.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/FunctionParam.java
@@ -27,6 +27,7 @@ import org.apache.inlong.sort.protocol.transformation.function.EncryptFunction;
import org.apache.inlong.sort.protocol.transformation.function.HopEndFunction;
import org.apache.inlong.sort.protocol.transformation.function.HopFunction;
import org.apache.inlong.sort.protocol.transformation.function.HopStartFunction;
+import org.apache.inlong.sort.protocol.transformation.function.JsonGetterFunction;
import org.apache.inlong.sort.protocol.transformation.function.MultiValueFilterFunction;
import org.apache.inlong.sort.protocol.transformation.function.RegexpReplaceFirstFunction;
import org.apache.inlong.sort.protocol.transformation.function.RegexpReplaceFunction;
@@ -96,7 +97,8 @@ import org.apache.inlong.sort.protocol.transformation.operator.OrOperator;
@JsonSubTypes.Type(value = RegexpReplaceFunction.class, name = "regexpReplace"),
@JsonSubTypes.Type(value = RegexpReplaceFirstFunction.class, name = "regexpReplaceFirst"),
@JsonSubTypes.Type(value = CascadeFunctionWrapper.class, name = "cascadeFunctionWrapper"),
- @JsonSubTypes.Type(value = EncryptFunction.class, name = "encrypt")
+ @JsonSubTypes.Type(value = EncryptFunction.class, name = "encrypt"),
+ @JsonSubTypes.Type(value = JsonGetterFunction.class, name = "jsonGetterFunction")
})
public interface FunctionParam {
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/JsonGetterFunction.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/JsonGetterFunction.java
new file mode 100644
index 000000000..6fad491ed
--- /dev/null
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/transformation/function/JsonGetterFunction.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.sort.protocol.transformation.function;
+
+import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+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.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.FunctionParam;
+import org.apache.inlong.sort.protocol.transformation.StringConstantParam;
+
+/**
+ * Json getter function, used to get a field value from a json string
+ */
+public class JsonGetterFunction implements CascadeFunction, Serializable {
+
+ private static final long serialVersionUID = -2701547146694616429L;
+
+ @JsonProperty("field")
+ private FieldInfo field;
+ @JsonProperty("key")
+ private StringConstantParam key;
+
+ /**
+ * JsonGetterFunction constructor
+ *
+ * @param field the field to be extracted
+ * @param key the key of the field in the json
+ */
+ @JsonCreator
+ public JsonGetterFunction(@JsonProperty("field") FieldInfo field,
+ @JsonProperty("key") StringConstantParam key) {
+ this.field = Preconditions.checkNotNull(field, "field is null");
+ this.key = Preconditions.checkNotNull(key, "key is null");
+ }
+
+ @Override
+ public String getName() {
+ return "JSON_GETTER";
+ }
+
+ @Override
+ public List<FunctionParam> getParams() {
+ return Arrays.asList(field, key);
+ }
+
+ @Override
+ public String format() {
+ return String.format("%s(CAST(%s AS STRING), %s)",
+ getName(), field.format(), key.format());
+ }
+
+ @Override
+ public ConstantParam apply(ConstantParam constantParam) {
+ return new ConstantParam(String.format("%s(%s, %s)", getName(),
+ constantParam.format(), key.format()));
+ }
+
+}
diff --git a/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/function/JsonGetterFunction.java b/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/function/JsonGetterFunction.java
new file mode 100644
index 000000000..de26eb7fe
--- /dev/null
+++ b/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/function/JsonGetterFunction.java
@@ -0,0 +1,51 @@
+/*
+ * 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.function;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.table.functions.ScalarFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * json getter function, used to get a field value from a json string
+ */
+public class JsonGetterFunction extends ScalarFunction {
+
+ private static final long serialVersionUID = -7185622027483662395L;
+
+ public static final Logger LOG = LoggerFactory.getLogger(JsonGetterFunction.class);
+
+ private final ObjectMapper mapper = new ObjectMapper();
+
+ /**
+ * eval is String replacement execution method
+ *
+ * @param field is the field to be replaced
+ * @return replaced value
+ */
+ public String eval(String field, String key) {
+ try {
+ return mapper.readTree(field).findValue(key).asText();
+ } catch (Exception e) {
+ LOG.error("json getter function error, key {}, field {}", key, field, e);
+ return null;
+ }
+ }
+
+}
diff --git a/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/FlinkSqlParser.java b/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/FlinkSqlParser.java
index 9cb307f7e..75dafa0dc 100644
--- a/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/FlinkSqlParser.java
+++ b/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/FlinkSqlParser.java
@@ -23,6 +23,7 @@ import org.apache.flink.table.api.TableEnvironment;
import org.apache.inlong.sort.formats.base.TableFormatUtils;
import org.apache.inlong.sort.formats.common.FormatInfo;
import org.apache.inlong.sort.function.EncryptFunction;
+import org.apache.inlong.sort.function.JsonGetterFunction;
import org.apache.inlong.sort.function.RegexpReplaceFirstFunction;
import org.apache.inlong.sort.function.RegexpReplaceFunction;
import org.apache.inlong.sort.parser.Parser;
@@ -108,6 +109,7 @@ public class FlinkSqlParser implements Parser {
tableEnv.createTemporarySystemFunction("REGEXP_REPLACE_FIRST", RegexpReplaceFirstFunction.class);
tableEnv.createTemporarySystemFunction("REGEXP_REPLACE", RegexpReplaceFunction.class);
tableEnv.createTemporarySystemFunction("ENCRYPT", EncryptFunction.class);
+ tableEnv.createTemporarySystemFunction("JSON_GETTER", JsonGetterFunction.class);
}
/**
diff --git a/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/NativeFlinkSqlParser.java b/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/NativeFlinkSqlParser.java
index bef80132f..e0cfdfac9 100644
--- a/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/NativeFlinkSqlParser.java
+++ b/inlong-sort/sort-core/src/main/java/org/apache/inlong/sort/parser/impl/NativeFlinkSqlParser.java
@@ -21,6 +21,7 @@ package org.apache.inlong.sort.parser.impl;
import com.google.common.base.Preconditions;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.inlong.sort.function.EncryptFunction;
+import org.apache.inlong.sort.function.JsonGetterFunction;
import org.apache.inlong.sort.function.RegexpReplaceFirstFunction;
import org.apache.inlong.sort.function.RegexpReplaceFunction;
import org.apache.inlong.sort.parser.Parser;
@@ -68,6 +69,7 @@ public class NativeFlinkSqlParser implements Parser {
tableEnv.createTemporarySystemFunction("REGEXP_REPLACE_FIRST", RegexpReplaceFirstFunction.class);
tableEnv.createTemporarySystemFunction("REGEXP_REPLACE", RegexpReplaceFunction.class);
tableEnv.createTemporarySystemFunction("ENCRYPT", EncryptFunction.class);
+ tableEnv.createTemporarySystemFunction("JSON_GETTER", JsonGetterFunction.class);
}
/**
diff --git a/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/function/JsonGetterFunctionTest.java b/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/function/JsonGetterFunctionTest.java
new file mode 100644
index 000000000..ccb4996f0
--- /dev/null
+++ b/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/function/JsonGetterFunctionTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.function;
+
+import java.util.ArrayList;
+import java.util.List;
+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.StringConstantParam;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link JsonGetterFunction}
+ */
+public class JsonGetterFunctionTest extends AbstractTestBase {
+
+ /**
+ * Test for JsonGetter function
+ *
+ * @throws Exception The exception may throw when test Encrypt function
+ */
+ @Test
+ public void testJsonGetterFunction() throws Exception {
+ 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 json getter
+ tableEnv.createTemporaryFunction("JSON_GETTER", JsonGetterFunction.class);
+ // step 2. Generate test data and convert to DataStream
+ List<Row> data = new ArrayList<>();
+ data.add(Row.of("{\"name\":\"abc\"}"));
+ TypeInformation<?>[] types = {
+ BasicTypeInfo.STRING_TYPE_INFO};
+ String[] names = {"content"};
+ RowTypeInfo typeInfo = new RowTypeInfo(types, names);
+ DataStream<Row> dataStream = env.fromCollection(data).returns(typeInfo);
+
+ // step 3. Convert from DataStream to Table and execute the json getter function
+ Table tempView = tableEnv.fromDataStream(dataStream).as("content");
+ tableEnv.createTemporaryView("temp_view", tempView);
+
+ org.apache.inlong.sort.protocol.transformation.function.JsonGetterFunction jsonGetterFunction =
+ new org.apache.inlong.sort.protocol.transformation.function.JsonGetterFunction(
+ new FieldInfo("content",
+ new StringFormatInfo()), new StringConstantParam("name"));
+
+ String sqlQuery = String.format("SELECT %s as content FROM temp_view", jsonGetterFunction.format());
+ Table outputTable = tableEnv.sqlQuery(sqlQuery);
+ // step 4. Get function execution result and parse it
+ DataStream<Row> resultSet = tableEnv.toAppendStream(outputTable, Row.class);
+ List<String> result = new ArrayList<>();
+
+ for (CloseableIterator<String> it = resultSet.map(s -> s.getField(0).toString()).executeAndCollect();
+ it.hasNext(); ) {
+ String next = it.next();
+ result.add(next);
+ }
+
+ // step 5. Whether the comparison results are as expected
+ String expect = "abc";
+ Assert.assertEquals(expect, result.get(0));
+ }
+
+}