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 3 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,14 @@ 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() != null ? msg.getData().length : 0, key, value, headers);

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

Expand Down Expand Up @@ -520,10 +529,12 @@ private CompletableFuture<Void> doCommitOffsets(Map<TopicPartition, OffsetAndMet

lastCommittedOffset.put(tp, offsetAndMetadata);
MessageId msgId = MessageIdUtils.getMessageId(offsetAndMetadata.offset());
if (consumer instanceof MultiTopicsConsumerImpl) {
msgId = new TopicMessageIdImpl(topicPartition.topic(), tp.topic(), msgId);
if (consumer != null) {
if (consumer instanceof MultiTopicsConsumerImpl) {
msgId = new TopicMessageIdImpl(topicPartition.topic(), tp.topic(), msgId);
}
futures.add(consumer.acknowledgeCumulativeAsync(msgId));
}
futures.add(consumer.acknowledgeCumulativeAsync(msgId));
});

return FutureUtil.waitForAll(futures);
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
@@ -0,0 +1,144 @@
/**
* 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.kafka.clients.consumer;

import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyCollection;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.util.Collections;
import java.util.Properties;
import org.apache.kafka.common.serialization.IntegerDeserializer;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.pulsar.client.api.ClientBuilder;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.util.MessageIdUtils;
import org.apache.pulsar.common.naming.TopicName;
import org.powermock.api.mockito.PowerMockito;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.testng.IObjectFactory;
import org.testng.annotations.ObjectFactory;
import org.testng.annotations.Test;

@PrepareForTest({TopicName.class, MessageIdUtils.class})
@PowerMockIgnore({"org.apache.logging.log4j.*"})
public class PulsarKafkaConsumerTest {

@ObjectFactory
public IObjectFactory getObjectFactory() {
return new org.powermock.modules.testng.PowerMockObjectFactory();
}

@Test
public void testPulsarKafkaConsumerWithHeaders() throws Exception {
PowerMockito.mockStatic(TopicName.class);
PowerMockito.mockStatic(MessageIdUtils.class);
swamymavuri marked this conversation as resolved.
Show resolved Hide resolved

TopicName topicName = mock(TopicName.class);

doReturn("topic").when(topicName).getPartitionedTopicName();

ClientBuilder mockClientBuilder = mock(ClientBuilder.class);
Consumer consumer = mock(Consumer.class);
Message msg = mock(Message.class);
MessageId msgId = mock(MessageId.class);

PulsarClient mockClient = mock(PulsarClient.class);

doReturn(mockClientBuilder).when(mockClientBuilder).serviceUrl(anyString());
when(TopicName.get(any())).thenReturn(topicName);
when(msg.getMessageId()).thenReturn(msgId);
doReturn(mockClient).when(mockClientBuilder).build();

Properties properties = new Properties();

properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class.getName());
properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, Collections.singletonList("pulsar://localhost:6650"));
properties.put(ConsumerConfig.GROUP_ID_CONFIG, "my-subscription-name");


PulsarKafkaConsumer<Integer, String> pulsarKafkaConsumerSpy =
spy(new PulsarKafkaConsumer<>(properties, new IntegerDeserializer(), new StringDeserializer()));

doNothing().when(pulsarKafkaConsumerSpy).seekToEnd(anyCollection());
PowerMockito.whenNew(PulsarKafkaConsumer.class).withAnyArguments().thenReturn(pulsarKafkaConsumerSpy);

PulsarKafkaConsumer<Integer, String> pulsarKafkaConsumer =
new PulsarKafkaConsumer<>(properties, new IntegerDeserializer(), new StringDeserializer());

pulsarKafkaConsumer.received(consumer, msg);
pulsarKafkaConsumer.poll(100);
pulsarKafkaConsumer.close();
}

@Test
public void testPulsarKafkaConsumer() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't understand what this test does. There are no assertions, no messages, no headers verification, ...

Copy link
Author

Choose a reason for hiding this comment

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

Added assertion statements for Header Verification

PowerMockito.mockStatic(TopicName.class);
PowerMockito.mockStatic(MessageIdUtils.class);

TopicName topicName = mock(TopicName.class);

doReturn("topic").when(topicName).getPartitionedTopicName();

ClientBuilder mockClientBuilder = mock(ClientBuilder.class);
Consumer<byte[]> consumer = mock(Consumer.class);
Message<byte[]> msg = mock(Message.class);
MessageId msgId = mock(MessageId.class);

PulsarClient mockClient = mock(PulsarClient.class);

doReturn(mockClientBuilder).when(mockClientBuilder).serviceUrl(anyString());
when(TopicName.get(any())).thenReturn(topicName);
when(msg.getMessageId()).thenReturn(msgId);
doReturn(mockClient).when(mockClientBuilder).build();

Properties properties = new Properties();

properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class.getName());
properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, Collections.singletonList("pulsar://localhost:6650"));
properties.put(ConsumerConfig.GROUP_ID_CONFIG, "my-subscription-name");


PulsarKafkaConsumer<Integer, String> pulsarKafkaConsumerSpy =
spy(new PulsarKafkaConsumer<>(properties, new IntegerDeserializer(), new StringDeserializer()));

doNothing().when(pulsarKafkaConsumerSpy).seekToEnd(anyCollection());
PowerMockito.whenNew(PulsarKafkaConsumer.class).withAnyArguments().thenReturn(pulsarKafkaConsumerSpy);

PulsarKafkaConsumer<Integer, String> pulsarKafkaConsumer =
new PulsarKafkaConsumer<>(properties, new IntegerDeserializer(), new StringDeserializer());

pulsarKafkaConsumer.poll(100);
pulsarKafkaConsumer.close();
}
}




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