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 9 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
@@ -0,0 +1,28 @@
/**
* 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.constants;

public final class MessageConstants {
private MessageConstants() {

}

public static final String KAFKA_MESSAGE_HEADER_PREFIX = "kafka.header.";
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,15 @@
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.codec.DecoderException;
import org.apache.commons.codec.binary.Hex;
import org.apache.kafka.clients.constants.MessageConstants;
import org.apache.kafka.common.Metric;
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 Down Expand Up @@ -404,10 +409,28 @@ public ConsumerRecords<K, V> poll(long timeoutMillis) {
timestamp = msg.getEventTime();
timestampType = TimestampType.CREATE_TIME;
}

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

ConsumerRecord<K, V> consumerRecord;
if (msg.getProperties() != null) {
Headers headers = new RecordHeaders();
msg.getProperties().forEach((k, v) -> {
if (k.startsWith(MessageConstants.KAFKA_MESSAGE_HEADER_PREFIX)) {
String originalKey = k.replace(MessageConstants.KAFKA_MESSAGE_HEADER_PREFIX, "");
try {
headers.add(originalKey, Hex.decodeHex(v));
} catch (DecoderException e) {
log.warn("Corrupted Header Key : {}", originalKey);
throw new RuntimeException(e);
}
}
});
consumerRecord = new ConsumerRecord<>(topic, partition, offset, timestamp, timestampType, -1L,
msg.hasKey() ? msg.getKey().length() : 0, msg.getData() != null ? msg.getData().length : 0,
key, value, headers);
} else {
consumerRecord = new ConsumerRecord<>(topic, partition, offset, timestamp, timestampType, -1L,
msg.hasKey() ? msg.getKey().length() : 0, msg.getData() != null ? msg.getData().length : 0,
key, value);
}
records.computeIfAbsent(tp, k -> new ArrayList<>()).add(consumerRecord);

// Update last offset seen by application
Expand Down Expand Up @@ -520,10 +543,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 @@ -18,6 +18,7 @@
*/
package org.apache.kafka.clients.producer;

import com.google.common.annotations.VisibleForTesting;
import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.Base64;
Expand All @@ -36,7 +37,9 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;

import lombok.Getter;
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.constants.MessageConstants;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.Cluster;
Expand Down Expand Up @@ -64,10 +67,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.google.common.annotations.VisibleForTesting;

import lombok.Getter;

public class PulsarKafkaProducer<K, V> implements Producer<K, V> {

private static final Logger log = LoggerFactory.getLogger(PulsarKafkaProducer.class);
Expand Down Expand Up @@ -387,6 +386,14 @@ private int buildMessage(TypedMessageBuilder<byte[]> builder, ProducerRecord<K,
builder.property(KafkaMessageRouter.PARTITION_ID, Integer.toString(partition));
}

if (record.headers() != null) {
record.headers().forEach(header -> {
String key = MessageConstants.KAFKA_MESSAGE_HEADER_PREFIX + header.key();
builder.property(key, Hex.encodeHexString(header.value()));
log.debug("Formatted Kafka Specific Headers Before : {}, After : {}", header.key(), key);
});
}

return value.length;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
/**
* 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.anyCollection;
import static org.mockito.ArgumentMatchers.anyString;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import org.apache.commons.codec.binary.Hex;
import org.apache.kafka.clients.constants.MessageConstants;
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.api.Schema;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.naming.TopicName;
import org.mockito.Mockito;
import org.testng.Assert;
import org.testng.annotations.Test;

public class PulsarKafkaConsumerTest {

@Test
public void testPulsarKafkaConsumerWithHeaders_noAck() throws Exception {
TopicName topicName = Mockito.mock(TopicName.class);
Hex hex = Mockito.mock(Hex.class);

Mockito.when(topicName.getPartitionedTopicName()).thenReturn("topic");

ClientBuilder mockClientBuilder = Mockito.mock(ClientBuilder.class);
Consumer consumer = Mockito.mock(Consumer.class);
Mockito.doReturn("topic").when(consumer).getTopic();
MessageId msgId = Mockito.mock(MessageId.class);
MessageMetadata messageMetadata = new MessageMetadata();
messageMetadata.setPublishTime(System.currentTimeMillis());
String topic = "topic";

TopicName topicNameAll = TopicName.get(topic);

Map<String, String> headerMap = new HashMap<>();
String header1 = MessageConstants.KAFKA_MESSAGE_HEADER_PREFIX + "header1";
String kafkaHeader = MessageConstants.KAFKA_MESSAGE_HEADER_PREFIX + header1;
headerMap.put(kafkaHeader, Hex.encodeHexString(header1.getBytes()));
Message<byte[]> msg =
new MessageImpl<byte[]>(topic, "1:1", headerMap, "string".getBytes(), Schema.BYTES, messageMetadata);

PulsarClient mockClient = Mockito.mock(PulsarClient.class);
PulsarClientImpl mockClientImpl = Mockito.mock(PulsarClientImpl.class);

CompletableFuture<Integer> mockNoOfPartitionFuture = CompletableFuture.completedFuture(1);

Mockito.doReturn(mockClientBuilder).when(mockClientBuilder).serviceUrl(Mockito.anyString());
Mockito.doReturn(mockClient).when(mockClientBuilder).build();

Mockito.when(mockClientImpl.getNumberOfPartitions(Mockito.anyString())).thenReturn(mockNoOfPartitionFuture);

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> pulsarKafkaConsumer =
new PulsarKafkaConsumer<>(properties, new IntegerDeserializer(), new StringDeserializer());

PulsarKafkaConsumer<Integer, String> pulsarKafkaConsumerSpy = Mockito.spy(pulsarKafkaConsumer);

Mockito.doNothing().when(pulsarKafkaConsumerSpy).seekToEnd(anyCollection());

pulsarKafkaConsumerSpy.received(consumer, msg);
pulsarKafkaConsumerSpy.poll(100);
pulsarKafkaConsumerSpy.close();

Assert.assertNotNull(msg.getProperty(kafkaHeader));
Mockito.verify(pulsarKafkaConsumerSpy).seekToEnd(anyCollection());
Mockito.verify(consumer, Mockito.times(0)).acknowledgeCumulativeAsync(Mockito.any(MessageId.class));
Mockito.verify(hex, Mockito.times(1)).decodeHex(Hex.encodeHexString(header1.getBytes()));
}

}

Loading