Skip to content

Commit 97c736c

Browse files
authored
Add IcebergChangeHandlerV2 handler for flattened data consuming, schema will be inferred by arrow (#23)
* Add iceberg v2 consumer for flattened data consuming, schema will be inferred by arrow * Add iceberg v2 consumer for flattened data consuming, schema will be inferred by arrow * Add iceberg v2 consumer for flattened data consuming, schema will be inferred by arrow
1 parent 99f3c9f commit 97c736c

File tree

8 files changed

+408
-4
lines changed

8 files changed

+408
-4
lines changed

README.md

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,13 @@ The Iceberg handlers are designed to stream CDC events directly into Apache Iceb
3434
* **Automatic Table Creation & Partitioning**: It automatically creates a new Iceberg table for each source table and partitions it by day on the `_consumed_at` timestamp for efficient time-series queries.
3535
* **Enriched Metadata**: It also adds `_consumed_at`, `_dbz_event_key`, and `_dbz_event_key_hash` columns for enhanced traceability.
3636

37+
* `IcebergChangeHandlerV2`: A more advanced handler that automatically infers the schema from the Debezium events and creates a well-structured Iceberg table accordingly.
38+
* **Use Case**: Ideal for scenarios where you want the pipeline to automatically create tables with native data types that mirror the source. This allows for direct querying of the data without needing to parse JSON.
39+
* **Schema and Features**:
40+
* **Automatic Schema Inference**: It inspects the first batch of records for a given table and infers the schema using PyArrow, preserving native data types (e.g., `LongType`, `TimestampType`).
41+
* **Robust Type Handling**: If a field's type cannot be inferred from the initial batch (e.g., it is always `null`), it safely falls back to `StringType` to prevent errors.
42+
* **Automatic Table Creation & Partitioning**: It automatically creates a new Iceberg table for each source table and partitions it by day on the `_consumed_at` timestamp for efficient time-series queries.
43+
* **Enriched Metadata**: It also adds `_consumed_at`, `_dbz_event_key`, and `_dbz_event_key_hash` columns for enhanced traceability.
3744

3845
### dlt (data load tool) Handler (`pydbzengine[dlt]`)
3946

pydbzengine/handlers/iceberg.py

Lines changed: 209 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,11 +1,13 @@
11
import datetime
2+
import io
23
import json
34
import logging
45
import uuid
56
from abc import abstractmethod
67
from typing import List, Dict
78

89
import pyarrow as pa
10+
from pyarrow import json as pa_json
911
from pyiceberg.catalog import Catalog
1012
from pyiceberg.exceptions import NoSuchTableError
1113
from pyiceberg.partitioning import PartitionSpec, PartitionField
@@ -141,7 +143,8 @@ def load_table(self, table_identifier):
141143
table = self.catalog.create_table(identifier=table_identifier,
142144
schema=self._target_schema,
143145
partition_spec=self.DEBEZIUM_TABLE_PARTITION_SPEC)
144-
self.log.info(f"Created iceberg table {'.'.join(table_identifier)} with daily partitioning on _consumed_at.")
146+
self.log.info(
147+
f"Created iceberg table {'.'.join(table_identifier)} with daily partitioning on _consumed_at.")
145148
return table
146149

147150
@property
@@ -200,3 +203,208 @@ def _target_schema(self) -> Schema:
200203
),
201204
)
202205

206+
207+
class IcebergChangeHandlerV2(BaseIcebergChangeHandler):
208+
"""
209+
A change handler that uses Apache Iceberg to process Debezium change events.
210+
This class receives batches of Debezium ChangeEvent objects and applies the changes
211+
to the corresponding Iceberg tables.
212+
"""
213+
214+
def __init__(self, catalog: "Catalog", destination_namespace: tuple, supports_variant: bool = False,
215+
event_flattening_enabled=False):
216+
super().__init__(catalog, destination_namespace, supports_variant)
217+
self.event_flattening_enabled = event_flattening_enabled
218+
219+
def _handle_table_changes(self, destination: str, records: List[ChangeEvent]):
220+
"""
221+
Handles changes for a specific table.
222+
Args:
223+
destination: The name of the table to apply the changes to.
224+
records: A list of ChangeEvent objects for the specified table.
225+
"""
226+
227+
table = self.get_table(destination)
228+
if table is None:
229+
table_identifier: tuple = self.destination_to_table_identifier(destination=destination)
230+
table = self._infer_and_create_table(records=records, table_identifier=table_identifier)
231+
#
232+
arrow_data = self._read_to_arrow_table(records=records, schema=table.schema())
233+
234+
# Populate all metadata columns (_consumed_at, _dbz_event_key, etc.)
235+
enriched_arrow_data = self._enrich_arrow_table_with_metadata(
236+
arrow_table=arrow_data,
237+
records=records
238+
)
239+
240+
self._handle_schema_changes(table=table, arrow_schema=enriched_arrow_data.schema)
241+
table.append(enriched_arrow_data)
242+
self.log.info(f"Appended {len(enriched_arrow_data)} records to table {'.'.join(table.name())}")
243+
244+
def _enrich_arrow_table_with_metadata(self, arrow_table: pa.Table, records: List[ChangeEvent]) -> pa.Table:
245+
num_records = len(arrow_table)
246+
dbz_event_keys = []
247+
dbz_event_key_hashes = []
248+
249+
for record in records:
250+
key = record.key()
251+
dbz_event_keys.append(key)
252+
key_hash = str(uuid.uuid5(uuid.NAMESPACE_DNS, key)) if key else None
253+
dbz_event_key_hashes.append(key_hash)
254+
255+
# Create PyArrow arrays for each metadata column
256+
consumed_at_array = pa.array([datetime.datetime.now(datetime.timezone.utc)] * num_records,
257+
type=pa.timestamp('us', tz='UTC'))
258+
dbz_event_key_array = pa.array(dbz_event_keys, type=pa.string())
259+
dbz_event_key_hash_array = pa.array(dbz_event_key_hashes, type=pa.string())
260+
261+
# Replace the null columns in the Arrow table with the populated arrays.
262+
# This uses set_column, which is efficient for replacing entire columns.
263+
enriched_table = arrow_table.set_column(
264+
arrow_table.schema.get_field_index("_consumed_at"),
265+
"_consumed_at",
266+
consumed_at_array
267+
)
268+
enriched_table = enriched_table.set_column(
269+
enriched_table.schema.get_field_index("_dbz_event_key"),
270+
"_dbz_event_key",
271+
dbz_event_key_array
272+
)
273+
enriched_table = enriched_table.set_column(
274+
enriched_table.schema.get_field_index("_dbz_event_key_hash"),
275+
"_dbz_event_key_hash",
276+
dbz_event_key_hash_array
277+
)
278+
279+
return enriched_table
280+
281+
def _read_to_arrow_table(self, records, schema=None):
282+
json_lines_buffer = io.BytesIO()
283+
for record in records:
284+
json_lines_buffer.write((record.value() + '\n').encode('utf-8'))
285+
json_lines_buffer.seek(0)
286+
287+
parse_options = None
288+
if schema:
289+
# If an Iceberg schema is provided, convert it to a PyArrow schema and use it for parsing.
290+
parse_options = pa_json.ParseOptions(explicit_schema=schema.as_arrow(), unexpected_field_behavior="infer")
291+
return pa_json.read_json(json_lines_buffer, parse_options=parse_options)
292+
293+
def get_table(self, destination: str) -> "Table":
294+
table_identifier: tuple = self.destination_to_table_identifier(destination=destination)
295+
return self.load_table(table_identifier=table_identifier)
296+
297+
def load_table(self, table_identifier):
298+
try:
299+
return self.catalog.load_table(identifier=table_identifier)
300+
except NoSuchTableError:
301+
return None
302+
303+
def _infer_and_create_table(self, records: List[ChangeEvent], table_identifier: tuple) -> Table:
304+
"""
305+
Infers a schema from a batch of records, creates a new Iceberg table with that schema,
306+
and sets up daily partitioning on the _consumed_at field.
307+
"""
308+
arrow_table = self._read_to_arrow_table(records)
309+
sanitized_fields = self._sanitize_schema_fields(data_schema=arrow_table.schema)
310+
311+
# Add metadata fields to the list of pyarrow fields
312+
sanitized_fields.extend([
313+
pa.field("_consumed_at", pa.timestamp('us', tz='UTC')),
314+
pa.field("_dbz_event_key", pa.string()),
315+
pa.field("_dbz_event_key_hash", pa.string()) # For UUIDType
316+
])
317+
318+
# Create a pyarrow schema first
319+
sanitized_arrow_schema = pa.schema(sanitized_fields)
320+
321+
# Create the table
322+
table = self.catalog.create_table(
323+
identifier=table_identifier,
324+
schema=sanitized_arrow_schema
325+
)
326+
# add partitioning
327+
with table.update_spec() as update_spec:
328+
update_spec.add_field(source_column_name="_consumed_at", transform=DayTransform(),
329+
partition_field_name="_consumed_at_day")
330+
331+
if self.event_flattening_enabled and False:
332+
# @TODO fix future. https://github.com/apache/iceberg-python/issues/1728
333+
identifier_fields = self._get_identifier_fields(sample_event=records[0],
334+
table_identifier=table_identifier
335+
)
336+
if identifier_fields:
337+
with table.update_schema(allow_incompatible_changes=True) as update_schema:
338+
for field in identifier_fields:
339+
update_schema._set_column_requirement(path=field, required=True)
340+
update_schema.set_identifier_fields(*identifier_fields)
341+
self.log.info(f"Created iceberg table {'.'.join(table_identifier)} with daily partitioning on _consumed_at.")
342+
return table
343+
344+
def _sanitize_schema_fields(self, data_schema: pa.Schema) -> list:
345+
"""
346+
Recursively traverses a PyArrow schema and replaces null types with a string fallback.
347+
This is useful when a schema is inferred from JSON where some fields are always null.
348+
"""
349+
new_fields = []
350+
for field in data_schema:
351+
field_type = field.type
352+
if pa.types.is_null(field_type):
353+
# Found a null type, replace it with a string as a fallback.
354+
new_fields.append(field.with_type(pa.string()))
355+
elif pa.types.is_struct(field_type):
356+
# Found a struct, so we recurse on its fields to sanitize them.
357+
# We can treat the struct's fields as a schema for the recursive call.
358+
nested_schema = pa.schema(field_type)
359+
sanitized_nested_schema = self._sanitize_schema_fields(nested_schema)
360+
# Recreate the field with the new, sanitized struct type.
361+
new_fields.append(field.with_type(pa.struct(sanitized_nested_schema)))
362+
else:
363+
# Not a null or struct, so we keep the field as is.
364+
new_fields.append(field)
365+
return new_fields
366+
367+
def _get_identifier_fields(self, sample_event: ChangeEvent, table_identifier: tuple) -> list:
368+
"""
369+
Parses the Debezium event key to extract primary key field names.
370+
371+
This method uses a series of guard clauses to validate the key and returns
372+
an empty list if any validation step fails.
373+
374+
Args:
375+
sample_event: A sample change event to inspect for the key.
376+
table_identifier: The identifier of the table, used for logging.
377+
378+
Returns:
379+
A list of key field names, or an empty list if the key cannot be determined.
380+
"""
381+
key_json_str = sample_event.key()
382+
table_name_str = '.'.join(table_identifier)
383+
384+
if not key_json_str:
385+
self.log.warning(f"Cannot determine identifier fields for {table_name_str}: event key is empty.")
386+
return []
387+
388+
try:
389+
key_data = json.loads(key_json_str)
390+
except json.JSONDecodeError:
391+
self.log.error(f"Failed to parse Debezium event key as JSON for table {table_name_str}: {key_json_str}")
392+
return []
393+
394+
if not isinstance(key_data, dict):
395+
self.log.warning(
396+
f"Event key for {table_name_str} is not a JSON object, cannot infer primary key. Key: {key_json_str}")
397+
return []
398+
399+
key_field_names = list(key_data.keys())
400+
if not key_field_names:
401+
self.log.warning(f"Event key for {table_name_str} is an empty JSON object, cannot infer primary key.")
402+
return []
403+
404+
self.log.info(f"Found potential primary key fields {key_field_names} for table {table_name_str}")
405+
return key_field_names
406+
407+
def _handle_schema_changes(self, table: "Table", arrow_schema: "pa.Schema"):
408+
with table.update_schema() as update:
409+
update.union_by_name(new_schema=arrow_schema)
410+
self.log.info(f"Schema for table {'.'.join(table.name())} has been updated.")
File renamed without changes.

tests/mock_events.py

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,29 @@
1+
from pydbzengine import ChangeEvent
2+
3+
4+
class MockChangeEvent(ChangeEvent):
5+
"""
6+
A concrete implementation of ChangeEvent for testing.
7+
"""
8+
9+
def __init__(self, key: str, value: str, destination: str, partition: int=1):
10+
self._key = key
11+
self._value = value
12+
self._destination = destination
13+
self._partition = partition
14+
15+
def key(self) -> str:
16+
"""Returns the record key."""
17+
return self._key
18+
19+
def value(self) -> str:
20+
"""Returns the record value (payload)."""
21+
return self._value
22+
23+
def destination(self) -> str:
24+
"""Returns the destination topic/table."""
25+
return self._destination
26+
27+
def partition(self) -> int:
28+
"""Returns the partition the record belongs to."""
29+
return self._partition

tests/test_change_handler.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
import logging
22
from typing import List
33

4-
from base_postgresql_test import BasePostgresqlTest
4+
from base_postgresql import BasePostgresqlTest
55
from pydbzengine import ChangeEvent, BasePythonChangeHandler
66
from pydbzengine import DebeziumJsonEngine
77

tests/test_dlt_handler.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33
import dlt
44
import duckdb
55

6-
from base_postgresql_test import BasePostgresqlTest
6+
from base_postgresql import BasePostgresqlTest
77
from pydbzengine import DebeziumJsonEngine
88
from pydbzengine.handlers.dlt import DltChangeHandler
99
from pydbzengine.helper import Utils

tests/test_iceberg_handler.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
from pyiceberg.schema import Schema
55
from pyiceberg.types import LongType, NestedField, StringType
66

7-
from base_postgresql_test import BasePostgresqlTest
7+
from base_postgresql import BasePostgresqlTest
88
from catalog_rest import CatalogRestContainer
99
from pydbzengine import DebeziumJsonEngine
1010
from pydbzengine.handlers.iceberg import IcebergChangeHandler

0 commit comments

Comments
 (0)