Hi Keith, thank you for the detailed feedback. 1. On motivation vs existing aggregation merge engine: The aggregation merge engine in 0.9 supports rbm32/rbm64 at the storage level, but BITMAP is not yet a first-class type in the DDL or type system. Users today must declare the column as BYTES (as shown in the 0.9 release example: uv_bitmap BYTES), and there are no SQL functions to build, merge, or query bitmaps from Flink SQL. This proposal adds the missing bridge: a proper BITMAP DDL type, SQL functions (BITMAP_BUILD, BITMAP_OR_AGG, BITMAP_CARDINALITY), and pushdown via applyAggregates(). The storage-side aggregation logic already exists; this proposal makes it accessible end-to-end.
2. On NULL semantics: BITMAP_OR(bitmap, NULL) returns NULL following standard SQL scalar function semantics where NULL inputs propagate to NULL outputs. BITMAP_OR_AGG follows aggregate function convention consistent with how SUM and AVG behave, where NULLs in individual rows are skipped and only a fully NULL input set returns NULL. This distinction follows FLIP-556 and StarRocks semantics. 3. On 32-bit scope: The proposal is scoped to 32-bit initially because RoaringBitmap32 covers integer values up to 2^32 (~4 billion), which is sufficient for most user ID and session ID use cases. RBM64 requires a fundamentally different internal structure; a map of RBM32 chunks which increases implementation and serialization complexity significantly. Starting with 32-bit keeps the initial scope focused and deliverable. RBM64 support is listed as a Could-Have in the MoSCoW deliverables and can follow in a subsequent iteration. Best regards, Prajwal Banakar On Wed, 11 Mar 2026 at 01:34, Keith Lee <[email protected]> wrote: > Hello Prajwal, > > Thank you for the detailed proposal. I enjoyed reading it and have a few > questions/comments. > > 1. On motivation, can you provide context on how this differs with > aggregation merge engine’s roaring bitmap implementation [1]? Specifically, > motivation part states that “users requiring high cardinality unique > counting … must execute client-side aggregation”. Aggregation merge engine > performs aggregation on server-side. The motivation section should clarify > how the proposed changes improve or complement aggregation merge engine, > which seems to have been considered as Section 2 references FIP-21 > Aggregation Merge Engine. Adding this context will help readers understand > the motivation of the proposal better. > > 2. Can you clarify the NULL semantics section specifically on the decision > on why BITMAP_OR(bitmap, NULL) returns NULL but BITMAP_OR_AGG only returns > null when all rows are NULL? > > 3. Why is the scope limited to 32 bit bitmaps? Adding the rationale behind > these e.g. how (if any) support of 64bit bitmaps would increase > implementation complexity. Articulating these may help other contributors > understand the complexity and perhaps come up with suggestions on how to > address them. > > Best regards > > Keith Lee > > [1] > > https://fluss.apache.org/blog/releases/0.9/#2-storage-level-processing--semantics > > > On Mon, 9 Mar 2026 at 05:31, Prajwal Banakar <[email protected]> > wrote: > > > Hi Devs, > > > > I have pushed a working prototype to my public fork demonstrating the > > BitmapType integrated with FieldRoaringBitmap32Agg. This includes four > > passing unit tests. > > > > The link to the prototype is available in the Google Doc, and you can > also > > find it here: > > https://github.com/Prajwal-banakar/fluss/tree/RoaringBitmap-prototype > > > > The Google Doc link remains the same. I look forward to your feedback. > > > > Best regards, > > > > Prajwal Banakar > > > > > > On Sun, 1 Mar, 2026, 11:49 am Prajwal Banakar, < > [email protected] > > > > > wrote: > > > > > Hi everyone, > > > > > > I would like to start a discussion on the proposal for Native Bitmap > > > Integration & Stateless Pushdown Aggregation. > > > > > > This proposal enables end-to-end native support for the BITMAP type in > > > Fluss and integrates it with the existing aggregation merge engine to > > > support server-side bitmap union pushdown. The goal is to reduce > network > > > transfer and offload DISTINCT-style aggregation from Flink to the > > > TabletServer. > > > > > > Key highlights of the proposal include: > > > > > > - Type System: Promoting BITMAP to a first-class logical type. > > > - UDF Suite: Introducing BITMAP_BUILD, BITMAP_OR_AGG, and > > > BITMAP_CARDINALITY (aligned with FLIP-556 and StarRocks semantics). > > > - Optimizer: Planner-based pushdown via applyAggregates in the Flink > > > connector. > > > - Safety: No changes to LogRecordBatch or WAL, making this strictly > > > additive and migration-free. > > > > > > You can find the full proposal document here: > > > > > > > > > https://docs.google.com/document/d/1sDhfkmo-w-UTvo2n3rsY1lytSSryswfkI83cSdka8s0/edit?usp=sharing > > > > > > I would appreciate feedback on the public interfaces, pushdown > > > constraints, and overall scope. > > > > > > Best regards, > > > Prajwal Banakar > > > > > >
