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

[Issue #45] [pulsar-client-kafka-compat] Handled Kafka record headers… #46

Open
wants to merge 16 commits into
base: master
Choose a base branch
from
Open
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 @@ -47,6 +47,8 @@
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.StringDeserializer;
Expand All @@ -70,6 +72,7 @@
import org.apache.pulsar.client.util.MessageIdUtils;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.FutureUtil;
import org.bouncycastle.util.encoders.Hex;
swamymavuri marked this conversation as resolved.
Show resolved Hide resolved

@Slf4j
public class PulsarKafkaConsumer<K, V> implements Consumer<K, V>, MessageListener<byte[]> {
Expand Down Expand Up @@ -405,8 +408,13 @@ public ConsumerRecords<K, V> poll(long timeoutMillis) {
timestampType = TimestampType.CREATE_TIME;
}

Headers headers = new RecordHeaders();
if (msg.getProperties() != null) {
msg.getProperties().forEach((k, v) -> headers.add(k, Hex.decode(v)));
}

ConsumerRecord<K, V> consumerRecord = new ConsumerRecord<>(topic, partition, offset, timestamp,
timestampType, -1, msg.hasKey() ? msg.getKey().length() : 0, msg.getData().length, key, value);
timestampType, -1L, msg.hasKey() ? msg.getKey().length() : 0, msg.getData().length, key, value, headers);

records.computeIfAbsent(tp, k -> new ArrayList<>()).add(consumerRecord);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;

import org.apache.commons.codec.binary.Hex;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this import used at all?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, while encoding the kafka record headers

import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.Cluster;
Expand Down Expand Up @@ -387,6 +388,13 @@ private int buildMessage(TypedMessageBuilder<byte[]> builder, ProducerRecord<K,
builder.property(KafkaMessageRouter.PARTITION_ID, Integer.toString(partition));
}

if (record.headers() != null) {
Map<String, String> headerProperties = new HashMap<>();
record.headers()
.forEach(header -> headerProperties.putIfAbsent(header.key(), Hex.encodeHexString(header.value())));
builder.properties(headerProperties);
}

return value.length;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@
import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.pulsar.client.api.ClientBuilder;
import org.apache.pulsar.client.api.ProducerBuilder;
Expand Down Expand Up @@ -246,7 +248,13 @@ public void testPulsarKafkaSendAvro() throws PulsarClientException {
foo.setField2("field2");
foo.setField3(3);

pulsarKafkaProducer.send(new ProducerRecord<>("topic", 1,foo, bar));
Headers headers = new RecordHeaders();
String header1 = "header1";
String header2 = "header2";
headers.add(header1,header1.getBytes());
headers.add(header2,header2.getBytes());

pulsarKafkaProducer.send(new ProducerRecord<>("topic", 1,foo, bar, headers));
swamymavuri marked this conversation as resolved.
Show resolved Hide resolved

// Verify
verify(mockTypedMessageBuilder, times(1)).sendAsync();
Expand Down