[ https://issues.apache.org/jira/browse/KAFKA-15838?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Eric Pangiawan updated KAFKA-15838: ----------------------------------- Description: ExtractField: Line 116-119 [https://github.com/a0x8o/kafka/blob/master/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java#L61-L68] InsertField: Line 163 - 195 [https://github.com/a0x8o/kafka/blob/master/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java#L163-L195] h1. Expect: Value `null` is valid for an optional filed, even though the field has a default value. Only when field is required, the class return default value fallback when value is `null`. h1. Actual: Always return default value if `null` was given. h1. Example: PostgreSQL DDL: {code:java} CREATE TABLE products( id varchar(255), color varchar(255), quantity float8 ); -- Set Default ALTER TABLE products ALTER COLUMN quantity SET DEFAULT 1.0; {code} Insert A Record: {code:java} INSERT INTO public.products VALUES('1', 'Blue', null); {code} Table Select *: {code:java} id | color | quantity ----+-------+---------- 1 | Blue | {code} Debezium Behavior when using ExtractField and InsertField class (in the event flattening SMT): {code:java} { "id":"1", "color":"Blue", "quantity":1.0, "__op":"c", "__ts_ms":1698127432079, "__source_ts_ms":1698127431679, "__db":"testing_db", "__schema":"public", "__table":"products", "__lsn":24470112, "__txId":957, "__snapshot":null, "__deleted":"false" } {code} The debezium code can be found [here|https://github.com/debezium/debezium/blob/2.4/debezium-core/src/main/java/io/debezium/transforms/ExtractNewRecordState.java#L116-L119] h1. Expected Output: {code:java} { "id":"1", "color":"Blue", "quantity":null, "__op":"c", "__ts_ms":1698127432079, "__source_ts_ms":1698127431679, "__db":"testing_db", "__schema":"public", "__table":"products", "__lsn":24470112, "__txId":957, "__snapshot":null, "__deleted":"false" }{code} h1. Temporary Solution: use getWithoutDefault() into ExtractField and InsertField instead of get() was: ExtractField: Line 116-119 [https://github.com/a0x8o/kafka/blob/master/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java#L61-L68] InsertField: Line 163 - 195 [https://github.com/a0x8o/kafka/blob/master/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java#L163-L195] h1. Expect: Value `null` is valid for an optional filed, even though the field has a default value. Only when field is required, the class return default value fallback when value is `null`. h1. Actual: Always return default value if `null` was given. h1. Example: PostgreSQL DDL: {code:java} CREATE TABLE products( id varchar(255), color varchar(255), quantity float8 ); -- Set Default ALTER TABLE products ALTER COLUMN quantity SET DEFAULT 1.0; {code} Insert A Record: {code:java} INSERT INTO public.products VALUES('1', 'Blue', null); {code} Table Select *: {code:java} id | color | quantity ----+-------+---------- 1 | Blue | {code} Debezium Behavior when using ExtractField and InsertField class (in the event flattening SMT): {code:java} { "id":"1", "color":"Blue", "quantity":1.0, "__op":"c", "__ts_ms":1698127432079, "__source_ts_ms":1698127431679, "__db":"testing_db", "__schema":"public", "__table":"products", "__lsn":24470112, "__txId":957, "__snapshot":null, "__deleted":"false" } {code} The debezium code can be found [here|https://github.com/debezium/debezium/blob/2.4/debezium-core/src/main/java/io/debezium/transforms/ExtractNewRecordState.java#L116-L119] h1. Expected Output: {code:java} { "id":"1", "color":"Blue", "quantity":null, "__op":"c", "__ts_ms":1698127432079, "__source_ts_ms":1698127431679, "__db":"testing_db", "__schema":"public", "__table":"products", "__lsn":24470112, "__txId":957, "__snapshot":null, "__deleted":"false" }{code} h1. Temporary Solution: use getWithoutDefault() into ExtractField and InsertField > [Connect] ExtractField and InsertField NULL Values are replaced by default > value even in NULLABLE fields > -------------------------------------------------------------------------------------------------------- > > Key: KAFKA-15838 > URL: https://issues.apache.org/jira/browse/KAFKA-15838 > Project: Kafka > Issue Type: Bug > Reporter: Eric Pangiawan > Priority: Major > > ExtractField: Line 116-119 > [https://github.com/a0x8o/kafka/blob/master/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java#L61-L68] > InsertField: Line 163 - 195 > [https://github.com/a0x8o/kafka/blob/master/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java#L163-L195] > h1. Expect: > Value `null` is valid for an optional filed, even though the field has a > default value. > Only when field is required, the class return default value fallback when > value is `null`. > h1. Actual: > Always return default value if `null` was given. > h1. Example: > PostgreSQL DDL: > {code:java} > CREATE TABLE products( > id varchar(255), > color varchar(255), > quantity float8 > ); > -- Set Default > ALTER TABLE products ALTER COLUMN quantity SET DEFAULT 1.0; {code} > Insert A Record: > {code:java} > INSERT INTO public.products VALUES('1', 'Blue', null); {code} > Table Select *: > {code:java} > id | color | quantity > ----+-------+---------- > 1 | Blue | {code} > Debezium Behavior when using ExtractField and InsertField class (in the event > flattening SMT): > {code:java} > { > "id":"1", > "color":"Blue", > "quantity":1.0, > "__op":"c", > "__ts_ms":1698127432079, > "__source_ts_ms":1698127431679, > "__db":"testing_db", > "__schema":"public", > "__table":"products", > "__lsn":24470112, > "__txId":957, > "__snapshot":null, > "__deleted":"false" > } {code} > The debezium code can be found > [here|https://github.com/debezium/debezium/blob/2.4/debezium-core/src/main/java/io/debezium/transforms/ExtractNewRecordState.java#L116-L119] > h1. Expected Output: > {code:java} > { > "id":"1", > "color":"Blue", > "quantity":null, > "__op":"c", > "__ts_ms":1698127432079, > "__source_ts_ms":1698127431679, > "__db":"testing_db", > "__schema":"public", > "__table":"products", > "__lsn":24470112, > "__txId":957, > "__snapshot":null, > "__deleted":"false" > }{code} > h1. Temporary Solution: > use getWithoutDefault() into ExtractField and InsertField instead of get() -- This message was sent by Atlassian Jira (v8.20.10#820010)