├── images ├── assignor-overview.png └── partitioner-overview.png ├── CODE_OF_CONDUCT.md ├── .gitignore ├── src ├── main │ └── java │ │ └── code │ │ └── buildon │ │ └── aws │ │ └── streaming │ │ └── kafka │ │ ├── DiscardPartitioner.java │ │ ├── Bucket.java │ │ ├── BucketPriorityConfig.java │ │ ├── BucketPriorityPartitioner.java │ │ └── BucketPriorityAssignor.java └── test │ └── java │ └── code │ └── buildon │ └── aws │ └── streaming │ └── kafka │ ├── BucketPriorityAssignorTest.java │ └── BucketPriorityPartitionerTest.java ├── LICENSE ├── pom.xml ├── CONTRIBUTING.md └── README.md /images/assignor-overview.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/build-on-aws/prioritizing-event-processing-with-apache-kafka/HEAD/images/assignor-overview.png -------------------------------------------------------------------------------- /images/partitioner-overview.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/build-on-aws/prioritizing-event-processing-with-apache-kafka/HEAD/images/partitioner-overview.png -------------------------------------------------------------------------------- /CODE_OF_CONDUCT.md: -------------------------------------------------------------------------------- 1 | ## Code of Conduct 2 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct). 3 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact 4 | opensource-codeofconduct@amazon.com with any additional questions or comments. 5 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class file 2 | *.class 3 | 4 | # Log file 5 | *.log 6 | 7 | # BlueJ files 8 | *.ctxt 9 | 10 | # Mobile Tools for Java (J2ME) 11 | .mtj.tmp/ 12 | 13 | # VSCode 14 | .classpath 15 | .project 16 | target 17 | .vscode 18 | .settings 19 | .target 20 | 21 | # Package Files # 22 | *.jar 23 | *.war 24 | *.nar 25 | *.ear 26 | *.zip 27 | *.tar.gz 28 | *.rar 29 | 30 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 31 | hs_err_pid* 32 | -------------------------------------------------------------------------------- /src/main/java/code/buildon/aws/streaming/kafka/DiscardPartitioner.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.util.Map; 4 | 5 | import org.apache.kafka.clients.producer.Partitioner; 6 | import org.apache.kafka.common.Cluster; 7 | 8 | public class DiscardPartitioner implements Partitioner { 9 | 10 | @Override 11 | public void configure(Map configs) { 12 | } 13 | 14 | @Override 15 | public int partition(String topic, Object key, byte[] keyBytes, 16 | Object value, byte[] valueBytes, Cluster cluster) { 17 | return -1; 18 | } 19 | 20 | @Override 21 | public void close() { 22 | } 23 | 24 | } 25 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy of 4 | this software and associated documentation files (the "Software"), to deal in 5 | the Software without restriction, including without limitation the rights to 6 | use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of 7 | the Software, and to permit persons to whom the Software is furnished to do so. 8 | 9 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 10 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS 11 | FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR 12 | COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER 13 | IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN 14 | CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. 15 | 16 | -------------------------------------------------------------------------------- /src/main/java/code/buildon/aws/streaming/kafka/Bucket.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.util.ArrayList; 4 | import java.util.List; 5 | import java.util.concurrent.atomic.AtomicInteger; 6 | 7 | import org.apache.kafka.common.TopicPartition; 8 | import org.apache.kafka.common.utils.Utils; 9 | 10 | public class Bucket implements Comparable { 11 | 12 | private int allocation; 13 | private List partitions; 14 | private AtomicInteger counter; 15 | 16 | public Bucket(int allocation) { 17 | this.allocation = allocation; 18 | partitions = new ArrayList<>(); 19 | counter = new AtomicInteger(-1); 20 | } 21 | 22 | public int nextPartition() { 23 | if (!partitions.isEmpty()) { 24 | int nextValue = counter.incrementAndGet(); 25 | int index = Utils.toPositive(nextValue) % partitions.size(); 26 | return partitions.get(index).partition(); 27 | } 28 | return -1; 29 | } 30 | 31 | @Override 32 | public int compareTo(Bucket bucket) { 33 | int result = 0; 34 | if (getAllocation() < bucket.getAllocation()) { 35 | result = 1; 36 | } else if (getAllocation() > bucket.getAllocation()) { 37 | result = -1; 38 | } 39 | return result; 40 | } 41 | 42 | public void decrementCounter() { 43 | counter.decrementAndGet(); 44 | } 45 | 46 | public int size(int numPartitions) { 47 | return Math.round(((float) allocation / 100) * numPartitions); 48 | } 49 | 50 | public int getAllocation() { 51 | return allocation; 52 | } 53 | 54 | public List getPartitions() { 55 | return partitions; 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 7 | 8 | 4.0.0 9 | 10 | code.buildon.aws.streaming.kafka 11 | bucket-priority-pattern 12 | Pattern that groups topic partitions into buckets so these buckets can be processed in a given priority order. 13 | 1.0.0 14 | 15 | 16 | 17 | riferrei 18 | Ricardo Ferreira 19 | https://riferrei.com 20 | 21 | Committer 22 | 23 | 24 | 25 | 26 | 27 | 3.4.1 28 | 2.0.7 29 | 5.9.3 30 | 31 | 32 | 33 | 34 | org.apache.kafka 35 | kafka-clients 36 | ${kafka.clients.version} 37 | 38 | 39 | org.slf4j 40 | slf4j-api 41 | ${slf4j.api.version} 42 | 43 | 44 | org.junit.jupiter 45 | junit-jupiter-engine 46 | ${junit.jupiter.version} 47 | test 48 | 49 | 50 | 51 | 52 | 53 | 54 | org.apache.maven.plugins 55 | maven-compiler-plugin 56 | 3.11.0 57 | 58 | 17 59 | 60 | 61 | 62 | 63 | 64 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing Guidelines 2 | 3 | Thank you for your interest in contributing to our project. Whether it's a bug report, new feature, correction, or additional 4 | documentation, we greatly value feedback and contributions from our community. 5 | 6 | Please read through this document before submitting any issues or pull requests to ensure we have all the necessary 7 | information to effectively respond to your bug report or contribution. 8 | 9 | 10 | ## Reporting Bugs/Feature Requests 11 | 12 | We welcome you to use the GitHub issue tracker to report bugs or suggest features. 13 | 14 | When filing an issue, please check existing open, or recently closed, issues to make sure somebody else hasn't already 15 | reported the issue. Please try to include as much information as you can. Details like these are incredibly useful: 16 | 17 | * A reproducible test case or series of steps 18 | * The version of our code being used 19 | * Any modifications you've made relevant to the bug 20 | * Anything unusual about your environment or deployment 21 | 22 | 23 | ## Contributing via Pull Requests 24 | Contributions via pull requests are much appreciated. Before sending us a pull request, please ensure that: 25 | 26 | 1. You are working against the latest source on the *main* branch. 27 | 2. You check existing open, and recently merged, pull requests to make sure someone else hasn't addressed the problem already. 28 | 3. You open an issue to discuss any significant work - we would hate for your time to be wasted. 29 | 30 | To send us a pull request, please: 31 | 32 | 1. Fork the repository. 33 | 2. Modify the source; please focus on the specific change you are contributing. If you also reformat all the code, it will be hard for us to focus on your change. 34 | 3. Ensure local tests pass. 35 | 4. Commit to your fork using clear commit messages. 36 | 5. Send us a pull request, answering any default questions in the pull request interface. 37 | 6. Pay attention to any automated CI failures reported in the pull request, and stay involved in the conversation. 38 | 39 | GitHub provides additional document on [forking a repository](https://help.github.com/articles/fork-a-repo/) and 40 | [creating a pull request](https://help.github.com/articles/creating-a-pull-request/). 41 | 42 | 43 | ## Finding contributions to work on 44 | Looking at the existing issues is a great way to find something to contribute on. As our projects, by default, use the default GitHub issue labels (enhancement/bug/duplicate/help wanted/invalid/question/wontfix), looking at any 'help wanted' issues is a great place to start. 45 | 46 | 47 | ## Code of Conduct 48 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct). 49 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact 50 | opensource-codeofconduct@amazon.com with any additional questions or comments. 51 | 52 | 53 | ## Security issue notifications 54 | If you discover a potential security issue in this project we ask that you notify AWS/Amazon Security via our [vulnerability reporting page](http://aws.amazon.com/security/vulnerability-reporting/). Please do **not** create a public github issue. 55 | 56 | 57 | ## Licensing 58 | 59 | See the [LICENSE](LICENSE) file for our project's licensing. We will ask you to confirm the licensing of your contribution. 60 | -------------------------------------------------------------------------------- /src/main/java/code/buildon/aws/streaming/kafka/BucketPriorityConfig.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.util.List; 4 | import java.util.Map; 5 | 6 | import org.apache.kafka.clients.consumer.RangeAssignor; 7 | import org.apache.kafka.common.config.AbstractConfig; 8 | import org.apache.kafka.common.config.ConfigDef; 9 | 10 | public class BucketPriorityConfig extends AbstractConfig { 11 | 12 | public BucketPriorityConfig(Map originals) { 13 | super(CONFIG, originals); 14 | } 15 | 16 | public String topic() { 17 | return getString(TOPIC_CONFIG); 18 | } 19 | 20 | public List buckets() { 21 | return getList(BUCKETS_CONFIG); 22 | } 23 | 24 | public List allocation() { 25 | return getList(ALLOCATION_CONFIG); 26 | } 27 | 28 | public String bucket() { 29 | return getString(BUCKET_CONFIG); 30 | } 31 | 32 | public String delimiter() { 33 | return getString(DELIMITER_CONFIG); 34 | } 35 | 36 | public String fallbackAssignor() { 37 | return getString(FALLBACK_ASSIGNOR_CONFIG); 38 | } 39 | 40 | private static final ConfigDef CONFIG; 41 | 42 | public static final String TOPIC_CONFIG = "bucket.priority.topic"; 43 | public static final String TOPIC_CONFIG_DOC = "Which topic should have its partitions mapped to buckets."; 44 | public static final String BUCKETS_CONFIG = "bucket.priority.buckets"; 45 | public static final String BUCKETS_CONFIG_DOC = "List of the bucket names."; 46 | public static final String BUCKET_CONFIG = "bucket.priority.bucket"; 47 | public static final String BUCKET_CONFIG_DOC = "Bucket that the consumer will be assigned to."; 48 | public static final String BUCKET_CONFIG_DEFAULT = ""; 49 | public static final String DELIMITER_CONFIG = "bucket.priority.delimiter"; 50 | public static final String DELIMITER_CONFIG_DOC = "Delimiter used to look up the bucket name in the key."; 51 | public static final String DELIMITER_CONFIG_DEFAULT = "-"; 52 | public static final String ALLOCATION_CONFIG = "bucket.priority.allocation"; 53 | public static final String ALLOCATION_CONFIG_DOC = "Allocation in percentage for each bucket."; 54 | public static final String FALLBACK_ASSIGNOR_CONFIG = "bucket.priority.fallback.assignor"; 55 | public static final String FALLBACK_ASSIGNOR_CONFIG_DOC = "Which assignor to use as fallback strategy."; 56 | public static final String FALLBACK_ASSIGNOR_CONFIG_DEFAULT = RangeAssignor.class.getName(); 57 | 58 | static { 59 | CONFIG = new ConfigDef() 60 | .define(TOPIC_CONFIG, 61 | ConfigDef.Type.STRING, 62 | ConfigDef.Importance.HIGH, 63 | TOPIC_CONFIG_DOC) 64 | .define(BUCKETS_CONFIG, 65 | ConfigDef.Type.LIST, 66 | ConfigDef.Importance.HIGH, 67 | BUCKETS_CONFIG_DOC) 68 | .define(ALLOCATION_CONFIG, 69 | ConfigDef.Type.LIST, 70 | ConfigDef.Importance.HIGH, 71 | ALLOCATION_CONFIG_DOC) 72 | .define(BUCKET_CONFIG, 73 | ConfigDef.Type.STRING, 74 | BUCKET_CONFIG_DEFAULT, 75 | ConfigDef.Importance.HIGH, 76 | BUCKET_CONFIG_DOC) 77 | .define(DELIMITER_CONFIG, 78 | ConfigDef.Type.STRING, 79 | DELIMITER_CONFIG_DEFAULT, 80 | ConfigDef.Importance.LOW, 81 | DELIMITER_CONFIG_DOC) 82 | .define( 83 | FALLBACK_ASSIGNOR_CONFIG, 84 | ConfigDef.Type.CLASS, 85 | FALLBACK_ASSIGNOR_CONFIG_DEFAULT, 86 | ConfigDef.Importance.LOW, 87 | FALLBACK_ASSIGNOR_CONFIG_DOC); 88 | } 89 | 90 | } 91 | -------------------------------------------------------------------------------- /src/main/java/code/buildon/aws/streaming/kafka/BucketPriorityPartitioner.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.util.ArrayList; 4 | import java.util.Comparator; 5 | import java.util.LinkedHashMap; 6 | import java.util.List; 7 | import java.util.Map; 8 | import java.util.concurrent.atomic.AtomicInteger; 9 | import java.util.stream.Collectors; 10 | 11 | import org.apache.kafka.common.Cluster; 12 | import org.apache.kafka.common.PartitionInfo; 13 | import org.apache.kafka.common.TopicPartition; 14 | import org.apache.kafka.clients.producer.Partitioner; 15 | import org.apache.kafka.clients.producer.RecordMetadata; 16 | import org.apache.kafka.common.errors.InvalidConfigurationException; 17 | import org.apache.kafka.common.utils.Utils; 18 | 19 | public class BucketPriorityPartitioner implements Partitioner { 20 | 21 | private BucketPriorityConfig config; 22 | private ThreadLocal lastBucket; 23 | private Map buckets; 24 | private int lastPartitionCount; 25 | 26 | @Override 27 | public void configure(Map configs) { 28 | config = new BucketPriorityConfig(configs); 29 | List allocation = new ArrayList<>(config.allocation().size()); 30 | for (String allocItem : config.allocation()) { 31 | allocItem = allocItem.replaceAll("%", "").trim(); 32 | allocation.add(Integer.parseInt(allocItem)); 33 | } 34 | if (config.buckets().size() != allocation.size()) { 35 | throw new InvalidConfigurationException("The bucket allocation " + 36 | "doesn't match with the number of buckets configured."); 37 | } 38 | int sumAllBuckets = allocation.stream().mapToInt(Integer::intValue).sum(); 39 | if (sumAllBuckets != 100) { 40 | throw new InvalidConfigurationException("The bucket allocation " + 41 | "is incorrect. The sum of all buckets needs to be 100."); 42 | } 43 | lastBucket = new ThreadLocal<>(); 44 | buckets = new LinkedHashMap<>(); 45 | for (int i = 0; i < config.buckets().size(); i++) { 46 | String bucketName = config.buckets().get(i).trim(); 47 | buckets.put(bucketName, new Bucket(allocation.get(i))); 48 | } 49 | // Sort the buckets with higher allocation to come 50 | // first than the others. This will help later during 51 | // the allocation if unassigned partitions are found. 52 | buckets = buckets.entrySet() 53 | .stream() 54 | .sorted(Map.Entry.comparingByValue()) 55 | .collect(Collectors.toMap( 56 | Map.Entry::getKey, 57 | Map.Entry::getValue, 58 | (oldValue, newValue) -> oldValue, LinkedHashMap::new)); 59 | } 60 | 61 | @Override 62 | public int partition(String topic, Object key, byte[] keyBytes, 63 | Object value, byte[] valueBytes, Cluster cluster) { 64 | int partition = RecordMetadata.UNKNOWN_PARTITION; 65 | // Try to apply the bucket priority partitioning logic. If 66 | // none of the conditions apply, allow the partition to be 67 | // set by the built-in partitioning logic from KIP-794. 68 | if (config.topic() != null && config.topic().equals(topic)) { 69 | if (key instanceof String) { 70 | String keyValue = (String) key; 71 | String[] keyValueParts = keyValue.split(config.delimiter()); 72 | if (keyValueParts.length >= 1) { 73 | String bucketName = keyValueParts[0].trim(); 74 | if (buckets.containsKey(bucketName)) { 75 | lastBucket.set(bucketName); 76 | partition = getPartition(bucketName, cluster); 77 | } 78 | } 79 | } 80 | } 81 | return partition; 82 | } 83 | 84 | private int getPartition(String bucketName, Cluster cluster) { 85 | int numPartitions = cluster.partitionCountForTopic(config.topic()); 86 | // Check if the # of partitions has changed 87 | // and trigger an update if that happened. 88 | if (lastPartitionCount != numPartitions) { 89 | updatePartitionsAssignment(cluster); 90 | lastPartitionCount = numPartitions; 91 | } 92 | Bucket bucket = buckets.get(bucketName); 93 | return bucket.nextPartition(); 94 | } 95 | 96 | private void updatePartitionsAssignment(Cluster cluster) { 97 | List partitions = cluster.partitionsForTopic(config.topic()); 98 | if (partitions.size() < buckets.size()) { 99 | StringBuilder message = new StringBuilder(); 100 | message.append("The number of partitions available for the topic '"); 101 | message.append(config.topic()).append("' is incompatible with the "); 102 | message.append("number of buckets. It needs to be at least "); 103 | message.append(buckets.size()).append("."); 104 | throw new InvalidConfigurationException(message.toString()); 105 | } 106 | // Sort partitions in ascendent order since 107 | // the partitions will be mapped into the 108 | // buckets from partition-0 to partition-n. 109 | partitions = partitions.stream() 110 | .sorted(Comparator.comparing(PartitionInfo::partition)) 111 | .collect(Collectors.toList()); 112 | // Design the layout of the distribution 113 | int distribution = 0; 114 | Map layout = new LinkedHashMap<>(); 115 | for (Map.Entry entry : buckets.entrySet()) { 116 | int bucketSize = entry.getValue().size(partitions.size()); 117 | layout.put(entry.getKey(), bucketSize); 118 | distribution += bucketSize; 119 | } 120 | // Check if there are unassigned partitions. 121 | // If so then distribute them over the buckets 122 | // starting from the top to bottom until there 123 | // are no partitions left. 124 | int remaining = partitions.size() - distribution; 125 | if (remaining > 0) { 126 | AtomicInteger counter = new AtomicInteger(-1); 127 | List availableBuckets = new ArrayList<>(); 128 | buckets.keySet().stream().forEach(bucket -> { 129 | availableBuckets.add(bucket); 130 | }); 131 | while (remaining > 0) { 132 | int nextValue = counter.incrementAndGet(); 133 | int index = Utils.toPositive(nextValue) % availableBuckets.size(); 134 | String bucketName = availableBuckets.get(index); 135 | int bucketSize = layout.get(bucketName); 136 | layout.put(bucketName, ++bucketSize); 137 | remaining--; 138 | } 139 | } 140 | // Finally assign the available partitions to buckets 141 | int partition = -1; 142 | TopicPartition topicPartition = null; 143 | bucketAssign: for (Map.Entry entry : buckets.entrySet()) { 144 | int bucketSize = layout.get(entry.getKey()); 145 | entry.getValue().getPartitions().clear(); 146 | for (int i = 0; i < bucketSize; i++) { 147 | topicPartition = new TopicPartition(config.topic(), ++partition); 148 | entry.getValue().getPartitions().add(topicPartition); 149 | if (partition == partitions.size() - 1) { 150 | break bucketAssign; 151 | } 152 | } 153 | } 154 | } 155 | 156 | @Override 157 | public void onNewBatch(String topic, Cluster cluster, int prevPartition) { 158 | // With the introduction of KIP-480 to enhance record production 159 | // throughput Kafka's API calls the partition() method twice resulting 160 | // in partitions being skipped. More information about this here: 161 | // https://issues.apache.org/jira/browse/KAFKA-9965 162 | // The temporary solution is to use the callback method 'onNewBatch' 163 | // to decrease the counter to stabilize the round-robin logic. 164 | String bucketName = lastBucket.get(); 165 | Bucket bucket = buckets.get(bucketName); 166 | if (bucket != null) { 167 | bucket.decrementCounter(); 168 | } 169 | lastBucket.remove(); 170 | } 171 | 172 | @Override 173 | public void close() { 174 | // Nothing to close 175 | } 176 | 177 | } 178 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Prioritizing Event Processing with Apache Kafka 2 | 3 | Implement event processing prioritization in [Apache Kafka](https://kafka.apache.org) is often a hard task because Kafka doesn't support broker-level reordering of messages like some messaging technologies do. This is not necessarily a limitation, since Kafka is a [distributed commit log](https://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying). With this data structure, messages are immutable, and so their ordering is within partitions. But this doesn't change the fact the developers may need to implement event processing prioritization with Kafka, anyway. 4 | 5 | This project addresses event processing prioritization via the bucket pattern. It groups partitions into simpler abstractions called buckets. Bigger buckets mean a higher priority, and smaller buckets mean less priority. The number of partitions associated with each bucket defines their size. The bucket pattern also addresses code simplicity by providing a way to do all of this without forcing developers to handle low-level code related to event partitioning and consumer assignment. 6 | 7 | Let's understand how this works with an example. 8 | 9 | ![Partitioner Overview](images/partitioner-overview.png) 10 | 11 | Here we can see that the partitions were grouped into the buckets `Platinum` and `Gold`. The Platinum bucket has a higher priority and therefore was configured to have `70%` of the allocation, whereas the Gold bucket has lower priority and therefore was configured to have only `30%`. This means that for a topic that contains `6` partitions, `4` of them will be associated with the Platinum bucket and `2` will be associated with the Gold bucket. To implement the prioritization, there has to be a process that ensures that messages with higher priority will end up in one the partitions from the Platinum bucket and messages with lower priority will end up in one the partitions from the Gold bucket. Consumers need to subscribe to the topic knowing which buckets they need to be associated with. This means that developers can decide to execute more consumers for the Platinum bucket and fewer consumers for the Gold bucket to ensure that they process high priority messages faster. 12 | 13 | To ensure that each message will end up in their respective bucket, use the `BucketPriorityPartitioner`. This partitioner uses data in the message key to decide which bucket to use and therefore which partition from the bucket the message should be written. This partitioner distributes the messages within the bucket using a round robin algorithm to maximize consumption parallelism. On the consumer side, use the `BucketPriorityAssignor` to ensure that the consumer will be assigned only to the partitions that represent the bucket they want to process. 14 | 15 | ![Assignor Overview](images/assignor-overview.png) 16 | 17 | With the bucket priority, you can implement event processing prioritization by having more consumers working on buckets with higher priorities, while buckets with less priority can have fewer consumers. Event processing prioritization can also be obtained by executing these consumers in an order that gives preference to processing high priority buckets before the less priority ones. While coordinating this execution may involve some extra coding from you (perhaps using some sort of scheduler) you don't have to implement low-level code to manage partition assignment and keep your consumers simple by leveraging the standard `subscribe()` and `poll()` methods. 18 | 19 | You can read more about the bucket priority pattern in this blog post: https://www.buildon.aws/posts/prioritizing-event-processing-with-apache-kafka 20 | 21 | ## Building the project 22 | 23 | The first thing you need to do to start using this partitioner is building it. In order to do that, you need to install the following dependencies: 24 | 25 | - [Java 11+](https://openjdk.java.net/) 26 | - [Apache Maven](https://maven.apache.org/) 27 | 28 | After installing these dependencies, execute the following command: 29 | 30 | ```bash 31 | mvn clean package 32 | ``` 33 | 34 | ## Using the partitioner 35 | 36 | To use the `BucketPriorityPartitioner` in your producer you need to register it in the configuration. 37 | 38 | ```bash 39 | Properties configs = new Properties(); 40 | 41 | configs.setProperty(ProducerConfig.PARTITIONER_CLASS_CONFIG, 42 | BucketPriorityPartitioner.class.getName()); 43 | 44 | KafkaProducer producer = new KafkaProducer<>(configs); 45 | ``` 46 | 47 | To work properly, you need to specify in the configuration which topic will have its partitions grouped into buckets. This is important because, in Kafka, topics are specified at a message level and not at a producer level. This means that the same producer can write messages on different topics, so the partitioner needs to know which topic will have their partitions grouped into buckets. 48 | 49 | ```bash 50 | configs.setProperty(BucketPriorityConfig.TOPIC_CONFIG, "orders"); 51 | ``` 52 | 53 | Finally, specify in the configuration which buckets will be configured and what is the partition allocation for each one of them. The partition allocation is specified in terms of percentage. Note that the usage of the symbol `%` is optional. 54 | 55 | ```bash 56 | configs.setProperty(BucketPriorityConfig.BUCKETS_CONFIG, "Platinum, Gold"); 57 | configs.setProperty(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 58 | ``` 59 | 60 | The partitioner ensures that all partitions from the topic will be assigned to the buckets. 61 | In case of the allocation result in some partitions being left behind because the distribution is not even, the remaining partitions will be assigned to the buckets using a round robin algorithm over the buckets sorted by allocation. 62 | 63 | ### Messages and buckets 64 | 65 | In order to specify which bucket should be used, your producer need to provide this information on the message key. The partitioner will inspect each key in the attempt to understand in which bucket the message should be written. For this reason, the key must be an instance of a [java.lang.String](https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/lang/String.html) and it needs to contain the bucket name either as one literal string or as the first part of a string separated by a delimiter. For example, to specify that the bucket is `Platinum` then following examples are valid: 66 | 67 | * Key = `"Platinum"` 68 | * Key = `"Platinum-001"` 69 | * Key = `"Platinum-Group01-001"` 70 | 71 | The default delimiter is `-` but you can change to something else: 72 | 73 | ```bash 74 | configs.setProperty(BucketPriorityConfig.DELIMITER_CONFIG, "|"); 75 | ``` 76 | 77 | ### Discarding messages 78 | 79 | Discarding any message that can't be sent to any of the buckets is also possible: 80 | 81 | ```bash 82 | configs.setProperty(BucketPriorityConfig.FALLBACK_PARTITIONER_CONFIG, 83 | "code.buildon.aws.streaming.kafka.DiscardPartitioner"); 84 | ``` 85 | 86 | ## Using the assignor 87 | 88 | To use the `BucketPriorityAssignor` in your consumer you need to register it in the configuration. 89 | 90 | ```bash 91 | Properties configs = new Properties(); 92 | 93 | configs.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, 94 | BucketPriorityAssignor.class.getName()); 95 | 96 | KafkaConsumer consumer = new KafkaConsumer<>(configs); 97 | ``` 98 | 99 | To work properly, you need to specify in the configuration which topic will have its partitions grouped into buckets. This is important because, in Kafka, consumers can subscribe to multiple topics. This means that the same consumer can read messages from different topics, so the assignor needs to know which topic will have their partitions grouped into buckets. 100 | 101 | ```bash 102 | configs.setProperty(BucketPriorityConfig.TOPIC_CONFIG, "orders"); 103 | ``` 104 | 105 | You also have to specify in the configuration which buckets will be configured and what is the partition allocation for each one of them. 106 | The partition allocation is specified in terms of percentage. Note that the usage of the symbol `%` is optional. Ideally, the partition allocation configuration needs to be the same used in the producer. 107 | 108 | 109 | ```bash 110 | configs.setProperty(BucketPriorityConfig.BUCKETS_CONFIG, "Platinum, Gold"); 111 | configs.setProperty(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 112 | ``` 113 | 114 | The assignor ensures that all partitions from the topic will be assigned to the buckets. 115 | In case of the allocation result in some partitions being left behind because the distribution is not even, the remaining partitions will be assigned to the buckets using a round robin algorithm over the buckets sorted by allocation. 116 | 117 | Finally you need to specify in the configuration which bucket the consumer will be associated. 118 | 119 | ```bash 120 | configs.setProperty(BucketPriorityConfig.BUCKET_CONFIG, "Platinum"); 121 | ``` 122 | 123 | ### What about the other topics? 124 | 125 | In Kafka, a consumer can subscribe to multiple topics, allowing the same consumer to read messages from partitions belonging to different topics. Because of this, the assignor ensures that only the topic specified in the configuration will have its partitions assigned to the consumers using the bucket priority logic. The other topics will have their partitions assigned to consumers using a fallback assignor. 126 | 127 | Here is an example of configuring the fallback assignor to round-robin: 128 | 129 | ```bash 130 | configs.setProperty(BucketPriorityConfig.FALLBACK_ASSIGNOR_CONFIG, 131 | "org.apache.kafka.clients.consumer.RoundRobinAssignor"); 132 | ``` 133 | 134 | If you don't configure a fallback assignor explicitly, Kafka's default assignor will be used. 135 | 136 | ## Security 137 | 138 | See [CONTRIBUTING](CONTRIBUTING.md#security-issue-notifications) for more information. 139 | 140 | ## License 141 | 142 | This project is licensed under the MIT-0 License. See the [LICENSE](./LICENSE) file. 143 | -------------------------------------------------------------------------------- /src/main/java/code/buildon/aws/streaming/kafka/BucketPriorityAssignor.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.nio.ByteBuffer; 4 | import java.nio.charset.Charset; 5 | import java.nio.charset.StandardCharsets; 6 | import java.util.ArrayList; 7 | import java.util.Comparator; 8 | import java.util.LinkedHashMap; 9 | import java.util.List; 10 | import java.util.Map; 11 | import java.util.Set; 12 | import java.util.concurrent.atomic.AtomicInteger; 13 | import java.util.stream.Collectors; 14 | 15 | import org.apache.kafka.common.Configurable; 16 | import org.apache.kafka.common.TopicPartition; 17 | import org.apache.kafka.common.errors.InvalidConfigurationException; 18 | import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; 19 | import org.apache.kafka.common.utils.Utils; 20 | 21 | public class BucketPriorityAssignor extends AbstractPartitionAssignor implements Configurable { 22 | 23 | private AbstractPartitionAssignor fallback; 24 | private BucketPriorityConfig config; 25 | private Map buckets; 26 | private int lastPartitionCount; 27 | 28 | @Override 29 | public String name() { 30 | return "bucket-priority"; 31 | } 32 | 33 | @Override 34 | public void configure(Map configs) { 35 | config = new BucketPriorityConfig(configs); 36 | List bucketAlloc = new ArrayList<>(config.allocation().size()); 37 | for (String allocItem : config.allocation()) { 38 | allocItem = allocItem.replaceAll("%", "").trim(); 39 | bucketAlloc.add(Integer.parseInt(allocItem)); 40 | } 41 | if (config.buckets().size() != bucketAlloc.size()) { 42 | throw new InvalidConfigurationException("The bucket allocation " + 43 | "doesn't match with the number of buckets configured."); 44 | } 45 | int oneHundredPerc = bucketAlloc.stream() 46 | .mapToInt(Integer::intValue) 47 | .sum(); 48 | if (oneHundredPerc != 100) { 49 | throw new InvalidConfigurationException("The bucket allocation " + 50 | "is incorrect. The sum of all buckets needs to be 100."); 51 | } 52 | try { 53 | fallback = config.getConfiguredInstance( 54 | BucketPriorityConfig.FALLBACK_ASSIGNOR_CONFIG, 55 | AbstractPartitionAssignor.class); 56 | } catch (Exception ex) { 57 | throw new InvalidConfigurationException("The fallback " + 58 | "assignor configured is invalid.", ex); 59 | } 60 | buckets = new LinkedHashMap<>(); 61 | for (int i = 0; i < config.buckets().size(); i++) { 62 | String bucketName = config.buckets().get(i).trim(); 63 | buckets.put(bucketName, new Bucket(bucketAlloc.get(i))); 64 | } 65 | // Sort the buckets with higher allocation to come 66 | // first than the others. This will help later during 67 | // the allocation if unassigned partitions are found. 68 | buckets = buckets.entrySet() 69 | .stream() 70 | .sorted(Map.Entry.comparingByValue()) 71 | .collect(Collectors.toMap( 72 | Map.Entry::getKey, 73 | Map.Entry::getValue, 74 | (oldValue, newValue) -> oldValue, LinkedHashMap::new)); 75 | } 76 | 77 | @Override 78 | public ByteBuffer subscriptionUserData(Set topics) { 79 | ByteBuffer userData = null; 80 | for (String topic : topics) { 81 | if (topic.equals(config.topic())) { 82 | String bucket = config.bucket(); 83 | Charset charset = StandardCharsets.UTF_8; 84 | userData = charset.encode(bucket); 85 | break; 86 | } 87 | } 88 | return userData; 89 | } 90 | 91 | @Override 92 | public Map> assign(Map partitionsPerTopic, 93 | Map subscriptions) { 94 | int numPartitions = partitionsPerTopic.get(config.topic()); 95 | // Check if the # of partitions has changed 96 | // and trigger an update if that happened. 97 | if (lastPartitionCount != numPartitions) { 98 | updatePartitionsAssignment(numPartitions); 99 | lastPartitionCount = numPartitions; 100 | } 101 | Map> assignments = new LinkedHashMap<>(); 102 | Map> consumersPerBucket = new LinkedHashMap<>(); 103 | Map otherTopicsSubscriptions = new LinkedHashMap<>(); 104 | for (String consumer : subscriptions.keySet()) { 105 | Subscription subscription = subscriptions.get(consumer); 106 | if (subscription.topics().contains(config.topic())) { 107 | assignments.put(consumer, new ArrayList<>()); 108 | ByteBuffer userData = subscription.userData(); 109 | Charset charset = StandardCharsets.UTF_8; 110 | String bucket = charset.decode(userData).toString(); 111 | if (buckets.containsKey(bucket)) { 112 | if (consumersPerBucket.containsKey(bucket)) { 113 | List consumers = consumersPerBucket.get(bucket); 114 | consumers.add(consumer); 115 | } else { 116 | List consumers = new ArrayList<>(); 117 | consumers.add(consumer); 118 | consumersPerBucket.put(bucket, consumers); 119 | } 120 | } 121 | } else { 122 | otherTopicsSubscriptions.put(consumer, subscription); 123 | } 124 | } 125 | // Evenly distribute the partitions across the 126 | // available consumers in a per-bucket basis. 127 | AtomicInteger counter = new AtomicInteger(-1); 128 | for (Map.Entry bucket : buckets.entrySet()) { 129 | List consumers = consumersPerBucket.get(bucket.getKey()); 130 | // Check if the bucket has consumers available... 131 | if (consumers != null && !consumers.isEmpty()) { 132 | for (TopicPartition partition : bucket.getValue().getPartitions()) { 133 | int nextValue = counter.incrementAndGet(); 134 | int index = Utils.toPositive(nextValue) % consumers.size(); 135 | String consumer = consumers.get(index); 136 | assignments.get(consumer).add(partition); 137 | } 138 | } 139 | } 140 | // If there are subscriptions for topics that 141 | // are not based on buckets then use the fallback 142 | // assignor to create their partition assignments. 143 | if (!otherTopicsSubscriptions.isEmpty()) { 144 | Map> fallbackAssignments = 145 | fallback.assign(partitionsPerTopic, otherTopicsSubscriptions); 146 | assignments.putAll(fallbackAssignments); 147 | } 148 | return assignments; 149 | } 150 | 151 | private void updatePartitionsAssignment(int numPartitions) { 152 | List partitions = partitions(config.topic(), numPartitions); 153 | if (partitions.size() < buckets.size()) { 154 | StringBuilder message = new StringBuilder(); 155 | message.append("The number of partitions available for the topic '"); 156 | message.append(config.topic()).append("' is incompatible with the "); 157 | message.append("number of buckets. It needs to be at least "); 158 | message.append(buckets.size()).append("."); 159 | throw new InvalidConfigurationException(message.toString()); 160 | } 161 | // Sort partitions in ascendent order since 162 | // the partitions will be mapped into the 163 | // buckets from partition-0 to partition-n. 164 | partitions = partitions.stream() 165 | .sorted(Comparator.comparing(TopicPartition::partition)) 166 | .collect(Collectors.toList()); 167 | // Design the layout of the distribution 168 | int distribution = 0; 169 | Map layout = new LinkedHashMap<>(); 170 | for (Map.Entry entry : buckets.entrySet()) { 171 | int bucketSize = entry.getValue().size(partitions.size()); 172 | layout.put(entry.getKey(), bucketSize); 173 | distribution += bucketSize; 174 | } 175 | // Check if there are unassigned partitions. 176 | // If so then distribute them over the buckets 177 | // starting from the top to bottom until there 178 | // are no partitions left. 179 | int remaining = partitions.size() - distribution; 180 | if (remaining > 0) { 181 | AtomicInteger counter = new AtomicInteger(-1); 182 | List availableBuckets = new ArrayList<>(); 183 | buckets.keySet().stream().forEach(bucket -> { 184 | availableBuckets.add(bucket); 185 | }); 186 | while (remaining > 0) { 187 | int nextValue = counter.incrementAndGet(); 188 | int index = Utils.toPositive(nextValue) % availableBuckets.size(); 189 | String bucketName = availableBuckets.get(index); 190 | int bucketSize = layout.get(bucketName); 191 | layout.put(bucketName, ++bucketSize); 192 | remaining--; 193 | } 194 | } 195 | // Finally assign the available partitions to buckets 196 | int partition = -1; 197 | TopicPartition topicPartition = null; 198 | bucketAssign: for (Map.Entry entry : buckets.entrySet()) { 199 | int bucketSize = layout.get(entry.getKey()); 200 | entry.getValue().getPartitions().clear(); 201 | for (int i = 0; i < bucketSize; i++) { 202 | topicPartition = new TopicPartition(config.topic(), ++partition); 203 | entry.getValue().getPartitions().add(topicPartition); 204 | if (partition == partitions.size() - 1) { 205 | break bucketAssign; 206 | } 207 | } 208 | } 209 | } 210 | 211 | } 212 | -------------------------------------------------------------------------------- /src/test/java/code/buildon/aws/streaming/kafka/BucketPriorityAssignorTest.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.nio.charset.StandardCharsets; 4 | import java.util.HashMap; 5 | import java.util.List; 6 | import java.util.Map; 7 | import java.util.Set; 8 | import java.util.TreeSet; 9 | 10 | import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; 11 | import org.apache.kafka.common.TopicPartition; 12 | import org.apache.kafka.common.config.ConfigException; 13 | import org.apache.kafka.common.errors.InvalidConfigurationException; 14 | import org.junit.jupiter.api.Test; 15 | 16 | import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; 17 | import static org.junit.jupiter.api.Assertions.assertEquals; 18 | import static org.junit.jupiter.api.Assertions.assertThrows; 19 | import static org.junit.jupiter.api.Assertions.assertTrue; 20 | 21 | public class BucketPriorityAssignorTest { 22 | 23 | @Test 24 | public void checkMissingConfiguration() { 25 | final Map configs = new HashMap<>(); 26 | final BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 27 | // Check if the topic configuration is missing 28 | assertThrows(ConfigException.class, () -> { 29 | assignor.configure(configs); 30 | }); 31 | // Check if the buckets configuration is missing 32 | assertThrows(ConfigException.class, () -> { 33 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 34 | assignor.configure(configs); 35 | }); 36 | // Check if the allocation configuration is missing 37 | assertThrows(ConfigException.class, () -> { 38 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 39 | assignor.configure(configs); 40 | }); 41 | // Check if complete configuration is gonna be enough 42 | assertDoesNotThrow(() -> { 43 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 44 | assignor.configure(configs); 45 | }); 46 | } 47 | 48 | @Test 49 | public void checkMatchingBucketConfiguration() { 50 | final Map configs = new HashMap<>(); 51 | final BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 52 | assertThrows(InvalidConfigurationException.class, () -> { 53 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 54 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 55 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%"); 56 | assignor.configure(configs); 57 | }); 58 | assertDoesNotThrow(() -> { 59 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 60 | assignor.configure(configs); 61 | }); 62 | } 63 | 64 | @Test 65 | public void checkAllocationPercentageConfiguration() { 66 | final Map configs = new HashMap<>(); 67 | final BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 68 | assertThrows(InvalidConfigurationException.class, () -> { 69 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 70 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 71 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 20%"); 72 | assignor.configure(configs); 73 | }); 74 | assertDoesNotThrow(() -> { 75 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 76 | assignor.configure(configs); 77 | }); 78 | } 79 | 80 | @Test 81 | public void checkIfFallbackPartitionerIsValid() { 82 | final Map configs = new HashMap<>(); 83 | final BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 84 | assertThrows(InvalidConfigurationException.class, () -> { 85 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 86 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 87 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 88 | configs.put(BucketPriorityConfig.FALLBACK_ASSIGNOR_CONFIG, 89 | BucketPriorityAssignorTest.class.getName()); 90 | assignor.configure(configs); 91 | }); 92 | } 93 | 94 | @Test 95 | public void checkIfMinNumberPartitionsIsRespected() { 96 | final String topic = "test"; 97 | final Map configs = new HashMap<>(); 98 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 99 | // Using two buckets implies having at least two partitions... 100 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 101 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 102 | configs.put(BucketPriorityConfig.BUCKET_CONFIG, "B1"); 103 | BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 104 | assignor.configure(configs); 105 | assertThrows(InvalidConfigurationException.class, () -> { 106 | Map partitionsPerTopic = Map.of(topic, 1); 107 | Map subscriptions = 108 | Map.of("consumer-0", new ConsumerPartitionAssignor.Subscription(List.of(topic))); 109 | assignor.assign(partitionsPerTopic, subscriptions); 110 | }); 111 | } 112 | 113 | @Test 114 | public void checkMultipleTopicsAssignment() { 115 | 116 | final String regularTopic = "regularTopic"; 117 | final String bucketTopic = "bucketTopic"; 118 | final Map configs = new HashMap<>(); 119 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, bucketTopic); 120 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 121 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 122 | BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 123 | assignor.configure(configs); 124 | 125 | // Create the partitions and the subscriptions 126 | Map partitionsPerTopic = Map.of(regularTopic, 6, bucketTopic, 6); 127 | Map subscriptions = new HashMap<>(); 128 | 129 | // Create 4 consumers, 2 for each topic 130 | int count = 0; 131 | for (int i = 0; i < 2; i++) { 132 | subscriptions.put(String.format("consumer-%d", count++), 133 | new ConsumerPartitionAssignor.Subscription( 134 | List.of(regularTopic))); 135 | } 136 | for (int i = 0; i < 2; i++) { 137 | subscriptions.put(String.format("consumer-%d", count++), 138 | new ConsumerPartitionAssignor.Subscription( 139 | List.of(bucketTopic), StandardCharsets.UTF_8.encode("B1"))); 140 | } 141 | 142 | // Execute the assignor 143 | Map> assignments = 144 | assignor.assign(partitionsPerTopic, subscriptions); 145 | 146 | // The expected output is that each of the 4 consumers 147 | // will have assignments and their assignments need to 148 | // be greather than zero. 149 | assertEquals(4, assignments.size()); 150 | assignments.values().forEach(v -> assertTrue(v.size() > 0)); 151 | 152 | } 153 | 154 | @Test 155 | public void checkPerBucketAssignmentWithoutRebalance() { 156 | 157 | final String bucketTopic = "bucketTopic"; 158 | final Map configs = new HashMap<>(); 159 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, bucketTopic); 160 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 161 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "80%, 20%"); 162 | BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 163 | assignor.configure(configs); 164 | 165 | // Create the partitions and the subscriptions 166 | Map partitionsPerTopic = Map.of(bucketTopic, 10); 167 | Map subscriptions = new HashMap<>(); 168 | 169 | int count = 0; 170 | // Create 8 consumers for the B1 bucket 171 | for (int i = 0; i < 8; i++) { 172 | subscriptions.put(String.format("consumer-%d", count++), 173 | new ConsumerPartitionAssignor.Subscription( 174 | List.of(bucketTopic), StandardCharsets.UTF_8.encode("B1"))); 175 | } 176 | // Create 2 consumers for the B2 bucket 177 | for (int i = 0; i < 2; i++) { 178 | subscriptions.put(String.format("consumer-%d", count++), 179 | new ConsumerPartitionAssignor.Subscription( 180 | List.of(bucketTopic), StandardCharsets.UTF_8.encode("B2"))); 181 | } 182 | 183 | // Execute the assignor 184 | Map> assignments = 185 | assignor.assign(partitionsPerTopic, subscriptions); 186 | 187 | // The expected output is that each of the 10 consumers 188 | // will have assignments and their assignments need to 189 | // be greather than zero. 190 | assertEquals(10, assignments.size()); 191 | assignments.values().forEach(v -> assertTrue(v.size() > 0)); 192 | 193 | // Also consumer-0 to consumer-7 should be working 194 | // on B1 while consumer-8 and consumer-9 should be 195 | // working on B2. 196 | final Set b1Consumers = new TreeSet<>(); 197 | final Set b2Consumers = new TreeSet<>(); 198 | assignments.entrySet().forEach(assignment -> { 199 | String consumer = assignment.getKey(); 200 | assignment.getValue().stream().forEach(tp -> { 201 | if (tp.partition() >= 0 && tp.partition() <= 7) { // B1 202 | b1Consumers.add(consumer); 203 | } 204 | if (tp.partition() >= 8 && tp.partition() <= 9) { // B2 205 | b2Consumers.add(consumer); 206 | } 207 | }); 208 | }); 209 | count = 0; 210 | Set expectedB1Consumers = new TreeSet<>(); 211 | for (int i = 0; i < 8; i++) { 212 | expectedB1Consumers.add(String.format("consumer-%d", count++)); 213 | } 214 | Set expectedB2Consumers = new TreeSet<>(); 215 | for (int i = 0; i < 2; i++) { 216 | expectedB2Consumers.add(String.format("consumer-%d", count++)); 217 | } 218 | 219 | // Check if the expected consumers is correct 220 | assertEquals(expectedB1Consumers, b1Consumers); 221 | assertEquals(expectedB2Consumers, b2Consumers); 222 | 223 | } 224 | 225 | @Test 226 | public void checkPerBucketAssignmentWithRebalance() { 227 | 228 | final String bucketTopic = "bucketTopic"; 229 | final Map configs = new HashMap<>(); 230 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, bucketTopic); 231 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 232 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "80%, 20%"); 233 | BucketPriorityAssignor assignor = new BucketPriorityAssignor(); 234 | assignor.configure(configs); 235 | 236 | // Create the partitions and the subscriptions 237 | Map partitionsPerTopic = Map.of(bucketTopic, 10); 238 | Map subscriptions = new HashMap<>(); 239 | 240 | int count = 0; 241 | // Create 8 consumers for the B1 bucket 242 | for (int i = 0; i < 8; i++) { 243 | subscriptions.put(String.format("consumer-%d", count++), 244 | new ConsumerPartitionAssignor.Subscription( 245 | List.of(bucketTopic), StandardCharsets.UTF_8.encode("B1"))); 246 | } 247 | // Create 2 consumers for the B2 bucket 248 | for (int i = 0; i < 2; i++) { 249 | subscriptions.put(String.format("consumer-%d", count++), 250 | new ConsumerPartitionAssignor.Subscription( 251 | List.of(bucketTopic), StandardCharsets.UTF_8.encode("B2"))); 252 | } 253 | 254 | // Execute the assignor 255 | Map> assignments = 256 | assignor.assign(partitionsPerTopic, subscriptions); 257 | 258 | // The expected output is that each of the 10 consumers 259 | // will have assignments and their assignments need to 260 | // be greather than zero. 261 | assertEquals(10, assignments.size()); 262 | assignments.values().forEach(v -> assertTrue(v.size() > 0)); 263 | 264 | // Also consumer-0 to consumer-7 should be working 265 | // on B1 while consumer-8 and consumer-9 should be 266 | // working on B2. 267 | final Set b1Consumers = new TreeSet<>(); 268 | final Set b2Consumers = new TreeSet<>(); 269 | assignments.entrySet().forEach(assignment -> { 270 | String consumer = assignment.getKey(); 271 | assignment.getValue().stream().forEach(tp -> { 272 | if (tp.partition() >= 0 && tp.partition() <= 7) { // B1 273 | b1Consumers.add(consumer); 274 | } 275 | if (tp.partition() >= 8 && tp.partition() <= 9) { // B2 276 | b2Consumers.add(consumer); 277 | } 278 | }); 279 | }); 280 | count = 0; 281 | Set expectedB1Consumers = new TreeSet<>(); 282 | for (int i = 0; i < 8; i++) { 283 | expectedB1Consumers.add(String.format("consumer-%d", count++)); 284 | } 285 | Set expectedB2Consumers = new TreeSet<>(); 286 | for (int i = 0; i < 2; i++) { 287 | expectedB2Consumers.add(String.format("consumer-%d", count++)); 288 | } 289 | 290 | // Check if the expected consumers is correct 291 | assertEquals(expectedB1Consumers, b1Consumers); 292 | assertEquals(expectedB2Consumers, b2Consumers); 293 | 294 | // ***************** Rebalance ***************** 295 | 296 | // Now let's simulate a rebalance triggered because two 297 | // consumers from each bucket presumably dropped. 298 | subscriptions.clear(); 299 | count = 0; 300 | // Create 6 consumers only for the B1 bucket 301 | for (int i = 0; i < 6; i++) { 302 | subscriptions.put(String.format("consumer-%d", count++), 303 | new ConsumerPartitionAssignor.Subscription( 304 | List.of(bucketTopic), StandardCharsets.UTF_8.encode("B1"))); 305 | } 306 | // Zero consumers will be created for B2 307 | 308 | // Execute the assignor one more time... 309 | assignments = assignor.assign(partitionsPerTopic, subscriptions); 310 | 311 | // The expected output is that now we have only 6 consumers 312 | // and each one of them still need to have their assignments 313 | // and their assignments need to be greather than zero. 314 | assertEquals(6, assignments.size()); 315 | assignments.values().forEach(v -> assertTrue(v.size() > 0)); 316 | 317 | // Also consumer-0 to consumer-5 should be working 318 | // on B1 while B2 should have zero consumers... 319 | b1Consumers.clear(); 320 | b2Consumers.clear(); 321 | assignments.entrySet().forEach(assignment -> { 322 | String consumer = assignment.getKey(); 323 | assignment.getValue().stream().forEach(tp -> { 324 | if (tp.partition() >= 0 && tp.partition() <= 7) { // B1 325 | b1Consumers.add(consumer); 326 | } 327 | if (tp.partition() >= 8 && tp.partition() <= 9) { // B2 328 | b2Consumers.add(consumer); 329 | } 330 | }); 331 | }); 332 | count = 0; 333 | expectedB1Consumers.clear(); 334 | for (int i = 0; i < 6; i++) { 335 | expectedB1Consumers.add(String.format("consumer-%d", count++)); 336 | } 337 | expectedB2Consumers.clear(); 338 | 339 | // Check if the expected consumers is correct 340 | assertEquals(expectedB1Consumers, b1Consumers); 341 | assertEquals(expectedB2Consumers, b2Consumers); 342 | 343 | } 344 | 345 | } 346 | -------------------------------------------------------------------------------- /src/test/java/code/buildon/aws/streaming/kafka/BucketPriorityPartitionerTest.java: -------------------------------------------------------------------------------- 1 | package code.buildon.aws.streaming.kafka; 2 | 3 | import java.lang.reflect.Field; 4 | import java.util.ArrayList; 5 | import java.util.HashMap; 6 | import java.util.List; 7 | import java.util.Map; 8 | import java.util.Set; 9 | import java.util.concurrent.atomic.AtomicInteger; 10 | 11 | import org.apache.kafka.clients.producer.MockProducer; 12 | import org.apache.kafka.clients.producer.ProducerRecord; 13 | import org.apache.kafka.common.Cluster; 14 | import org.apache.kafka.common.Node; 15 | import org.apache.kafka.common.PartitionInfo; 16 | import org.apache.kafka.common.config.ConfigException; 17 | import org.apache.kafka.common.errors.InvalidConfigurationException; 18 | import org.apache.kafka.common.serialization.StringSerializer; 19 | import org.junit.jupiter.api.Test; 20 | 21 | import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; 22 | import static org.junit.jupiter.api.Assertions.assertEquals; 23 | import static org.junit.jupiter.api.Assertions.assertThrows; 24 | 25 | public class BucketPriorityPartitionerTest { 26 | 27 | @Test 28 | public void checkMissingConfiguration() { 29 | final Map configs = new HashMap<>(); 30 | try (BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner()) { 31 | // Check if the topic configuration is missing 32 | assertThrows(ConfigException.class, () -> { 33 | partitioner.configure(configs); 34 | }); 35 | // Check if the buckets configuration is missing 36 | assertThrows(ConfigException.class, () -> { 37 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 38 | partitioner.configure(configs); 39 | }); 40 | // Check if the allocation configuration is missing 41 | assertThrows(ConfigException.class, () -> { 42 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 43 | partitioner.configure(configs); 44 | }); 45 | // Check if complete configuration is gonna be enough 46 | assertDoesNotThrow(() -> { 47 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 48 | partitioner.configure(configs); 49 | }); 50 | } 51 | } 52 | 53 | @Test 54 | public void checkMatchingBucketConfiguration() { 55 | final Map configs = new HashMap<>(); 56 | try (BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner()) { 57 | assertThrows(InvalidConfigurationException.class, () -> { 58 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 59 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 60 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%"); 61 | partitioner.configure(configs); 62 | }); 63 | assertDoesNotThrow(() -> { 64 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 65 | partitioner.configure(configs); 66 | }); 67 | } 68 | } 69 | 70 | @Test 71 | public void checkAllocationPercentageConfiguration() { 72 | final Map configs = new HashMap<>(); 73 | try (BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner()) { 74 | assertThrows(InvalidConfigurationException.class, () -> { 75 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, "test"); 76 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 77 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 20%"); 78 | partitioner.configure(configs); 79 | }); 80 | assertDoesNotThrow(() -> { 81 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 82 | partitioner.configure(configs); 83 | }); 84 | } 85 | } 86 | 87 | @Test 88 | public void checkIfMinNumberPartitionsIsRespected() { 89 | final String topic = "test"; 90 | final Map configs = new HashMap<>(); 91 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 92 | // Using two buckets implies having at least two partitions... 93 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 94 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "70%, 30%"); 95 | BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner(); 96 | partitioner.configure(configs); 97 | // Create a topic with only one partition... 98 | PartitionInfo partition0 = new PartitionInfo(topic, 0, null, null, null); 99 | List partitions = List.of(partition0); 100 | Cluster cluster = createCluster(partitions); 101 | try (MockProducer producer = new MockProducer<>(cluster, 102 | true, partitioner, new StringSerializer(), new StringSerializer())) { 103 | assertThrows(InvalidConfigurationException.class, () -> { 104 | producer.send(new ProducerRecord(topic, "B1-001", "value")); 105 | }); 106 | } 107 | } 108 | 109 | @Test 110 | public void checkBucketAllocationGivenEvenAllocationConfig() { 111 | 112 | final String topic = "test"; 113 | final Map configs = new HashMap<>(); 114 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 115 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2, B3"); 116 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "50%, 30%, 20%"); 117 | BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner(); 118 | partitioner.configure(configs); 119 | 120 | // Create 10 partitions for buckets B1, B2, and B3 121 | List partitions = new ArrayList<>(); 122 | for (int i = 0; i < 10; i++) { 123 | partitions.add(new PartitionInfo(topic, i, null, null, null)); 124 | } 125 | Cluster cluster = createCluster(partitions); 126 | 127 | try (MockProducer producer = new MockProducer<>(cluster, 128 | true, partitioner, new StringSerializer(), new StringSerializer())) { 129 | 130 | ProducerRecord record = null; 131 | // Produce 10 records to the 'B1' bucket that must 132 | // be composed of the partitions [0, 1, 2, 3, 4] 133 | final AtomicInteger b1Count = new AtomicInteger(0); 134 | for (int i = 0; i < 10; i++) { 135 | String recordKey = "B1-" + i; 136 | record = new ProducerRecord(topic, recordKey, "value"); 137 | producer.send(record, (metadata, exception) -> { 138 | int chosenPartition = metadata.partition(); 139 | if (chosenPartition >= 0 && chosenPartition <= 4) { 140 | b1Count.incrementAndGet(); 141 | } 142 | }); 143 | } 144 | 145 | // Produce 10 records to the 'B2' bucket that must 146 | // be composed of the partitions [5, 6, 7] 147 | final AtomicInteger b2Count = new AtomicInteger(0); 148 | for (int i = 0; i < 10; i++) { 149 | String recordKey = "B2-" + i; 150 | record = new ProducerRecord(topic, recordKey, "value"); 151 | producer.send(record, (metadata, exception) -> { 152 | int chosenPartition = metadata.partition(); 153 | if (chosenPartition >= 5 && chosenPartition <= 7) { 154 | b2Count.incrementAndGet(); 155 | } 156 | }); 157 | } 158 | 159 | // Produce 10 records to the 'B3' bucket that must 160 | // be composed of the partitions [8, 9] 161 | final AtomicInteger b3Count = new AtomicInteger(0); 162 | for (int i = 0; i < 10; i++) { 163 | String recordKey = "B3-" + i; 164 | record = new ProducerRecord(topic, recordKey, "value"); 165 | producer.send(record, (metadata, exception) -> { 166 | int chosenPartition = metadata.partition(); 167 | if (chosenPartition >= 8 && chosenPartition <= 9) { 168 | b3Count.incrementAndGet(); 169 | } 170 | }); 171 | } 172 | 173 | // The expected output is: 174 | // - B1 should contain 10 records 175 | // - B2 should contain 10 records 176 | // - B3 should contain 10 records 177 | assertEquals(10, b1Count.get()); 178 | assertEquals(10, b2Count.get()); 179 | assertEquals(10, b3Count.get()); 180 | 181 | } 182 | 183 | } 184 | 185 | @Test 186 | public void checkBucketAllocationGivenUnevenAllocationConfig() { 187 | 188 | final String topic = "test"; 189 | final Map configs = new HashMap<>(); 190 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 191 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2, B3"); 192 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "55%, 40%, 5%"); 193 | BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner(); 194 | partitioner.configure(configs); 195 | 196 | // Create 10 partitions for buckets B1, B2, and B3 197 | List partitions = new ArrayList<>(); 198 | for (int i = 0; i < 10; i++) { 199 | partitions.add(new PartitionInfo(topic, i, null, null, null)); 200 | } 201 | Cluster cluster = createCluster(partitions); 202 | 203 | try (MockProducer producer = new MockProducer<>(cluster, 204 | true, partitioner, new StringSerializer(), new StringSerializer())) { 205 | 206 | ProducerRecord record = null; 207 | // Produce 10 records to the 'B1' bucket that must 208 | // be composed of the partitions [0, 1, 2, 3, 4, 5] 209 | final AtomicInteger b1Count = new AtomicInteger(0); 210 | for (int i = 0; i < 10; i++) { 211 | String recordKey = "B1-" + i; 212 | record = new ProducerRecord(topic, recordKey, "value"); 213 | producer.send(record, (metadata, exception) -> { 214 | int chosenPartition = metadata.partition(); 215 | if (chosenPartition >= 0 && chosenPartition <= 5) { 216 | b1Count.incrementAndGet(); 217 | } 218 | }); 219 | } 220 | 221 | // Produce 10 records to the 'B2' bucket that must 222 | // be composed of the partitions [6, 7, 8, 9] 223 | final AtomicInteger b2Count = new AtomicInteger(0); 224 | for (int i = 0; i < 10; i++) { 225 | String recordKey = "B2-" + i; 226 | record = new ProducerRecord(topic, recordKey, "value"); 227 | producer.send(record, (metadata, exception) -> { 228 | int chosenPartition = metadata.partition(); 229 | if (chosenPartition >= 6 && chosenPartition <= 9) { 230 | b2Count.incrementAndGet(); 231 | } 232 | }); 233 | } 234 | 235 | // Produce 10 records to the 'B3' bucket that must 236 | // be composed of zero partitions [] because is uneven 237 | final AtomicInteger b3Count = new AtomicInteger(0); 238 | for (int i = 0; i < 10; i++) { 239 | String recordKey = "B3-" + i; 240 | record = new ProducerRecord(topic, recordKey, "value"); 241 | producer.send(record, (metadata, exception) -> { 242 | // Getting a partition set to -1 means that 243 | // the record didn't get written anywhere and 244 | // therefore we can expect that the counter 245 | // will never be incremented below. 246 | if (metadata.partition() != -1) { 247 | b3Count.incrementAndGet(); 248 | } 249 | }); 250 | } 251 | 252 | // The expected output is: 253 | // - B1 should contain 10 records 254 | // - B2 should contain 10 records 255 | // - B3 should contain 0 records 256 | assertEquals(10, b1Count.get()); 257 | assertEquals(10, b2Count.get()); 258 | assertEquals(0, b3Count.get()); 259 | 260 | } 261 | 262 | } 263 | 264 | @Test 265 | public void checkBucketAllocationGivenUnevenPartitionNumber() { 266 | 267 | final String topic = "test"; 268 | final Map configs = new HashMap<>(); 269 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 270 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2, B3"); 271 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "55%, 40%, 5%"); 272 | BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner(); 273 | partitioner.configure(configs); 274 | 275 | // Create 5 partitions for buckets B1, B2, and B3 276 | List partitions = new ArrayList<>(); 277 | for (int i = 0; i < 5; i++) { 278 | partitions.add(new PartitionInfo(topic, i, null, null, null)); 279 | } 280 | Cluster cluster = createCluster(partitions); 281 | 282 | try (MockProducer producer = new MockProducer<>(cluster, 283 | true, partitioner, new StringSerializer(), new StringSerializer())) { 284 | 285 | ProducerRecord record = null; 286 | // Produce 10 records to the 'B1' bucket that must 287 | // be composed of the partitions [0, 1, 2] 288 | final AtomicInteger b1Count = new AtomicInteger(0); 289 | for (int i = 0; i < 10; i++) { 290 | String recordKey = "B1-" + i; 291 | record = new ProducerRecord(topic, recordKey, "value"); 292 | producer.send(record, (metadata, exception) -> { 293 | int chosenPartition = metadata.partition(); 294 | if (chosenPartition >= 0 && chosenPartition <= 2) { 295 | b1Count.incrementAndGet(); 296 | } 297 | }); 298 | } 299 | 300 | // Produce 10 records to the 'B2' bucket that must 301 | // be composed of the partitions [3, 4] 302 | final AtomicInteger b2Count = new AtomicInteger(0); 303 | for (int i = 0; i < 10; i++) { 304 | String recordKey = "B2-" + i; 305 | record = new ProducerRecord(topic, recordKey, "value"); 306 | producer.send(record, (metadata, exception) -> { 307 | int chosenPartition = metadata.partition(); 308 | if (chosenPartition >= 3 && chosenPartition <= 4) { 309 | b2Count.incrementAndGet(); 310 | } 311 | }); 312 | } 313 | 314 | // Produce 10 records to the 'B3' bucket that must 315 | // be composed of zero partitions [] because is uneven 316 | final AtomicInteger b3Count = new AtomicInteger(0); 317 | for (int i = 0; i < 10; i++) { 318 | String recordKey = "B3-" + i; 319 | record = new ProducerRecord(topic, recordKey, "value"); 320 | producer.send(record, (metadata, exception) -> { 321 | // Getting a partition set to -1 means that 322 | // the record didn't get written anywhere and 323 | // therefore we can expect that the counter 324 | // will never be incremented below. 325 | if (metadata.partition() != -1) { 326 | b3Count.incrementAndGet(); 327 | } 328 | }); 329 | } 330 | 331 | // The expected output is: 332 | // - B1 should contain 10 records 333 | // - B2 should contain 10 records 334 | // - B3 should contain 0 records 335 | assertEquals(10, b1Count.get()); 336 | assertEquals(10, b2Count.get()); 337 | assertEquals(0, b3Count.get()); 338 | 339 | } 340 | 341 | } 342 | 343 | @Test 344 | public void checkRoundRobinBucketDataDistribution() { 345 | 346 | final String topic = "test"; 347 | final Map configs = new HashMap<>(); 348 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 349 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 350 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "80%, 20%"); 351 | BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner(); 352 | partitioner.configure(configs); 353 | 354 | // Create 10 partitions for buckets B1 and B2 that 355 | // will create the following partition assignment: 356 | // B1 = [0, 1, 2, 3, 4, 5, 6, 7] 357 | // B2 = [8, 9] 358 | List partitions = new ArrayList<>(); 359 | for (int i = 0; i < 10; i++) { 360 | partitions.add(new PartitionInfo(topic, i, null, null, null)); 361 | } 362 | Cluster cluster = createCluster(partitions); 363 | 364 | try (MockProducer producer = new MockProducer<>(cluster, 365 | true, partitioner, new StringSerializer(), new StringSerializer())) { 366 | 367 | final Map distribution = new HashMap<>(); 368 | ProducerRecord record = null; 369 | // Produce 32 records to the 'B1' bucket that 370 | // should distribute 4 records per partition. 371 | for (int i = 0; i < 32; i++) { 372 | String recordKey = "B1-" + i; 373 | record = new ProducerRecord(topic, recordKey, "value"); 374 | producer.send(record, (metadata, exception) -> { 375 | int chosenPartition = metadata.partition(); 376 | int currentCount = 0; 377 | if (distribution.containsKey(chosenPartition)) { 378 | currentCount = distribution.get(chosenPartition); 379 | } 380 | distribution.put(chosenPartition, ++currentCount); 381 | }); 382 | } 383 | 384 | // Produce 32 records to the 'B2' bucket that 385 | // should distribute 16 records per partition. 386 | for (int i = 0; i < 32; i++) { 387 | String recordKey = "B2-" + i; 388 | record = new ProducerRecord(topic, recordKey, "value"); 389 | producer.send(record, (metadata, exception) -> { 390 | int chosenPartition = metadata.partition(); 391 | int currentCount = 0; 392 | if (distribution.containsKey(chosenPartition)) { 393 | currentCount = distribution.get(chosenPartition); 394 | } 395 | distribution.put(chosenPartition, ++currentCount); 396 | }); 397 | } 398 | 399 | // The expected output is: 400 | // - 4 records on each partition of B1 401 | // - 16 records on each partition of B2 402 | Map expected = new HashMap<>(); 403 | // B1 404 | expected.put(0, 4); 405 | expected.put(1, 4); 406 | expected.put(2, 4); 407 | expected.put(3, 4); 408 | expected.put(4, 4); 409 | expected.put(5, 4); 410 | expected.put(6, 4); 411 | expected.put(7, 4); 412 | // B2 413 | expected.put(8, 16); 414 | expected.put(9, 16); 415 | assertEquals(expected, distribution); 416 | 417 | } 418 | 419 | } 420 | 421 | @Test 422 | public void checkBucketsResizeDueToPartitionsIncrease() { 423 | 424 | final String topic = "test"; 425 | final Map configs = new HashMap<>(); 426 | configs.put(BucketPriorityConfig.TOPIC_CONFIG, topic); 427 | configs.put(BucketPriorityConfig.BUCKETS_CONFIG, "B1, B2"); 428 | configs.put(BucketPriorityConfig.ALLOCATION_CONFIG, "80%, 20%"); 429 | BucketPriorityPartitioner partitioner = new BucketPriorityPartitioner(); 430 | partitioner.configure(configs); 431 | 432 | // Create 10 partitions for buckets B1 and B2 that 433 | // will create the following partition assignment: 434 | // B1 = [0, 1, 2, 3, 4, 5, 6, 7] 435 | // B2 = [8, 9] 436 | List partitions = new ArrayList<>(); 437 | for (int i = 0; i < 10; i++) { 438 | partitions.add(new PartitionInfo(topic, i, null, null, null)); 439 | } 440 | Cluster cluster = createCluster(partitions); 441 | 442 | try (MockProducer producer = new MockProducer<>(cluster, 443 | true, partitioner, new StringSerializer(), new StringSerializer())) { 444 | 445 | final Map distribution = new HashMap<>(); 446 | ProducerRecord record = null; 447 | // Produce 32 records to the 'B1' bucket that 448 | // should distribute 4 records per partition. 449 | for (int i = 0; i < 32; i++) { 450 | String recordKey = "B1-" + i; 451 | record = new ProducerRecord(topic, recordKey, "value"); 452 | producer.send(record, (metadata, exception) -> { 453 | int chosenPartition = metadata.partition(); 454 | int currentCount = 0; 455 | if (distribution.containsKey(chosenPartition)) { 456 | currentCount = distribution.get(chosenPartition); 457 | } 458 | distribution.put(chosenPartition, ++currentCount); 459 | }); 460 | } 461 | 462 | // Produce 32 records to the 'B2' bucket that 463 | // should distribute 16 records per partition. 464 | for (int i = 0; i < 32; i++) { 465 | String recordKey = "B2-" + i; 466 | record = new ProducerRecord(topic, recordKey, "value"); 467 | producer.send(record, (metadata, exception) -> { 468 | int chosenPartition = metadata.partition(); 469 | int currentCount = 0; 470 | if (distribution.containsKey(chosenPartition)) { 471 | currentCount = distribution.get(chosenPartition); 472 | } 473 | distribution.put(chosenPartition, ++currentCount); 474 | }); 475 | } 476 | 477 | // The expected output is: 478 | // - 4 records on each partition of B1 479 | // - 16 records on each partition of B2 480 | Map expected = new HashMap<>(); 481 | // B1 482 | expected.put(0, 4); 483 | expected.put(1, 4); 484 | expected.put(2, 4); 485 | expected.put(3, 4); 486 | expected.put(4, 4); 487 | expected.put(5, 4); 488 | expected.put(6, 4); 489 | expected.put(7, 4); 490 | // B2 491 | expected.put(8, 16); 492 | expected.put(9, 16); 493 | assertEquals(expected, distribution); 494 | 495 | // Now let's force the partitions to be reallocated into 496 | // the buckets because the number of partitions has been 497 | // increased (doubled) by the user. 498 | increasePartitionNumber(topic, 20, producer); 499 | // Adding 10 more partitions will create the 500 | // following new partition assignment: 501 | // B1 = [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 502 | // B2 = [16, 17, 18, 19] 503 | 504 | // Clear the distribution counter so we can check again 505 | distribution.clear(); 506 | 507 | // Produce 32 records to the 'B1' bucket that 508 | // should distribute 2 records per partition. 509 | for (int i = 0; i < 32; i++) { 510 | String recordKey = "B1-" + i; 511 | record = new ProducerRecord(topic, recordKey, "value"); 512 | producer.send(record, (metadata, exception) -> { 513 | int chosenPartition = metadata.partition(); 514 | int currentCount = 0; 515 | if (distribution.containsKey(chosenPartition)) { 516 | currentCount = distribution.get(chosenPartition); 517 | } 518 | distribution.put(chosenPartition, ++currentCount); 519 | }); 520 | } 521 | 522 | // Produce 32 records to the 'B2' bucket that 523 | // should distribute 8 records per partition. 524 | for (int i = 0; i < 32; i++) { 525 | String recordKey = "B2-" + i; 526 | record = new ProducerRecord(topic, recordKey, "value"); 527 | producer.send(record, (metadata, exception) -> { 528 | int chosenPartition = metadata.partition(); 529 | int currentCount = 0; 530 | if (distribution.containsKey(chosenPartition)) { 531 | currentCount = distribution.get(chosenPartition); 532 | } 533 | distribution.put(chosenPartition, ++currentCount); 534 | }); 535 | } 536 | expected.clear(); 537 | 538 | // The expected output is: 539 | // - 2 records on each partition of B1 540 | // - 8 records on each partition of B2 541 | // B1 542 | expected.put(0, 2); 543 | expected.put(1, 2); 544 | expected.put(2, 2); 545 | expected.put(3, 2); 546 | expected.put(4, 2); 547 | expected.put(5, 2); 548 | expected.put(6, 2); 549 | expected.put(7, 2); 550 | expected.put(8, 2); 551 | expected.put(9, 2); 552 | expected.put(10, 2); 553 | expected.put(11, 2); 554 | expected.put(12, 2); 555 | expected.put(13, 2); 556 | expected.put(14, 2); 557 | expected.put(15, 2); 558 | // B2 559 | expected.put(16, 8); 560 | expected.put(17, 8); 561 | expected.put(18, 8); 562 | expected.put(19, 8); 563 | assertEquals(expected, distribution); 564 | 565 | } 566 | 567 | } 568 | 569 | @SuppressWarnings("rawtypes") 570 | private void increasePartitionNumber(String topic, int newPartitionCount, 571 | MockProducer producer) { 572 | List currPartitions = producer.partitionsFor(topic); 573 | List targetPartitions = new ArrayList<>(newPartitionCount); 574 | targetPartitions.addAll(currPartitions); 575 | for (int i = targetPartitions.size(); i < newPartitionCount; i++) { 576 | targetPartitions.add(new PartitionInfo(topic, i, null, null, null)); 577 | } 578 | Cluster newCluster = createCluster(targetPartitions); 579 | // Not pretty but since the MockProducer class doesn't provide 580 | // a way to replace the existing cluster we need to inject it 581 | Class mockProducerClass = MockProducer.class; 582 | Field clusterField = null; 583 | try { 584 | clusterField = mockProducerClass.getDeclaredField("cluster"); 585 | clusterField.setAccessible(true); 586 | clusterField.set(producer, newCluster); 587 | } catch (Exception ex) { 588 | } 589 | } 590 | 591 | private Cluster createCluster(List partitions) { 592 | return new Cluster("newCluster", new ArrayList(), 593 | partitions, Set.of(), Set.of()); 594 | } 595 | 596 | } 597 | --------------------------------------------------------------------------------