Skip to content

Conversation

@yruslan
Copy link
Collaborator

@yruslan yruslan commented Oct 23, 2025

Closes #648

Summary by CodeRabbit

  • New Features

    • Kafka Avro source and sink now support configuring custom Kafka metadata and key column names and permit metadata as a nested struct.
    • Sink automatically drops the Kafka metadata column before sending records to Kafka and logs a warning when it does so.
    • Source emits warnings when configured metadata/key names conflict with payload fields.
  • Documentation

    • README updated with examples for the new optional Kafka configuration parameters.

@github-actions
Copy link

github-actions bot commented Oct 23, 2025

Unit Test Coverage

Overall Project 83.59% -0.59% 🍏
Files changed 10.19%

Module Coverage
pramen:core Jacoco Report 85.41% -0.31%
pramen-extras Jacoco Report 67.32% -3.04%
Files
Module File Coverage
pramen:core Jacoco Report OffsetManagerUtils.scala 98.19% 🍏
SparkUtils.scala 87.64% -14.25%
IncrementalIngestionJob.scala 74.63% -4.06%
pramen-extras Jacoco Report KafkaAvroSink.scala 0% -48.63%
KafkaAvroSource.scala 0% -30.43%

@yruslan yruslan requested a review from Copilot October 23, 2025 10:00
Copy link

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull Request Overview

This PR implements issue #648 by restructuring how Kafka-generated columns are organized in the data schema. The changes group Kafka metadata (partition, offset, timestamp, timestamp_type) into a configurable struct column and allow customization of column names for both the Kafka metadata struct and the key column.

Key changes:

  • Kafka metadata fields are now grouped into a configurable struct column (default: "kafka") instead of individual top-level columns
  • Column names for Kafka metadata and key are now configurable via custom.kafka.column and key.column.name configuration options
  • Updated offset management utilities to work with nested struct fields instead of flat columns

Reviewed Changes

Copilot reviewed 8 out of 8 changed files in this pull request and generated 6 comments.

Show a summary per file
File Description
KafkaAvroSource.scala Restructured to create a struct column for Kafka metadata and made column names configurable
KafkaAvroSink.scala Added logic to drop the configurable Kafka metadata column before writing to Kafka
OffsetManagerUtilsSuite.scala Updated test to create Kafka metadata as a struct field
SparkUtils.scala Added getNestedField utility method to retrieve nested struct fields from schema
IncrementalIngestionJob.scala Updated to validate nested Kafka offset and partition fields within struct
OffsetManagerUtils.scala Modified to work with nested Kafka metadata fields using dot notation
OffsetValue.scala Removed now-unused constant field name definitions
README.md Added documentation for new configuration options

Tip: Customize your code reviews with copilot-instructions.md. Create the file or learn how to get started.

@yruslan yruslan changed the title #648 Kafka Source: Group kafka generated columns into a configurable struct column. #648 Kafka Source: Group Kafka generated columns into a configurable struct column. Oct 23, 2025
@yruslan
Copy link
Collaborator Author

yruslan commented Oct 23, 2025

@coderabbitai full review

@coderabbitai
Copy link

coderabbitai bot commented Oct 23, 2025

✅ Actions performed

Full review triggered.

@coderabbitai
Copy link

coderabbitai bot commented Oct 23, 2025

Note

Other AI code review bot(s) detected

CodeRabbit has detected other AI code review bot(s) in this pull request and will avoid duplicating their findings in the review comments. This may lead to a less comprehensive review.

Walkthrough

Refactors Kafka metadata handling: removes two hardcoded Kafka field constants, adds a nested-schema resolver, makes Kafka metadata column name configurable in source and sink, wraps metadata into a configurable struct on read, and drops that metadata column before writing to Kafka. Tests and utilities updated to use dynamic field names.

Changes

Cohort / File(s) Summary
Documentation
README.md
Added examples for custom.kafka.column usage in Kafka source and sink docs, and noted the sink drops the configured metadata column before writing.
API Cleanup
pramen/api/src/main/scala/za/co/absa/pramen/api/offset/OffsetValue.scala
Removed public constants KAFKA_PARTITION_FIELD and KAFKA_OFFSET_FIELD.
Schema Utilities
pramen/core/src/main/scala/za/co/absa/pramen/core/utils/SparkUtils.scala
Added getNestedField(schema: StructType, fieldName: String) to resolve dot-separated nested field paths with descriptive errors.
Offset & Ingestion Logic
pramen/core/src/main/scala/za/co/absa/pramen/core/bookkeeper/OffsetManagerUtils.scala, pramen/core/src/main/scala/za/co/absa/pramen/core/pipeline/IncrementalIngestionJob.scala
Replaced hardcoded Kafka field names with dynamically computed nested field names and switched lookups to SparkUtils.getNestedField(). Adjusted aggregations and row-index mappings accordingly.
Tests
pramen/core/src/test/scala/za/co/absa/pramen/core/tests/bookkeeper/OffsetManagerUtilsSuite.scala
Test data changed from flat kafka_offset/kafka_partition columns to a nested kafka struct; updated test calls to use the struct column.
Kafka Adapters (Source & Sink)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala, pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala
Source: added custom.kafka.column and key.column.name config keys, temporarily assembles metadata into tmp_pramen_kafka/tmp_pramen_kafka_key, projects and renames to configured names, and logs conflicts. Sink: reads configured kafka column name, drops that column (if present) before sending to Kafka and logs a warning.

Sequence Diagram(s)

sequenceDiagram
    participant Kafka as Kafka Topic
    participant Source as KafkaAvroSource
    participant Config as Config
    participant Schema as Schema Builder
    participant DF as DataFrame
    participant Sink as KafkaAvroSink

    Config->>Source: read custom.kafka.column / key.column.name
    Source->>Kafka: subscribe & read records
    Kafka-->>Source: record + partition, offset, timestamp, key
    Source->>Schema: build payload fields
    Source->>DF: create tmp_pramen_kafka (partition,offset,timestamp,...)
    Source->>DF: create tmp_pramen_kafka_key (key)
    Source->>DF: project tmp_pramen_kafka_key, data.*, tmp_pramen_kafka
    Source->>DF: drop original configured metadata columns (if conflict) and rename temps → configured names
    DF-->>Sink: dataframe with configured nested kafka column
    Sink->>Sink: if configured kafka column exists -> drop it (log)
    Sink->>Kafka: write records (without kafka metadata column)
Loading

Estimated code review effort

🎯 3 (Moderate) | ⏱️ ~25 minutes

Possibly related PRs

  • #644 Add Kafka Avro batch source #647: Modifies Kafka offset/partition handling and related constants/types; directly related to the removal and refactoring of Kafka field constants and KafkaValue usage.

Poem

🐇 I nest your Kafka bits with care,

tmp fields tucked in my rabbit lair,
No hardcoded names to chase,
Configured fields find their place,
I drop them gently before they share.

Pre-merge checks and finishing touches

✅ Passed checks (5 passed)
Check name Status Explanation
Description Check ✅ Passed Check skipped - CodeRabbit’s high-level summary is enabled.
Title Check ✅ Passed The PR title "#648 Kafka Source: Group Kafka generated columns into a configurable struct column" directly and clearly summarizes the main change in the changeset. It references the related issue number (#648), is concise and specific, and accurately describes the primary objective of gathering Kafka-specific fields under a configurable struct field. The title is neither vague nor overly generic, and it provides sufficient clarity for teammates reviewing the git history.
Linked Issues Check ✅ Passed The changeset comprehensively addresses all five coding objectives from issue #648: grouping Kafka-specific fields into a struct field is implemented in KafkaAvroSource with a temporary struct approach [KafkaAvroSource], making the field name configurable is achieved through new CUSTOM_KAFKA_COLUMN_KEY constant and runtime config lookup [KafkaAvroSource], positioning the Kafka field at the end is reflected in the implementation [KafkaAvroSource], handling payload field conflicts with warnings is mentioned in the source changes [KafkaAvroSource], and dropping Kafka offset fields from DataFrames in KafkaSink is implemented to prevent name collisions [KafkaAvroSink]. Supporting changes include adding SparkUtils.getNestedField for nested field resolution [SparkUtils], updating OffsetManagerUtils and IncrementalIngestionJob to use dynamic field names instead of hard-coded constants [OffsetValue, OffsetManagerUtils, IncrementalIngestionJob], and test updates demonstrating the struct implementation [OffsetManagerUtilsSuite].
Out of Scope Changes Check ✅ Passed All code changes in the pull request are directly related to implementing the objectives of issue #648. The README.md updates document the new Kafka configuration feature, the removal of hard-coded constants from OffsetValue supports the shift to dynamic field names, the adaptations in OffsetManagerUtils and IncrementalIngestionJob enable dynamic field lookups, the new SparkUtils.getNestedField utility directly supports nested field access required for the struct feature, the test updates demonstrate the new nested struct approach, and the modifications to KafkaAvroSource and KafkaAvroSink implement the core functionality. No changes appear to be out of scope or unrelated to the stated PR objectives.
Docstring Coverage ✅ Passed No functions found in the changes. Docstring coverage check skipped.
✨ Finishing touches
🧪 Generate unit tests (beta)
  • Create PR with unit tests
  • Post copyable unit tests in a comment
  • Commit unit tests in branch feature/648-kafka-keys-as-a-struct-field

Thanks for using CodeRabbit! It's free for OSS, and your support helps us grow. If you like it, consider giving us a shout-out.

❤️ Share

Comment @coderabbitai help to get the list of available commands and usage tips.

@coderabbitai
Copy link

coderabbitai bot commented Oct 23, 2025

✅ Actions performed

Review triggered.

Note: CodeRabbit is an incremental review system and does not re-review already reviewed commits. This command is applicable only when automatic reviews are paused.

Copy link

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 0

♻️ Duplicate comments (4)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (1)

42-43: Fix grammar in documentation comment.

The comment contains grammatical issues that were flagged in previous reviews:

  • "struct field that Kafka record metadata" should be "struct field that contains Kafka record metadata"
  • "if It exists" should be "if it exists"

Apply this diff:

-  *    # [Optional] Set name for the struct field that contains Kafka record metadata. This column will be dropped if It exists before sending data to Kafka.
+  *    # [Optional] Set name for the struct field that contains Kafka record metadata. This column will be dropped if it exists before sending data to Kafka.
README.md (2)

942-946: Fix grammar in documentation comments.

The comments contain grammatical issues flagged in previous reviews:

  • Line 942: "struct field that Kafka record metadata" should be "struct field that contains Kafka record metadata"

Apply this diff:

-    # [Optional] Set name for the struct field that Kafka record metadata
+    # [Optional] Set name for the struct field that contains Kafka record metadata

1017-1019: Fix grammar in documentation comments.

The comments contain grammatical issues flagged in previous reviews:

  • Line 1017: "struct field that Kafka record metadata" should be "struct field that contains Kafka record metadata"
  • "if It exists" should be "if it exists"

Apply this diff:

-    # [Optional] Set name for the struct field that contains Kafka record metadata. This column will be dropped if It exists before sending data to Kafka.
+    # [Optional] Set name for the struct field that contains Kafka record metadata. This column will be dropped if it exists before sending data to Kafka.
pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (1)

48-51: Fix grammar in documentation comment.

The comment contains a grammatical issue flagged in previous reviews:

  • Line 48: "struct field that Kafka record metadata" should be "struct field that contains Kafka record metadata"

Apply this diff:

-  *    # [Optional] Set name for the struct field that contains Kafka record metadata
+  *    # [Optional] Set name for the struct field that contains Kafka record metadata

Wait, looking at line 48 more carefully, it actually already says "contains" - let me check the actual text again... No, line 48 shows "struct field that contains Kafka record metadata" which is correct. But the past review comment suggests it should say "contains" or "for". Let me look at the exact text provided.

Actually, reviewing the provided text at line 48-49, I see:

  *    # [Optional] Set name for the struct field that contains Kafka record metadata

This is already correct! The past review comment was about an older version that said "struct field that Kafka record metadata" without "contains" or "for". The current code already has "contains" so this is fine.

Let me skip this comment since the grammar is already correct.

🧹 Nitpick comments (1)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (1)

199-224: LGTM: Kafka metadata grouped into configurable struct at schema end.

The implementation correctly:

  • Creates a temporary tmp_pramen_kafka struct containing partition, offset, timestamp, and timestamp_type (lines 199-204)
  • Uses temporary tmp_pramen_kafka_key column for key data (lines 211, 213)
  • Projects data fields first (data.*), then Kafka metadata (line 219), placing Kafka fields at the end of the schema per PR objectives
  • Drops conflicting payload fields (lines 220-221) before renaming temporaries (lines 222-223)

However, the PR objectives state: "If the chosen Kafka struct field name conflicts with an existing payload field, the Kafka field should replace the payload field and a warning should be generated."

Currently, lines 220-221 silently drop the conflicting columns without logging a warning. Consider adding a check and warning:

// Before line 219
val payloadFields = df2.select("data.*").schema.fieldNames.toSet
if (payloadFields.contains(kafkaColumnName)) {
  log.warn(s"Payload field '$kafkaColumnName' conflicts with Kafka metadata struct name and will be replaced.")
}
if (payloadFields.contains(keyColumnName)) {
  log.warn(s"Payload field '$keyColumnName' conflicts with Kafka key column name and will be replaced.")
}

Would you like me to open an issue to track this enhancement?

📜 Review details

Configuration used: CodeRabbit UI

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 68971be and fdc5cb4.

📒 Files selected for processing (8)
  • README.md (2 hunks)
  • pramen/api/src/main/scala/za/co/absa/pramen/api/offset/OffsetValue.scala (0 hunks)
  • pramen/core/src/main/scala/za/co/absa/pramen/core/bookkeeper/OffsetManagerUtils.scala (1 hunks)
  • pramen/core/src/main/scala/za/co/absa/pramen/core/pipeline/IncrementalIngestionJob.scala (2 hunks)
  • pramen/core/src/main/scala/za/co/absa/pramen/core/utils/SparkUtils.scala (1 hunks)
  • pramen/core/src/test/scala/za/co/absa/pramen/core/tests/bookkeeper/OffsetManagerUtilsSuite.scala (2 hunks)
  • pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (4 hunks)
  • pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (5 hunks)
💤 Files with no reviewable changes (1)
  • pramen/api/src/main/scala/za/co/absa/pramen/api/offset/OffsetValue.scala
🔇 Additional comments (12)
pramen/core/src/test/scala/za/co/absa/pramen/core/tests/bookkeeper/OffsetManagerUtilsSuite.scala (2)

19-19: LGTM: Import added for struct function.

The addition of struct to the imports is necessary for creating the nested Kafka metadata structure in the test.


61-66: LGTM: Test updated to reflect new nested Kafka struct approach.

The test now correctly creates a nested "kafka" struct containing offset and partition fields, and passes "kafka" to getMinMaxValueFromData. This aligns with the PR's objective of grouping Kafka metadata into a configurable struct field.

pramen/core/src/main/scala/za/co/absa/pramen/core/bookkeeper/OffsetManagerUtils.scala (2)

30-31: LGTM: Dynamic Kafka field names support nested structures.

The construction of kafkaOffsetFieldName and kafkaPartitionFieldName using dot notation enables access to nested struct fields (e.g., "kafka.offset", "kafka.partition"). This aligns with the PR's goal of grouping Kafka metadata into a configurable struct.


34-41: LGTM: Aggregation logic correctly uses dynamic field names.

The updated groupBy, aggregation, and ordering operations now reference the dynamically constructed field names. The positional access pattern (getAs[Int](0), getAs[Long](1), getAs[Long](2)) in lines 40-41 correctly extracts partition (position 0), min_offset (position 1), and max_offset (position 2) from the aggregated results.

pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (3)

23-24: LGTM: Imports added for configuration handling.

The imports enable access to the shared CUSTOM_KAFKA_COLUMN_KEY constant and configuration utilities, ensuring consistency with the source's configuration model.


122-122: LGTM: Configurable Kafka column name with sensible default.

The kafkaColumnName is derived from the optional custom.kafka.column configuration, defaulting to "kafka". This aligns with the source's default and the PR's objective of making the Kafka struct field name configurable.


141-143: LGTM: Kafka metadata column dropped before writing to Kafka.

Dropping the configured kafkaColumnName column before writing prevents name collisions when data is re-ingested from Kafka, addressing one of the PR's key objectives.

pramen/core/src/main/scala/za/co/absa/pramen/core/utils/SparkUtils.scala (1)

618-648: LGTM: Well-designed nested field resolution utility.

The getNestedField method provides a clean API for resolving nested StructType fields using dot notation. Key strengths:

  • Clear validation: Empty field names are rejected at line 643-645
  • Recursive traversal: The helper getNestedFieldInArray correctly handles nested StructTypes
  • Case-insensitive matching: Root field lookup (line 621) uses equalsIgnoreCase, improving usability
  • Error handling: Provides informative error messages listing available fields (line 636-637) and reporting type mismatches (line 632)

The implementation supports the PR's dynamic Kafka field access pattern effectively.

pramen/core/src/main/scala/za/co/absa/pramen/core/pipeline/IncrementalIngestionJob.scala (2)

33-33: LGTM: Import added for nested field utilities.

The SparkUtils import is necessary for accessing the getNestedField method used in Kafka offset validation.


310-324: LGTM: Kafka offset validation adapted for dynamic nested fields.

The validation logic correctly:

  • Constructs dynamic field names (kafkaOffsetFieldName, kafkaPartitionFieldName) using dot notation (lines 311-312)
  • Retrieves nested fields via SparkUtils.getNestedField (lines 313-314)
  • Validates data types remain unchanged (LongType for offset, IntegerType for partition)
  • Provides clear error messages referencing the dynamic field names (lines 317, 322)

This implementation aligns with the PR's objective of supporting configurable Kafka struct field names.

pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (2)

110-113: LGTM: Configurable Kafka metadata field names with sensible defaults.

The introduction of CUSTOM_KAFKA_COLUMN_KEY and KEY_COLUMN_KEY as public constants enables configuration reuse across components. The defaults ("kafka" and "kafka_key") provide a good out-of-box experience.


118-119: LGTM: Offset info uses configurable field name.

Using kafkaColumnName in getOffsetInfo ensures offset management tracks the user-configured struct field, maintaining consistency with the dynamic field naming approach used throughout the PR.

Copy link

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 0

🧹 Nitpick comments (2)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (1)

143-148: Consider using the exact field name from schema for clarity.

While Spark's drop() is case-insensitive by default, the code would be more robust and self-documenting if it retrieved the actual field name from the schema:

-    val dfWithoutKafkaField = if (df.schema.fields.exists(_.name.equalsIgnoreCase(kafkaColumnName))) {
+    val dfWithoutKafkaField = df.schema.fields.find(_.name.equalsIgnoreCase(kafkaColumnName)) match {
+      case Some(field) =>
-      log.warn(s"Dropping '$kafkaColumnName' field from the input DataFrame before sending to Kafka because the output topic has its own metadata.")
-      df.drop(kafkaColumnName)
-    } else {
+        log.warn(s"Dropping '${field.name}' field from the input DataFrame before sending to Kafka because the output topic has its own metadata.")
+        df.drop(field.name)
+      case None =>
-      df
+        df
     }

This avoids relying on Spark's case-sensitivity configuration and makes the intent explicit.

pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (1)

227-234: LGTM! Final DataFrame transformation correctly handles conflicts and ordering.

The logic properly:

  • Flattens the payload
  • Drops conflicting fields from payload
  • Renames temporary Kafka columns to configured names
  • Ensures Kafka metadata appears at the end of the schema

Minor suggestion: Update the comment on lines 227-228 to reference the configured column names rather than hardcoded names:

-    // Put data fields to the root level of the schema, and if data struct already has kafka_key and kafka fields,
-    // drop them
+    // Put data fields to the root level of the schema, and if data struct already has fields
+    // matching the configured Kafka metadata column names, drop them
📜 Review details

Configuration used: CodeRabbit UI

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between fdc5cb4 and cd10447.

📒 Files selected for processing (2)
  • pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (4 hunks)
  • pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (5 hunks)
🧰 Additional context used
🧬 Code graph analysis (2)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (4)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/writer/TableWriterKafka.scala (7)
  • pramen (66-66)
  • pramen (68-78)
  • pramen (80-86)
  • pramen (88-98)
  • pramen (100-118)
  • pramen (120-138)
  • write (39-64)
pramen/api/src/main/scala/za/co/absa/pramen/api/SinkResult.scala (1)
  • SinkResult (19-24)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (2)
  • KafkaAvroSource (107-240)
  • KafkaAvroSource (242-253)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/utils/ConfigUtils.scala (2)
  • ConfigUtils (24-205)
  • getOptionString (43-49)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (3)
pramen/api/src/main/scala/za/co/absa/pramen/api/offset/OffsetValue.scala (2)
  • OffsetValue (34-154)
  • KafkaValue (80-125)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/utils/ConfigUtils.scala (2)
  • ConfigUtils (24-205)
  • getOptionString (43-49)
pramen/api/src/main/scala/za/co/absa/pramen/api/offset/OffsetInfo.scala (1)
  • OffsetInfo (19-19)
⏰ Context from checks skipped due to timeout of 90000ms. You can increase the timeout in your CodeRabbit configuration to a maximum of 15 minutes (900000ms). (8)
  • GitHub Check: Test Spark 3.5.5 on Scala 2.12.20
  • GitHub Check: Test Spark 3.5.5 on Scala 2.13.16
  • GitHub Check: Test Spark 3.3.4 on Scala 2.13.16
  • GitHub Check: Test Spark 3.4.4 on Scala 2.13.16
  • GitHub Check: Test Spark 3.3.4 on Scala 2.12.20
  • GitHub Check: Test Spark 3.4.4 on Scala 2.12.20
  • GitHub Check: Test Spark 2.4.8 on Scala 2.11.12
  • GitHub Check: Test Coverage on Scala 2.12.18
🔇 Additional comments (12)
pramen/extras/src/main/scala/za/co/absa/pramen/extras/sink/KafkaAvroSink.scala (4)

21-21: LGTM! Imports are properly used.

The new imports support logging and configuration-driven Kafka column handling.

Also applies to: 24-25


43-45: LGTM! Documentation is clear and accurate.

The documentation properly describes the configurable Kafka metadata column and its handling behavior.


123-124: LGTM! Configuration initialization follows best practices.

The logger and configurable column name are properly initialized with a sensible default.


150-150: LGTM! Write logic correctly uses the modified DataFrame.

The implementation properly writes the DataFrame with Kafka metadata removed.

pramen/extras/src/main/scala/za/co/absa/pramen/extras/source/KafkaAvroSource.scala (8)

21-22: LGTM! Imports support the new struct-based metadata handling.

The imports are properly used throughout the implementation.

Also applies to: 25-25


111-111: LGTM! Initialization follows best practices.

The logger and configurable column names are properly initialized with sensible defaults.

Also applies to: 113-113, 115-116


121-121: LGTM! Offset column name is now configurable.

The change aligns with the PR objective to make Kafka metadata configurable. Since the offset column is now a struct, ensure that the offset management system (OffsetManagerUtils) properly handles nested field access for partition and offset values.


49-53: LGTM! Documentation is clear and addresses previous feedback.

The documentation properly describes both configurable parameters.


202-207: LGTM! Kafka metadata properly grouped into struct.

The implementation correctly groups partition, offset, timestamp, and timestamp_type into a single struct, using a temporary name to avoid conflicts during transformation.


214-217: LGTM! Key column handling supports both Avro and raw formats.

The implementation correctly handles both scenarios using a consistent temporary naming pattern.


219-225: LGTM! Excellent defensive programming with conflict warnings.

The payload field conflict detection and warnings help users understand when their data fields will be replaced by Kafka metadata, directly addressing the PR objectives.


244-245: LGTM! New configuration constants promote consistency.

The public constants centralize configuration key definitions and enable safe sharing with other components like KafkaAvroSink.

@yruslan yruslan merged commit 2981884 into main Oct 23, 2025
9 checks passed
@yruslan yruslan deleted the feature/648-kafka-keys-as-a-struct-field branch October 23, 2025 12:29
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Kafka source: gather all Kafka-specific fields under a struct field

1 participant