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

[fix][io] KCA sink: handle null values with KeyValue<Avro,Avro> schema #19861

Merged
merged 4 commits into from
Mar 21, 2023
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -420,8 +420,8 @@ protected SinkRecord toSinkRecord(Record<GenericObject> sourceRecord) {

if (nativeObject instanceof KeyValue) {
KeyValue kv = (KeyValue) nativeObject;
key = KafkaConnectData.getKafkaConnectData(kv.getKey(), keySchema);
value = KafkaConnectData.getKafkaConnectData(kv.getValue(), valueSchema);
key = KafkaConnectData.getKafkaConnectDataFromSchema(kv.getKey(), keySchema);
value = KafkaConnectData.getKafkaConnectDataFromSchema(kv.getValue(), valueSchema);
} else if (nativeObject != null) {
throw new IllegalStateException("Cannot extract KeyValue data from " + nativeObject.getClass());
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,13 @@ private static List<Object> arrayToList(Object nativeObject, Schema kafkaValueSc
return out;
}


public static Object getKafkaConnectDataFromSchema(Object nativeObject, Schema kafkaSchema) {
if (kafkaSchema != null && nativeObject == null) {
return null;
}
return getKafkaConnectData(nativeObject, kafkaSchema);
}
@SuppressWarnings("unchecked")
public static Object getKafkaConnectData(Object nativeObject, Schema kafkaSchema) {
if (kafkaSchema == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,9 @@
import org.apache.pulsar.client.api.schema.Field;
import org.apache.pulsar.client.api.schema.GenericObject;
import org.apache.pulsar.client.api.schema.GenericRecord;
import org.apache.pulsar.client.api.schema.GenericSchema;
import org.apache.pulsar.client.api.schema.RecordSchemaBuilder;
import org.apache.pulsar.client.api.schema.SchemaBuilder;
import org.apache.pulsar.client.api.schema.SchemaDefinition;
import org.apache.pulsar.client.impl.BatchMessageIdImpl;
import org.apache.pulsar.client.impl.MessageIdImpl;
Expand All @@ -60,6 +63,7 @@
import org.apache.pulsar.client.impl.schema.JSONSchema;
import org.apache.pulsar.client.impl.schema.SchemaInfoImpl;
import org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord;
import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema;
import org.apache.pulsar.client.util.MessageIdUtils;
import org.apache.pulsar.common.schema.KeyValue;
import org.apache.pulsar.common.schema.SchemaInfo;
Expand Down Expand Up @@ -690,6 +694,20 @@ public void schemaKeyValueSchemaTest() throws Exception {
Assert.assertEquals(key, 11);
}

@Test
public void schemaKeyValueSchemaNullValueTest() throws Exception {
RecordSchemaBuilder builder = SchemaBuilder
.record("test");
builder.field("test").type(SchemaType.STRING);
GenericSchema<GenericRecord> schema = GenericAvroSchema.of(builder.build(SchemaType.AVRO));
KeyValue<Integer, String> kv = new KeyValue<>(11, null);
SinkRecord sinkRecord = recordSchemaTest(kv, Schema.KeyValue(Schema.INT32, schema), 11,
"INT32", null, "STRUCT");
Assert.assertNull(sinkRecord.value());
int key = (int) sinkRecord.key();
Assert.assertEquals(key, 11);
}

dlg99 marked this conversation as resolved.
Show resolved Hide resolved
@Test
public void kafkaLogicalTypesTimestampTest() {
Schema schema = new TestSchema(SchemaInfoImpl.builder()
Expand Down