You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by tk...@apache.org on 2023/11/14 18:10:03 UTC

(phoenix) branch PHOENIX-628-feature updated: PHOENIX-7099 : Implement JSON_EXISTS function on json object (#1732)

This is an automated email from the ASF dual-hosted git repository.

tkhurana pushed a commit to branch PHOENIX-628-feature
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/PHOENIX-628-feature by this push:
     new 83798d12d7 PHOENIX-7099 : Implement JSON_EXISTS function on json object (#1732)
83798d12d7 is described below

commit 83798d12d71dbbd2c015e13589f808742322fca0
Author: RanganathG <ra...@gmail.com>
AuthorDate: Tue Nov 14 23:39:56 2023 +0530

    PHOENIX-7099 : Implement JSON_EXISTS function on json object (#1732)
    
    * PHOENIX-7099 : Implement JSON_EXISTS function on json object
    
    * Added ASF license
    Remove duplicate copy/paster
    
    * Added another test scenario
---
 .../phoenix/end2end/json/JsonFunctionsIT.java      |  68 ++++++++++++-
 .../it/resources/json/json_functions_basic.json    |   3 +-
 .../apache/phoenix/expression/ExpressionType.java  |   3 +-
 .../expression/function/JsonExistsFunction.java    | 106 +++++++++++++++++++++
 .../apache/phoenix/parse/JsonExistsParseNode.java  |  45 +++++++++
 .../apache/phoenix/util/json/BsonDataFormat.java   |  15 +++
 .../apache/phoenix/util/json/BsonJsonProvider.java |   5 +-
 .../apache/phoenix/util/json/JsonDataFormat.java   |   8 ++
 8 files changed, 247 insertions(+), 6 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/json/JsonFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/json/JsonFunctionsIT.java
index 822097de32..f01f278b58 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/json/JsonFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/json/JsonFunctionsIT.java
@@ -135,7 +135,7 @@ public class JsonFunctionsIT extends ParallelStatsDisabledIT {
             assertEquals("Manchester", rs.getString(2));
             assertEquals("alto1", rs.getString(3));
             assertEquals("[\"Sport\", \"alto1\", \"UpsertSelectVal\"]", rs.getString(4));
-            assertEquals("{\"type\": 1, \"address\": {\"town\": \"Manchester\", \"county\": \"Avon\", \"country\": \"England\"}, \"tags\": [\"Sport\", \"alto1\", \"UpsertSelectVal\"]}", rs.getString(5));
+            assertEquals("{\"type\": 1, \"address\": {\"town\": \"Manchester\", \"county\": \"Avon\", \"country\": \"England\", \"exists\": true}, \"tags\": [\"Sport\", \"alto1\", \"UpsertSelectVal\"]}", rs.getString(5));
             assertEquals("UpsertSelectVal", rs.getString(6));
 
             // Now check for empty match
@@ -344,6 +344,72 @@ public class JsonFunctionsIT extends ParallelStatsDisabledIT {
                 " (JSON_MODIFY(jsoncol, '$.info.tags[2]', '\"newValue\"')) include (col)");
     }
 
+    @Test
+    public void testJsonExists() throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String ddl = "create table " + tableName + " (pk integer primary key, col integer, jsoncol json)";
+            conn.createStatement().execute(ddl);
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES (?,?,?)");
+            stmt.setInt(1, 1);
+            stmt.setInt(2, 2);
+            stmt.setString(3, basicJson);
+            stmt.execute();
+            stmt.setInt(1, 2);
+            stmt.setInt(2, 3);
+            stmt.setString(3, getJsonString(BASIC_JSON, "$[1]"));
+            stmt.execute();
+            conn.commit();
+
+            String query ="SELECT JSON_VALUE(jsoncol, '$.type'), JSON_VALUE(jsoncol, '$.info.address.town') " +
+                    " FROM " + tableName +
+                    " WHERE JSON_EXISTS(jsoncol, '$.info.address.town')";
+            ResultSet rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("Basic", rs.getString(1));
+            assertEquals("Bristol", rs.getString(2));
+            assertTrue(rs.next());
+            assertEquals("Normal", rs.getString(1));
+            assertEquals("Bristol2", rs.getString(2));
+            assertFalse(rs.next());
+
+            query ="SELECT JSON_VALUE(jsoncol, '$.type'), JSON_VALUE(jsoncol, '$.info.address.town') " +
+                    " FROM " + tableName +
+                    " WHERE JSON_EXISTS(jsoncol, '$.info.address.exists')";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("Bristol", rs.getString(2));
+            assertFalse(rs.next());
+
+            query ="SELECT JSON_VALUE(jsoncol, '$.type'), JSON_VALUE(jsoncol, '$.info.address.town') " +
+                    " FROM " + tableName +
+                    " WHERE NOT JSON_EXISTS(jsoncol, '$.info.address.exists')";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("Bristol2", rs.getString(2));
+            assertFalse(rs.next());
+
+            query ="SELECT JSON_VALUE(jsoncol, '$.type'), JSON_VALUE(jsoncol, '$.info.address.town') " +
+                    " FROM " + tableName +
+                    " WHERE JSON_EXISTS(jsoncol, '$.info.address.name')";
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            query ="SELECT JSON_VALUE(jsoncol, '$.type'), JSON_VALUE(jsoncol, '$.info.address.town') " +
+                    " FROM " + tableName +
+                    " WHERE JSON_EXISTS(jsoncol, '$.existsFail')";
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            query ="SELECT JSON_VALUE(jsoncol, '$.type'), JSON_VALUE(jsoncol, '$.info.address.town') " +
+                    " FROM " + tableName +
+                    " WHERE JSON_EXISTS(jsoncol, '$.existsFail[*]')";
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+        }
+    }
+
     private void checkInvalidJsonIndexExpression(Properties props, String tableName,
             String indexName, String indexExpression) {
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
diff --git a/phoenix-core/src/it/resources/json/json_functions_basic.json b/phoenix-core/src/it/resources/json/json_functions_basic.json
index fc2050c823..6b44afe281 100644
--- a/phoenix-core/src/it/resources/json/json_functions_basic.json
+++ b/phoenix-core/src/it/resources/json/json_functions_basic.json
@@ -5,7 +5,8 @@
     "address": {
       "town": "Bristol",
       "county": "Avon",
-      "country": "England"
+      "country": "England",
+      "exists": true
     },
     "tags": [
       "Sport",
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index fb8a485ce7..956cd76490 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -195,7 +195,8 @@ public enum ExpressionType {
     PhoenixRowTimestampFunction(PhoenixRowTimestampFunction.class),
     JsonValueFunction(JsonValueFunction.class),
     JsonModifyFunction(JsonModifyFunction.class),
-    JsonQueryFunction(JsonQueryFunction.class)
+    JsonQueryFunction(JsonQueryFunction.class),
+    JsonExistsFunction(JsonExistsFunction.class)
     ;
 
     ExpressionType(Class<? extends Expression> clazz) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/JsonExistsFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/JsonExistsFunction.java
new file mode 100644
index 0000000000..997942c065
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/JsonExistsFunction.java
@@ -0,0 +1,106 @@
+/*
+ * 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.phoenix.expression.function;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.parse.JsonExistsParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions;
+import org.apache.phoenix.util.json.JsonDataFormat;
+import org.apache.phoenix.util.json.JsonDataFormatFactory;
+
+import java.util.List;
+
+/**
+ * Built-in function for JSON_EXISTS JSON_EXISTS(<column_with_json/json_string>, <path>) JSON_EXISTS
+ * determines whether a JSON value satisfies a search criterion.
+ */
+@FunctionParseNode.BuiltInFunction(name = JsonExistsFunction.NAME,
+        nodeClass = JsonExistsParseNode.class,
+        args = { @FunctionParseNode.Argument(allowedTypes = { PJson.class, PVarbinary.class }),
+                @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }) })
+public class JsonExistsFunction extends ScalarFunction {
+
+    public static final String NAME = "JSON_EXISTS";
+    private final JsonDataFormat
+            jsonDataFormat =
+            JsonDataFormatFactory.getJsonDataFormat(JsonDataFormatFactory.DataFormat.BSON);
+
+    // This is called from ExpressionType newInstance
+    public JsonExistsFunction() {
+
+    }
+
+    public JsonExistsFunction(List<Expression> children) {
+        super(children);
+        Preconditions.checkNotNull(getJSONPathExpr());
+    }
+
+    @Override
+    public String getName() {
+        return NAME;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        if (!getColValExpr().evaluate(tuple, ptr)) {
+            return false;
+        }
+        if (ptr == null || ptr.getLength() == 0) {
+            return true;
+        }
+
+        // Column name or JSON string
+        Object top = PJson.INSTANCE.toObject(ptr, getColValExpr().getSortOrder());
+
+        if (!getJSONPathExpr().evaluate(tuple, ptr)) {
+            return false;
+        }
+
+        if (ptr.getLength() == 0) {
+            return true;
+        }
+
+        String
+                jsonPathExprStr =
+                (String) PVarchar.INSTANCE.toObject(ptr, getJSONPathExpr().getSortOrder());
+        if (jsonPathExprStr == null) {
+            return true;
+        }
+
+        boolean isPathValid = jsonDataFormat.isPathValid(top, jsonPathExprStr);
+        ptr.set(PBoolean.INSTANCE.toBytes(isPathValid));
+        return true;
+    }
+
+    private Expression getColValExpr() {
+        return getChildren().get(0);
+    }
+
+    private Expression getJSONPathExpr() {
+        return getChildren().get(1);
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PBoolean.INSTANCE;
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/JsonExistsParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/JsonExistsParseNode.java
new file mode 100644
index 0000000000..ee10f97542
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/JsonExistsParseNode.java
@@ -0,0 +1,45 @@
+/*
+ * 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.phoenix.parse;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.FunctionExpression;
+import org.apache.phoenix.expression.function.JsonExistsFunction;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PJson;
+
+import java.sql.SQLException;
+import java.util.List;
+
+public class JsonExistsParseNode extends FunctionParseNode {
+
+    public JsonExistsParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public FunctionExpression create(List<Expression> children, StatementContext context)
+            throws SQLException {
+        PDataType dataType = children.get(0).getDataType();
+        if (!dataType.isCoercibleTo(PJson.INSTANCE)) {
+            throw new SQLException(dataType + " type is unsupported for JSON_EXISTS().");
+        }
+        return new JsonExistsFunction(children);
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonDataFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonDataFormat.java
index b114240f86..a25a2c3792 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonDataFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonDataFormat.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.util.json;
 import com.jayway.jsonpath.Configuration;
 import com.jayway.jsonpath.JsonPath;
 import com.jayway.jsonpath.Option;
+import com.jayway.jsonpath.PathNotFoundException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.bson.BsonBinaryReader;
 import org.bson.BsonDocument;
@@ -114,6 +115,20 @@ public class BsonDataFormat implements JsonDataFormat {
         return updated.getByteBuffer().asNIO();
     }
 
+    // Ref: https://github.com/json-path/JsonPath/pull/828
+    @Override
+    public boolean isPathValid(Object top, String path) {
+        try{
+            Configuration conf = Configuration.builder().jsonProvider(new BsonJsonProvider()).build();
+            BsonDocument root = fromRaw((RawBsonDocument) top);
+            JsonPath.using(conf).parse(root).read(path);
+            return true;
+        }
+        catch (PathNotFoundException e){
+            return false;
+        }
+    }
+
     private BsonValue getBsonValue(String jsonPathExprStr, RawBsonDocument top) {
         Configuration conf = getConfiguration();
         BsonValue value = JsonPath.using(conf).parse(top).read(jsonPathExprStr, BsonValue.class);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonJsonProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonJsonProvider.java
index 28bea22f21..c39e7c95fc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonJsonProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/json/BsonJsonProvider.java
@@ -112,12 +112,11 @@ public class BsonJsonProvider extends AbstractJsonProvider {
     public Object getMapValue(final Object obj, final String key) {
         BsonDocument bsonDocument = toBsonDocument(obj);
         Object o = bsonDocument.get(key);
-        /*if (!jsonObject.has(key)) {
+        if (!bsonDocument.containsKey(key)) {
             return UNDEFINED;
         } else {
             return unwrap(o);
-        }*/
-        return unwrap(o);
+        }
     }
 
     @Override
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonDataFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonDataFormat.java
index 79cf1ac069..0d1189151e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonDataFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonDataFormat.java
@@ -75,4 +75,12 @@ public interface JsonDataFormat {
      * @return
      */
     ByteBuffer updateValue(Object top, String jsonPathExprStr, String newVal);
+
+    /**
+     * Checks if the path is valid in a JSON document.
+     * @param top
+     * @param path
+     * @return
+     */
+    boolean isPathValid(Object top, String path);
 }
\ No newline at end of file