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


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala:
##########
@@ -476,6 +485,11 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, 
nullableInput: Boolean)
             call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 =>
         generateNullLiteral(resultType)
 
+      // We only support JSON function operands within JSON_OBJECT
+      case (operand: RexNode, _)
+          if isJsonObjectOrArrayOperand(call) && 
isJsonFunctionOperand(operand) =>

Review Comment:
   Since we use `OrArrayOperand`? Do we support JSON_ARRAY? 



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala:
##########
@@ -476,6 +485,11 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, 
nullableInput: Boolean)
             call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 =>
         generateNullLiteral(resultType)
 
+      // We only support JSON function operands within JSON_OBJECT
+      case (operand: RexNode, _)

Review Comment:
   ```suggestion
         case (operand: RexNode, i)
   ```
   also check that the index is even. otherwise JSON() function could be used 
for the key?



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java:
##########
@@ -861,9 +862,13 @@ public static ApiExpression withoutColumns(Object head, 
Object... tail) {
      * jsonObject(JsonOnNull.ABSENT, "K1", nullOf(DataTypes.STRING())) // "{}"
      *
      * // {"K1":{"K2":"V"}}
+     * jsonObject(JsonOnNull.NULL, "K1", json('{"K2":"V"}'))

Review Comment:
   thanks for updating the JavaDocs of `json()`. Seems this line still needs an 
update.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/JsonFunction.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.flink.table.runtime.functions.scalar;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableRuntimeException;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.SpecializedFunction.SpecializedContext;
+
+import javax.annotation.Nullable;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#JSON}. */
+@Internal
+public class JsonFunction extends BuiltInScalarFunction {

Review Comment:
   Remove?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala:
##########
@@ -459,6 +460,14 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, 
nullableInput: Boolean)
 
   override def visitCall(call: RexCall): GeneratedExpression = {
     val resultType = FlinkTypeFactory.toLogicalType(call.getType)
+
+    // throw exception if json function is called outside JSON_OBJECT
+    if (isJsonFunctionOperand(call)) {
+      throw new ValidationException(
+        "The JSON function is currently only supported inside a JSON_OBJECT." +

Review Comment:
   ```suggestion
           "The JSON() function is currently only supported inside a 
JSON_OBJECT() function." +
   ```



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java:
##########
@@ -2819,6 +2819,15 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
                     .runtimeDeferred()
                     .build();
 
+    public static final BuiltInFunctionDefinition JSON =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("JSON")
+                    .kind(SCALAR)
+                    
.inputTypeStrategy(sequence(logical(LogicalTypeFamily.CHARACTER_STRING)))
+                    
.outputTypeStrategy(nullableIfArgs(explicit(DataTypes.STRING())))
+                    
.runtimeClass("org.apache.flink.table.runtime.functions.scalar.JsonFunction")

Review Comment:
   replace with `runtimeDeferred()`?



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to