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/05 01:29:14 UTC

[GitHub] [incubator-pinot] Jackie-Jiang opened a new pull request #6877: Enhance JSON index to support nested array

Jackie-Jiang opened a new pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877


   ## Description
   Enhance JSON index to support top-level value, top-level array, and nested array (multi-dimensional array)
   Change JSON_MATCH filter expression to be JSONPath compatible (e.g. `'"$.a[*].b.c[0]"=123'`)
   Other misc fixes including `null` and empty array/object handling
   
   In order to support nested array:
    - Json index format is changed and a new version (2) is added. Version 1 is deprecated, but still supported. New generated json index will be of version 2.
    - Array access must be explicit. In order to match any element in an array, `[*]` must be added to the filter, e.g. `'"$.array[*]"=''foo'''`.
   
   ## Upgrade Notes
   Please upgrade the servers before uploading the segments with the new index.
   
   ## Release Notes
   With the new JSON index, array wildcard access must be explicit
   


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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r628388394



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/json/BaseJsonIndexCreator.java
##########
@@ -130,7 +132,7 @@ void addToPostingList(String value) {
   void generateIndexFile()
       throws IOException {
     ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_LENGTH);
-    headerBuffer.putInt(VERSION);
+    headerBuffer.putInt(VERSION_2);

Review comment:
       No, v1 and v2 have different format, and the handling logic is also slightly different (notice the delimiter for array access). That's why we need to maintain the version info so that the new query can work on both v1 and v2 segments




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


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

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r628472654



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonMatchQueriesTest.java
##########
@@ -0,0 +1,199 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for JSON_MATCH predicate.
+ */
+public class JsonMatchQueriesTest extends BaseQueriesTest {

Review comment:
       Can we rename this test file to `JsonMatchV2QueriesTest` and rename the existing `JsonMatchPredicateTest` to `JsonMatchV1QueriesTest` so that the scope of these tests is clear with respect to different versions of JsonMatch?




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r630436915



##########
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:
       I see. Yes, this is a good optimization, especially for `IN` and `NOT IN` predicate where the key is the same and only the value changes.
   Since this is out of the scope of this PR, I added a TODO in the code and we can address it in a separate PR.
   




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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6877: Enhance JSON index to support nested array

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#issuecomment-832402698


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#6877](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cbaaf6f) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/bb2519494a15a3125dd509286d60ffd124f91c37?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (bb25194) will **decrease** coverage by `8.04%`.
   > The diff coverage is `68.25%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6877/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6877      +/-   ##
   ============================================
   - Coverage     73.55%   65.51%   -8.05%     
     Complexity       12       12              
   ============================================
     Files          1423     1423              
     Lines         70029    70046      +17     
     Branches      10119    10128       +9     
   ============================================
   - Hits          51509    45888    -5621     
   - Misses        15103    20881    +5778     
   + Partials       3417     3277     -140     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | integration | `?` | `?` | |
   | unittests | `65.51% <68.25%> (+0.01%)` | `12.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...t/index/readers/json/ImmutableJsonIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvanNvbi9JbW11dGFibGVKc29uSW5kZXhSZWFkZXIuamF2YQ==) | `75.36% <44.44%> (-9.86%)` | `0.00 <0.00> (ø)` | |
   | [...ent/local/realtime/impl/json/MutableJsonIndex.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2pzb24vTXV0YWJsZUpzb25JbmRleC5qYXZh) | `85.82% <63.15%> (+0.11%)` | `0.00 <0.00> (ø)` | |
   | [...ain/java/org/apache/pinot/spi/utils/JsonUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvSnNvblV0aWxzLmphdmE=) | `66.66% <86.66%> (-2.87%)` | `0.00 <0.00> (ø)` | |
   | [...nt/creator/impl/inv/json/BaseJsonIndexCreator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9pbnYvanNvbi9CYXNlSnNvbkluZGV4Q3JlYXRvci5qYXZh) | `92.42% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...a/org/apache/pinot/minion/metrics/MinionMeter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [.../apache/pinot/minion/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [...t/core/startree/plan/StarTreeDocIdSetPlanNode.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9wbGFuL1N0YXJUcmVlRG9jSWRTZXRQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | ... and [342 more](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [bb25194...cbaaf6f](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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


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

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r626970042



##########
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'

Review comment:
       I am wondering if the `..` usage conflict with the JsonPath deep scan operator `..`? (example: JsonPath expression `$..foo..bar`, will find all keys `bar` that appear anywhere within key `foo` that in turn may appear any where in the document). I don't think we need to support JsonPath deep scan operator `..` for now, but just want to make sure it won't lead to issues later on.




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r628619787



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonMatchQueriesTest.java
##########
@@ -0,0 +1,199 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for JSON_MATCH predicate.
+ */
+public class JsonMatchQueriesTest extends BaseQueriesTest {

Review comment:
       In that case, I would recommend directly modifying the `JsonMatchPredicateTest` to the new `JsonDataTypeTest`. We don't need to keep it because I already add the legacy queries test into the `JsonMatchPredicateTest`, so both versions query are covered in one test.




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


[GitHub] [incubator-pinot] codecov-commenter commented on pull request #6877: Enhance JSON index to support nested array

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#issuecomment-832402698


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#6877](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cce9b2c) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1c09b7866b4d1c8267847f464e9c4618787b5a15?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1c09b78) will **decrease** coverage by `0.00%`.
   > The diff coverage is `68.25%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6877/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6877      +/-   ##
   ============================================
   - Coverage     65.48%   65.48%   -0.01%     
     Complexity       12       12              
   ============================================
     Files          1421     1421              
     Lines         69980    69997      +17     
     Branches      10112    10119       +7     
   ============================================
   + Hits          45825    45835      +10     
   - Misses        20874    20878       +4     
   - Partials       3281     3284       +3     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | unittests | `65.48% <68.25%> (-0.01%)` | `12.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...t/index/readers/json/ImmutableJsonIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvanNvbi9JbW11dGFibGVKc29uSW5kZXhSZWFkZXIuamF2YQ==) | `75.36% <44.44%> (-9.86%)` | `0.00 <0.00> (ø)` | |
   | [...ent/local/realtime/impl/json/MutableJsonIndex.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2pzb24vTXV0YWJsZUpzb25JbmRleC5qYXZh) | `85.82% <63.15%> (+0.11%)` | `0.00 <0.00> (ø)` | |
   | [...ain/java/org/apache/pinot/spi/utils/JsonUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvSnNvblV0aWxzLmphdmE=) | `66.66% <86.66%> (-2.87%)` | `0.00 <0.00> (ø)` | |
   | [...nt/creator/impl/inv/json/BaseJsonIndexCreator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9pbnYvanNvbi9CYXNlSnNvbkluZGV4Q3JlYXRvci5qYXZh) | `92.42% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...pache/pinot/core/util/SortedRangeIntersection.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL1NvcnRlZFJhbmdlSW50ZXJzZWN0aW9uLmphdmE=) | `80.88% <0.00%> (-7.36%)` | `0.00% <0.00%> (ø%)` | |
   | [...rg/apache/pinot/broker/routing/RoutingManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9Sb3V0aW5nTWFuYWdlci5qYXZh) | `71.94% <0.00%> (-0.72%)` | `0.00% <0.00%> (ø%)` | |
   | [...lix/core/realtime/PinotRealtimeSegmentManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JlYWx0aW1lL1Bpbm90UmVhbHRpbWVTZWdtZW50TWFuYWdlci5qYXZh) | `79.48% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [.../controller/helix/core/SegmentDeletionManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL1NlZ21lbnREZWxldGlvbk1hbmFnZXIuamF2YQ==) | `74.59% <0.00%> (+0.81%)` | `0.00% <0.00%> (ø%)` | |
   | [.../java/org/apache/pinot/spi/data/TimeFieldSpec.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9UaW1lRmllbGRTcGVjLmphdmE=) | `90.90% <0.00%> (+2.27%)` | `0.00% <0.00%> (ø%)` | |
   | ... and [4 more](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [1c09b78...cce9b2c](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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


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

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r630096361



##########
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:
       I wouldn't suggest making the tuple type mutable, no. The `String` class caches its own hash code so there is no cache to maintain, so no added complexity. I typically observe an integer multiple reduction in latency of hash map lookups when making this transformation, because it makes the lookup genuinely O(1) rather than O(k) where k is the length of the concatenated string. As mentioned, this would also reduce allocation per lookup.
   
   I agree this is out of the scope of the PR since it is an existing pattern, and that there is no profiling data to refer to in order to motivate the change, moreover, the cost of the lookup will be dwarfed by the subsequent bitmap intersection so you might not see it with a sampling profiler, but this looked like an efficiency smell when I was browsing this PR.
   
   




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r629778303



##########
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:
       Are you suggesting storing tuples like `[leftPart, JsonUtils.ARRAY_INDEX_KEY, BaseJsonIndexCreator.KEY_VALUE_SEPARATOR, arrayIndex]` and cache the hashcode of each part?
   I can see that it might be able to reuse some strings and reduce some hashcode calculation, but it also has the following potential fallbacks:
   - The tuple will be mutable, so maintaining the hashcode cache won't be easy. The extra logic of maintaining the cache might bring other overhead
   - The tuple comparison logic won't be very straight forward. It might have to end up concatenate the strings in the tuple, or split the query string in order to compare them, which eliminate the benefit of the tuple
   - The code won't be as readable as the string concatenation
   
   This optimization is out of the scope of this PR. We can have a separate PR implementing this optimization if this part of the code become the hotspot




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r627883138



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/json/BaseJsonIndexCreator.java
##########
@@ -130,7 +132,7 @@ void addToPostingList(String value) {
   void generateIndexFile()
       throws IOException {
     ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_LENGTH);
-    headerBuffer.putInt(VERSION);
+    headerBuffer.putInt(VERSION_2);

Review comment:
       The `JsonUtils.flatten()` logic changes, thus the storage format is different




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


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

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r627855461



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/json/BaseJsonIndexCreator.java
##########
@@ -130,7 +132,7 @@ void addToPostingList(String value) {
   void generateIndexFile()
       throws IOException {
     ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_LENGTH);
-    headerBuffer.putInt(VERSION);
+    headerBuffer.putInt(VERSION_2);

Review comment:
       why do we need to bump the version when the index generation logic does not change?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/json/ImmutableJsonIndexReader.java
##########
@@ -151,43 +152,91 @@ private MutableRoaringBitmap getMatchingFlattenedDocIds(FilterContext filter) {
    */
   private MutableRoaringBitmap 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();
 
-    // 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'
     MutableRoaringBitmap matchingDocIds = null;
-    int leftBracketIndex;
-    while ((leftBracketIndex = key.indexOf('[')) > 0) {
-      int rightBracketIndex = key.indexOf(']');
-      Preconditions.checkState(rightBracketIndex > leftBracketIndex, "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);
+    if (_version == BaseJsonIndexCreator.VERSION_2) {

Review comment:
       can we make this an opt at query level, but not at index level?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/json/ImmutableJsonIndexReader.java
##########
@@ -50,15 +50,16 @@
 public class ImmutableJsonIndexReader implements JsonIndexReader {
   // NOTE: Use long type for _numDocs to comply with the RoaringBitmap APIs.
   private final long _numDocs;
+  private final int _version;
   private final StringDictionary _dictionary;
   private final BitmapInvertedIndexReader _invertedIndex;
   private final PinotDataBuffer _docIdMapping;
 
   public ImmutableJsonIndexReader(PinotDataBuffer dataBuffer, int numDocs) {
     _numDocs = numDocs;
-
-    int version = dataBuffer.getInt(0);
-    Preconditions.checkState(version == BaseJsonIndexCreator.VERSION, "Unsupported json index version: %s", version);
+    _version = dataBuffer.getInt(0);
+    Preconditions.checkState(_version == BaseJsonIndexCreator.VERSION_1 || _version == BaseJsonIndexCreator.VERSION_2,

Review comment:
       can this be extracted to server configurations?




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r628602361



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonMatchQueriesTest.java
##########
@@ -0,0 +1,199 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for JSON_MATCH predicate.
+ */
+public class JsonMatchQueriesTest extends BaseQueriesTest {

Review comment:
       All the queries from `JsonMatchPredicateTest` got included in `JsonDataTypeTest`, so the sole purpose of JsonMatchPredicateTest now is to ensure that JSON_MATCH V1 doesn't regress. Hence, my suggestion to rename it to JsonMatchV1QueriesTest and after that we don't need to maintain this test file. I will modify `JsonDataTypeTest` to use V2 JSON_MATCH and add a few more tests and it will continue to test json_extract_scalar with JSON data type. While we deprecate JsonMatchV1QueriesTest along with JSON_MATCH V1, the new test cases will be able to validate all the new json functionality.




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r627883425



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/json/ImmutableJsonIndexReader.java
##########
@@ -50,15 +50,16 @@
 public class ImmutableJsonIndexReader implements JsonIndexReader {
   // NOTE: Use long type for _numDocs to comply with the RoaringBitmap APIs.
   private final long _numDocs;
+  private final int _version;
   private final StringDictionary _dictionary;
   private final BitmapInvertedIndexReader _invertedIndex;
   private final PinotDataBuffer _docIdMapping;
 
   public ImmutableJsonIndexReader(PinotDataBuffer dataBuffer, int numDocs) {
     _numDocs = numDocs;
-
-    int version = dataBuffer.getInt(0);
-    Preconditions.checkState(version == BaseJsonIndexCreator.VERSION, "Unsupported json index version: %s", version);
+    _version = dataBuffer.getInt(0);
+    Preconditions.checkState(_version == BaseJsonIndexCreator.VERSION_1 || _version == BaseJsonIndexCreator.VERSION_2,

Review comment:
       We want the reader to support both versions in case there are segments with different version




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r628572042



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonMatchQueriesTest.java
##########
@@ -0,0 +1,199 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for JSON_MATCH predicate.
+ */
+public class JsonMatchQueriesTest extends BaseQueriesTest {

Review comment:
       The current `JsonMatchPredicateTest` has several tests on `json_extract_scalar`, so I think it might be better to rename it to `JsonQueriesTest`?
   I'll add some V1 format queries into the new `JsonMatchQueriesTest` to ensure the backward compatibility




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


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

Posted by GitBox <gi...@apache.org>.
richardstartin commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r630097220



##########
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:
       (i.e. don't let this comment block merging the PR)




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r628572042



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonMatchQueriesTest.java
##########
@@ -0,0 +1,199 @@
+/**
+ * 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.pinot.queries;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Queries test for JSON_MATCH predicate.
+ */
+public class JsonMatchQueriesTest extends BaseQueriesTest {

Review comment:
       The current `JsonMatchPredicateTest` has several tests on `json_extract_scalar`, and the focus is on value type, so I think it might be better to rename it to `JsonValueTypeQueriesTest`?
   I'll add some V1 format queries into the new `JsonMatchQueriesTest` to ensure the backward compatibility




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


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #6877: Enhance JSON index to support nested array

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877


   


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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6877: Enhance JSON index to support nested array

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#issuecomment-832402698


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#6877](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (63b308b) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/bb2519494a15a3125dd509286d60ffd124f91c37?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (bb25194) will **decrease** coverage by `8.07%`.
   > The diff coverage is `68.25%`.
   
   > :exclamation: Current head 63b308b differs from pull request most recent head 99987af. Consider uploading reports for the commit 99987af to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6877/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6877      +/-   ##
   ============================================
   - Coverage     73.55%   65.47%   -8.08%     
     Complexity       12       12              
   ============================================
     Files          1423     1423              
     Lines         70029    70046      +17     
     Branches      10119    10128       +9     
   ============================================
   - Hits          51509    45864    -5645     
   - Misses        15103    20897    +5794     
   + Partials       3417     3285     -132     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | integration | `?` | `?` | |
   | unittests | `65.47% <68.25%> (-0.03%)` | `12.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...t/index/readers/json/ImmutableJsonIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvanNvbi9JbW11dGFibGVKc29uSW5kZXhSZWFkZXIuamF2YQ==) | `75.36% <44.44%> (-9.86%)` | `0.00 <0.00> (ø)` | |
   | [...ent/local/realtime/impl/json/MutableJsonIndex.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2pzb24vTXV0YWJsZUpzb25JbmRleC5qYXZh) | `85.82% <63.15%> (+0.11%)` | `0.00 <0.00> (ø)` | |
   | [...ain/java/org/apache/pinot/spi/utils/JsonUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvSnNvblV0aWxzLmphdmE=) | `66.66% <86.66%> (-2.87%)` | `0.00 <0.00> (ø)` | |
   | [...nt/creator/impl/inv/json/BaseJsonIndexCreator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9pbnYvanNvbi9CYXNlSnNvbkluZGV4Q3JlYXRvci5qYXZh) | `92.42% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...a/org/apache/pinot/minion/metrics/MinionMeter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [.../apache/pinot/minion/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [...pache/pinot/common/utils/grpc/GrpcQueryClient.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvZ3JwYy9HcnBjUXVlcnlDbGllbnQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | [...t/core/startree/plan/StarTreeDocIdSetPlanNode.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS9wbGFuL1N0YXJUcmVlRG9jSWRTZXRQbGFuTm9kZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (ø%)` | |
   | ... and [343 more](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [2d86114...99987af](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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


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

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r627921488



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/json/BaseJsonIndexCreator.java
##########
@@ -130,7 +132,7 @@ void addToPostingList(String value) {
   void generateIndexFile()
       throws IOException {
     ByteBuffer headerBuffer = ByteBuffer.allocate(HEADER_LENGTH);
-    headerBuffer.putInt(VERSION);
+    headerBuffer.putInt(VERSION_2);

Review comment:
       Don't fully get it. Is my understanding right that V1 is a special case of V2, and therefore the handling logic is subset of V2's?




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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6877: Enhance JSON index to support nested array

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#issuecomment-832402698


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#6877](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cce9b2c) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/1c09b7866b4d1c8267847f464e9c4618787b5a15?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1c09b78) will **increase** coverage by `8.08%`.
   > The diff coverage is `68.25%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6877/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6877      +/-   ##
   ============================================
   + Coverage     65.48%   73.56%   +8.08%     
     Complexity       12       12              
   ============================================
     Files          1421     1421              
     Lines         69980    69997      +17     
     Branches      10112    10119       +7     
   ============================================
   + Hits          45825    51492    +5667     
   + Misses        20874    15085    -5789     
   - Partials       3281     3420     +139     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | integration | `43.34% <0.00%> (?)` | `7.00 <0.00> (?)` | |
   | unittests | `65.48% <68.25%> (-0.01%)` | `12.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...t/index/readers/json/ImmutableJsonIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvanNvbi9JbW11dGFibGVKc29uSW5kZXhSZWFkZXIuamF2YQ==) | `75.36% <44.44%> (-9.86%)` | `0.00 <0.00> (ø)` | |
   | [...ent/local/realtime/impl/json/MutableJsonIndex.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2pzb24vTXV0YWJsZUpzb25JbmRleC5qYXZh) | `85.82% <63.15%> (+0.11%)` | `0.00 <0.00> (ø)` | |
   | [...ain/java/org/apache/pinot/spi/utils/JsonUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvSnNvblV0aWxzLmphdmE=) | `66.66% <86.66%> (-2.87%)` | `0.00 <0.00> (ø)` | |
   | [...nt/creator/impl/inv/json/BaseJsonIndexCreator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9pbnYvanNvbi9CYXNlSnNvbkluZGV4Q3JlYXRvci5qYXZh) | `92.42% <100.00%> (ø)` | `0.00 <0.00> (ø)` | |
   | [...ava/org/apache/pinot/core/transport/TlsConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS90cmFuc3BvcnQvVGxzQ29uZmlnLmphdmE=) | `94.11% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...org/apache/pinot/core/auth/BasicAuthPrincipal.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9hdXRoL0Jhc2ljQXV0aFByaW5jaXBhbC5qYXZh) | `80.00% <0.00%> (ø)` | `0.00% <0.00%> (ø%)` | |
   | [...ment/creator/impl/SegmentColumnarIndexCreator.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9TZWdtZW50Q29sdW1uYXJJbmRleENyZWF0b3IuamF2YQ==) | `86.70% <0.00%> (+0.55%)` | `0.00% <0.00%> (ø%)` | |
   | [.../function/DistinctCountHLLAggregationFunction.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9EaXN0aW5jdENvdW50SExMQWdncmVnYXRpb25GdW5jdGlvbi5qYXZh) | `44.91% <0.00%> (+0.59%)` | `0.00% <0.00%> (ø%)` | |
   | [...pql/parsers/PinotQuery2BrokerRequestConverter.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wcWwvcGFyc2Vycy9QaW5vdFF1ZXJ5MkJyb2tlclJlcXVlc3RDb252ZXJ0ZXIuamF2YQ==) | `90.90% <0.00%> (+0.69%)` | `0.00% <0.00%> (ø%)` | |
   | [...java/org/apache/pinot/common/utils/DataSchema.java](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvRGF0YVNjaGVtYS5qYXZh) | `75.00% <0.00%> (+0.87%)` | `0.00% <0.00%> (ø%)` | |
   | ... and [340 more](https://codecov.io/gh/apache/incubator-pinot/pull/6877/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [1c09b78...cce9b2c](https://codecov.io/gh/apache/incubator-pinot/pull/6877?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r627782784



##########
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'

Review comment:
       Good question. Since we won't support JsonPath deep scan for now, users should not put expression `$..foo..bar` as the predicate expression in the `JSON_MATCH` expression. In the future if we do need to support deep scan (it will significantly increase the size of the json index as we need a separate entry for each combination), we can change the storage format and upgrade the version, similar to what I did in this PR.




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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6877:
URL: https://github.com/apache/incubator-pinot/pull/6877#discussion_r627883534



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/json/ImmutableJsonIndexReader.java
##########
@@ -151,43 +152,91 @@ private MutableRoaringBitmap getMatchingFlattenedDocIds(FilterContext filter) {
    */
   private MutableRoaringBitmap 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();
 
-    // 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'
     MutableRoaringBitmap matchingDocIds = null;
-    int leftBracketIndex;
-    while ((leftBracketIndex = key.indexOf('[')) > 0) {
-      int rightBracketIndex = key.indexOf(']');
-      Preconditions.checkState(rightBracketIndex > leftBracketIndex, "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);
+    if (_version == BaseJsonIndexCreator.VERSION_2) {

Review comment:
       Same here, we want to handle the case where both version segments exist




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