Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-12522: Cast SMT should allow null value records to pass through #10375

Merged
merged 1 commit into from
May 21, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,10 @@ public void configure(Map<String, ?> props) {

@Override
public R apply(R record) {
if (operatingValue(record) == null) {
return record;
}

if (operatingSchema(record) == null) {
return applySchemaless(record);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,42 @@ public void testConfigMixWholeAndFieldTransformation() {
assertThrows(ConfigException.class, () -> xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int8,int32")));
}

@Test
public void castNullValueRecordWithSchema() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64"));
SourceRecord original = new SourceRecord(null, null, "topic", 0,
Schema.STRING_SCHEMA, "key", Schema.STRING_SCHEMA, null);
SourceRecord transformed = xformValue.apply(original);
assertEquals(original, transformed);
}

@Test
public void castNullValueRecordSchemaless() {
xformValue.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64"));
SourceRecord original = new SourceRecord(null, null, "topic", 0,
Schema.STRING_SCHEMA, "key", null, null);
SourceRecord transformed = xformValue.apply(original);
assertEquals(original, transformed);
}

@Test
public void castNullKeyRecordWithSchema() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64"));
SourceRecord original = new SourceRecord(null, null, "topic", 0,
Schema.STRING_SCHEMA, null, Schema.STRING_SCHEMA, "value");
SourceRecord transformed = xformKey.apply(original);
assertEquals(original, transformed);
}

@Test
public void castNullKeyRecordSchemaless() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "foo:int64"));
SourceRecord original = new SourceRecord(null, null, "topic", 0,
null, null, Schema.STRING_SCHEMA, "value");
SourceRecord transformed = xformKey.apply(original);
assertEquals(original, transformed);
}

@Test
public void castWholeRecordKeyWithSchema() {
xformKey.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int8"));
Expand Down