Pulsar adaptor for Apache Kafka

Pulsar provides an easy option for applications that are currently written using the Apache Kafka Java client API.

Using the Pulsar Kafka compatibility wrapper

In an existing application, change the regular Kafka client dependency and replace it with the Pulsar Kafka wrapper. Remove the following dependency in pom.xml:

  1. <dependency>
  2. <groupId>org.apache.kafka</groupId>
  3. <artifactId>kafka-clients</artifactId>
  4. <version>0.10.2.1</version>
  5. </dependency>

Then include this dependency for the Pulsar Kafka wrapper:

  1. <dependency>
  2. <groupId>org.apache.pulsar</groupId>
  3. <artifactId>pulsar-client-kafka</artifactId>
  4. <version>2.10.0</version>
  5. </dependency>

With the new dependency, the existing code works without any changes. You need to adjust the configuration, and make sure it points the producers and consumers to Pulsar service rather than Kafka, and uses a particular Pulsar topic.

Using the Pulsar Kafka compatibility wrapper together with existing kafka client

When migrating from Kafka to Pulsar, the application might use the original kafka client and the pulsar kafka wrapper together during migration. You should consider using the unshaded pulsar kafka client wrapper.

  1. <dependency>
  2. <groupId>org.apache.pulsar</groupId>
  3. <artifactId>pulsar-client-kafka-original</artifactId>
  4. <version>2.10.0</version>
  5. </dependency>

When using this dependency, construct producers using org.apache.kafka.clients.producer.PulsarKafkaProducer instead of org.apache.kafka.clients.producer.KafkaProducer and org.apache.kafka.clients.producer.PulsarKafkaConsumer for consumers.

Producer example

  1. // Topic needs to be a regular Pulsar topic
  2. String topic = "persistent://public/default/my-topic";
  3. Properties props = new Properties();
  4. // Point to a Pulsar service
  5. props.put("bootstrap.servers", "pulsar://localhost:6650");
  6. props.put("key.serializer", IntegerSerializer.class.getName());
  7. props.put("value.serializer", StringSerializer.class.getName());
  8. Producer<Integer, String> producer = new KafkaProducer(props);
  9. for (int i = 0; i < 10; i++) {
  10. producer.send(new ProducerRecord<Integer, String>(topic, i, "hello-" + i));
  11. log.info("Message {} sent successfully", i);
  12. }
  13. producer.close();

Consumer example

  1. String topic = "persistent://public/default/my-topic";
  2. Properties props = new Properties();
  3. // Point to a Pulsar service
  4. props.put("bootstrap.servers", "pulsar://localhost:6650");
  5. props.put("group.id", "my-subscription-name");
  6. props.put("enable.auto.commit", "false");
  7. props.put("key.deserializer", IntegerDeserializer.class.getName());
  8. props.put("value.deserializer", StringDeserializer.class.getName());
  9. Consumer<Integer, String> consumer = new KafkaConsumer(props);
  10. consumer.subscribe(Arrays.asList(topic));
  11. while (true) {
  12. ConsumerRecords<Integer, String> records = consumer.poll(100);
  13. records.forEach(record -> {
  14. log.info("Received record: {}", record);
  15. });
  16. // Commit last offset
  17. consumer.commitSync();
  18. }

Complete Examples

You can find the complete producer and consumer examples here.

Compatibility matrix

Currently the Pulsar Kafka wrapper supports most of the operations offered by the Kafka API.

Producer

APIs:

Producer MethodSupportedNotes
Future<RecordMetadata> send(ProducerRecord<K, V> record)Yes
Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback)Yes
void flush()Yes
List<PartitionInfo> partitionsFor(String topic)No
Map<MetricName, ? extends Metric> metrics()No
void close()Yes
void close(long timeout, TimeUnit unit)Yes

Properties:

Config propertySupportedNotes
acksIgnoredDurability and quorum writes are configured at the namespace level
auto.offset.resetYesIt uses a default value of earliest if you do not give a specific setting.
batch.sizeIgnored
bootstrap.serversYes
buffer.memoryIgnored
client.idIgnored
compression.typeYesAllows gzip and lz4. No snappy.
connections.max.idle.msYesOnly support up to 2,147,483,647,000(Integer.MAX_VALUE * 1000) ms of idle time
interceptor.classesYes
key.serializerYes
linger.msYesControls the group commit time when batching messages
max.block.msIgnored
max.in.flight.requests.per.connectionIgnoredIn Pulsar ordering is maintained even with multiple requests in flight
max.request.sizeIgnored
metric.reportersIgnored
metrics.num.samplesIgnored
metrics.sample.window.msIgnored
partitioner.classYes
receive.buffer.bytesIgnored
reconnect.backoff.msIgnored
request.timeout.msIgnored
retriesIgnoredPulsar client retries with exponential backoff until the send timeout expires.
send.buffer.bytesIgnored
timeout.msYes
value.serializerYes

Consumer

The following table lists consumer APIs.

Consumer MethodSupportedNotes
Set<TopicPartition> assignment()No
Set<String> subscription()Yes
void subscribe(Collection<String> topics)Yes
void subscribe(Collection<String> topics, ConsumerRebalanceListener callback)No
void assign(Collection<TopicPartition> partitions)No
void subscribe(Pattern pattern, ConsumerRebalanceListener callback)No
void unsubscribe()Yes
ConsumerRecords<K, V> poll(long timeoutMillis)Yes
void commitSync()Yes
void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets)Yes
void commitAsync()Yes
void commitAsync(OffsetCommitCallback callback)Yes
void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCommitCallback callback)Yes
void seek(TopicPartition partition, long offset)Yes
void seekToBeginning(Collection<TopicPartition> partitions)Yes
void seekToEnd(Collection<TopicPartition> partitions)Yes
long position(TopicPartition partition)Yes
OffsetAndMetadata committed(TopicPartition partition)Yes
Map<MetricName, ? extends Metric> metrics()No
List<PartitionInfo> partitionsFor(String topic)No
Map<String, List<PartitionInfo>> listTopics()No
Set<TopicPartition> paused()No
void pause(Collection<TopicPartition> partitions)No
void resume(Collection<TopicPartition> partitions)No
Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartition, Long> timestampsToSearch)No
Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions)No
Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions)No
void close()Yes
void close(long timeout, TimeUnit unit)Yes
void wakeup()No

Properties:

Config propertySupportedNotes
group.idYesMaps to a Pulsar subscription name
max.poll.recordsYes
max.poll.interval.msIgnoredMessages are “pushed” from broker
session.timeout.msIgnored
heartbeat.interval.msIgnored
bootstrap.serversYesNeeds to point to a single Pulsar service URL
enable.auto.commitYes
auto.commit.interval.msIgnoredWith auto-commit, acks are sent immediately to broker
partition.assignment.strategyIgnored
auto.offset.resetYesOnly support earliest and latest.
fetch.min.bytesIgnored
fetch.max.bytesIgnored
fetch.max.wait.msIgnored
interceptor.classesYes
metadata.max.age.msIgnored
max.partition.fetch.bytesIgnored
send.buffer.bytesIgnored
receive.buffer.bytesIgnored
client.idIgnored

Customize Pulsar configurations

You can configure Pulsar authentication provider directly from the Kafka properties.

Pulsar client properties

Config propertyDefaultNotes
pulsar.authentication.classConfigure to auth provider. For example, org.apache.pulsar.client.impl.auth.AuthenticationTls.
pulsar.authentication.params.mapMap which represents parameters for the Authentication-Plugin.
pulsar.authentication.params.stringString which represents parameters for the Authentication-Plugin, for example, key1:val1,key2:val2.
pulsar.use.tlsfalseEnable TLS transport encryption.
pulsar.tls.trust.certs.file.pathPath for the TLS trust certificate store.
pulsar.tls.allow.insecure.connectionfalseAccept self-signed certificates from brokers.
pulsar.operation.timeout.ms30000General operations timeout.
pulsar.stats.interval.seconds60Pulsar client lib stats printing interval.
pulsar.num.io.threads1The number of Netty IO threads to use.
pulsar.connections.per.broker1The maximum number of connection to each broker.
pulsar.use.tcp.nodelaytrueTCP no-delay.
pulsar.concurrent.lookup.requests50000The maximum number of concurrent topic lookups.
pulsar.max.number.rejected.request.per.connection50The threshold of errors to forcefully close a connection.
pulsar.keepalive.interval.ms30000Keep alive interval for each client-broker-connection.

Pulsar producer properties

Config propertyDefaultNotes
pulsar.producer.nameSpecify the producer name.
pulsar.producer.initial.sequence.idSpecify baseline for sequence ID of this producer.
pulsar.producer.max.pending.messages1000Set the maximum size of the message queue pending to receive an acknowledgment from the broker.
pulsar.producer.max.pending.messages.across.partitions50000Set the maximum number of pending messages across all the partitions.
pulsar.producer.batching.enabledtrueControl whether automatic batching of messages is enabled for the producer.
pulsar.producer.batching.max.messages1000The maximum number of messages in a batch.
pulsar.block.if.producer.queue.fullSpecify the block producer if queue is full.
pulsar.crypto.reader.factory.class.nameSpecify the CryptoReader-Factory(CryptoKeyReaderFactory) classname which allows producer to create CryptoKeyReader.

Pulsar consumer Properties

Config propertyDefaultNotes
pulsar.consumer.nameSpecify the consumer name.
pulsar.consumer.receiver.queue.size1000Set the size of the consumer receiver queue.
pulsar.consumer.acknowledgments.group.time.millis100Set the maximum amount of group time for consumers to send the acknowledgments to the broker.
pulsar.consumer.total.receiver.queue.size.across.partitions50000Set the maximum size of the total receiver queue across partitions.
pulsar.consumer.subscription.topics.modePersistentOnlySet the subscription topic mode for consumers.
pulsar.crypto.reader.factory.class.nameSpecify the CryptoReader-Factory(CryptoKeyReaderFactory) classname which allows consumer to create CryptoKeyReader.