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



##########
File path: docs/data/sql_functions.yml
##########
@@ -571,6 +571,22 @@ collection:
     table: MAP.at(ANY)
     description: Returns the value specified by key value in map.
 
+json:
+  - sql: JSON_EXISTS(jsonValue, path [ { TRUE | FALSE | UNKNOWN | ERROR } ON 
ERROR ])
+    table: STRING.jsonExists(STRING path [, JsonExistsOnError onError])
+    description: |
+      Determines whether a JSON string satisfies a given path search criterion.
+
+      This follows the ISO/IEC TR 19075-6 specification for JSON support in 
SQL.
+
+      ```
+      SELECT JSON_EXISTS(

Review comment:
       can we add a couple more examples? at least one with strict mode. We 
should try to summarize `ISO/IEC TR 19075-6 specification` a bit. E.g. what is 
the default error behavior?

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonExistsFunctionITCase.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.expressions.JsonExistsOnError;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+
+import org.apache.commons.io.IOUtils;
+import org.junit.runners.Parameterized;
+
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.nullOf;
+
+/** Tests for {@link BuiltInFunctionDefinitions#JSON_EXISTS}. */
+public class JsonExistsFunctionITCase extends BuiltInFunctionTestBase {
+
+    @Parameterized.Parameters(name = "{index}: {0}")
+    public static List<TestSpec> testData() throws Exception {
+        final InputStream jsonResource =
+                
JsonExistsFunctionITCase.class.getResourceAsStream("/json/json-exists.json");
+        if (jsonResource == null) {
+            throw new IllegalStateException(
+                    String.format(
+                            "%s: Missing test data.", 
JsonExistsFunctionITCase.class.getName()));
+        }
+
+        final String jsonValue = IOUtils.toString(jsonResource, 
Charset.defaultCharset());

Review comment:
       do we have our own Flink util somewhere? depending on an external 
dependency for such little things should be avoided

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
##########
@@ -1261,4 +1263,47 @@ public OutType sha2(InType hashLength) {
         return toApiSpecificExpression(
                 unresolvedCall(SHA2, toExpr(), 
objectToExpression(hashLength)));
     }
+
+    // JSON functions
+
+    /**
+     * Returns whether a JSON string satisfies a given search criterion.
+     *
+     * <p>The given JSON path follows the ISO9075-2 standard.

Review comment:
       synchronize docs with docs of other method below

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
##########
@@ -1261,4 +1263,47 @@ public OutType sha2(InType hashLength) {
         return toApiSpecificExpression(
                 unresolvedCall(SHA2, toExpr(), 
objectToExpression(hashLength)));
     }
+
+    // JSON functions
+
+    /**
+     * Returns whether a JSON string satisfies a given search criterion.
+     *
+     * <p>The given JSON path follows the ISO9075-2 standard.
+     *
+     * <p>Examples:
+     *
+     * <pre>{@code
+     * $("f0").jsonExists("lax $.a.b[0].c", JsonExistsOnError.TRUE)
+     * }</pre>
+     *
+     * @param path JSON path to search for.
+     * @param onError Behavior in case of an error.
+     * @return {@code true} if the JSON string satisfies the search criterion.
+     */
+    public OutType jsonExists(String path, JsonExistsOnError onError) {

Review comment:
       Did you check that the path must be a literal and SQL does not support 
to dynamically set it via a field like `SELECT JSON_EXIST(f1, f2)`?

##########
File path: flink-python/pyflink/table/expression.py
##########
@@ -1354,6 +1370,27 @@ def sha2(self, hash_length: Union[int, 
'Expression[int]']) -> 'Expression[str]':
         """
         return _binary_op("sha2")(self, hash_length)
 
+    # ---------------------------- JSON functions -----------------------------
+
+    def json_exists(self, path: str, on_error: JsonExistsOnError = None) -> 
'Expression[bool]':
+        """
+        Determines whether a JSON string satisfies a given search criterion.
+
+        This follows the ISO/IEC TR 19075-6 specification for JSON support in 
SQL.
+
+        Example:
+        ::
+
+            >>> col('f0').json_exists('lax $.a.b[0].c')
+
+        Known Limitations: Lax mode currently does not unnest arrays 
automatically. This is an

Review comment:
       nit: I think we can remove the limitations from the JavaDocs and only 
mention them in the website docs

##########
File path: tools/maven/suppressions.xml
##########
@@ -47,7 +47,7 @@ under the License.
 
            <!-- Have to use guava directly -->
            <suppress
-                       files="OverConvertRule.java|CustomizedConvertRule.java"
+                       files="OverConvertRule.java|InConverter.java"

Review comment:
       we can also disable the check locally without bothering the rest of the 
project via something like...
   ```
       // CHECKSTYLE.OFF: MethodName
       public static ApiExpression $(String name) {
           return new ApiExpression(unresolvedRef(name));
       }
       // CHECKSTYLE.ON: MethodName
   ```

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinition.java
##########
@@ -195,16 +195,31 @@ public Builder notDeterministic() {
             return this;
         }
 
+        /**
+         * Specifies that this {@link BuiltInFunctionDefinition} is 
implemented during code
+         * generation.
+         */
         public Builder runtimeProvided() {
             this.isRuntimeProvided = true;
             return this;
         }
 
+        /** Specifies the runtime class implementing this {@link 
BuiltInFunctionDefinition}. */
         public Builder runtimeClass(String runtimeClass) {
             this.runtimeClass = runtimeClass;
             return this;
         }
 
+        /**
+         * Specifies that this {@link BuiltInFunctionDefinition} will be 
mapped to a Calcite
+         * function.
+         */
+        public Builder runtimeDeferred() {

Review comment:
       nit: `runtimeTransformed` or `runtimeConverted`?

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/JsonExistsOnError.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.expressions;

Review comment:
       move to API package?

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
##########
@@ -1477,6 +1477,31 @@
                     .outputTypeStrategy(TypeStrategies.MISSING)
                     .build();
 
+    // 
--------------------------------------------------------------------------------------------
+    // JSON functions
+    // 
--------------------------------------------------------------------------------------------
+
+    public static final BuiltInFunctionDefinition JSON_EXISTS =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("JSON_EXISTS")
+                    .kind(SCALAR)
+                    .inputTypeStrategy(
+                            or(
+                                    sequence(
+                                            
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                            and(
+                                                    
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                                    LITERAL)),
+                                    sequence(
+                                            
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                            and(
+                                                    
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                                    LITERAL),
+                                            logical(LogicalTypeRoot.SYMBOL))))
+                    
.outputTypeStrategy(nullable(explicit(DataTypes.BOOLEAN())))

Review comment:
       this is not correct, even when both args are NOT NULL the function can 
still return NULL depending on the JsonExistsOnError strategy

##########
File path: docs/data/sql_functions.yml
##########
@@ -571,6 +571,22 @@ collection:
     table: MAP.at(ANY)
     description: Returns the value specified by key value in map.
 
+json:
+  - sql: JSON_EXISTS(jsonValue, path [ { TRUE | FALSE | UNKNOWN | ERROR } ON 
ERROR ])
+    table: STRING.jsonExists(STRING path [, JsonExistsOnError onError])
+    description: |
+      Determines whether a JSON string satisfies a given path search criterion.
+
+      This follows the ISO/IEC TR 19075-6 specification for JSON support in 
SQL.
+
+      ```
+      SELECT JSON_EXISTS(

Review comment:
       One question that arises for me seeing this the first time? Does return 
true only if there is a value like string or is `$.items` also fine in your 
example?

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
##########
@@ -1477,6 +1477,31 @@
                     .outputTypeStrategy(TypeStrategies.MISSING)
                     .build();
 
+    // 
--------------------------------------------------------------------------------------------
+    // JSON functions
+    // 
--------------------------------------------------------------------------------------------
+
+    public static final BuiltInFunctionDefinition JSON_EXISTS =
+            BuiltInFunctionDefinition.newBuilder()
+                    .name("JSON_EXISTS")
+                    .kind(SCALAR)
+                    .inputTypeStrategy(
+                            or(
+                                    sequence(
+                                            
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                            and(
+                                                    
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                                    LITERAL)),
+                                    sequence(
+                                            
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                            and(
+                                                    
logical(LogicalTypeFamily.CHARACTER_STRING),
+                                                    LITERAL),
+                                            logical(LogicalTypeRoot.SYMBOL))))

Review comment:
       Can we add a special symbol strategy to be more specific which kind of 
symbol we expect? e.g. `symbol(JsonExistsOnError.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: [email protected]

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


Reply via email to