Table of Contents |
---|
Status
Current state: "Under DiscussionAdopted"
Discussion thread:
JIRA: KAFKA-13602
...
Lot of times, in Kafka Streams users want to send a record to more than one partition on the sink topic. Currently, if a user wants to replicate a message into N partitions, the only way of doing that is to replicate the message N times and then plug-in a new custom partitioner to write the message N times into N different partitions. To give a more concrete example, let's say there's a sink topic with 10 partitions and a user wants to send records only to even numbered partitions.
...
Code Block | ||
---|---|---|
| ||
final int numPartitions = 10; final KStream<String, String> inputStream = builder.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String())); for (int i = 0; i < numPartitions; i += 2) { final int evenNumberedPartition = i; inputStream.to("output-topic", Produced.with(Serdes.String(), Serdes.String(), (topic, key, val, numPartitions) -> evenNumberedPartition)); } |
This seems a little cumbersome way to broadcast. Also, there seems to be no way of dropping a record within the partitionerAs can be seen, there's no implicit way of sending the records to multiple partitions. This KIP aims to make this process simpler in Kafka Streams. As a side note, this KIP also adds a provision to drop records using a custom partitioner.
Public Interfaces
The StreamPartitioner method would have a new method added called partitions()
and the current partition()
method would be marked as deprecated. The partitions()
method would be marked as default within which it would invoke partition()
method and construct a singleton list set out of it. Here's how the interface would look like now:
Code Block | ||
---|---|---|
| ||
import java.util.ListSet; publicimport java.util.Collections; public interface StreamPartitioner<K, V> { /** * Determine the partition number for a record with the given key and value and the current number of partitions. * * @param topic the topic name this record is sent to * @param key the key of the record * @param value the value of the record * @param numPartitions the total number of partitions * @return an integer between 0 and {@code numPartitions-1}, or {@code null} if the default partitioning logic should be used */ @Deprecated Integer partition(String topic, K key, V value, int numPartitions); /** * Determine the List of partition numbers to which a record, with the given key and value and the current number * of partitions, should be multi-casted to. Note that returning* a@param singletopic valuedthe listtopic withname valuethis -1record is asent shorthandto * for@param broadcastingkey the recordkey toof all the partitions of the topic. *record * @param topicvalue the topicvalue nameof thisthe record is sent to * @param keynumPartitions the total keynumber of the recordpartitions * @param@return valuean theOptional value of theSet record of integers between 0 and * @param numPartitions the total number of partitions * @return a List of integers between 0 and {@code numPartitions-1}, null would mean a broadcast to all partitions * while an empty list would mean the record would not be sent to any partitions. * the partitions of the topic. */ {@code numPartitions-1}, * Empty optional means use default partitioner * Optional of an empty set means the record won't be sent to any partitions i.e dropped. * Optional of Set of integers means the partitions to which the record should be sent to. * */ default List<Integer>Optional<Set<Integer>> partitions(String topic, K key, V value, int numPartitions) { return Collections.singletonListInteger partition = partition(topic, key, value, numPartitions); return partition == null ? Optional.empty() : Optional.of(Collections.singleton(partition(topic, key, value, numPartitions))); } } |
The return type is a Set so that for cases of a faulty implementation of partitions
method which might return same partition number multiple times, we would still record it only once.
Proposed ChangesProposed Changes
- RecordCollector and it's implementation RecordCollectorImpl has the method which accepts a StreamPartitioner object and pushes it to the partitions returned by the partition method. This is the core piece of logic which would allow multi/broadcasting records. Here are the high level changes. Note that when the
partitions()
method return an empty list set meaning we won't send the record to any partition, we would also be updating the droppedRecordsSensor.Code Block language java java @Override public <K, V> void send(final String topic, final K key, final V value, @Override public <K, V> void send(final String topic final Headers headers, final KLong keytimestamp, final VSerializer<K> valuekeySerializer, final Serializer<V> valueSerializer, final Headers headers, final StreamPartitioner<? super K, ? super V> partitioner) { final Long timestamp, if (partitioner != null) { // Existing logic to fetch partitions // Changes finalto Serializer<K>be keySerializer, made due to the KIP if (partitions.size() > 0) { final Serializer<V> valueSerializer, final Optional<Set<Integer>> maybeMulticastPartitions = partitioner.partitions(topic, key, value, partitions.size()); if (!maybeMulticastPartitions.isPresent()) { final StreamPartitioner<? super K, ? super V> partitioner) { List<Integer> multicastPartitions; // New change. Use default ifpartitioner (partitioner != null) { // Existing logic to fetch partitions // Changes to be made due to the KIPsend(topic, key, value, headers, null, timestamp, keySerializer, valueSerializer, processorNodeId, context); if (partitions.size() > 0) } else { multicastPartitions = partitioner.partitions(topic, key, value, partitions.size() Set<Integer> multicastPartitions = maybeMulticastPartitions.get(); if (multicastPartitions.isEmpty()) { // New change. // If a record is not to be sent to any partitionspartition, mark it as being // as a dropped record. dropped. log.info("Not sending the record to any partition"); droppedRecordsSensor.record(); } else { // broadcast iffor (final int multicastPartition: multicastPartitions == null) { ) { send(topic, key, value, // Broadcast to all partitions headers, multicastPartition, timestamp, keySerializer, valueSerializer, processorNodeId, context); multicastPartitions = partitions.stream().map(PartitionInfo::partition).collect(Collectors.toList()); } } } // iterate over the subset of partitions and send to those. } else { throw new StreamsException("Could not get partition information for topic " for+ (inttopic p:+ multicastPartitions)" { for task " + taskId + send(topic, key, value, headers, p, timestamp, keySerializer, valueSerializer); ". This can happen if the topic does not exist."); } } else { } send(topic, key, value, headers, null, timestamp, keySerializer, valueSerializer, processorNodeId, context); } // Existing logic. } }
- StreamPartitioner is also used in FK-join and Interactive queries. The queries. For FK-join and IQ, the invocation of
partition()
would be replaced bypartitions()
and a runtime check would be added to ensure that the returned list set is singleton.
Example Usage
...
Code Block | ||
---|---|---|
| ||
// Define a partitioner class which sends to even numbered partitions public static class EvenPartitioner<K, V> implements StreamPartitioner<K, V> { @Override public Integer partition(String topic, K key, V value, int numPartitions) { return null; } @Override public List<Integer>Set<Integer> partitions(String topic, K key, V value, int numPartitions) { final List<Integer>Set<Integer> partitions = new ArrayList<>HashSet<>(); for (int i = 0; i < numPartitions; i += 2) { partitions.add(i); } return partitions; } } // Broadcasting public static class BroadcastingPartitioner<K, V> implements StreamPartitioner<K, V> { @Override public Integer partition(String topic, K key, V value, int numPartitions) { return null; } @Override public List<Integer>Set<Integer> partitions(String topic, K key, V value, int numPartitions) { return null IntStream.range(0, numPartitions).boxed().collect(Collectors.toSet()); } } // Don't send to any partitions public static class DroppingPartitioner<K, V> implements StreamPartitioner<K, V> { @Override public Integer partition(String topic, K key, V value, int numPartitions) { return null; } @Override public List<Integer>Set<Integer> partitions(String topic, K key, V value, int numPartitions) { return Collections.emptyListemptySet(); } } // Build Stream. final KStream<String, String> inputStream = builder.stream("input-topic", Consumed.with(Serdes.String(), Serdes.String())); // Send to even numbered partitions inputStream.to("output-topic", Produced.with(Serdes.String(), Serdes.String(), (topic, key, val, numPartitions) -> new EvenPartitioner())); // BroadcaseBroadcast inputStream.to("output-topic", Produced.with(Serdes.String(), Serdes.String(), (topic, key, val, numPartitions) -> new BroadcastingPartitioner())); // Send to even numbered partitions inputStream.to("output-topic", Produced.with(Serdes.String(), Serdes.String(), (topic, key, val, numPartitions) -> new DroppingPartitioner())); } |
...
- StreamPartitioner is also used in FK-join and Interactive queriesIQ. The invocation of
partition()
would be replaced bypartitions()
and a runtime check would be added to ensure that the returned list set is singleton. - Adding sufficient logging
...
- extend to() / addSink() with a "broadcast" option/config in KafkaStreams.
- add toAllPartitions() / addBroadcastSink() methods in KafkaStreams.
- allow StreamPartitioner to return `-1` for "all partitions".
- Adding a separate class to handle multi casting. This was rejected in favour of enhancing the current
StreamPartitioner
interface.
Test Plan
- Add new tests(unit/integration) similar to the
partition
tests for the newly introducedpartitions
method. - Add tests to fail when a non singleton set is returned for FK joins and IQ when querying.