You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/05/10 21:06:26 UTC

[GitHub] [incubator-pinot] richardstartin commented on a change in pull request #6877: Enhance JSON index to support nested array

richardstartin commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r629683586



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/json/MutableJsonIndex.java
##########
@@ -186,43 +186,54 @@ private RoaringBitmap getMatchingFlattenedDocIds(FilterContext filter) {
    */
   private RoaringBitmap getMatchingFlattenedDocIds(Predicate predicate) {
     ExpressionContext lhs = predicate.getLhs();
-    Preconditions.checkState(lhs.getType() == ExpressionContext.Type.IDENTIFIER,
+    Preconditions.checkArgument(lhs.getType() == ExpressionContext.Type.IDENTIFIER,
         "Left-hand side of the predicate must be an identifier, got: %s (%s). Put double quotes around the identifier if needed.",
         lhs, lhs.getType());
     String key = lhs.getIdentifier();
 
+    // Support 2 formats:
+    // - JSONPath format (e.g. "$.a[1].b"='abc', "$[0]"=1, "$"='abc')
+    // - Legacy format (e.g. "a[1].b"='abc')
+    if (key.charAt(0) == '$') {
+      key = key.substring(1);
+    } else {
+      key = JsonUtils.KEY_SEPARATOR + key;
+    }
+
     // Process the array index within the key if exists
-    // E.g. "foo[0].bar[1].foobar"='abc' -> foo.$index=0 && foo.bar.$index=1 && foo.bar.foobar='abc'
+    // E.g. "[*]"=1 -> "."='1'
+    // E.g. "[0]"=1 -> ".$index"='0' && "."='1'
+    // E.g. "[0][1]"=1 -> ".$index"='0' && "..$index"='1' && ".."='1'
+    // E.g. ".foo[*].bar[*].foobar"='abc' -> ".foo..bar..foobar"='abc'
+    // E.g. ".foo[0].bar[1].foobar"='abc' -> ".foo.$index"='0' && ".foo..bar.$index"='1' && ".foo..bar..foobar"='abc'
+    // E.g. ".foo[0][1].bar"='abc' -> ".foo.$index"='0' && ".foo..$index"='1' && ".foo...bar"='abc'
     RoaringBitmap matchingDocIds = null;
     int leftBracketIndex;
-    while ((leftBracketIndex = key.indexOf('[')) > 0) {
-      int rightBracketIndex = key.indexOf(']');
-      Preconditions.checkState(rightBracketIndex > leftBracketIndex, "Missing right bracket in key: %s", key);
+    while ((leftBracketIndex = key.indexOf('[')) >= 0) {
+      int rightBracketIndex = key.indexOf(']', leftBracketIndex + 2);
+      Preconditions.checkArgument(rightBracketIndex > 0, "Missing right bracket in key: %s", key);
 
       String leftPart = key.substring(0, leftBracketIndex);
-      int arrayIndex;
-      try {
-        arrayIndex = Integer.parseInt(key.substring(leftBracketIndex + 1, rightBracketIndex));
-      } catch (Exception e) {
-        throw new IllegalStateException("Invalid key: " + key);
-      }
+      String arrayIndex = key.substring(leftBracketIndex + 1, rightBracketIndex);
       String rightPart = key.substring(rightBracketIndex + 1);
 
-      // foo[1].bar -> foo.$index=1
-      String searchKey =
-          leftPart + JsonUtils.KEY_SEPARATOR + JsonUtils.ARRAY_INDEX_KEY + BaseJsonIndexCreator.KEY_VALUE_SEPARATOR
-              + arrayIndex;
-      RoaringBitmap docIds = _postingListMap.get(searchKey);
-      if (docIds != null) {
-        if (matchingDocIds == null) {
-          matchingDocIds = docIds.clone();
+      if (!arrayIndex.equals(JsonUtils.WILDCARD)) {
+        // "[0]"=1 -> ".$index"='0' && "."='1'
+        // ".foo[1].bar"='abc' -> ".foo.$index"=1 && ".foo..bar"='abc'
+        String searchKey = leftPart + JsonUtils.ARRAY_INDEX_KEY + BaseJsonIndexCreator.KEY_VALUE_SEPARATOR + arrayIndex;
+        RoaringBitmap docIds = _postingListMap.get(searchKey);

Review comment:
       Can I suggest using a different type for the key of the `_postingListMap` here and change the key type to some kind of tuple? It won't make any difference in this particular case because the strings making up `searchKey` are all new strings, but down on L233/244 the input strings may not have been modified. The problem with concatenating strings to create a key for a `HashMap` lookup is that they may be already have their hashcode computed and cached, making the hashcode of the concatentation very cheap to compute (just some arithmetic with precomputed hashcodes), but creating a new `String` means the hash code will need to be computed again. 
   
   The allocation of the tuple will also be a lot smaller than constructing a new `String`, but with enough inlining the tuple will typically be scalarised, eliminating its allocation.




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org