You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "MeihanLi (via GitHub)" <gi...@apache.org> on 2023/04/26 22:30:07 UTC

[GitHub] [pinot] MeihanLi opened a new issue, #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

MeihanLi opened a new issue, #10691:
URL: https://github.com/apache/pinot/issues/10691

   We recently came into issues when creating star tree index for DISTINCTCOUNTBITMAP. BaseChunkSVForwardIndexWriter can not be initialized because the chunk size is larger than Integer.MAX_VALUE and this checking is introduced by #8286. And We are able to reproduce the issue on a table with only one small segment.
   
   Does anyone know why forward index are created (BaseSingleTreeBuilder: createForwardIndexes) in the process. And is this considered a bug/unexpected behavior or something else?
   
   
   Star-tree index config as below:
   ```
         "enableDefaultStarTree": false,
         "starTreeIndexConfigs": [
           {
             "dimensionsSplitOrder": [
               "time_bucket_derived",
               "time_bucket"
             ],
             "skipStarNodeCreationForDimensions": [],
             "functionColumnPairs": [
               "DISTINCTCOUNTBITMAP__active_riders"
             ],
             "maxLeafRecords": 0
           }
         ],
   ```
   
   columns metadata as below:
   ```
       {
         "name": "time_bucket",
         "dataType": "LONG"
       },
       {
         "name": "time_bucket_derived",
         "dataType": "LONG"
       },
       {
         "name": "active_riders",
         "dataType": "BYTES",
         "defaultNullValue": ""
       },
   
      Cardinality:
      time_bucket: 1440
      time_bucket_derived: 24
      Max length of active riders: 18360
   ```
   
   time_bucket_derived is configured as:
   ```
       "ingestionConfig": {
         "transformConfigs": [
           {
             "columnName": "time_bucket_derived",
             "transformFunction": "dateTimeConvert(time_bucket, '1:MILLISECONDS:EPOCH', '1:MILLISECONDS:EPOCH', '1:HOURS')"
           }
         ],
   ```
   
   
   
   
     "diskSizeInBytes": 1158298974,
     "numSegments": 1,
     "numRows": 3260450,
   
   
   
   Errors as below:
   
   ```
   2023/04/26 13:55:06.884 ERROR [HelixInstanceDataManager] [pool-46-thread-1] Caught exception while reloading segment: rta_fact_trip_with_ten_dimensions_performance_testing_1682294399_1682294399_00000 in table: rta_fact_trip_with_ten_dimensions_performance_testing_OFFLINE
   java.lang.IllegalArgumentException: chunk size limited to 2GB
   	at org.apache.pinot.shaded.com.google.common.base.Preconditions.checkArgument(Preconditions.java:122) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.io.writer.impl.BaseChunkSVForwardIndexWriter.<init>(BaseChunkSVForwardIndexWriter.java:77) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.io.writer.impl.VarByteChunkSVForwardIndexWriter.<init>(VarByteChunkSVForwardIndexWriter.java:81) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator.<init>(SingleValueVarByteRawIndexCreator.java:81) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator.<init>(SingleValueVarByteRawIndexCreator.java:59) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.BaseSingleTreeBuilder.createForwardIndexes(BaseSingleTreeBuilder.java:476) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.BaseSingleTreeBuilder.build(BaseSingleTreeBuilder.java:320) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.OffHeapSingleTreeBuilder.build(OffHeapSingleTreeBuilder.java:42) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.MultipleTreesBuilder.build(MultipleTreesBuilder.java:141) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.index.loader.SegmentPreProcessor.processStarTrees(SegmentPreProcessor.java:260) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.index.loader.SegmentPreProcessor.process(SegmentPreProcessor.java:135) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.preprocessSegment(ImmutableSegmentLoader.java:267) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.preprocess(ImmutableSegmentLoader.java:142) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.load(ImmutableSegmentLoader.java:109) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.load(ImmutableSegmentLoader.java:91) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.core.data.manager.BaseTableDataManager.reloadSegment(BaseTableDataManager.java:412) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.server.starter.helix.HelixInstanceDataManager.reloadSegmentWithMetadata(HelixInstanceDataManager.java:449) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.server.starter.helix.HelixInstanceDataManager.lambda$reloadSegmentsWithMetadata$6(HelixInstanceDataManager.java:376) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1736) [?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) [?:?]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) [?:?]
   	at java.lang.Thread.run(Thread.java:834) [?:?]
   2023/04/26 13:55:06.886 ERROR [HelixTask] [HelixTaskExecutor-message_handle_thread_89] Exception while executing a message. java.lang.RuntimeException: Caught exception while reloading segment:  in table: rta_fact_trip_with_ten_dimensions_performance_testing_OFFLINE msgId: 90e95b84-3f2c-4f36-944d-775ee6233bd6 type: USER_DEFINE_MSG
   java.lang.RuntimeException: Caught exception while reloading segment:  in table: rta_fact_trip_with_ten_dimensions_performance_testing_OFFLINE
   	at org.apache.pinot.server.starter.helix.SegmentMessageHandlerFactory$SegmentReloadMessageHandler.handleMessage(SegmentMessageHandlerFactory.java:155) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.helix.messaging.handling.HelixTask.call(HelixTask.java:97) [pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.helix.messaging.handling.HelixTask.call(HelixTask.java:49) [pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) [?:?]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) [?:?]
   	at java.lang.Thread.run(Thread.java:834) [?:?]
   Caused by: java.lang.RuntimeException: Failed to reload 1/1 segments: [rta_fact_trip_with_ten_dimensions_performance_testing_1682294399_1682294399_00000] in table: rta_fact_trip_with_ten_dimensions_performance_testing_OFFLINE
   	at org.apache.pinot.server.starter.helix.HelixInstanceDataManager.reloadSegmentsWithMetadata(HelixInstanceDataManager.java:392) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.server.starter.helix.HelixInstanceDataManager.reloadAllSegments(HelixInstanceDataManager.java:326) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.server.starter.helix.SegmentMessageHandlerFactory$SegmentReloadMessageHandler.handleMessage(SegmentMessageHandlerFactory.java:148) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	... 6 more
   Caused by: java.lang.IllegalArgumentException: chunk size limited to 2GB
   	at org.apache.pinot.shaded.com.google.common.base.Preconditions.checkArgument(Preconditions.java:122) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.io.writer.impl.BaseChunkSVForwardIndexWriter.<init>(BaseChunkSVForwardIndexWriter.java:77) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.io.writer.impl.VarByteChunkSVForwardIndexWriter.<init>(VarByteChunkSVForwardIndexWriter.java:81) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator.<init>(SingleValueVarByteRawIndexCreator.java:81) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator.<init>(SingleValueVarByteRawIndexCreator.java:59) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.BaseSingleTreeBuilder.createForwardIndexes(BaseSingleTreeBuilder.java:476) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.BaseSingleTreeBuilder.build(BaseSingleTreeBuilder.java:320) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.OffHeapSingleTreeBuilder.build(OffHeapSingleTreeBuilder.java:42) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.startree.v2.builder.MultipleTreesBuilder.build(MultipleTreesBuilder.java:141) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.index.loader.SegmentPreProcessor.processStarTrees(SegmentPreProcessor.java:260) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.segment.index.loader.SegmentPreProcessor.process(SegmentPreProcessor.java:135) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.preprocessSegment(ImmutableSegmentLoader.java:267) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.preprocess(ImmutableSegmentLoader.java:142) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.load(ImmutableSegmentLoader.java:109) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader.load(ImmutableSegmentLoader.java:91) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.core.data.manager.BaseTableDataManager.reloadSegment(BaseTableDataManager.java:412) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.server.starter.helix.HelixInstanceDataManager.reloadSegmentWithMetadata(HelixInstanceDataManager.java:449) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at org.apache.pinot.server.starter.helix.HelixInstanceDataManager.lambda$reloadSegmentsWithMetadata$6(HelixInstanceDataManager.java:376) ~[pinot-all-0.13.0-SNAPSHOT-jar-with-dependencies.jar:0.13.0-SNAPSHOT-2bde942a4cc5a6ddc45f497a674908da88bea41a]
   	at java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1736) ~[?:?]
   	... 3 more
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org.apache.org

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] [pinot] deemoliu commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "deemoliu (via GitHub)" <gi...@apache.org>.
deemoliu commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1679661856

   is this issue already been address by the following config?
   ```
       {
         "name": "columnName",
         ...
         "properties": {
           "rawIndexWriterVersion": 3
         }
       }
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] Jackie-Jiang commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1526272726

   It shouldn't be too much effort to make the version configurable


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] MeihanLi commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "MeihanLi (via GitHub)" <gi...@apache.org>.
MeihanLi commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1526306215

   Thanks, I will raise a pr to make it configurable. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] MeihanLi commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "MeihanLi (via GitHub)" <gi...@apache.org>.
MeihanLi commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1526067042

   Thanks Jackie. Yes, the largest entry size we saw is ~25M. How many efforts it will take to upgrade to v4 and deprecate the old 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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] Jackie-Jiang commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1524212793

   Is `active_riders` already serialized `RoaringBitmap`? 
   I think it throws exception because star-tree builder is still using the V2 raw forward index, where the chunk size is calculated based on the largest entry. If the largest bitmap size is larger than 2M, then the chunk size will be greater than 2GB.
   
   To fix this, we want to allow star-tree to switch to higher version of raw forward index. Further more, we might want to consider making V4 the default raw forward index version, and deprecate the old versions from the builder side.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [I] Unable to create Star-Tree index for DISTINCTCOUNTBITMAP [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1811921993

   #11744 adds the support for compression, which can unblock this issue. We may consider also adding the raw index version using similar way


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] MeihanLi commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "MeihanLi (via GitHub)" <gi...@apache.org>.
MeihanLi commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1524120107

   Can you help provide some insights here? cc @Jackie-Jiang @chenboat 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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] [pinot] deemoliu commented on issue #10691: Unable to create Star-Tree index for DISTINCTCOUNTBITMAP

Posted by "deemoliu (via GitHub)" <gi...@apache.org>.
deemoliu commented on issue #10691:
URL: https://github.com/apache/pinot/issues/10691#issuecomment-1679361112

   Hi @Jackie-Jiang and @MeihanLi 
   
   - is this version of raw forward index configured at cluster level, or it table level?
   - if we already have V2 raw forward index in a test table, will it be backward compatible to move to V4?
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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