twalthr commented on code in PR #24704:
URL: https://github.com/apache/flink/pull/24704#discussion_r1609988821


##########
flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj:
##########
@@ -0,0 +1,8457 @@
+/*
+ * 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.
+ */
+<@pp.dropOutputFile />

Review Comment:
   Can you add a comment that is similar to other Calcite copied files, so that 
we know when we can drop this file again in the future. Which Calcite version 
is required, where is the change located.



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java:
##########
@@ -2046,11 +2046,72 @@ public OutType jsonQuery(
             JsonQueryWrapper wrappingBehavior,
             JsonQueryOnEmptyOrError onEmpty,
             JsonQueryOnEmptyOrError onError) {
+        return jsonQuery(path, DataTypes.STRING(), wrappingBehavior, onEmpty, 
onError);
+    }
+
+    /**
+     * Extracts JSON values from a JSON string.
+     *
+     * <p>This follows the ISO/IEC TR 19075-6 specification for JSON support 
in SQL. The result is

Review Comment:
   Reduce code duplication and simplify this JavaDoc. Link to the other and 
focus on the data type in following paragraphs. Also the examples should show 
the usage of the `datatype` parameter.



##########
flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/fun/SqlJsonQueryFunction.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.calcite.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlJsonQueryEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlJsonQueryWrapperBehavior;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeTransforms;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * The <code>JSON_QUERY</code> function.
+ *
+ * <p>This class was copied over from Calcite to support RETURNING clause in 
JSON_QUERY
+ * (CALCITE-6365). When upgrading to Calcite version that has the change, 
please remove the entire

Review Comment:
   we could already name a Calcite version?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/SqlJsonUtils.java:
##########
@@ -235,39 +251,85 @@ private static String jsonQuery(
                 }
             }
             if (value == null || context.mode == PathMode.LAX && 
isScalarObject(value)) {
-                switch (emptyBehavior) {
-                    case ERROR:
-                        throw emptyResultOfJsonQueryFuncNotAllowed();
-                    case NULL:
-                        return null;
-                    case EMPTY_ARRAY:
-                        return "[]";
-                    case EMPTY_OBJECT:
-                        return "{}";
-                    default:
-                        throw 
illegalEmptyBehaviorInJsonQueryFunc(emptyBehavior.toString());
-                }
+                return emptyResultForJsonQuery(emptyBehavior, returnType);
             } else if (context.mode == PathMode.STRICT && 
isScalarObject(value)) {
                 exc = 
arrayOrObjectValueRequiredInStrictModeOfJsonQueryFunc(value.toString());
             } else {
                 try {
-                    return jsonize(value);
+                    switch (returnType) {
+                        case STRING:
+                            return jsonize(value);
+                        case ARRAY:
+                            return new GenericArrayData(
+                                    ((List<Object>) value)

Review Comment:
   avoid streams in runtime code and go with a regular loop and pre-initalized 
array?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/SqlJsonUtils.java:
##########
@@ -235,39 +251,85 @@ private static String jsonQuery(
                 }
             }
             if (value == null || context.mode == PathMode.LAX && 
isScalarObject(value)) {
-                switch (emptyBehavior) {
-                    case ERROR:
-                        throw emptyResultOfJsonQueryFuncNotAllowed();
-                    case NULL:
-                        return null;
-                    case EMPTY_ARRAY:
-                        return "[]";
-                    case EMPTY_OBJECT:
-                        return "{}";
-                    default:
-                        throw 
illegalEmptyBehaviorInJsonQueryFunc(emptyBehavior.toString());
-                }
+                return emptyResultForJsonQuery(emptyBehavior, returnType);
             } else if (context.mode == PathMode.STRICT && 
isScalarObject(value)) {
                 exc = 
arrayOrObjectValueRequiredInStrictModeOfJsonQueryFunc(value.toString());
             } else {
                 try {
-                    return jsonize(value);
+                    switch (returnType) {
+                        case STRING:
+                            return jsonize(value);
+                        case ARRAY:
+                            return new GenericArrayData(
+                                    ((List<Object>) value)
+                                            .stream()
+                                                    .map(
+                                                            o ->
+                                                                    o != null
+                                                                            ? 
StringData.fromString(
+                                                                               
     o.toString())
+                                                                            : 
null)
+                                                    
.toArray(StringData[]::new));
+                        default:
+                            throw new RuntimeException("illegal return type");
+                    }
                 } catch (Exception e) {
                     exc = e;
                 }
             }
         }
-        switch (errorBehavior) {
+        return errorResultForJsonQuery(errorBehavior, returnType, exc);
+    }
+
+    private static Object emptyResultForJsonQuery(
+            JsonQueryOnEmptyOrError emptyBehavior, JsonQueryReturnType 
returnType) {
+        switch (emptyBehavior) {
+            case ERROR:
+                throw emptyResultOfJsonQueryFuncNotAllowed();
+            case NULL:
+                return null;
+            case EMPTY_ARRAY:
+                switch (returnType) {
+                    case ARRAY:
+                        return new GenericArrayData(new StringData[0]);
+                    case STRING:
+                        return "[]";
+                    default:
+                        throw new RuntimeException("illegal return type");

Review Comment:
   Do we want to use the new TableRuntimeException? Or is this one only 
intended for explicit errors.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to