You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/11/10 22:37:02 UTC

[GitHub] [druid] wjhypo opened a new pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

wjhypo opened a new pull request #11307:
URL: https://github.com/apache/druid/pull/11307


   <!-- Thanks for trying to help us make Apache Druid be the best it can be! Please fill out as much of the following information as is possible (where relevant, and remove it when irrelevant) to help make the intention and scope of this PR clear in order to ease review. -->
   
   <!-- Please read the doc for contribution (https://github.com/apache/druid/blob/master/CONTRIBUTING.md) before making this PR. Also, once you open a PR, please _avoid using force pushes and rebasing_ since these make it difficult for reviewers to see what you've changed in response to their reviews. See [the 'If your pull request shows conflicts with master' section](https://github.com/apache/druid/blob/master/CONTRIBUTING.md#if-your-pull-request-shows-conflicts-with-master) for more details. -->
   
   Fixes #11301
   
   <!-- If you are a committer, follow the PR action item checklist for committers:
   https://github.com/apache/druid/blob/master/dev/committer-instructions.md#pr-and-issue-action-item-checklist-for-committers. -->
   
   ### Description
   
   
   
   
   
   <!-- Describe the goal of this PR, what problem are you fixing. If there is a corresponding issue (referenced above), it's not necessary to repeat the description here, however, you may choose to keep one summary sentence. -->
   
   <!-- Describe your patch: what did you change in code? How did you fix the problem? -->
   
   <!-- If there are several relatively logically separate changes in this PR, create a mini-section for each of them. For example: -->
   
   
   
   - Choice of algorithms
   
   If `enableInMemoryBitmap` is set to true in `tuningConfig` in the supervisor spec, a bitmap is maintained during the data is in incremental index for each dimension that uses bitmap in the schema, i.e., for a dimension, only if bitmap is to be created when creating the immutable segment during handoff, will a corresponding temporary in memory bitmap be created.
   
   This in-memory bitmap is not serialized or reused during persistence to completely separate from the other bitmap created in the immutable segment for clean logic separation.
   
   In terms of the implementation of this in-memory bitmap, currently user can't specify it anywhere and only Roaring bitmap is used as the default implementation. The PR doesn't add choices to pick other compressed bitmaps like Concise Bitmap because we don't see clear performance benefits to add other implementations at the moment.
   
   In terms of the implementation of Roaring bitmap, there are 3 implementations available in the Roaring bitmap library: `MutableRoaringBitmap`, `ImmutableRoaringBitmap` and `RoaringBitmap`. `RoaringBitmap` is supposed to be more performant than the other two in this case because it doesn't need to maintain a byte buffer backend and byte buffer backend is only convenient if we need to do Mmap or if there's serde involved, which is the case for batch ingested immutable segments querying. However, the implementation in the PR tries reusing the logic in the current Java classes built for the batch ingested immutable segments, so the byte buffer backed implementation `MutableRoaringBitmap` is used to store row indexes for each new rows. During query time, a snapshot of the bitmap is needed so `MutableRoaringBitmap` is cast to `ImmutableRoaringBitmap`, which involves copying at one of the stages of the conversion. During the copy, for high cardinality dimensions, the number of row indexe
 s for a given row value should be small so the copying overhead should be minimal; for low cardinality dimensions, supposedly there are more data to copy than the previous case, though we don't see obvious issue during testing and we can add metrics on the copying time and as long as the benefit outweigh the no bitmap full scan case, it should still be worthwhile.
   
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error conditions handled, such as when there are insufficient resources?
   
   Enable in memory bitmap construction in supervisor spec
   ```
   {
     "type": "kafka",
     "tuningConfig": {
       ...
       "enableInMemoryBitmap": true
       ...
     }
   }
   ```
   
    - Class organization and design (how the logic is split between classes, inheritance, composition, design patterns)
   
    The implementation tries piggybacking on the bitmap query related classes of batch generated immutable segment index (QueryableIndex and its support classes) as much as possible to reduce duplicate work.
   
   ### Benchmark
   
   #### Production Cluster Benchmark
   
   <img width="631" alt="Screen Shot 2021-08-24 at 5 10 07 PM" src="https://user-images.githubusercontent.com/8614743/130705911-fb6d749b-07f2-400b-ac5d-419209f164c4.png">
   Figure 1: Production cluster use case 1 P99 latency reduction on Middle Managers. The diagram shows Middle Manager P99 latency reduction (more than 30x) with the same middle manager capacity before and after enabling in memory bitmap for use case 1.
   
   <img width="446" alt="Screen Shot 2021-08-24 at 5 17 41 PM" src="https://user-images.githubusercontent.com/8614743/130706345-faaf17d6-4fa8-42c9-8941-40f05e96ef5a.png">
   Figure 2: Production cluster use case 1 QPS on Middle Managers. The diagram shows the corresponding increase of QPS throughput before and after enabling in-memory bitmap for use case 1 in Figure 1.
   
   <img width="642" alt="Screen Shot 2021-08-24 at 5 12 10 PM" src="https://user-images.githubusercontent.com/8614743/130705930-6a133036-ccc6-45fa-8d4b-db2d9e06846f.png">
   Figure 3: Production cluster use case 2 P90 latency (ms) and infrastructure cost reduction on Middle Managers. The diagram shows a 10x reduction in Middle Manager P90 latency before and after enabling in-memory bitmap for use case 2. The diagram also shows a follow-up 68.89% host capacity reduction without impacting latency (the green to red line switch).
   
   #### Unit Test Benchmark
   
   Additional unit test benchmark result is also included in the comments of this PR: https://github.com/apache/druid/pull/11307#issuecomment-859209115
   
   ### Correctness Validation
   We added a query context flag that controls the broker to only return results from middle manager hosts (not included in the PR). We added a temporary query time flag `useInMemoryBitmapInQuery` that can control whether to use the in-memory bitmap index during query time even though the in-memory bitmap index is already created during ingestion (this flag is removed in the PR created because it doesn't serve any purpose other than validation during development). We stopped the Kafka producer temporarily and left some records in memory in the incremental index (~852,000 records, when reaching 1M, persistence will trigger) before persistence triggered to verify that the results are the same when `useInMemoryBitmapInQuery` is set to true or false.
   
   <img width="371" alt="Screen Shot 2021-08-24 at 5 29 39 PM" src="https://user-images.githubusercontent.com/8614743/130707097-084c4dbf-775c-490f-9e3e-a8bb7b91424a.png">
   Figure 4: Number of events ingested on Middle Managers for verification data source. The diagram shows the number of events ingested dropped to 0 and remained 0 as we stopped the Kafka producer.
   
   <img width="356" alt="Screen Shot 2021-08-24 at 5 29 28 PM" src="https://user-images.githubusercontent.com/8614743/130707099-77a83632-568e-48aa-b1f2-c3e296f23ae3.png">
   Figure 5: Number of events in incremental index on Middle Managers for verification data source. The diagram shows the number of events in incremental index remained a little over 800k and temporarily not qualified for persistence threshold as we stopped the Kafka producer.
   
   We ran the follow queries (some sensitive business related column name and filter value in the queries before are replaced with dummy value but they were issued with real value during the verification).
   
   ```
   Query 1: aggregate sanity check. This query doesn't use in-memory bitmap in reality because no filter is there.
   SELECT __time, SUM(spend)/1000000.00 as spend,
   sum(paid_events) as paid_events,
   sum(cnt) as cnt, 
   sum(cnt_gross) as cnt_gross
   FROM verification_data_source
   WHERE __time >= timestamp'2021-07-20 20:00:00' 
   GROUP BY __time
   ORDER BY __time
   
   Query 2: filter with random value. Column `advertiserid` has in-memory bitmap index created.
   SELECT __time, SUM(spend)/1000000.00 as spend, sum(paid_events) as paid_events, sum(cnt) as cnt, sum(cnt_gross) as cnt_gross FROM verification_data_source WHERE advertiserid = '123' and __time >= timestamp'2021-07-20 20:00:00' group by __time
   
   Query 3: filter on one of top 10 large advertisers. Column `advertiserid` and column `apptype` have in-memory bitmap index created.
   SELECT __time, SUM(spend)/1000000.00 as spend, sum(paid_events) as paid_events, sum(cnt) as cnt, sum(cnt_gross) as cnt_gross FROM verification_data_source WHERE advertiserid = '123' and apptype = 3 and __time >= timestamp'2021-07-20 20:00:00' GROUP BY __time ORDER BY __time
   
   Query 4: filter on one of top 10 small advertiser. Column `advertiserid` and column `apptype` have in-memory bitmap index created.
   SELECT __time, SUM(spend)/1000000.00 as spend, sum(paid_events) as paid_events, sum(cnt) as cnt, sum(cnt_gross) as cnt_gross FROM verification_data_source WHERE advertiserid = '123' and __time >= timestamp'2021-07-20 20:00:00' GROUP BY __time ORDER BY __time
   ```
   We verified that non empty results are return and they are the same whether `useInMemoryBitmapInQuery` is set to true or false for all the above queries.
   
   <!--
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
    -->
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design (or naming) decision point and compare the alternatives with the designs that you've implemented (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), link to that discussion from this PR description and explain what have changed in your final design compared to your original proposal or the consensus version in the end of the discussion. If something hasn't changed since the original discussion, you can omit a detailed discussion of those aspects of the design here, perhaps apart from brief mentioning for the sake of readability of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. 
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `MyFoo`
    * `OurBar`
    * `TheirBaz`
   
   <hr>
   
   -->
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.
   


-- 
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@druid.apache.org

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



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


[GitHub] [druid] FrankChen021 commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-855248560


   BTW, do you have any performance report about:
   1. how much it costs if `enableInMemoryBitmap` is turned on ?
   2. how much it improves the performance if `enableInMemoryBitmap` is turned 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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] FrankChen021 commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-851200397


   What's the motivation that an extra parameter `useInMemoryBitmapInQuery` is needed in query context ? I mean, if `enableInMemoryBitmap` is turned on, what is the case that `useInMemoryBitmapInQuery` should be `false` ?


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] FrankChen021 commented on a change in pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r645451553



##########
File path: processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
##########
@@ -270,7 +270,8 @@ public VectorCursor makeVectorCursor(
       VirtualColumns virtualColumns,
       Granularity gran,
       boolean descending,
-      @Nullable QueryMetrics<?> queryMetrics
+      @Nullable QueryMetrics<?> queryMetrics,
+      boolean useInMemoryBitmapInQuery

Review comment:
       `useInMemoryBitmapInQuery` is not used inside this function. Is this the final version ? This new parameter introduces lots of changes in other files.




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on a change in pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r649641217



##########
File path: processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
##########
@@ -270,7 +270,8 @@ public VectorCursor makeVectorCursor(
       VirtualColumns virtualColumns,
       Granularity gran,
       boolean descending,
-      @Nullable QueryMetrics<?> queryMetrics
+      @Nullable QueryMetrics<?> queryMetrics,
+      boolean useInMemoryBitmapInQuery

Review comment:
       Removed `useInMemoryBitmapInQuery` from the code which removed about ~30 changed files.




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-859209115






-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-905052846


   Mark this PR as WIP to work on 1) enhance description 2) resolve recent conflicts to master branch 3) improve unit test coverage and documentation


-- 
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@druid.apache.org

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



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


[GitHub] [druid] wjhypo commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-905052161


   > when enable bitmap in IncrementalIndex during real time ingestion,have you considered adding vector query?
   
   Hi Mrxiashu, I have not visited the vector query part because the performance enhancement we got from enabling bitmap in incremental index was good enough for us at the time. However, vector query is a complementary optimization that we can further look into.
   


-- 
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@druid.apache.org

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



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


[GitHub] [druid] FrankChen021 commented on a change in pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r748805400



##########
File path: docs/configuration/index.md
##########
@@ -1403,6 +1403,7 @@ Additional peon configs include:
 |`druid.indexer.task.restoreTasksOnRestart`|If true, MiddleManagers will attempt to stop tasks gracefully on shutdown and restore them on restart.|false|
 |`druid.indexer.task.ignoreTimestampSpecForDruidInputSource`|If true, tasks using the [Druid input source](../ingestion/native-batch.md#druid-input-source) will ignore the provided timestampSpec, and will use the `__time` column of the input datasource. This option is provided for compatibility with ingestion specs written before Druid 0.22.0.|false|
 |`druid.indexer.server.maxChatRequests`|Maximum number of concurrent requests served by a task's chat handler. Set to 0 to disable limiting.|0|
+|`druid.indexer.task.enableInMemoryBitmap`| If true, stream ingestion will enable in memory bitmap for applicable dimensions when data is still in memory during real time writes before disk persistence triggers. Queries can leverage the bitmaps to avoid a full scan to speed up for this stage of data. |false|

Review comment:
       From implementation we can see that bitmap index are applied to dimensions with type of String and __time column, it's better to put this detail in the doc.




-- 
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@druid.apache.org

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



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


[GitHub] [druid] FrankChen021 commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-854530053


   @wjhypo That makes senses to me. 


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on a change in pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r645979759



##########
File path: processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
##########
@@ -270,7 +270,8 @@ public VectorCursor makeVectorCursor(
       VirtualColumns virtualColumns,
       Granularity gran,
       boolean descending,
-      @Nullable QueryMetrics<?> queryMetrics
+      @Nullable QueryMetrics<?> queryMetrics,
+      boolean useInMemoryBitmapInQuery

Review comment:
       Hi Frank, this case is caused by `IncrementalIndexStorageAdapter` and `QueryableIndexStorageAdapter` extending the same interface and lots of other classes implementing the same interface introduced changes accordingly as well. However, based on the previous discussion on the necessity of `useInMemoryBitmapInQuery`, this param probably can be omitted and just use `enableInMemoryBitmap` as the only switch, let me remove this param and simplify the code.




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-853323924


   > What's the motivation that an extra parameter `useInMemoryBitmapInQuery` is needed in query context ? I mean, if `enableInMemoryBitmap` is turned on, what is the case that `useInMemoryBitmapInQuery` should be `false` ?
   
   Hi Frank, good point. Once the feature is launched in production, I don't think we need to set `useInMemoryBitmapInQuery` to `false` again. The two knobs setting are more or less the remnant during our development to separate turning on writes and reads of this feature to be cautious. We first turned on the writes with the feature to monitor issues including task memory etc., then we found the system was stable and turned on reads with this feature to monitor potential query issues and the query performance and in case we found any issue, we would just update the query context which is faster than updating the supervisor spec in some way. On the other hand, assuming every user of this feature will verify this feature first in a dev cluster, we can also remove `useInMemoryBitmapInQuery` and leave just `enableInMemoryBitmap`, in which case even if we want to flip back to old behavior we can just update the supervisor spec again to set `enableInMemoryBitmap` to false, which is not mu
 ch slower than using the query context and simpler than having two confusing knobs now. I can leave `enableInMemoryBitmap` if that's more desirable?
   
   
   
   


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on a change in pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r752778734



##########
File path: docs/configuration/index.md
##########
@@ -1403,6 +1403,7 @@ Additional peon configs include:
 |`druid.indexer.task.restoreTasksOnRestart`|If true, MiddleManagers will attempt to stop tasks gracefully on shutdown and restore them on restart.|false|
 |`druid.indexer.task.ignoreTimestampSpecForDruidInputSource`|If true, tasks using the [Druid input source](../ingestion/native-batch.md#druid-input-source) will ignore the provided timestampSpec, and will use the `__time` column of the input datasource. This option is provided for compatibility with ingestion specs written before Druid 0.22.0.|false|
 |`druid.indexer.server.maxChatRequests`|Maximum number of concurrent requests served by a task's chat handler. Set to 0 to disable limiting.|0|
+|`druid.indexer.task.enableInMemoryBitmap`| If true, stream ingestion will enable in memory bitmap for applicable dimensions when data is still in memory during real time writes before disk persistence triggers. Queries can leverage the bitmaps to avoid a full scan to speed up for this stage of data. |false|

Review comment:
       In memory bitmap index is only applied to string type. There is only a wrapper without bitmap index to __time to make the query engine not break. I'll document this.
   

##########
File path: processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
##########
@@ -303,16 +322,48 @@ protected IncrementalIndex(
       DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(
           dimName,
           capabilities,
-          dimSchema.getMultiValueHandling()
+          dimSchema.getMultiValueHandling(),
+          enableInMemoryBitmap
       );
-      addNewDimension(dimName, handler);
+      DimensionDesc desc = addNewDimension(dimName, handler);
+
+      if (enableInMemoryBitmap && type.equals(ColumnType.STRING)) {

Review comment:
       Good catch!
   
   https://github.com/apache/druid/blob/master/docs/ingestion/ingestion-spec.md
   in dimensionSpec, createBitmapIndex (default to true)
   ```
   For string typed dimensions, whether or not bitmap indexes should be created for the column in generated segments. Creating a bitmap index requires more storage, but speeds up certain kinds of filtering (especially equality and prefix filtering). Only supported for string typed dimensions.	
   ```
   
   So it means currently bitmap index in batch immutable segments only support string type columns, in this PR I try to follow the same design by still only supporting string typed dimensions but extending the bitmap index support from batch immutable segments to real-time incremental index. In-memory bitmaps will be enabled if both `createBitmapIndex` and `enableInMemoryBitmap` are true. 
   
   Not sure if it makes sense to have any such case: `createBitmapIndex` is false (bitmap index is disabled in batch immutable segments) and `enableInMemoryBitmap` is true (bitmap index is enabled in incremental index), to avoid confusion, I avoided this case by disabling in-memory bitmap in this case. Let me know if you have other thoughts.
   
   I'll also document this in detail.




-- 
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@druid.apache.org

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



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


[GitHub] [druid] Mrxiashu commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
Mrxiashu commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-904365208


   when enable bitmap in IncrementalIndex during real time ingestion,have you considered adding vector query?


-- 
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@druid.apache.org

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



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


[GitHub] [druid] FrankChen021 commented on a change in pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r748805464



##########
File path: processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java
##########
@@ -57,16 +64,42 @@ private static String emptyToNullIfNeeded(@Nullable Object o)
     return o != null ? NullHandling.emptyToNullIfNeeded(o.toString()) : null;
   }
 
+  /**
+   * The same idea as {@link org.apache.druid.query.aggregation.datasketches.tuple.ArrayOfDoublesSketchBuildBufferAggregator#NUM_STRIPES}
+   *
+   * Locking per bitmap
+   */
+  private static final int NUM_STRIPES = 64;
+
   private final MultiValueHandling multiValueHandling;
   private final boolean hasBitmapIndexes;
   private final boolean hasSpatialIndexes;
   private volatile boolean hasMultipleValues = false;
 
-  public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes, boolean hasSpatialIndexes)
+  private final boolean enableInMemoryBitmap;
+  // Used only as a read lock to check if modification is needed for inMemoryBitmaps
+  private final ReentrantReadWriteLock inMemoryBitmapsReadLock;
+  // Used as a write lock to modify inMemoryBitmaps. Can't use inMemoryBitmapsReadLock because it's not easy to upgrade
+  // the same lock after holding a read lock first.
+  private final ReentrantLock inMemoryBitmapsWriteLock;
+  // Per bitmap lock
+  private final Striped<Lock> stripedLock;
+  private final List<MutableBitmap> inMemoryBitmaps;
+  private final BitmapFactory inMemoryBitmapFactory;
+
+  public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes,
+                                boolean hasSpatialIndexes, boolean enableInMemoryBitmap)
   {
     this.multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
     this.hasBitmapIndexes = hasBitmapIndexes;
     this.hasSpatialIndexes = hasSpatialIndexes;
+
+    this.enableInMemoryBitmap = enableInMemoryBitmap;
+    this.inMemoryBitmaps = new ArrayList<>();
+    this.inMemoryBitmapFactory = new RoaringBitmapFactory();
+    this.inMemoryBitmapsReadLock = new ReentrantReadWriteLock();
+    this.inMemoryBitmapsWriteLock = new ReentrantLock();
+    this.stripedLock = Striped.lock(NUM_STRIPES);

Review comment:
       If bitmap index is not enabled, do we need to initialize these variables?




-- 
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@druid.apache.org

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



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


[GitHub] [druid] FrankChen021 commented on a change in pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r748805230



##########
File path: processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java
##########
@@ -303,16 +322,48 @@ protected IncrementalIndex(
       DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(
           dimName,
           capabilities,
-          dimSchema.getMultiValueHandling()
+          dimSchema.getMultiValueHandling(),
+          enableInMemoryBitmap
       );
-      addNewDimension(dimName, handler);
+      DimensionDesc desc = addNewDimension(dimName, handler);
+
+      if (enableInMemoryBitmap && type.equals(ColumnType.STRING)) {

Review comment:
       Why only String dimension is supported?




-- 
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@druid.apache.org

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



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


[GitHub] [druid] wjhypo closed pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo closed pull request #11307:
URL: https://github.com/apache/druid/pull/11307


   


-- 
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@druid.apache.org

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



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


[GitHub] [druid] FrankChen021 commented on a change in pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r645997613



##########
File path: processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
##########
@@ -270,7 +270,8 @@ public VectorCursor makeVectorCursor(
       VirtualColumns virtualColumns,
       Granularity gran,
       boolean descending,
-      @Nullable QueryMetrics<?> queryMetrics
+      @Nullable QueryMetrics<?> queryMetrics,
+      boolean useInMemoryBitmapInQuery

Review comment:
       Understood.




-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] FrankChen021 commented on pull request #11307: Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
FrankChen021 commented on pull request #11307:
URL: https://github.com/apache/druid/pull/11307#issuecomment-855249506


   I also notice that `enableInMemoryBitmap` is designed in the supervisor spec, which means for each implementation of SeekableStreamXXX, they need to add this new switch. 
   
   Could this switch be a global switch so that every streaming indexing service don't care about it ? I don't have a strong opinion on it , I ask this question because I see there's a new streaming indexing service extension(#11240), if this switch is designed on supervisor spec, the new extension needs to make some change too.


-- 
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@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] wjhypo commented on a change in pull request #11307: [WIP] Add an option to enable bitmap in IncrementalIndex during real time ingestion

Posted by GitBox <gi...@apache.org>.
wjhypo commented on a change in pull request #11307:
URL: https://github.com/apache/druid/pull/11307#discussion_r752779114



##########
File path: processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java
##########
@@ -57,16 +64,42 @@ private static String emptyToNullIfNeeded(@Nullable Object o)
     return o != null ? NullHandling.emptyToNullIfNeeded(o.toString()) : null;
   }
 
+  /**
+   * The same idea as {@link org.apache.druid.query.aggregation.datasketches.tuple.ArrayOfDoublesSketchBuildBufferAggregator#NUM_STRIPES}
+   *
+   * Locking per bitmap
+   */
+  private static final int NUM_STRIPES = 64;
+
   private final MultiValueHandling multiValueHandling;
   private final boolean hasBitmapIndexes;
   private final boolean hasSpatialIndexes;
   private volatile boolean hasMultipleValues = false;
 
-  public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes, boolean hasSpatialIndexes)
+  private final boolean enableInMemoryBitmap;
+  // Used only as a read lock to check if modification is needed for inMemoryBitmaps
+  private final ReentrantReadWriteLock inMemoryBitmapsReadLock;
+  // Used as a write lock to modify inMemoryBitmaps. Can't use inMemoryBitmapsReadLock because it's not easy to upgrade
+  // the same lock after holding a read lock first.
+  private final ReentrantLock inMemoryBitmapsWriteLock;
+  // Per bitmap lock
+  private final Striped<Lock> stripedLock;
+  private final List<MutableBitmap> inMemoryBitmaps;
+  private final BitmapFactory inMemoryBitmapFactory;
+
+  public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes,
+                                boolean hasSpatialIndexes, boolean enableInMemoryBitmap)
   {
     this.multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling;
     this.hasBitmapIndexes = hasBitmapIndexes;
     this.hasSpatialIndexes = hasSpatialIndexes;
+
+    this.enableInMemoryBitmap = enableInMemoryBitmap;
+    this.inMemoryBitmaps = new ArrayList<>();
+    this.inMemoryBitmapFactory = new RoaringBitmapFactory();
+    this.inMemoryBitmapsReadLock = new ReentrantReadWriteLock();
+    this.inMemoryBitmapsWriteLock = new ReentrantLock();
+    this.stripedLock = Striped.lock(NUM_STRIPES);

Review comment:
       Good catch! I'll avoid the initialization behind a flag.




-- 
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@druid.apache.org

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



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