You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/26 12:44:52 UTC

[GitHub] [flink] twalthr commented on a change in pull request #17562: [FLINK-16206][table-planner] Support JSON_ARRAYAGG

twalthr commented on a change in pull request #17562:
URL: https://github.com/apache/flink/pull/17562#discussion_r736487847



##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/JsonArrayAggFunction.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.dataview.ListView;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule;
+import org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.util.RawValue;
+
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.createArrayNode;
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.getNodeFactory;
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.serializeJson;
+
+/**
+ * Implementation for {@link BuiltInFunctionDefinitions#JSON_ARRAYAGG_ABSENT_ON_NULL} / {@link
+ * BuiltInFunctionDefinitions#JSON_ARRAYAGG_NULL_ON_NULL}.
+ *
+ * <p>Note that this function only ever receives strings to accumulate because {@link
+ * WrapJsonAggFunctionArgumentsRule} wraps arguments into {@link
+ * BuiltInFunctionDefinitions#JSON_STRING}.
+ */
+@Internal
+public class JsonArrayAggFunction
+        extends BuiltInAggregateFunction<String, JsonArrayAggFunction.Accumulator> {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Marker that represents a {@code null} since {@link ListView} does not allow {@code null}s.
+     *
+     * <p>Note that due to {@link WrapJsonAggFunctionArgumentsRule} and the fact that this function
+     * already only receives JSON strings, this value cannot be created by the user and is thus safe
+     * to use.
+     */
+    private static final StringData NULL_STR = StringData.fromString("null");
+
+    private final transient List<DataType> argumentTypes;
+    private final SqlJsonConstructorNullClause onNull;
+
+    public JsonArrayAggFunction(LogicalType[] argumentTypes, SqlJsonConstructorNullClause onNull) {

Review comment:
       let's make this a boolean flag to not have Calcite deps in runtime code

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/JsonArrayAggFunction.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.dataview.ListView;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule;
+import org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.util.RawValue;
+
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.createArrayNode;
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.getNodeFactory;
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.serializeJson;
+
+/**
+ * Implementation for {@link BuiltInFunctionDefinitions#JSON_ARRAYAGG_ABSENT_ON_NULL} / {@link
+ * BuiltInFunctionDefinitions#JSON_ARRAYAGG_NULL_ON_NULL}.
+ *
+ * <p>Note that this function only ever receives strings to accumulate because {@link
+ * WrapJsonAggFunctionArgumentsRule} wraps arguments into {@link
+ * BuiltInFunctionDefinitions#JSON_STRING}.
+ */
+@Internal
+public class JsonArrayAggFunction
+        extends BuiltInAggregateFunction<String, JsonArrayAggFunction.Accumulator> {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Marker that represents a {@code null} since {@link ListView} does not allow {@code null}s.
+     *
+     * <p>Note that due to {@link WrapJsonAggFunctionArgumentsRule} and the fact that this function
+     * already only receives JSON strings, this value cannot be created by the user and is thus safe
+     * to use.
+     */
+    private static final StringData NULL_STR = StringData.fromString("null");
+
+    private final transient List<DataType> argumentTypes;
+    private final SqlJsonConstructorNullClause onNull;
+
+    public JsonArrayAggFunction(LogicalType[] argumentTypes, SqlJsonConstructorNullClause onNull) {
+        this.argumentTypes =
+                Arrays.stream(argumentTypes)
+                        .map(DataTypeUtils::toInternalDataType)
+                        .collect(Collectors.toList());
+
+        this.onNull = onNull;
+    }
+
+    @Override
+    public List<DataType> getArgumentDataTypes() {
+        return argumentTypes;
+    }
+
+    @Override
+    public DataType getOutputDataType() {
+        return DataTypes.STRING();
+    }
+
+    @Override
+    public DataType getAccumulatorDataType() {
+        return DataTypes.STRUCTURED(
+                Accumulator.class,
+                DataTypes.FIELD("list", ListView.newListViewDataType(DataTypes.STRING())));
+    }
+
+    @Override
+    public Accumulator createAccumulator() {
+        return new Accumulator();
+    }
+
+    public void resetAccumulator(Accumulator acc) {
+        acc.list.clear();
+    }
+
+    public void accumulate(Accumulator acc, StringData itemData) throws Exception {
+        if (itemData == null) {
+            switch (onNull) {
+                case NULL_ON_NULL:
+                    acc.list.add(NULL_STR);
+                    break;
+                case ABSENT_ON_NULL:
+                    break;
+                default:
+                    throw new TableException(
+                            String.format("Unsupported ON NULL behavior: %s", onNull));
+            }
+        } else {
+            acc.list.add(itemData);
+        }
+    }
+
+    public void retract(Accumulator acc, StringData itemData) throws Exception {
+        if (itemData == null) {
+            acc.list.remove(NULL_STR);
+        } else {
+            acc.list.remove(itemData);
+        }
+    }
+
+    public void merge(Accumulator acc, Iterable<Accumulator> others) throws Exception {
+        for (final Accumulator other : others) {
+            acc.list.addAll(other.list.getList());
+        }
+    }
+
+    @Override
+    public String getValue(Accumulator acc) {
+        final ArrayNode rootNode = createArrayNode();
+        try {
+            for (final StringData item : acc.list.get()) {
+                final JsonNode itemNode =
+                        getNodeFactory().rawValueNode(new RawValue(item.toString()));

Review comment:
       can we also somehow sort the list? otherwise the `merge` might behave non-deterministic

##########
File path: flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/JsonArrayAggFunction.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.dataview.ListView;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule;
+import org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.utils.DataTypeUtils;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.util.RawValue;
+
+import org.apache.calcite.sql.SqlJsonConstructorNullClause;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.createArrayNode;
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.getNodeFactory;
+import static org.apache.flink.table.runtime.functions.SqlJsonUtils.serializeJson;
+
+/**
+ * Implementation for {@link BuiltInFunctionDefinitions#JSON_ARRAYAGG_ABSENT_ON_NULL} / {@link
+ * BuiltInFunctionDefinitions#JSON_ARRAYAGG_NULL_ON_NULL}.
+ *
+ * <p>Note that this function only ever receives strings to accumulate because {@link
+ * WrapJsonAggFunctionArgumentsRule} wraps arguments into {@link
+ * BuiltInFunctionDefinitions#JSON_STRING}.
+ */
+@Internal
+public class JsonArrayAggFunction
+        extends BuiltInAggregateFunction<String, JsonArrayAggFunction.Accumulator> {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * Marker that represents a {@code null} since {@link ListView} does not allow {@code null}s.
+     *
+     * <p>Note that due to {@link WrapJsonAggFunctionArgumentsRule} and the fact that this function
+     * already only receives JSON strings, this value cannot be created by the user and is thus safe
+     * to use.
+     */
+    private static final StringData NULL_STR = StringData.fromString("null");
+
+    private final transient List<DataType> argumentTypes;
+    private final SqlJsonConstructorNullClause onNull;
+
+    public JsonArrayAggFunction(LogicalType[] argumentTypes, SqlJsonConstructorNullClause onNull) {
+        this.argumentTypes =
+                Arrays.stream(argumentTypes)
+                        .map(DataTypeUtils::toInternalDataType)
+                        .collect(Collectors.toList());
+
+        this.onNull = onNull;
+    }
+
+    @Override
+    public List<DataType> getArgumentDataTypes() {
+        return argumentTypes;
+    }
+
+    @Override
+    public DataType getOutputDataType() {
+        return DataTypes.STRING();
+    }
+
+    @Override
+    public DataType getAccumulatorDataType() {
+        return DataTypes.STRUCTURED(
+                Accumulator.class,
+                DataTypes.FIELD("list", ListView.newListViewDataType(DataTypes.STRING())));

Review comment:
       `STRING().bridgedTo(StringData.class)`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org