This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git
The following commit(s) were added to refs/heads/master by this push:
new 97cb118890 [docs] Add rbm32/rbm64 function documentation (#6172)
97cb118890 is described below
commit 97cb1188903ae7bda6f88878829652a61a9fb616
Author: Arnav Balyan <[email protected]>
AuthorDate: Thu Aug 28 11:18:37 2025 +0530
[docs] Add rbm32/rbm64 function documentation (#6172)
---
.../primary-key-table/merge-engine/aggregation.md | 177 +++++++++++++++++++++
1 file changed, 177 insertions(+)
diff --git a/docs/content/primary-key-table/merge-engine/aggregation.md
b/docs/content/primary-key-table/merge-engine/aggregation.md
index c022978f11..92e0f79f64 100644
--- a/docs/content/primary-key-table/merge-engine/aggregation.md
+++ b/docs/content/primary-key-table/merge-engine/aggregation.md
@@ -121,10 +121,187 @@ Current supported aggregate functions and data types are:
The rbm32 function aggregates multiple serialized 32-bit RoaringBitmap into
a single RoaringBitmap.
It supports VARBINARY data type which must be serialized 32-bit
RoaringBitmap.
+ RoaringBitmap is a compressed bitmap that efficiently represents sets of
integers. The rbm32 aggregator is useful for scenarios where you need to merge
multiple bitmap sets or use case requiring set union operations on large
integer datasets.
+
+ **Example:**
+
+ ```sql
+ -- Create a table to store user visit data with bitmap aggregation
+ CREATE TABLE user_visits (
+ user_id INT PRIMARY KEY NOT ENFORCED,
+ visit_bitmap VARBINARY
+ ) WITH (
+ 'merge-engine' = 'aggregation',
+ 'fields.visit_bitmap.aggregate-function' = 'rbm32'
+ );
+
+ -- Register a UDF to create RoaringBitmap from user IDs
+ -- CREATE TEMPORARY FUNCTION TO_BITMAP AS 'BitmapUDF';
+
+ -- Insert data with bitmap representations
+ INSERT INTO user_visits VALUES
+ (1, TO_BITMAP(100, 101, 102)), -- User 1 visited pages 100, 101, 102
+ (2, TO_BITMAP(101, 103)), -- User 2 visited pages 101, 103
+ (3, TO_BITMAP(102, 104)); -- User 3 visited pages 102, 104
+
+ -- When the same user_id is inserted again, the bitmaps will be merged
+ INSERT INTO user_visits VALUES
+ (1, TO_BITMAP(103, 105)), -- User 1 also visited pages 103, 105
+ (2, TO_BITMAP(104, 106)); -- User 2 also visited pages 104, 106
+
+ -- The final result will have merged bitmaps for each user
+ -- User 1: pages 100, 101, 102, 103, 105
+ -- User 2: pages 101, 103, 104, 106
+ -- User 3: pages 102, 104
+ ```
+
### rbm64
The rbm64 function aggregates multiple serialized 64-bit Roaring64Bitmap
into a single Roaring64Bitmap.
It supports VARBINARY data type which must be serialized 64-bit
RoaringBitmap.
+ Similar to rbm32, but supports 64-bit integers, making it suitable for
scenarios with very large integer values
+ or when you need to represent sets with values beyond the 32-bit range (up
to 2^31-1).
+
+ **Example:**
+
+ ```sql
+ -- Create a table to store large-scale user interaction data
+ CREATE TABLE user_interactions (
+ session_id BIGINT PRIMARY KEY NOT ENFORCED,
+ interaction_bitmap VARBINARY
+ ) WITH (
+ 'merge-engine' = 'aggregation',
+ 'fields.interaction_bitmap.aggregate-function' = 'rbm64'
+ );
+
+ -- Register a UDF to create Roaring64Bitmap from large user IDs
+ -- CREATE TEMPORARY FUNCTION TO_BITMAP64 AS 'Bitmap64UDF';
+
+ -- Insert data with 64-bit bitmap representations
+ INSERT INTO user_interactions VALUES
+ (1001, TO_BITMAP64(1000000001L, 1000000002L, 1000000003L)),
+ (1002, TO_BITMAP64(1000000002L, 1000000004L)),
+ (1003, TO_BITMAP64(1000000003L, 1000000005L));
+
+ -- Merge additional interactions
+ INSERT INTO user_interactions VALUES
+ (1001, TO_BITMAP64(1000000004L, 1000000006L)),
+ (1002, TO_BITMAP64(1000000005L, 1000000007L));
+
+ -- The final result will have merged 64-bit bitmaps for each session
+ ```
+
+The `rbm32` and `rbm64` aggregators work by:
+1. Deserializing the input VARBINARY data into RoaringBitmap objects
+2. Performing bitwise OR operations to merge the bitmaps
+3. Serializing the result back to VARBINARY format
+
+**Working with RoaringBitmap Functions:**
+
+Paimon currently does not provide built-in Flink UDFs for bitmap creation. You
have two options:
+
+1. Create bitmaps programmatically: in your application code and insert
serialized bytes
+2. Create custom Flink UDFs: to convert raw integers to serialized bitmap
format
+
+Here are examples of both approaches:
+
+**Option 1: Programmatic Bitmap Creation (Java/Scala)**
+
+```java
+// Create bitmaps programmatically
+RoaringBitmap32 bitmap1 = RoaringBitmap32.bitmapOf(100, 101, 102);
+RoaringBitmap32 bitmap2 = RoaringBitmap32.bitmapOf(101, 103);
+RoaringBitmap32 bitmap3 = RoaringBitmap32.bitmapOf(102, 104);
+
+byte[] serialized1 = bitmap1.serialize();
+byte[] serialized2 = bitmap2.serialize();
+byte[] serialized3 = bitmap3.serialize();
+
+INSERT INTO user_page_visits VALUES
+ (1, CAST(x'...' AS VARBINARY)), -- serialized1 bytes
+ (2, CAST(x'...' AS VARBINARY)), -- serialized2 bytes
+ (3, CAST(x'...' AS VARBINARY)); -- serialized3 bytes
+```
+
+**Option 2: Custom Flink UDFs**
+
+```sql
+-- Create the aggregation table
+CREATE TABLE user_page_visits (
+ user_id INT PRIMARY KEY NOT ENFORCED,
+ page_visits VARBINARY
+) WITH (
+ 'merge-engine' = 'aggregation',
+ 'fields.page_visits.aggregate-function' = 'rbm32'
+);
+
+-- Register custom UDFs (you need to implement these)
+
+-- Use the UDFs
+INSERT INTO user_page_visits VALUES
+ (1, TO_BITMAP(100, 101, 102)),
+ (2, TO_BITMAP(101, 103)),
+ (3, TO_BITMAP(102, 104));
+
+-- Query with UDFs
+SELECT user_id, FROM_BITMAP(page_visits) as unique_pages
+FROM user_page_visits;
+
+SELECT user_id,
+ BITMAP_CONTAINS(page_visits, 101) as visited_page_101
+FROM user_page_visits
+WHERE user_id = 1;
+```
+
+**Sample UDF Implementation (Java):**
+
+```java
+// BitmapUDF - Converts integers to serialized RoaringBitmap
+public static class BitmapUDF extends ScalarFunction {
+ public byte[] eval(Integer... values) {
+ RoaringBitmap32 bitmap = new RoaringBitmap32();
+ for (Integer value : values) {
+ if (value != null) {
+ bitmap.add(value);
+ }
+ }
+ return bitmap.serialize();
+ }
+}
+
+// BitmapCountUDF - Gets cardinality from serialized RoaringBitmap
+public static class BitmapCountUDF extends ScalarFunction {
+ public Long eval(byte[] bitmapBytes) {
+ if (bitmapBytes == null) {
+ return 0L;
+ }
+ try {
+ RoaringBitmap32 bitmap = new RoaringBitmap32();
+ bitmap.deserialize(ByteBuffer.wrap(bitmapBytes));
+ return bitmap.getCardinality();
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to deserialize bitmap", e);
+ }
+ }
+}
+
+// BitmapContainsUDF - Checks if value exists in bitmap
+public static class BitmapContainsUDF extends ScalarFunction {
+ public Boolean eval(byte[] bitmapBytes, Integer value) {
+ if (bitmapBytes == null || value == null) {
+ return false;
+ }
+ try {
+ RoaringBitmap32 bitmap = new RoaringBitmap32();
+ bitmap.deserialize(ByteBuffer.wrap(bitmapBytes));
+ return bitmap.contains(value);
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to deserialize bitmap", e);
+ }
+ }
+}
+```
+
### nested_update
The nested_update function collects multiple rows into one array<row>
(so-called 'nested table'). It supports ARRAY<ROW> data types.