Table of Contents |
---|
Status
Warning |
---|
Writing in progress... Not ready for review. |
Current state: "Under Discussion"
Discussion thread: https://www.mail-archive.com/dev@kafka.apache.org/msg124881.html
JIRA: KAFKA-13913
Motivation
To have more flexibility, builders should be provided for the following objects
KafkaAdminClient
- KafkaProducer
- KafkaConsumer
- KafkaStreams
...
From a user point of view, builders would be used as follow
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.clients.admin;
KafkaAdminClient kafkaAdminClient = new KafkaAdminClientBuilder(<MAP_OR_PROPERTIES_OR_CONFIG>)
.withMetricsReporter(<METRICS_REPORTER>)
...
.build(); |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.clients.producer; KafkaProducer kafkaProducer = new KafkaProducerBuilder<String, MyPojo>(<MAP_OR_PROPERTIES_OR_CONFIG>) .withKeySerializer(<KEY_SERIALIZER>) .withValueSerializer(<VALUE_SERIALIZER>) .withInterceptors(<LIST_OF_INTERCEPTORS>) .withPartitioner(<PARTITIONER>) .withMetricsReporter(<METRICS_REPORTER>) ... .build(); |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.clients.consumer; KafkaConsumer consumer = new KafkaConsumerBuilder<String, MyPojo>(<MAP_OR_PROPERTIES_OR_CONFIG>) .withKeyDeserializer(<KEY_DESERIALIZER>) .withValueDeserializer(<VALUE_DESERIALIZER>) .withInterceptors(<LIST_OF_INTERCEPTORS>) .withMetricsReporter(<METRICS_REPORTER>) ... .build(); |
Code Block | ||||
---|---|---|---|---|
| ||||
package org.apache.kafka.clients.streams; KafkaStreams kafkaStreams = new KafkaStreamsBuilder(<TOPOLOGY>, <MAP_OR_PROPERTIES_OR_CONFIG>) .withProducerInterceptors(<LIST_OF_PRODUER_INTERCEPTORS>) .withConsumerInterceptors(<LIST_OF_CONSUMER_INTERCEPTORS>) .withTime(<TIME>) .withKafkaClientSupplier(<KAFKA_CLIENT_SUPPLIER>) .withMetricsReporter(<METRICS_REPORTER>) ... .build(); |
To ease the integration with existing clients, a static builder may be added on related clients.
This KIP can be seen as the continuity of the KIP-832.
Proposed Changes
Three Four new builders will be added
- KafkaAdminClientBuilder
- KafkaProducerBuilder
- KafkaConsumerBuilder
- KafkaStreamsBuilder
...