Skip to content

Commit

Permalink
[kafka] Implement KeyValue support for KafkaBytesSource (apache#10002)
Browse files Browse the repository at this point in the history
  • Loading branch information
eolivelli authored Mar 25, 2021
1 parent 28e7e7c commit e4b4627
Show file tree
Hide file tree
Showing 9 changed files with 573 additions and 220 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -253,4 +253,9 @@ public String getTopicName() {
}
});
}

@Override
public String toString() {
return "KeyValueSchema(" + keyValueEncodingType + "," + keySchema + "," + valueSchema + ")";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -71,39 +71,11 @@ private Schema<ByteBuffer> fetchSchema(int schemaId) {
.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();
}
};
return new ByteBufferSchemaWrapper(schemaInfo);
} 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;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/**
* 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 org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.schema.SchemaInfo;

import java.nio.ByteBuffer;
import java.util.function.Supplier;

/**
* This is a ByteBuffer schema that reports SchemaInfo from another Schema instance.
*/
class ByteBufferSchemaWrapper implements Schema<ByteBuffer> {
private final Supplier<SchemaInfo> original;

public ByteBufferSchemaWrapper(Schema original) {
this(original::getSchemaInfo);
}

public ByteBufferSchemaWrapper(SchemaInfo info) {
this(() -> info);
}

public ByteBufferSchemaWrapper(Supplier<SchemaInfo> original) {
this.original = original;
}

@Override
public byte[] encode(ByteBuffer message) {
return getBytes(message);
}

@Override
public SchemaInfo getSchemaInfo() {
return original.get();
}

@Override
public Schema<ByteBuffer> clone() {
return new ByteBufferSchemaWrapper(original);
}

@Override
public ByteBuffer decode(byte[] bytes, byte[] schemaVersion) {
throw new UnsupportedOperationException();
}

public String toString() {
return original.get().toString();
}

static byte[] getBytes(ByteBuffer buffer) {
int remaining = buffer.remaining();
if (buffer.hasArray() && buffer.arrayOffset() == 0) {
// do not copy data if the ByteBuffer is a simple wrapper over an array
byte[] array = buffer.array();
if (array.length == remaining) {
return array;
}
}
buffer.mark();
byte[] avroEncodedData = new byte[remaining];
buffer.get(avroEncodedData);
buffer.reset();
return avroEncodedData;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.schema.KeyValue;
import org.apache.pulsar.common.schema.KeyValueEncodingType;
import org.apache.pulsar.functions.api.KVRecord;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.io.core.PushSource;
import org.apache.pulsar.io.core.SourceContext;
Expand Down Expand Up @@ -128,9 +131,10 @@ public void start() {
CompletableFuture<?>[] futures = new CompletableFuture<?>[consumerRecords.count()];
int index = 0;
for (ConsumerRecord<Object, Object> consumerRecord : consumerRecords) {
KafkaRecord record = new KafkaRecord(consumerRecord,
extractValue(consumerRecord),
extractSchema(consumerRecord));
KafkaRecord record = buildRecord(consumerRecord);
if (LOG.isDebugEnabled()) {
LOG.debug("Write record {} {} {}", record.getKey(), record.getValue(), record.getSchema());
}
consume(record);
futures[index] = record.getCompletableFuture();
index++;
Expand All @@ -153,14 +157,10 @@ public void start() {
runnerThread.start();
}

public Object extractValue(ConsumerRecord<Object, Object> consumerRecord) {
return consumerRecord.value();
}

public abstract Schema<V> extractSchema(ConsumerRecord<Object, Object> consumerRecord);
public abstract KafkaRecord buildRecord(ConsumerRecord<Object, Object> consumerRecord);

@Slf4j
static private class KafkaRecord<V> implements Record<V> {
protected static class KafkaRecord<V> implements Record<V> {
private final ConsumerRecord<String, ?> record;
private final V value;
private final Schema<V> schema;
Expand Down Expand Up @@ -203,4 +203,31 @@ public Schema<V> getSchema() {
return schema;
}
}
protected static class KeyValueKafkaRecord<V> extends KafkaRecord implements KVRecord<Object, Object> {

private final Schema<Object> keySchema;
private final Schema<Object> valueSchema;

public KeyValueKafkaRecord(ConsumerRecord record, KeyValue value,
Schema<Object> keySchema, Schema<Object> valueSchema) {
super(record, value, null);
this.keySchema = keySchema;
this.valueSchema = valueSchema;
}

@Override
public Schema<Object> getKeySchema() {
return keySchema;
}

@Override
public Schema<Object> getValueSchema() {
return valueSchema;
}

@Override
public KeyValueEncodingType getKeyValueEncodingType() {
return KeyValueEncodingType.SEPARATED;
}
}
}
Loading

0 comments on commit e4b4627

Please sign in to comment.