forked from apache/pulsar
-
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.
Pulsar IO - KafkaSource - allow to manage Avro Encoded messages (apac…
…he#9448) ### Motivation Currently KafkaSource allows only to deal with strings and byte arrays, it does not support records with Schema. In Kafka we have the ability to encode messages using Avro and there is a Schema Registry (by Confluent®) ### Modifications Summary of changes: - allow current KafkaSource (`KafkaBytesSource`) to deal with `io.confluent.kafka.serializers.KafkaAvroDeserializer ` and copy the raw bytes to the Pulsar topic, setting appropriately the Schema - this source support Schema Evolution end-to-end (i.e. add fields to the original schema in the Kafka world, and see the new fields in the Pulsar topic, without any reconfiguration or restart) - add Confluent® Schema Registry Client to the Kafka Connector NAR, the license is compatible with Apache 2 license and we can redistribute it - the configuration of the Schema Registry Client is done done in the consumerProperties property of the source (usually you add schema.registry.url) - add integration tests with Kafka and Schema Registry ### Verifying this change The patch introduces new integration tests. The integration tests launch a Kafka Container and also a Confluent Schema Registry Container
- Loading branch information
Showing
10 changed files
with
786 additions
and
27 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
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
109 changes: 109 additions & 0 deletions
109
pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/AvroSchemaCache.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,109 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.pulsar.io.kafka; | ||
|
||
import com.google.common.cache.CacheBuilder; | ||
import com.google.common.cache.CacheLoader; | ||
import com.google.common.cache.LoadingCache; | ||
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; | ||
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.pulsar.client.api.Schema; | ||
import org.apache.pulsar.common.schema.SchemaInfo; | ||
import org.apache.pulsar.common.schema.SchemaType; | ||
|
||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
import java.nio.charset.StandardCharsets; | ||
import java.util.Collections; | ||
import java.util.concurrent.ExecutionException; | ||
|
||
@Slf4j | ||
final class AvroSchemaCache { | ||
private final LoadingCache<Integer, Schema<ByteBuffer>> cache = CacheBuilder | ||
.newBuilder() | ||
.maximumSize(100) | ||
.build(new CacheLoader<Integer, Schema<ByteBuffer>>() { | ||
@Override | ||
public Schema<ByteBuffer> load(Integer schemaId) throws Exception { | ||
return fetchSchema(schemaId); | ||
} | ||
}); | ||
|
||
private final SchemaRegistryClient schemaRegistryClient; | ||
|
||
public AvroSchemaCache(SchemaRegistryClient schemaRegistryClient) { | ||
this.schemaRegistryClient = schemaRegistryClient; | ||
} | ||
|
||
public Schema<ByteBuffer> get(int schemaId) { | ||
try { | ||
return cache.get(schemaId); | ||
} catch (ExecutionException err) { | ||
throw new RuntimeException(err.getCause()); | ||
} | ||
} | ||
|
||
private Schema<ByteBuffer> fetchSchema(int schemaId) { | ||
try { | ||
org.apache.avro.Schema schema = schemaRegistryClient.getById(schemaId); | ||
String definition = schema.toString(false); | ||
log.info("Schema {} definition {}", schemaId, definition); | ||
SchemaInfo schemaInfo = SchemaInfo.builder() | ||
.type(SchemaType.AVRO) | ||
.name(schema.getName()) | ||
.properties(Collections.emptyMap()) | ||
.schema(definition.getBytes(StandardCharsets.UTF_8) | ||
).build(); | ||
return new Schema<ByteBuffer>() { | ||
@Override | ||
public byte[] encode(ByteBuffer message) { | ||
return getBytes(message); | ||
} | ||
|
||
@Override | ||
public SchemaInfo getSchemaInfo() { | ||
return schemaInfo; | ||
} | ||
|
||
@Override | ||
public Schema<ByteBuffer> clone() { | ||
return this; | ||
} | ||
|
||
@Override | ||
public ByteBuffer decode(byte[] bytes, byte[] schemaVersion) { | ||
throw new UnsupportedOperationException(); | ||
} | ||
}; | ||
} catch (IOException | RestClientException e) { | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
|
||
|
||
private static byte[] getBytes(ByteBuffer buffer) { | ||
buffer.mark(); | ||
byte[] avroEncodedData = new byte[buffer.remaining()]; | ||
buffer.get(avroEncodedData); | ||
buffer.reset(); | ||
return avroEncodedData; | ||
} | ||
|
||
} |
32 changes: 32 additions & 0 deletions
32
pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/BytesWithKafkaSchema.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,32 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.pulsar.io.kafka; | ||
|
||
import lombok.Value; | ||
|
||
import java.nio.ByteBuffer; | ||
|
||
/** | ||
* This is a wrapper around a Byte array (the Avro encoded record) and a schema id in the Kafka Schema Registry. | ||
*/ | ||
@Value | ||
public class BytesWithKafkaSchema { | ||
private final ByteBuffer value; | ||
private final int schemaId; | ||
} |
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
Oops, something went wrong.