forked from tchiotludo/akhq
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Support schema reference in record header for avro deserialization
supports producers like Spring Cloud Stream close tchiotludo#231 relate to tchiotludo#234
- Loading branch information
Showing
4 changed files
with
199 additions
and
30 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
65 changes: 65 additions & 0 deletions
65
src/main/java/org/kafkahq/repositories/AvroWireFormatConverter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
package org.kafkahq.repositories; | ||
|
||
import io.confluent.kafka.schemaregistry.client.SchemaMetadata; | ||
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; | ||
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.commons.lang3.ArrayUtils; | ||
import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
import org.apache.kafka.common.header.Header; | ||
|
||
import javax.inject.Singleton; | ||
import java.io.ByteArrayOutputStream; | ||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
import java.util.Iterator; | ||
import java.util.regex.Matcher; | ||
import java.util.regex.Pattern; | ||
|
||
/** | ||
* Converts an avro payload to the kafka avro wire format (https://docs.confluent.io/current/schema-registry/serializer-formatter.html#wire-format) | ||
* Some producers (like Spring Cloud Stream) do write this wire format, but use the raw avro binary encoding (without magic byte and schema id) | ||
* and put the reference to the schema in a header field. This converter will add the magic byte and schema id to the byte[] to | ||
* be wire format compatible if the following conditions are met: | ||
* - magic byte is not already present | ||
* - schema reference (subject and version) can be found in the message header | ||
* - schema can be fetch from the registry | ||
*/ | ||
@Singleton | ||
@Slf4j | ||
public class AvroWireFormatConverter { | ||
|
||
private static final byte MAGIC_BYTE = 0; | ||
private static final Pattern AVRO_CONTENT_TYPE_PATTERN = Pattern.compile("\"?application/vnd\\.(.+)\\.v(\\d+)\\+avro\"?"); | ||
|
||
public byte[] convertValueToWireFormat(ConsumerRecord<byte[], byte[]> record, SchemaRegistryClient registryClient) { | ||
Iterator<Header> contentTypeIter = record.headers().headers("contentType").iterator(); | ||
byte[] value = record.value(); | ||
if (contentTypeIter.hasNext() && | ||
ArrayUtils.isNotEmpty(value) && | ||
ByteBuffer.wrap(value).get() != MAGIC_BYTE) { | ||
String headerValue = new String(contentTypeIter.next().value()); | ||
Matcher matcher = AVRO_CONTENT_TYPE_PATTERN.matcher(headerValue); | ||
if (matcher.matches()) { | ||
String subject = matcher.group(1); | ||
int version = Integer.parseInt(matcher.group(2)); | ||
value = prependWireFormatHeader(value, registryClient, subject, version); | ||
} | ||
} | ||
return value; | ||
} | ||
|
||
private byte[] prependWireFormatHeader(byte[] value, SchemaRegistryClient registryClient, String subject, int version) { | ||
try { | ||
SchemaMetadata schemaMetadata = registryClient.getSchemaMetadata(subject, version); | ||
ByteArrayOutputStream out = new ByteArrayOutputStream(); | ||
out.write(MAGIC_BYTE); | ||
out.write(ByteBuffer.allocate(4).putInt(schemaMetadata.getId()).array()); | ||
out.write(value); | ||
value = out.toByteArray(); | ||
} catch (IOException | RestClientException e) { | ||
// ignore on purpose, dont prepend anything | ||
} | ||
return value; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
107 changes: 107 additions & 0 deletions
107
src/test/java/org/kafkahq/repositories/AvroWireFormatConverterTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
package org.kafkahq.repositories; | ||
|
||
import io.confluent.kafka.schemaregistry.client.SchemaMetadata; | ||
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; | ||
import io.confluent.kafka.serializers.AvroSchemaUtils; | ||
import io.confluent.kafka.serializers.KafkaAvroDeserializer; | ||
import lombok.AllArgsConstructor; | ||
import lombok.Data; | ||
import lombok.SneakyThrows; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.avro.Schema; | ||
import org.apache.avro.generic.GenericData; | ||
import org.apache.avro.io.DatumWriter; | ||
import org.apache.avro.io.Encoder; | ||
import org.apache.avro.io.EncoderFactory; | ||
import org.apache.avro.reflect.ReflectData; | ||
import org.apache.avro.reflect.ReflectDatumWriter; | ||
import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
import org.apache.kafka.common.header.internals.RecordHeader; | ||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import java.io.ByteArrayOutputStream; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
import static org.junit.jupiter.api.Assertions.assertNull; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
|
||
@Slf4j | ||
public class AvroWireFormatConverterTest { | ||
|
||
private AvroWireFormatConverter avroWireFormatConverter; | ||
private SchemaRegistryClient schemaRegistryClient; | ||
|
||
@Data | ||
@AllArgsConstructor | ||
private static class MyRecord { | ||
private int anInt; | ||
private String aString; | ||
} | ||
|
||
@BeforeEach | ||
@SneakyThrows | ||
public void before() { | ||
avroWireFormatConverter = new AvroWireFormatConverter(); | ||
schemaRegistryClient = mock(SchemaRegistryClient.class); | ||
|
||
ReflectData reflectData = ReflectData.get(); | ||
Schema schema = reflectData.getSchema(MyRecord.class); | ||
int id = 100; | ||
when(schemaRegistryClient.getById(id)).thenReturn(schema); | ||
when(schemaRegistryClient.getSchemaMetadata("mySubject", 1)).thenReturn(new SchemaMetadata(id, 1, "")); | ||
} | ||
|
||
@Test | ||
public void convertValueToWireFormatNull() { | ||
byte[] convertedValue = avroWireFormatConverter.convertValueToWireFormat(new ConsumerRecord<>("topic", 1, 0, new byte[0], null), schemaRegistryClient); | ||
assertNull(convertedValue); | ||
} | ||
|
||
@Test | ||
public void convertValueToWireFormatEmptyValue() { | ||
byte[] convertedValue = avroWireFormatConverter.convertValueToWireFormat(new ConsumerRecord<>("topic", 1, 0, new byte[0], new byte[0]), schemaRegistryClient); | ||
assertEquals(0, convertedValue.length); | ||
} | ||
|
||
@Test | ||
@SneakyThrows | ||
public void convertValueToWireFormatWrongContentType() { | ||
MyRecord record = new MyRecord(42, "leet"); | ||
byte[] avroPayload = serializeAvro(record); | ||
|
||
ConsumerRecord<byte[], byte[]> consumerRecord = new ConsumerRecord<>("topic", 1, 0, new byte[0], avroPayload); | ||
consumerRecord.headers().add(new RecordHeader("contentType", "mySubject.v1".getBytes())); | ||
byte[] convertedValue = avroWireFormatConverter.convertValueToWireFormat(consumerRecord, schemaRegistryClient); | ||
|
||
assertEquals(convertedValue, avroPayload); | ||
} | ||
|
||
@Test | ||
@SneakyThrows | ||
public void convertValueToWireFormatWireFormat() { | ||
MyRecord record = new MyRecord(42, "leet"); | ||
byte[] avroPayload = serializeAvro(record); | ||
|
||
ConsumerRecord<byte[], byte[]> consumerRecord = new ConsumerRecord<>("topic", 1, 0, new byte[0], avroPayload); | ||
consumerRecord.headers().add(new RecordHeader("contentType", "application/vnd.mySubject.v1+avro".getBytes())); | ||
byte[] convertedValue = avroWireFormatConverter.convertValueToWireFormat(consumerRecord, schemaRegistryClient); | ||
|
||
KafkaAvroDeserializer kafkaAvroDeserializer = new KafkaAvroDeserializer(schemaRegistryClient); | ||
GenericData.Record deserializedRecord = (GenericData.Record) kafkaAvroDeserializer.deserialize(null, convertedValue); | ||
assertEquals(record.getAnInt(), deserializedRecord.get(0)); | ||
assertEquals(record.getAString(), deserializedRecord.get(1).toString()); | ||
} | ||
|
||
@SneakyThrows | ||
private byte[] serializeAvro(MyRecord record) { | ||
Schema schema = AvroSchemaUtils.getSchema(record, true); | ||
DatumWriter<MyRecord> writer = new ReflectDatumWriter<>(schema); | ||
ByteArrayOutputStream stream = new ByteArrayOutputStream(); | ||
Encoder encoder = EncoderFactory.get().binaryEncoder(stream, null); | ||
writer.write(record, encoder); | ||
encoder.flush(); | ||
return stream.toByteArray(); | ||
} | ||
} |