From 011ccb38cf440a625e689177910d8b35a2aac0e5 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 27 Feb 2023 12:02:31 +0800 Subject: [PATCH 01/50] Implement NetworkIngressAssignor --- .../org/astraea/common/assignor/Assignor.java | 22 ++- .../assignor/NetworkIngressAssignor.java | 146 ++++++++++++++++++ 2 files changed, 164 insertions(+), 4 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 18e935d593..fec0e9c8af 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -18,9 +18,11 @@ import java.net.InetSocketAddress; import java.time.Duration; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; @@ -34,7 +36,7 @@ import org.astraea.common.admin.TopicPartition; import org.astraea.common.consumer.ConsumerConfigs; import org.astraea.common.cost.HasPartitionCost; -import org.astraea.common.cost.ReplicaLeaderSizeCost; +import org.astraea.common.cost.NetworkIngressCost; import org.astraea.common.metrics.collector.MetricCollector; import org.astraea.common.partitioner.PartitionerUtils; @@ -62,8 +64,6 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurabl */ protected abstract Map> assign( Map subscriptions, ClusterInfo clusterInfo); - // TODO: replace the topicPartitions by ClusterInfo after Assignor is able to handle Admin - // https://github.com/skiptests/astraea/issues/1409 /** * Parse config to get JMX port and cost function type. @@ -74,6 +74,20 @@ protected void configure(Configuration config) {} // -----------------------[helper]-----------------------// + /** + * get all topics which consumers subscribe. + * + * @param subscriptions Consumers' subscriptions + * @return A set of topics the consumers subscribe + */ + protected Set topics( + Map subscriptions) { + return subscriptions.values().stream() + .map(org.astraea.common.assignor.Subscription::topics) + .flatMap(Collection::stream) + .collect(Collectors.toUnmodifiableSet()); + } + /** * check the nodes which wasn't register yet. * @@ -150,7 +164,7 @@ public final void configure(Map configs) { var defaultJMXPort = config.integer(JMX_PORT); this.costFunction = costFunctions.isEmpty() - ? HasPartitionCost.of(Map.of(new ReplicaLeaderSizeCost(), 1D)) + ? HasPartitionCost.of(Map.of(new NetworkIngressCost(), 1D)) : HasPartitionCost.of(costFunctions); this.jmxPortGetter = id -> Optional.ofNullable(customJMXPort.get(id)).or(() -> defaultJMXPort); this.costFunction.metricSensor().ifPresent(metricCollector::addMetricSensor); diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java new file mode 100644 index 0000000000..4f16624a6c --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.astraea.common.assignor; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.astraea.common.Utils; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Replica; +import org.astraea.common.admin.TopicPartition; + +public class NetworkIngressAssignor extends Assignor { + + @Override + protected Map> assign( + Map subscriptions, + ClusterInfo clusterInfo) { + var consumers = subscriptions.keySet(); + // 1. check unregister node. if there are unregister nodes, register them + registerUnregisterNode(clusterInfo); + // wait for clusterBean + Utils.sleep(Duration.ofSeconds(1)); + var clusterBean = metricCollector.clusterBean(); + + // 2. parse subscription , get all topic consumer subscribe + var topics = topics(subscriptions); + var networkCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); + + // key = broker id, value = partition and its cost + var tpCostPerBroker = + clusterInfo + .replicaStream() + .filter(Replica::isLeader) + .filter(Replica::isOnline) + .filter(replica -> topics.contains(replica.topic())) + .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) + .entrySet() + .stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .map( + replica -> + Map.entry( + replica.topicPartition(), + networkCost.get(replica.topicPartition()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + return greedyAssign(tpCostPerBroker, consumers); + } + + /** + * register unregistered nodes if present. if we didn't register unregistered nodes, we would miss + * the beanObjects from the nodes + * + * @param clusterInfo Currently cluster information. + */ + private void registerUnregisterNode(ClusterInfo clusterInfo) { + var unregister = checkUnregister(clusterInfo.nodes()); + if (!unregister.isEmpty()) registerJMX(unregister); + } + + // visible for test + /** + * perform assign algorithm to get balanced assignment and ensure that 1. each consumer would + * receive the cost that are as close as possible to each other. 2. similar loads within a node + * would be assigned to the same consumer. + * + * @param costs the tp and their cost within a node + * @param consumers consumers' name + * @return the assignment + */ + Map> greedyAssign( + Map> costs, Set consumers) { + // initial + var assignment = new HashMap>(); + for (var consumer : consumers) { + assignment.put(consumer, new ArrayList<>()); + } + var costPerConsumer = + assignment.keySet().stream() + .map(c -> Map.entry(c, (double) 0)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + costs + .values() + .forEach( + costPerBroker -> { + var sortedCost = new LinkedHashMap(); + costPerBroker.entrySet().stream() + .sorted(Map.Entry.comparingByValue()) + .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); + var tmpCostPerConsumer = new HashMap<>(costPerConsumer); + Supplier largestCostConsumer = + () -> + Collections.max(tmpCostPerConsumer.entrySet(), Map.Entry.comparingByValue()) + .getKey(); + var consumer = largestCostConsumer.get(); + var lastValue = Collections.min(sortedCost.values()); + + for (var e : sortedCost.entrySet()) { + var tp = e.getKey(); + var cost = e.getValue(); + // TODO: threshold need to be set an appropriate value + if (cost - lastValue > 0.05) { + tmpCostPerConsumer.remove(consumer); + consumer = largestCostConsumer.get(); + } + + assignment.get(consumer).add(tp); + costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); + lastValue = cost; + } + }); + return assignment; + } + + @Override + public String name() { + return "NetworkIngress"; + } +} From f77a8c7ebbe8ce1b8ef83dd1a924f99d6ef97f91 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 27 Feb 2023 18:08:10 +0800 Subject: [PATCH 02/50] Add a test for checking greedyAssign --- .../assignor/NetworkIngressAssignor.java | 3 +- .../assignor/NetworkIngressAssignorTest.java | 186 ++++++++++++++++++ 2 files changed, 187 insertions(+), 2 deletions(-) create mode 100644 common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 4f16624a6c..bdcadfcb0a 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -85,7 +85,6 @@ private void registerUnregisterNode(ClusterInfo clusterInfo) { if (!unregister.isEmpty()) registerJMX(unregister); } - // visible for test /** * perform assign algorithm to get balanced assignment and ensure that 1. each consumer would * receive the cost that are as close as possible to each other. 2. similar loads within a node @@ -141,6 +140,6 @@ Map> greedyAssign( @Override public String name() { - return "NetworkIngress"; + return "networkIngress"; } } diff --git a/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java new file mode 100644 index 0000000000..d820faebd0 --- /dev/null +++ b/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.astraea.common.assignor; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.astraea.common.DataRate; +import org.astraea.common.admin.ClusterBean; +import org.astraea.common.admin.ClusterInfoBuilder; +import org.astraea.common.admin.Replica; +import org.astraea.common.cost.NetworkIngressCost; +import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.broker.ServerMetrics; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class NetworkIngressAssignorTest { + @Test + void testGreedyAssign() { + var clusterInfo = + ClusterInfoBuilder.builder() + .addNode(Set.of(1, 2, 3)) + .addFolders( + Map.of( + 1, + Set.of("/folder0", "/folder1"), + 2, + Set.of("/folder0", "/folder1"), + 3, + Set.of("/folder0", "/folder1"))) + .addTopic( + "a", + 9, + (short) 1, + replica -> { + var factor = getFactor(replica.partition()); + return Replica.builder(replica) + .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .build(); + }) + .addTopic( + "b", + 9, + (short) 1, + replica -> { + var factor = getFactor(replica.partition()); + return Replica.builder(replica) + .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .build(); + }) + .addTopic( + "c", + 9, + (short) 1, + replica -> { + var factor = getFactor(replica.partition()); + return Replica.builder(replica) + .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .build(); + }) + .build(); + var clusterBean = + ClusterBean.of( + Map.of( + 1, + List.of( + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "a", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "b", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "c", + DataRate.MB.of(90).perSecond().byteRate())), + 2, + List.of( + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "a", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "b", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "c", + DataRate.MB.of(90).perSecond().byteRate())), + 3, + List.of( + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "a", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "b", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "c", + DataRate.MB.of(90).perSecond().byteRate())))); + + var networkCost = new NetworkIngressCost(); + var cost = networkCost.partitionCost(clusterInfo, clusterBean).value(); + + var topics = Set.of("a", "b", "c"); + var tpCostPerBroker = + clusterInfo + .replicaStream() + .filter(Replica::isLeader) + .filter(Replica::isOnline) + .filter(replica -> topics.contains(replica.topic())) + .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) + .entrySet() + .stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .map( + replica -> + Map.entry( + replica.topicPartition(), + cost.get(replica.topicPartition()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var assignor = new NetworkIngressAssignor(); + var assignment = assignor.greedyAssign(tpCostPerBroker, Set.of("haha1", "haha2", "haha3")); + var costPerConsumer = + assignment.entrySet().stream() + .map( + e -> { + var tps = e.getValue(); + var tpCost = + tpCostPerBroker.values().stream() + .flatMap(v -> v.entrySet().stream()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return Map.entry(e.getKey(), tps.stream().mapToDouble(tpCost::get).sum()); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + var sortedCostPerConsumer = new LinkedHashMap(); + costPerConsumer.entrySet().stream() + .sorted(Map.Entry.comparingByValue()) + .forEach(e -> sortedCostPerConsumer.put(e.getKey(), e.getValue())); + sortedCostPerConsumer.forEach((ignore, c) -> Assertions.assertTrue(c > 0.7 && c < 1.3)); + } + + static ServerMetrics.Topic.Meter bandwidth( + ServerMetrics.Topic metric, String topic, double fifteenRate) { + var domainName = "kafka.server"; + var properties = + Map.of("type", "BrokerTopicMetric", "topic", topic, "name", metric.metricName()); + var attributes = Map.of("FifteenMinuteRate", fifteenRate); + return new ServerMetrics.Topic.Meter(new BeanObject(domainName, properties, attributes)); + } + + private int getFactor(int partition) { + if (partition > 2 && partition < 6) return 3; + else if (partition >= 6) return 5; + return 1; + } +} From 34f8492f0be90450c95c04087ef56fb27cf1cec7 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 27 Feb 2023 18:23:59 +0800 Subject: [PATCH 03/50] Add round-robin assign when the cost equals zero --- .../common/assignor/NetworkIngressAssignor.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index bdcadfcb0a..6dfb43f66d 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -109,6 +109,15 @@ Map> greedyAssign( .values() .forEach( costPerBroker -> { + if(costPerBroker.values().stream().mapToDouble(x -> x ). sum() == 0) { + // if there are no cost, round-robin assign per node + var iter = consumers.iterator(); + for(var tp : costPerBroker.keySet()) { + assignment.get(iter.next()).add(tp); + if(!iter.hasNext()) + iter = consumers.iterator(); + } + } else { var sortedCost = new LinkedHashMap(); costPerBroker.entrySet().stream() .sorted(Map.Entry.comparingByValue()) @@ -134,7 +143,7 @@ Map> greedyAssign( costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); lastValue = cost; } - }); + }}); return assignment; } From 17fa82c8c06b9730e95d10641133856267cfaac8 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 27 Feb 2023 18:25:13 +0800 Subject: [PATCH 04/50] spotless --- .../assignor/NetworkIngressAssignor.java | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 6dfb43f66d..b26adc25b5 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -109,41 +109,41 @@ Map> greedyAssign( .values() .forEach( costPerBroker -> { - if(costPerBroker.values().stream().mapToDouble(x -> x ). sum() == 0) { + if (costPerBroker.values().stream().mapToDouble(x -> x).sum() == 0) { // if there are no cost, round-robin assign per node var iter = consumers.iterator(); - for(var tp : costPerBroker.keySet()) { + for (var tp : costPerBroker.keySet()) { assignment.get(iter.next()).add(tp); - if(!iter.hasNext()) - iter = consumers.iterator(); + if (!iter.hasNext()) iter = consumers.iterator(); } } else { - var sortedCost = new LinkedHashMap(); - costPerBroker.entrySet().stream() - .sorted(Map.Entry.comparingByValue()) - .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); - var tmpCostPerConsumer = new HashMap<>(costPerConsumer); - Supplier largestCostConsumer = - () -> - Collections.max(tmpCostPerConsumer.entrySet(), Map.Entry.comparingByValue()) - .getKey(); - var consumer = largestCostConsumer.get(); - var lastValue = Collections.min(sortedCost.values()); + var sortedCost = new LinkedHashMap(); + costPerBroker.entrySet().stream() + .sorted(Map.Entry.comparingByValue()) + .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); + var tmpCostPerConsumer = new HashMap<>(costPerConsumer); + Supplier largestCostConsumer = + () -> + Collections.max(tmpCostPerConsumer.entrySet(), Map.Entry.comparingByValue()) + .getKey(); + var consumer = largestCostConsumer.get(); + var lastValue = Collections.min(sortedCost.values()); - for (var e : sortedCost.entrySet()) { - var tp = e.getKey(); - var cost = e.getValue(); - // TODO: threshold need to be set an appropriate value - if (cost - lastValue > 0.05) { - tmpCostPerConsumer.remove(consumer); - consumer = largestCostConsumer.get(); - } + for (var e : sortedCost.entrySet()) { + var tp = e.getKey(); + var cost = e.getValue(); + // TODO: threshold need to be set an appropriate value + if (cost - lastValue > 0.05) { + tmpCostPerConsumer.remove(consumer); + consumer = largestCostConsumer.get(); + } - assignment.get(consumer).add(tp); - costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); - lastValue = cost; + assignment.get(consumer).add(tp); + costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); + lastValue = cost; + } } - }}); + }); return assignment; } From ed52d24f4e0f2a4e55dd3fd9acdf217b4d48cf18 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Thu, 2 Mar 2023 10:01:50 +0800 Subject: [PATCH 05/50] add throwing exception when there is no mbeanObjects --- .../org/astraea/common/assignor/NetworkIngressAssignor.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index b26adc25b5..440264ff76 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -38,14 +38,15 @@ protected Map> assign( Map subscriptions, ClusterInfo clusterInfo) { var consumers = subscriptions.keySet(); + var topics = topics(subscriptions); // 1. check unregister node. if there are unregister nodes, register them registerUnregisterNode(clusterInfo); // wait for clusterBean - Utils.sleep(Duration.ofSeconds(1)); var clusterBean = metricCollector.clusterBean(); + if(clusterBean.all().isEmpty() || !clusterBean.topics().containsAll(topics)) + throw new RuntimeException("no enough metrics"); // 2. parse subscription , get all topic consumer subscribe - var topics = topics(subscriptions); var networkCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); // key = broker id, value = partition and its cost From 9a1f57eae02be3bbaeb38f4ee6a5bf242d5911f3 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Thu, 2 Mar 2023 16:49:36 +0800 Subject: [PATCH 06/50] spotless --- .../org/astraea/common/assignor/NetworkIngressAssignor.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 440264ff76..9caea20ffd 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -16,7 +16,6 @@ */ package org.astraea.common.assignor; -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -26,7 +25,6 @@ import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; -import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; @@ -43,7 +41,7 @@ protected Map> assign( registerUnregisterNode(clusterInfo); // wait for clusterBean var clusterBean = metricCollector.clusterBean(); - if(clusterBean.all().isEmpty() || !clusterBean.topics().containsAll(topics)) + if (clusterBean.all().isEmpty() || !clusterBean.topics().containsAll(topics)) throw new RuntimeException("no enough metrics"); // 2. parse subscription , get all topic consumer subscribe From 152e5bb8b02720b6d1aa95c7c04fa4467b7de1c3 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Thu, 2 Mar 2023 18:37:30 +0800 Subject: [PATCH 07/50] add more condition to verify whether there are sufficient metrics or not --- .../astraea/common/assignor/NetworkIngressAssignor.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 9caea20ffd..85d808e0de 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -16,6 +16,7 @@ */ package org.astraea.common.assignor; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -28,6 +29,7 @@ import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; +import org.astraea.common.cost.NoSufficientMetricsException; public class NetworkIngressAssignor extends Assignor { @@ -41,8 +43,11 @@ protected Map> assign( registerUnregisterNode(clusterInfo); // wait for clusterBean var clusterBean = metricCollector.clusterBean(); - if (clusterBean.all().isEmpty() || !clusterBean.topics().containsAll(topics)) - throw new RuntimeException("no enough metrics"); + if (clusterBean.all().isEmpty() + || !clusterBean.topics().containsAll(topics) + || !clusterInfo.topicPartitions().containsAll(clusterBean.partitions())) + throw new NoSufficientMetricsException( + costFunction, Duration.ofSeconds(1), "no enough metrics"); // 2. parse subscription , get all topic consumer subscribe var networkCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); From a3e99b8589fc193b6b3badafa69a61b8afcdb93d Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 4 Mar 2023 14:56:58 +0800 Subject: [PATCH 08/50] Add a parameter to set the waiting time that wait for fetch beanObject --- .../main/java/org/astraea/common/assignor/Assignor.java | 3 +++ .../astraea/common/assignor/NetworkIngressAssignor.java | 9 +++------ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 9cc7869328..ba00587637 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -44,6 +44,8 @@ /** Abstract assignor implementation which does some common work (e.g., configuration). */ public abstract class Assignor implements ConsumerPartitionAssignor, Configurable { public static final String JMX_PORT = "jmx.port"; + public static final String MAX_WAIT_BEAN = "max.wait.bean"; + int maxWaitBean = 3; Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -154,6 +156,7 @@ public final void configure(Map configs) { configs.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); + config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); var costFunctions = Utils.costFunctions(config, HasPartitionCost.class); var customJMXPort = PartitionerUtils.parseIdJMXPort(config); var defaultJMXPort = config.integer(JMX_PORT); diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 85d808e0de..dcb22bde8e 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -26,10 +26,10 @@ import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; -import org.astraea.common.cost.NoSufficientMetricsException; public class NetworkIngressAssignor extends Assignor { @@ -42,12 +42,9 @@ protected Map> assign( // 1. check unregister node. if there are unregister nodes, register them registerUnregisterNode(clusterInfo); // wait for clusterBean + Utils.waitFor( + () -> !metricCollector.clusterBean().all().isEmpty(), Duration.ofSeconds(maxWaitBean)); var clusterBean = metricCollector.clusterBean(); - if (clusterBean.all().isEmpty() - || !clusterBean.topics().containsAll(topics) - || !clusterInfo.topicPartitions().containsAll(clusterBean.partitions())) - throw new NoSufficientMetricsException( - costFunction, Duration.ofSeconds(1), "no enough metrics"); // 2. parse subscription , get all topic consumer subscribe var networkCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); From 18a98ce95883b2f6c099600dd00b195c99900076 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 4 Mar 2023 15:12:40 +0800 Subject: [PATCH 09/50] mask ClusterInfo with subscribed topics --- .../common/assignor/NetworkIngressAssignor.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index dcb22bde8e..1c353c3c9a 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -38,7 +38,7 @@ protected Map> assign( Map subscriptions, ClusterInfo clusterInfo) { var consumers = subscriptions.keySet(); - var topics = topics(subscriptions); + var subscribedTopics = topics(subscriptions); // 1. check unregister node. if there are unregister nodes, register them registerUnregisterNode(clusterInfo); // wait for clusterBean @@ -46,8 +46,11 @@ protected Map> assign( () -> !metricCollector.clusterBean().all().isEmpty(), Duration.ofSeconds(maxWaitBean)); var clusterBean = metricCollector.clusterBean(); - // 2. parse subscription , get all topic consumer subscribe - var networkCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); + // 2. get the network cost of all subscribed topic + var networkCost = + costFunction + .partitionCost(ClusterInfo.masked(clusterInfo, subscribedTopics::contains), clusterBean) + .value(); // key = broker id, value = partition and its cost var tpCostPerBroker = @@ -55,7 +58,7 @@ protected Map> assign( .replicaStream() .filter(Replica::isLeader) .filter(Replica::isOnline) - .filter(replica -> topics.contains(replica.topic())) + .filter(replica -> subscribedTopics.contains(replica.topic())) .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) .entrySet() .stream() From 748259ef73e8de8dae0c81f1d49e3acefbd39b59 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 5 Mar 2023 21:43:16 +0800 Subject: [PATCH 10/50] Add calculating the traffic interval to the score of cost --- .../org/astraea/common/assignor/Assignor.java | 5 + .../assignor/NetworkIngressAssignor.java | 153 +++++++++--- .../assignor/NetworkIngressAssignorTest.java | 233 +++++++++--------- 3 files changed, 249 insertions(+), 142 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index ba00587637..e83355e351 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -45,7 +45,9 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurable { public static final String JMX_PORT = "jmx.port"; public static final String MAX_WAIT_BEAN = "max.wait.bean"; + public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; int maxWaitBean = 3; + long maxTrafficMiBInterval = 10; Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -157,6 +159,9 @@ public final void configure(Map configs) { .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); + config + .integer(MAX_TRAFFIC_MIB_INTERVAL) + .ifPresent(value -> this.maxTrafficMiBInterval = (long) value); var costFunctions = Utils.costFunctions(config, HasPartitionCost.class); var customJMXPort = PartitionerUtils.parseIdJMXPort(config); var defaultJMXPort = config.integer(JMX_PORT); diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 1c353c3c9a..2fa6fdd57b 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -19,17 +19,26 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.astraea.common.DataRate; import org.astraea.common.Utils; +import org.astraea.common.admin.BrokerTopic; +import org.astraea.common.admin.ClusterBean; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; +import org.astraea.common.cost.NoSufficientMetricsException; +import org.astraea.common.metrics.HasBeanObject; +import org.astraea.common.metrics.broker.HasRate; +import org.astraea.common.metrics.broker.ServerMetrics; public class NetworkIngressAssignor extends Assignor { @@ -51,31 +60,9 @@ protected Map> assign( costFunction .partitionCost(ClusterInfo.masked(clusterInfo, subscribedTopics::contains), clusterBean) .value(); - - // key = broker id, value = partition and its cost - var tpCostPerBroker = - clusterInfo - .replicaStream() - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .filter(replica -> subscribedTopics.contains(replica.topic())) - .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) - .entrySet() - .stream() - .map( - e -> - Map.entry( - e.getKey(), - e.getValue().stream() - .map( - replica -> - Map.entry( - replica.topicPartition(), - networkCost.get(replica.topicPartition()))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - return greedyAssign(tpCostPerBroker, consumers); + var costPerBroker = costPerBroker(clusterInfo, subscribedTopics, networkCost); + var intervalPerBroker = convertTrafficToCost(clusterInfo, clusterBean, costPerBroker); + return greedyAssign(costPerBroker, consumers, intervalPerBroker); } /** @@ -99,7 +86,9 @@ private void registerUnregisterNode(ClusterInfo clusterInfo) { * @return the assignment */ Map> greedyAssign( - Map> costs, Set consumers) { + Map> costs, + Set consumers, + Map limitedPerBroker) { // initial var assignment = new HashMap>(); for (var consumer : consumers) { @@ -110,19 +99,19 @@ Map> greedyAssign( .map(c -> Map.entry(c, (double) 0)) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); costs - .values() + .entrySet() .forEach( costPerBroker -> { - if (costPerBroker.values().stream().mapToDouble(x -> x).sum() == 0) { + if (costPerBroker.getValue().values().stream().mapToDouble(x -> x).sum() == 0) { // if there are no cost, round-robin assign per node var iter = consumers.iterator(); - for (var tp : costPerBroker.keySet()) { + for (var tp : costPerBroker.getValue().keySet()) { assignment.get(iter.next()).add(tp); if (!iter.hasNext()) iter = consumers.iterator(); } } else { var sortedCost = new LinkedHashMap(); - costPerBroker.entrySet().stream() + costPerBroker.getValue().entrySet().stream() .sorted(Map.Entry.comparingByValue()) .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); var tmpCostPerConsumer = new HashMap<>(costPerConsumer); @@ -136,8 +125,8 @@ Map> greedyAssign( for (var e : sortedCost.entrySet()) { var tp = e.getKey(); var cost = e.getValue(); - // TODO: threshold need to be set an appropriate value - if (cost - lastValue > 0.05) { + + if (cost - lastValue > limitedPerBroker.get(costPerBroker.getKey())) { tmpCostPerConsumer.remove(consumer); consumer = largestCostConsumer.get(); } @@ -151,6 +140,106 @@ Map> greedyAssign( return assignment; } + Map> costPerBroker( + ClusterInfo clusterInfo, Set topics, Map cost) { + return clusterInfo + .replicaStream() + .filter(Replica::isLeader) + .filter(Replica::isOnline) + .filter(replica -> topics.contains(replica.topic())) + .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) + .entrySet() + .stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .map( + replica -> + Map.entry( + replica.topicPartition(), cost.get(replica.topicPartition()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + // visible for test + /** + * For all nodes, calculate the cost of given traffic. The assignor would use the cost to produce + * the assignment. + * + * @param clusterInfo the clusterInfo + * @param clusterBean the clusterBean + * @param tpCostPerBroker the partition cost of every broker + * @return the Map from broker id to the cost of given traffic + */ + Map convertTrafficToCost( + ClusterInfo clusterInfo, + ClusterBean clusterBean, + Map> tpCostPerBroker) { + var interval = DataRate.MiB.of(maxTrafficMiBInterval).perSecond().byteRate(); + var partitionsTraffic = + replicaLeaderLocation(clusterInfo).entrySet().stream() + .flatMap( + e -> { + var bt = e.getKey(); + var totalReplicaSize = e.getValue().stream().mapToLong(Replica::size).sum(); + var totalShare = + (double) + clusterBean + .brokerTopicMetrics(bt, ServerMetrics.Topic.Meter.class) + .filter( + bean -> bean.type().equals(ServerMetrics.Topic.BYTES_IN_PER_SEC)) + .max(Comparator.comparingLong(HasBeanObject::createdTimestamp)) + .map(HasRate::fifteenMinuteRate) + .orElse(0.0); + + if (Double.isNaN(totalShare) || totalShare < 0.0 || totalReplicaSize < 0) { + throw new NoSufficientMetricsException( + costFunction, + Duration.ofSeconds(1), + "no enough metric to calculate traffic"); + } + var calculateShare = + (Function) + (replica) -> + totalReplicaSize > 0 + ? (long) ((totalShare * replica.size()) / totalReplicaSize) + : 0L; + return e.getValue().stream() + .map(r -> Map.entry(r.topicPartition(), calculateShare.apply(r))); + }) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + + return tpCostPerBroker.entrySet().stream() + .map( + e -> { + var tpCost = + e.getValue().entrySet().stream() + .filter(entry -> entry.getValue() > 0.0) + .findFirst() + .get(); + var traffic = partitionsTraffic.get(tpCost.getKey()); + var normalizedCost = tpCost.getValue(); + + var result = normalizedCost / (traffic / interval); + return Map.entry(e.getKey(), result); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private Map> replicaLeaderLocation(ClusterInfo clusterInfo) { + return clusterInfo + .replicaStream() + .filter(Replica::isLeader) + .filter(Replica::isOnline) + .map( + replica -> Map.entry(BrokerTopic.of(replica.nodeInfo().id(), replica.topic()), replica)) + .collect( + Collectors.groupingBy( + Map.Entry::getKey, + Collectors.mapping(Map.Entry::getValue, Collectors.toUnmodifiableList()))); + } + @Override public String name() { return "networkIngress"; diff --git a/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java index d820faebd0..a3316a7e49 100644 --- a/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java @@ -16,15 +16,18 @@ */ package org.astraea.common.assignor; -import java.util.LinkedHashMap; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; -import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.astraea.common.DataRate; import org.astraea.common.admin.ClusterBean; +import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoBuilder; import org.astraea.common.admin.Replica; +import org.astraea.common.admin.TopicPartition; import org.astraea.common.cost.NetworkIngressCost; import org.astraea.common.metrics.BeanObject; import org.astraea.common.metrics.broker.ServerMetrics; @@ -32,49 +35,40 @@ import org.junit.jupiter.api.Test; public class NetworkIngressAssignorTest { + @Test - void testGreedyAssign() { + void testConvertTrafficToCost() { + var assignor = new NetworkIngressAssignor(); + var cost = new NetworkIngressCost(); + var aFactorList = new ArrayList(); + IntStream.range(0, 3) + .forEach( + i -> { + var rand = new Random(); + aFactorList.add(i, rand.nextDouble()); + }); var clusterInfo = ClusterInfoBuilder.builder() - .addNode(Set.of(1, 2, 3)) - .addFolders( - Map.of( - 1, - Set.of("/folder0", "/folder1"), - 2, - Set.of("/folder0", "/folder1"), - 3, - Set.of("/folder0", "/folder1"))) + .addNode(Set.of(1)) + .addFolders(Map.of(1, Set.of("/folder0", "/folder1"))) .addTopic( "a", - 9, + 3, (short) 1, replica -> { - var factor = getFactor(replica.partition()); + var factor = aFactorList.get(replica.partition()); return Replica.builder(replica) - .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .size((long) (factor * DataRate.MiB.of(100).perSecond().byteRate())) .build(); }) .addTopic( "b", - 9, - (short) 1, - replica -> { - var factor = getFactor(replica.partition()); - return Replica.builder(replica) - .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) - .build(); - }) - .addTopic( - "c", - 9, + 1, (short) 1, - replica -> { - var factor = getFactor(replica.partition()); - return Replica.builder(replica) - .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) - .build(); - }) + replica -> + Replica.builder(replica) + .size((long) DataRate.MiB.of(10).perSecond().byteRate()) + .build()) .build(); var clusterBean = ClusterBean.of( @@ -84,89 +78,108 @@ void testGreedyAssign() { bandwidth( ServerMetrics.Topic.BYTES_IN_PER_SEC, "a", - DataRate.MB.of(90).perSecond().byteRate()), + DataRate.MiB.of(100).perSecond().byteRate()), bandwidth( ServerMetrics.Topic.BYTES_IN_PER_SEC, "b", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "c", - DataRate.MB.of(90).perSecond().byteRate())), + DataRate.MiB.of(10).perSecond().byteRate())))); + var costPerBroker = + assignor.costPerBroker( + clusterInfo, Set.of("a", "b"), cost.partitionCost(clusterInfo, clusterBean).value()); + var resultOf10MiBCost = assignor.convertTrafficToCost(clusterInfo, clusterBean, costPerBroker); + var _10MiBCost = costPerBroker.get(1).get(TopicPartition.of("b-0")); + Assertions.assertEquals(resultOf10MiBCost.get(1), _10MiBCost); + } + + static ClusterInfo buildClusterInfo() { + return ClusterInfoBuilder.builder() + .addNode(Set.of(1, 2, 3)) + .addFolders( + Map.of( + 1, + Set.of("/folder0", "/folder1"), 2, - List.of( - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "a", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "b", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "c", - DataRate.MB.of(90).perSecond().byteRate())), + Set.of("/folder0", "/folder1"), 3, - List.of( - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "a", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "b", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "c", - DataRate.MB.of(90).perSecond().byteRate())))); - - var networkCost = new NetworkIngressCost(); - var cost = networkCost.partitionCost(clusterInfo, clusterBean).value(); - - var topics = Set.of("a", "b", "c"); - var tpCostPerBroker = - clusterInfo - .replicaStream() - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .filter(replica -> topics.contains(replica.topic())) - .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) - .entrySet() - .stream() - .map( - e -> - Map.entry( - e.getKey(), - e.getValue().stream() - .map( - replica -> - Map.entry( - replica.topicPartition(), - cost.get(replica.topicPartition()))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - var assignor = new NetworkIngressAssignor(); - var assignment = assignor.greedyAssign(tpCostPerBroker, Set.of("haha1", "haha2", "haha3")); - var costPerConsumer = - assignment.entrySet().stream() - .map( - e -> { - var tps = e.getValue(); - var tpCost = - tpCostPerBroker.values().stream() - .flatMap(v -> v.entrySet().stream()) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - return Map.entry(e.getKey(), tps.stream().mapToDouble(tpCost::get).sum()); - }) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Set.of("/folder0", "/folder1"))) + .addTopic( + "a", + 9, + (short) 1, + replica -> { + var factor = getFactor(replica.partition()); + return Replica.builder(replica) + .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .build(); + }) + .addTopic( + "b", + 9, + (short) 1, + replica -> { + var factor = getFactor(replica.partition()); + return Replica.builder(replica) + .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .build(); + }) + .addTopic( + "c", + 9, + (short) 1, + replica -> { + var factor = getFactor(replica.partition()); + return Replica.builder(replica) + .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) + .build(); + }) + .build(); + } - var sortedCostPerConsumer = new LinkedHashMap(); - costPerConsumer.entrySet().stream() - .sorted(Map.Entry.comparingByValue()) - .forEach(e -> sortedCostPerConsumer.put(e.getKey(), e.getValue())); - sortedCostPerConsumer.forEach((ignore, c) -> Assertions.assertTrue(c > 0.7 && c < 1.3)); + static ClusterBean buildClusterBean() { + return ClusterBean.of( + Map.of( + 1, + List.of( + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "a", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "b", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "c", + DataRate.MB.of(90).perSecond().byteRate())), + 2, + List.of( + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "a", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "b", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "c", + DataRate.MB.of(90).perSecond().byteRate())), + 3, + List.of( + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "a", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "b", + DataRate.MB.of(90).perSecond().byteRate()), + bandwidth( + ServerMetrics.Topic.BYTES_IN_PER_SEC, + "c", + DataRate.MB.of(90).perSecond().byteRate())))); } static ServerMetrics.Topic.Meter bandwidth( @@ -178,7 +191,7 @@ static ServerMetrics.Topic.Meter bandwidth( return new ServerMetrics.Topic.Meter(new BeanObject(domainName, properties, attributes)); } - private int getFactor(int partition) { + private static int getFactor(int partition) { if (partition > 2 && partition < 6) return 3; else if (partition >= 6) return 5; return 1; From 25f84c85a04f1de18623954f35645571fbc224a5 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 6 Mar 2023 20:51:48 +0800 Subject: [PATCH 11/50] tweak and add comment --- .../main/java/org/astraea/common/assignor/Assignor.java | 6 ++---- .../astraea/common/assignor/NetworkIngressAssignor.java | 8 ++++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index e83355e351..b338ed2fdb 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -47,7 +47,7 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurabl public static final String MAX_WAIT_BEAN = "max.wait.bean"; public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; int maxWaitBean = 3; - long maxTrafficMiBInterval = 10; + int maxTrafficMiBInterval = 10; Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -159,9 +159,7 @@ public final void configure(Map configs) { .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); - config - .integer(MAX_TRAFFIC_MIB_INTERVAL) - .ifPresent(value -> this.maxTrafficMiBInterval = (long) value); + config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); var costFunctions = Utils.costFunctions(config, HasPartitionCost.class); var customJMXPort = PartitionerUtils.parseIdJMXPort(config); var defaultJMXPort = config.integer(JMX_PORT); diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java index 2fa6fdd57b..3c66d834a6 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java @@ -103,6 +103,7 @@ Map> greedyAssign( .forEach( costPerBroker -> { if (costPerBroker.getValue().values().stream().mapToDouble(x -> x).sum() == 0) { + // TODO: use logLeaderSize cost to assign when there is no network ingress cost // if there are no cost, round-robin assign per node var iter = consumers.iterator(); for (var tp : costPerBroker.getValue().keySet()) { @@ -110,11 +111,14 @@ Map> greedyAssign( if (!iter.hasNext()) iter = consumers.iterator(); } } else { + // let networkIngress cost be ascending order var sortedCost = new LinkedHashMap(); costPerBroker.getValue().entrySet().stream() .sorted(Map.Entry.comparingByValue()) .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); + // maintain the temp loading of the consumer var tmpCostPerConsumer = new HashMap<>(costPerConsumer); + // get the consumer with the largest load Supplier largestCostConsumer = () -> Collections.max(tmpCostPerConsumer.entrySet(), Map.Entry.comparingByValue()) @@ -129,11 +133,11 @@ Map> greedyAssign( if (cost - lastValue > limitedPerBroker.get(costPerBroker.getKey())) { tmpCostPerConsumer.remove(consumer); consumer = largestCostConsumer.get(); + lastValue = cost; } assignment.get(consumer).add(tp); costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); - lastValue = cost; } } }); @@ -217,7 +221,7 @@ Map convertTrafficToCost( e.getValue().entrySet().stream() .filter(entry -> entry.getValue() > 0.0) .findFirst() - .get(); + .orElseThrow(); var traffic = partitionsTraffic.get(tpCost.getKey()); var normalizedCost = tpCost.getValue(); From 29c569c493935d00067e0a210003542a7106c597 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 7 Mar 2023 13:36:37 +0800 Subject: [PATCH 12/50] rename and add comment --- ...Assignor.java => SimilarCostAssignor.java} | 22 ++++++++++++++++++- ...Test.java => SimilarCostAssignorTest.java} | 4 ++-- 2 files changed, 23 insertions(+), 3 deletions(-) rename common/src/main/java/org/astraea/common/assignor/{NetworkIngressAssignor.java => SimilarCostAssignor.java} (88%) rename common/src/test/java/org/astraea/common/assignor/{NetworkIngressAssignorTest.java => SimilarCostAssignorTest.java} (98%) diff --git a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java b/common/src/main/java/org/astraea/common/assignor/SimilarCostAssignor.java similarity index 88% rename from common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java rename to common/src/main/java/org/astraea/common/assignor/SimilarCostAssignor.java index 3c66d834a6..1d504e7164 100644 --- a/common/src/main/java/org/astraea/common/assignor/NetworkIngressAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/SimilarCostAssignor.java @@ -40,7 +40,27 @@ import org.astraea.common.metrics.broker.HasRate; import org.astraea.common.metrics.broker.ServerMetrics; -public class NetworkIngressAssignor extends Assignor { +/** + * This assignor scores the partitions by cost function(s) that user given. Each cost function + * evaluate the partitions' cost in each node by metrics depend on which cost function user use. The + * default cost function ranks partitions that are in the same node by NetworkIngressCost{@link + * org.astraea.common.cost.NetworkIngressCost} + * + *

When get the partitions' cost of each node, assignor would assign partitions to consumers base + * on node. Each consumer would get the partitions with "the similar cost" from same node. + * + *

The important configs are JMX port, MAX_WAIT_BEAN, MAX_TRAFFIC_MiB_INTERVAL. Most cost + * function need the JMX metrics to score partitions. Normally, all brokers use the same JMX port, + * so you could just define the `jmx.port=12345`. If one of brokers uses different JMX client port, + * you can define `broker.1001.jmx.port=3456` (`1001` is the broker id) to replace the value of + * `jmx.port`. If the jmx port is undefined, only local mbean client is created for each cost + * function. + * + *

MAX_WAIT_BEAN is the config of setting the amount of time waiting for fetch ClusterBean. + * MAX_TRAFFIC_MiB_INTERVAL is the config of setting how traffic similar is. You can define these + * config by `max.wait.bean=10` or `max.traffic.mib.interval=15` + */ +public class SimilarCostAssignor extends Assignor { @Override protected Map> assign( diff --git a/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/SimilarCostAssignorTest.java similarity index 98% rename from common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java rename to common/src/test/java/org/astraea/common/assignor/SimilarCostAssignorTest.java index a3316a7e49..d20d3be7c1 100644 --- a/common/src/test/java/org/astraea/common/assignor/NetworkIngressAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/SimilarCostAssignorTest.java @@ -34,11 +34,11 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -public class NetworkIngressAssignorTest { +public class SimilarCostAssignorTest { @Test void testConvertTrafficToCost() { - var assignor = new NetworkIngressAssignor(); + var assignor = new SimilarCostAssignor(); var cost = new NetworkIngressCost(); var aFactorList = new ArrayList(); IntStream.range(0, 3) From e92a050744fbe14529f1b7d180a79a656d68a50b Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Wed, 8 Mar 2023 16:02:49 +0800 Subject: [PATCH 13/50] rename and add more comment --- .../org/astraea/common/assignor/Assignor.java | 16 --- ...stAssignor.java => CostAwareAssignor.java} | 133 ++++++++++-------- ...orTest.java => CostAwareAssignorTest.java} | 9 +- 3 files changed, 78 insertions(+), 80 deletions(-) rename common/src/main/java/org/astraea/common/assignor/{SimilarCostAssignor.java => CostAwareAssignor.java} (72%) rename common/src/test/java/org/astraea/common/assignor/{SimilarCostAssignorTest.java => CostAwareAssignorTest.java} (96%) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index b338ed2fdb..575538f32f 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -18,11 +18,9 @@ import java.net.InetSocketAddress; import java.time.Duration; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; @@ -74,20 +72,6 @@ protected void configure(Configuration config) {} // -----------------------[helper]-----------------------// - /** - * get all topics which consumers subscribe. - * - * @param subscriptions Consumers' subscriptions - * @return A set of topics the consumers subscribe - */ - protected Set topics( - Map subscriptions) { - return subscriptions.values().stream() - .map(org.astraea.common.assignor.Subscription::topics) - .flatMap(Collection::stream) - .collect(Collectors.toUnmodifiableSet()); - } - /** * check the nodes which wasn't register yet. * diff --git a/common/src/main/java/org/astraea/common/assignor/SimilarCostAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java similarity index 72% rename from common/src/main/java/org/astraea/common/assignor/SimilarCostAssignor.java rename to common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 1d504e7164..43b068c522 100644 --- a/common/src/main/java/org/astraea/common/assignor/SimilarCostAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -18,6 +18,7 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -26,7 +27,6 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.astraea.common.DataRate; import org.astraea.common.Utils; @@ -60,29 +60,36 @@ * MAX_TRAFFIC_MiB_INTERVAL is the config of setting how traffic similar is. You can define these * config by `max.wait.bean=10` or `max.traffic.mib.interval=15` */ -public class SimilarCostAssignor extends Assignor { +public class CostAwareAssignor extends Assignor { @Override protected Map> assign( Map subscriptions, ClusterInfo clusterInfo) { - var consumers = subscriptions.keySet(); - var subscribedTopics = topics(subscriptions); // 1. check unregister node. if there are unregister nodes, register them registerUnregisterNode(clusterInfo); + var subscribedTopics = + subscriptions.values().stream() + .map(org.astraea.common.assignor.Subscription::topics) + .flatMap(Collection::stream) + .collect(Collectors.toUnmodifiableSet()); + // wait for clusterBean Utils.waitFor( - () -> !metricCollector.clusterBean().all().isEmpty(), Duration.ofSeconds(maxWaitBean)); + () -> + !metricCollector.clusterBean().all().isEmpty() + && metricCollector.clusterBean().topics().containsAll(subscribedTopics), + Duration.ofSeconds(maxWaitBean)); var clusterBean = metricCollector.clusterBean(); - // 2. get the network cost of all subscribed topic - var networkCost = + // 2. get the partition cost of all subscribed topic + var partitionCost = costFunction .partitionCost(ClusterInfo.masked(clusterInfo, subscribedTopics::contains), clusterBean) .value(); - var costPerBroker = costPerBroker(clusterInfo, subscribedTopics, networkCost); - var intervalPerBroker = convertTrafficToCost(clusterInfo, clusterBean, costPerBroker); - return greedyAssign(costPerBroker, consumers, intervalPerBroker); + var costPerBroker = wrapCostBaseOnNode(clusterInfo, subscribedTopics, partitionCost); + var intervalPerBroker = estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); + return greedyAssign(costPerBroker, subscriptions, intervalPerBroker); } /** @@ -97,20 +104,20 @@ private void registerUnregisterNode(ClusterInfo clusterInfo) { } /** - * perform assign algorithm to get balanced assignment and ensure that 1. each consumer would - * receive the cost that are as close as possible to each other. 2. similar loads within a node + * perform assign algorithm to get balanced assignment and ensure that similar loads within a node * would be assigned to the same consumer. * * @param costs the tp and their cost within a node - * @param consumers consumers' name + * @param subscription All subscription for consumers * @return the assignment */ Map> greedyAssign( Map> costs, - Set consumers, + Map subscription, Map limitedPerBroker) { // initial var assignment = new HashMap>(); + var consumers = subscription.keySet(); for (var consumer : consumers) { assignment.put(consumer, new ArrayList<>()); } @@ -118,53 +125,50 @@ Map> greedyAssign( assignment.keySet().stream() .map(c -> Map.entry(c, (double) 0)) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - costs - .entrySet() - .forEach( - costPerBroker -> { - if (costPerBroker.getValue().values().stream().mapToDouble(x -> x).sum() == 0) { - // TODO: use logLeaderSize cost to assign when there is no network ingress cost - // if there are no cost, round-robin assign per node - var iter = consumers.iterator(); - for (var tp : costPerBroker.getValue().keySet()) { - assignment.get(iter.next()).add(tp); - if (!iter.hasNext()) iter = consumers.iterator(); - } - } else { - // let networkIngress cost be ascending order - var sortedCost = new LinkedHashMap(); - costPerBroker.getValue().entrySet().stream() - .sorted(Map.Entry.comparingByValue()) - .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); - // maintain the temp loading of the consumer - var tmpCostPerConsumer = new HashMap<>(costPerConsumer); - // get the consumer with the largest load - Supplier largestCostConsumer = - () -> - Collections.max(tmpCostPerConsumer.entrySet(), Map.Entry.comparingByValue()) - .getKey(); - var consumer = largestCostConsumer.get(); - var lastValue = Collections.min(sortedCost.values()); + Function, String> largestLoadConsumer = + (consumerCost) -> + Collections.max(consumerCost.entrySet(), Map.Entry.comparingByValue()).getKey(); - for (var e : sortedCost.entrySet()) { - var tp = e.getKey(); - var cost = e.getValue(); + costs.forEach( + (id, tpsCost) -> { + // let networkIngress cost be ascending order + var sortedCost = new LinkedHashMap(); + tpsCost.entrySet().stream() + .sorted(Map.Entry.comparingByValue()) + .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); + // maintain the temp cost of the consumer + var tmpCostPerConsumer = new HashMap<>(costPerConsumer); + // get the consumer with the largest load + var consumer = largestLoadConsumer.apply(tmpCostPerConsumer); + var lastValue = Collections.min(sortedCost.values()); - if (cost - lastValue > limitedPerBroker.get(costPerBroker.getKey())) { - tmpCostPerConsumer.remove(consumer); - consumer = largestCostConsumer.get(); - lastValue = cost; - } + for (var e : sortedCost.entrySet()) { + var tp = e.getKey(); + var cost = e.getValue(); - assignment.get(consumer).add(tp); - costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); - } - } - }); + if (cost - lastValue > limitedPerBroker.get(id)) { + tmpCostPerConsumer.remove(consumer); + consumer = largestLoadConsumer.apply(tmpCostPerConsumer); + lastValue = cost; + } + + assignment.get(consumer).add(tp); + costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); + } + }); return assignment; } - Map> costPerBroker( + /** + * Wrap the partition and cost based on nodes. This method is used to process special cost, e.g., + * `LogSizeCost` and `NetworkIngressCost` + * + * @param clusterInfo the cluster information that admin fetch + * @param topics total topics that consumers subscribed + * @param cost partition cost calculated by cost function + * @return Map from each broker id to partitions' cost + */ + Map> wrapCostBaseOnNode( ClusterInfo clusterInfo, Set topics, Map cost) { return clusterInfo .replicaStream() @@ -186,21 +190,23 @@ Map> costPerBroker( .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } + // visible for test /** - * For all nodes, calculate the cost of given traffic. The assignor would use the cost to produce - * the assignment. + * For all nodes, estimate the cost of given traffic. The assignor would use the interval cost to + * assign the partition with similar cost to the same consumer. * * @param clusterInfo the clusterInfo * @param clusterBean the clusterBean * @param tpCostPerBroker the partition cost of every broker - * @return the Map from broker id to the cost of given traffic + * @return Map from broker id to the cost of given traffic */ - Map convertTrafficToCost( + Map estimateIntervalTraffic( ClusterInfo clusterInfo, ClusterBean clusterBean, Map> tpCostPerBroker) { var interval = DataRate.MiB.of(maxTrafficMiBInterval).perSecond().byteRate(); + // get partitions' cost var partitionsTraffic = replicaLeaderLocation(clusterInfo).entrySet().stream() .flatMap( @@ -237,6 +243,7 @@ Map convertTrafficToCost( return tpCostPerBroker.entrySet().stream() .map( e -> { + // select a partition with its network ingress cost var tpCost = e.getValue().entrySet().stream() .filter(entry -> entry.getValue() > 0.0) @@ -244,13 +251,19 @@ Map convertTrafficToCost( .orElseThrow(); var traffic = partitionsTraffic.get(tpCost.getKey()); var normalizedCost = tpCost.getValue(); - + // convert the interval value to cost var result = normalizedCost / (traffic / interval); return Map.entry(e.getKey(), result); }) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } + /** + * the helper method that estimate the interval traffic + * + * @param clusterInfo cluster info + * @return Map from BrokerTopic to Replica + */ private Map> replicaLeaderLocation(ClusterInfo clusterInfo) { return clusterInfo .replicaStream() diff --git a/common/src/test/java/org/astraea/common/assignor/SimilarCostAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java similarity index 96% rename from common/src/test/java/org/astraea/common/assignor/SimilarCostAssignorTest.java rename to common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index d20d3be7c1..5bfe09ae3c 100644 --- a/common/src/test/java/org/astraea/common/assignor/SimilarCostAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -34,11 +34,11 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -public class SimilarCostAssignorTest { +public class CostAwareAssignorTest { @Test void testConvertTrafficToCost() { - var assignor = new SimilarCostAssignor(); + var assignor = new CostAwareAssignor(); var cost = new NetworkIngressCost(); var aFactorList = new ArrayList(); IntStream.range(0, 3) @@ -84,9 +84,10 @@ void testConvertTrafficToCost() { "b", DataRate.MiB.of(10).perSecond().byteRate())))); var costPerBroker = - assignor.costPerBroker( + assignor.wrapCostBaseOnNode( clusterInfo, Set.of("a", "b"), cost.partitionCost(clusterInfo, clusterBean).value()); - var resultOf10MiBCost = assignor.convertTrafficToCost(clusterInfo, clusterBean, costPerBroker); + var resultOf10MiBCost = + assignor.estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); var _10MiBCost = costPerBroker.get(1).get(TopicPartition.of("b-0")); Assertions.assertEquals(resultOf10MiBCost.get(1), _10MiBCost); } From 9d85e8e835175db2f0f0f7a75c6d06fd7e24fb45 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Wed, 8 Mar 2023 17:30:55 +0800 Subject: [PATCH 14/50] Move Kafka configuration to ours --- .../org/astraea/common/assignor/Assignor.java | 37 ++++++++++--------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 575538f32f..0425e1446d 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -68,7 +68,25 @@ protected abstract Map> assign( * * @param config configuration */ - protected void configure(Configuration config) {} + protected void configure(Configuration config) { + config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); + config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); + config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); + var costFunctions = Utils.costFunctions(config, HasPartitionCost.class); + var customJMXPort = PartitionerUtils.parseIdJMXPort(config); + var defaultJMXPort = config.integer(JMX_PORT); + this.costFunction = + costFunctions.isEmpty() + ? HasPartitionCost.of(Map.of(new NetworkIngressCost(), 1D)) + : HasPartitionCost.of(costFunctions); + this.jmxPortGetter = id -> Optional.ofNullable(customJMXPort.get(id)).or(() -> defaultJMXPort); + metricCollector = + MetricCollector.local() + .interval(Duration.ofSeconds(1)) + .expiration(Duration.ofSeconds(15)) + .addMetricSensors(this.costFunction.metricSensor().stream().collect(Collectors.toSet())) + .build(); + } // -----------------------[helper]-----------------------// @@ -141,23 +159,6 @@ public final void configure(Map configs) { Configuration.of( configs.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); - config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); - config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); - config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); - var costFunctions = Utils.costFunctions(config, HasPartitionCost.class); - var customJMXPort = PartitionerUtils.parseIdJMXPort(config); - var defaultJMXPort = config.integer(JMX_PORT); - this.costFunction = - costFunctions.isEmpty() - ? HasPartitionCost.of(Map.of(new NetworkIngressCost(), 1D)) - : HasPartitionCost.of(costFunctions); - this.jmxPortGetter = id -> Optional.ofNullable(customJMXPort.get(id)).or(() -> defaultJMXPort); - metricCollector = - MetricCollector.local() - .interval(Duration.ofSeconds(1)) - .expiration(Duration.ofSeconds(15)) - .addMetricSensors(this.costFunction.metricSensor().stream().collect(Collectors.toSet())) - .build(); configure(config); } } From c2a8293e31c89e66861f6cb9e1f608eeae6aa6f2 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Wed, 8 Mar 2023 18:30:09 +0800 Subject: [PATCH 15/50] Add test --- .../assignor/CostAwareAssignorTest.java | 40 ++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 5bfe09ae3c..be596e7a0f 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -17,6 +17,7 @@ package org.astraea.common.assignor; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -37,7 +38,7 @@ public class CostAwareAssignorTest { @Test - void testConvertTrafficToCost() { + void testEstimateIntervalTraffic() { var assignor = new CostAwareAssignor(); var cost = new NetworkIngressCost(); var aFactorList = new ArrayList(); @@ -92,6 +93,43 @@ void testConvertTrafficToCost() { Assertions.assertEquals(resultOf10MiBCost.get(1), _10MiBCost); } + @Test + void testWrapCostBaseOnNode() { + var assignor = new CostAwareAssignor(); + var clusterInfo = buildClusterInfo(); + var subTopics = Set.of("a", "b"); + var topics = Set.of("a", "b", "c"); + var cost = new HashMap(); + var rand = new Random(); + IntStream.range(0, 9) + .forEach( + i -> { + cost.put(TopicPartition.of("a", i), rand.nextDouble()); + cost.put(TopicPartition.of("b", i), rand.nextDouble()); + cost.put(TopicPartition.of("c", i), rand.nextDouble()); + }); + + var brokerTp = assignor.wrapCostBaseOnNode(clusterInfo, topics, cost); + brokerTp.forEach((id, tps) -> Assertions.assertEquals(9, tps.size())); + clusterInfo + .replicaStream() + .forEach( + r -> { + var tps = brokerTp.get(r.nodeInfo().id()); + Assertions.assertTrue(tps.containsKey(r.topicPartition())); + }); + + var brokerSubTp = assignor.wrapCostBaseOnNode(clusterInfo, subTopics, cost); + brokerSubTp.forEach((id, tps) -> Assertions.assertEquals(6, tps.size())); + ClusterInfo.masked(clusterInfo, subTopics::contains) + .replicaStream() + .forEach( + r -> { + var tps = brokerSubTp.get(r.nodeInfo().id()); + Assertions.assertTrue(tps.containsKey(r.topicPartition())); + }); + } + static ClusterInfo buildClusterInfo() { return ClusterInfoBuilder.builder() .addNode(Set.of(1, 2, 3)) From 5f389f162681341f6a08509f97c041eb608f5db8 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Wed, 8 Mar 2023 19:33:50 +0800 Subject: [PATCH 16/50] Add ClusterInfo masked --- .../org/astraea/common/admin/ClusterInfo.java | 16 +++++- .../astraea/common/admin/ClusterInfoTest.java | 56 +++++++++++++++++++ .../assignor/CostAwareAssignorTest.java | 7 ++- 3 files changed, 77 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java index b7b16d637d..abe1c54a55 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java @@ -25,6 +25,7 @@ import java.util.Optional; import java.util.Set; import java.util.function.BinaryOperator; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -36,7 +37,20 @@ static ClusterInfo empty() { } // ---------------------[helpers]---------------------// - + /** Mask specific topics from a {@link ClusterInfo}. */ + static ClusterInfo masked(ClusterInfo clusterInfo, Predicate topicFilter) { + final var nodes = List.copyOf(clusterInfo.nodes()); + final var topics = + clusterInfo.topics().entrySet().stream() + .filter(e -> topicFilter.test(e.getKey())) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + final var replicas = + clusterInfo + .replicaStream() + .filter(replica -> topicFilter.test(replica.topic())) + .collect(Collectors.toUnmodifiableList()); + return of(clusterInfo.clusterId(), nodes, topics, replicas); + } /** * Find a subset of topic/partitions in the source allocation, that has any non-fulfilled log * placement in the given target allocation. Note that the given two allocations must have the diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java index 38429fc88e..5ec3c9866b 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java @@ -58,6 +58,62 @@ public static ClusterInfo of(List replicas) { replicas); } + @Test + void testReplicaLeadersAndMaskedCluster() { + var replicas = + List.of( + Replica.builder() + .topic("test-1") + .partition(0) + .nodeInfo(NodeInfo.of(0, "", -1)) + .lag(-1) + .size(-1) + .isLeader(true) + .isSync(true) + .isFuture(false) + .isOffline(false) + .isPreferredLeader(false) + .path("/data-folder-01") + .build(), + Replica.builder() + .topic("test-1") + .partition(1) + .nodeInfo(NodeInfo.of(1, "", -1)) + .lag(-1) + .size(-1) + .isLeader(false) + .isSync(true) + .isFuture(false) + .isOffline(false) + .isPreferredLeader(false) + .path("/data-folder-02") + .build(), + Replica.builder() + .topic("test-1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "", -1)) + .lag(-1) + .size(-1) + .isLeader(false) + .isSync(true) + .isFuture(false) + .isOffline(false) + .isPreferredLeader(false) + .path("/data-folder-01") + .build()); + + var clusterInfo = ClusterInfoTest.of(replicas); + var maskedClusterInfoHasReplicas = ClusterInfo.masked(clusterInfo, t -> t.equals("test-1")); + var maskedClusterInfoNoReplicas = + ClusterInfo.masked(clusterInfo, t -> t.equals("No topic name the same.")); + + Assertions.assertNotEquals(0, maskedClusterInfoHasReplicas.nodes().size()); + Assertions.assertNotEquals(0, maskedClusterInfoHasReplicas.replicas().size()); + Assertions.assertEquals(0, maskedClusterInfoNoReplicas.replicas().size()); + + Assertions.assertNotEquals(0, clusterInfo.replicaLeaders(BrokerTopic.of(0, "test-1")).size()); + } + @Test void testEmptyCluster() { var emptyCluster = ClusterInfo.empty(); diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index be596e7a0f..d8e1c12328 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -16,6 +16,7 @@ */ package org.astraea.common.assignor; +import java.text.DecimalFormat; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -90,7 +91,11 @@ void testEstimateIntervalTraffic() { var resultOf10MiBCost = assignor.estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); var _10MiBCost = costPerBroker.get(1).get(TopicPartition.of("b-0")); - Assertions.assertEquals(resultOf10MiBCost.get(1), _10MiBCost); + var format = new DecimalFormat("#.####"); + // Assertions.assertEquals(_10MiBCost, resultOf10MiBCost.get(1)); + Assertions.assertEquals( + Double.parseDouble(format.format(_10MiBCost)), + Double.parseDouble(format.format(resultOf10MiBCost.get(1)))); } @Test From b1ae99a7990809b4f9a98639eddbd624bf86d405 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Fri, 10 Mar 2023 13:53:54 +0800 Subject: [PATCH 17/50] remove masked --- .../org/astraea/common/admin/ClusterInfo.java | 15 ----- .../org/astraea/common/assignor/Assignor.java | 3 + .../common/assignor/CostAwareAssignor.java | 5 +- .../astraea/common/admin/ClusterInfoTest.java | 56 ------------------- .../assignor/CostAwareAssignorTest.java | 11 ---- 5 files changed, 4 insertions(+), 86 deletions(-) diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java index abe1c54a55..827ac07eb2 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java @@ -25,7 +25,6 @@ import java.util.Optional; import java.util.Set; import java.util.function.BinaryOperator; -import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -37,20 +36,6 @@ static ClusterInfo empty() { } // ---------------------[helpers]---------------------// - /** Mask specific topics from a {@link ClusterInfo}. */ - static ClusterInfo masked(ClusterInfo clusterInfo, Predicate topicFilter) { - final var nodes = List.copyOf(clusterInfo.nodes()); - final var topics = - clusterInfo.topics().entrySet().stream() - .filter(e -> topicFilter.test(e.getKey())) - .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); - final var replicas = - clusterInfo - .replicaStream() - .filter(replica -> topicFilter.test(replica.topic())) - .collect(Collectors.toUnmodifiableList()); - return of(clusterInfo.clusterId(), nodes, topics, replicas); - } /** * Find a subset of topic/partitions in the source allocation, that has any non-fulfilled log * placement in the given target allocation. Note that the given two allocations must have the diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index d2f790d56a..75d9d416f6 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -45,8 +45,10 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurabl public static final String JMX_PORT = "jmx.port"; public static final String MAX_WAIT_BEAN = "max.wait.bean"; public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; + public static final String MAX_UPPER_BOUND_MIB = "max.upper.bound.mib"; int maxWaitBean = 3; int maxTrafficMiBInterval = 10; + int maxUpperBoundMiB = 40; Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -73,6 +75,7 @@ protected void configure(Configuration config) { config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); + config.integer(MAX_UPPER_BOUND_MIB).ifPresent(value -> this.maxUpperBoundMiB = value); var costFunctions = Utils.costFunctions(config.filteredPrefixConfigs(COST_PREFIX), HasPartitionCost.class); var customJMXPort = PartitionerUtils.parseIdJMXPort(config); diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 43b068c522..21d7fa1d08 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -83,10 +83,7 @@ protected Map> assign( var clusterBean = metricCollector.clusterBean(); // 2. get the partition cost of all subscribed topic - var partitionCost = - costFunction - .partitionCost(ClusterInfo.masked(clusterInfo, subscribedTopics::contains), clusterBean) - .value(); + var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); var costPerBroker = wrapCostBaseOnNode(clusterInfo, subscribedTopics, partitionCost); var intervalPerBroker = estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); return greedyAssign(costPerBroker, subscriptions, intervalPerBroker); diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java index 5ec3c9866b..38429fc88e 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java @@ -58,62 +58,6 @@ public static ClusterInfo of(List replicas) { replicas); } - @Test - void testReplicaLeadersAndMaskedCluster() { - var replicas = - List.of( - Replica.builder() - .topic("test-1") - .partition(0) - .nodeInfo(NodeInfo.of(0, "", -1)) - .lag(-1) - .size(-1) - .isLeader(true) - .isSync(true) - .isFuture(false) - .isOffline(false) - .isPreferredLeader(false) - .path("/data-folder-01") - .build(), - Replica.builder() - .topic("test-1") - .partition(1) - .nodeInfo(NodeInfo.of(1, "", -1)) - .lag(-1) - .size(-1) - .isLeader(false) - .isSync(true) - .isFuture(false) - .isOffline(false) - .isPreferredLeader(false) - .path("/data-folder-02") - .build(), - Replica.builder() - .topic("test-1") - .partition(2) - .nodeInfo(NodeInfo.of(0, "", -1)) - .lag(-1) - .size(-1) - .isLeader(false) - .isSync(true) - .isFuture(false) - .isOffline(false) - .isPreferredLeader(false) - .path("/data-folder-01") - .build()); - - var clusterInfo = ClusterInfoTest.of(replicas); - var maskedClusterInfoHasReplicas = ClusterInfo.masked(clusterInfo, t -> t.equals("test-1")); - var maskedClusterInfoNoReplicas = - ClusterInfo.masked(clusterInfo, t -> t.equals("No topic name the same.")); - - Assertions.assertNotEquals(0, maskedClusterInfoHasReplicas.nodes().size()); - Assertions.assertNotEquals(0, maskedClusterInfoHasReplicas.replicas().size()); - Assertions.assertEquals(0, maskedClusterInfoNoReplicas.replicas().size()); - - Assertions.assertNotEquals(0, clusterInfo.replicaLeaders(BrokerTopic.of(0, "test-1")).size()); - } - @Test void testEmptyCluster() { var emptyCluster = ClusterInfo.empty(); diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index d8e1c12328..0198cb27a5 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -102,7 +102,6 @@ void testEstimateIntervalTraffic() { void testWrapCostBaseOnNode() { var assignor = new CostAwareAssignor(); var clusterInfo = buildClusterInfo(); - var subTopics = Set.of("a", "b"); var topics = Set.of("a", "b", "c"); var cost = new HashMap(); var rand = new Random(); @@ -123,16 +122,6 @@ void testWrapCostBaseOnNode() { var tps = brokerTp.get(r.nodeInfo().id()); Assertions.assertTrue(tps.containsKey(r.topicPartition())); }); - - var brokerSubTp = assignor.wrapCostBaseOnNode(clusterInfo, subTopics, cost); - brokerSubTp.forEach((id, tps) -> Assertions.assertEquals(6, tps.size())); - ClusterInfo.masked(clusterInfo, subTopics::contains) - .replicaStream() - .forEach( - r -> { - var tps = brokerSubTp.get(r.nodeInfo().id()); - Assertions.assertTrue(tps.containsKey(r.topicPartition())); - }); } static ClusterInfo buildClusterInfo() { From 1e0b6bd814fd618d7442e72dc156746f0c168165 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 11 Mar 2023 19:47:14 +0800 Subject: [PATCH 18/50] add new assign methods --- .../org/astraea/common/assignor/Assignor.java | 6 +- .../common/assignor/CostAwareAssignor.java | 127 +++++++++++++++++- .../assignor/CostAwareAssignorTest.java | 112 +++++++++++++++ 3 files changed, 240 insertions(+), 5 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 75d9d416f6..5c0205203e 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -47,8 +47,8 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurabl public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; public static final String MAX_UPPER_BOUND_MIB = "max.upper.bound.mib"; int maxWaitBean = 3; - int maxTrafficMiBInterval = 10; - int maxUpperBoundMiB = 40; + double maxTrafficMiBInterval = 10; + double maxUpperBoundMiB = 40; Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -76,6 +76,8 @@ protected void configure(Configuration config) { config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); config.integer(MAX_UPPER_BOUND_MIB).ifPresent(value -> this.maxUpperBoundMiB = value); + if (maxUpperBoundMiB < maxTrafficMiBInterval) + throw new IllegalArgumentException("max traffic interval cannot larger than max upperbound"); var costFunctions = Utils.costFunctions(config.filteredPrefixConfigs(COST_PREFIX), HasPartitionCost.class); var customJMXPort = PartitionerUtils.parseIdJMXPort(config); diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 21d7fa1d08..b2498c7c80 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -27,7 +27,10 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.astraea.common.DataRate; import org.astraea.common.Utils; import org.astraea.common.admin.BrokerTopic; @@ -66,7 +69,6 @@ public class CostAwareAssignor extends Assignor { protected Map> assign( Map subscriptions, ClusterInfo clusterInfo) { - // 1. check unregister node. if there are unregister nodes, register them registerUnregisterNode(clusterInfo); var subscribedTopics = subscriptions.values().stream() @@ -82,7 +84,6 @@ protected Map> assign( Duration.ofSeconds(maxWaitBean)); var clusterBean = metricCollector.clusterBean(); - // 2. get the partition cost of all subscribed topic var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); var costPerBroker = wrapCostBaseOnNode(clusterInfo, subscribedTopics, partitionCost); var intervalPerBroker = estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); @@ -156,6 +157,126 @@ Map> greedyAssign( return assignment; } + Map> nodeAssignment( + Map partitionCost, + Map consumerCost, + Double interval) { + // TODO: avoid numberOfConsumer < intervalAssignment.size() + var intervalAssignment = groupPartitionWithInterval(partitionCost, interval); + var groupNumberOfNonInterval = consumerCost.size() - intervalAssignment.size(); + Map> result; + + if (groupNumberOfNonInterval == 0) { + var upperBound = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); + var dontCareSimilarCost = + partitionCost.entrySet().stream() + .filter(e -> e.getValue() >= upperBound) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + intervalAssignment.get(upperBound).putAll(dontCareSimilarCost); + result = intervalAssignment; + } else { + var dontCareSimilar = + groupPartitionWithoutInterval(partitionCost, interval, groupNumberOfNonInterval); + result = + Stream.concat(intervalAssignment.entrySet().stream(), dontCareSimilar.entrySet().stream()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + var assignOrder = + result.entrySet().stream() + .map( + e -> { + var id = e.getKey(); + var total = e.getValue().values().stream().mapToDouble(x -> x).sum(); + return Map.entry(id, total); + }) + .sorted(Map.Entry.comparingByValue()) + .map(Map.Entry::getKey) + .collect(Collectors.toList()); + Collections.reverse(assignOrder); + + var tmpConsumerCost = new HashMap<>(consumerCost); + Supplier lowestCostConsumer = + () -> Collections.min(tmpConsumerCost.entrySet(), Map.Entry.comparingByValue()).getKey(); + + return assignOrder.stream() + .map( + id -> { + var consumer = lowestCostConsumer.get(); + + tmpConsumerCost.remove(consumer); + consumerCost.compute( + consumer, + (ignore, cost) -> + cost + result.get(id).values().stream().mapToDouble(x -> x).sum()); + return Map.entry( + consumer, + result.get(id).keySet().stream().collect(Collectors.toUnmodifiableList())); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + protected Map> groupPartitionWithoutInterval( + Map partitionCost, Double interval, int groupNumber) { + var upperBound = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); + var dontCareSimilarCost = + partitionCost.entrySet().stream() + .filter(e -> e.getValue() >= upperBound) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + if (groupNumber == 1) return Map.of(1.0, new HashMap<>(dontCareSimilarCost)); + + var result = + IntStream.range(0, groupNumber) + .mapToObj(i -> Map.entry((double) i, new HashMap())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var tmpCost = result.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> 0.0)); + Supplier minCost = + () -> Collections.min(tmpCost.entrySet(), Map.Entry.comparingByValue()).getKey(); + dontCareSimilarCost.forEach( + (tp, cost) -> { + var min = minCost.get(); + result.get(min).put(tp, cost); + tmpCost.computeIfPresent(min, (ignore, costValue) -> costValue + cost); + }); + return result.entrySet().stream() + .filter(e -> !e.getValue().isEmpty()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + protected Map> groupPartitionWithInterval( + Map partitionCost, Double interval) { + // upper = 50, interval = 10 + // 0~10, 10~20, 20~30, 30~40, 40~50 + // upper = 35, interval = 10 + // 0~10, 10~20, 20~30, 30~35 + var upperBoundCost = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); + var groupNumbers = (int) Math.ceil(maxUpperBoundMiB / maxTrafficMiBInterval); + var intervals = + IntStream.range(1, groupNumbers + 1) + .mapToObj( + i -> + Map.entry( + Math.min(interval * i, upperBoundCost), + new HashMap())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var orderedList = intervals.keySet().stream().sorted().collect(Collectors.toUnmodifiableList()); + // Aggregate similar traffic to the same key of intervals. + // If cost is larger than upperBound cost, put it into dontCareSimilarCost + partitionCost.entrySet().stream() + .filter(e -> e.getValue() < upperBoundCost) + .forEach( + e -> { + for (var i : orderedList) { + if (e.getValue() < i) { + intervals.get(i).put(e.getKey(), e.getValue()); + break; + } + } + }); + return intervals.entrySet().stream() + .filter(e -> !e.getValue().isEmpty()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } /** * Wrap the partition and cost based on nodes. This method is used to process special cost, e.g., * `LogSizeCost` and `NetworkIngressCost` @@ -202,7 +323,7 @@ Map estimateIntervalTraffic( ClusterInfo clusterInfo, ClusterBean clusterBean, Map> tpCostPerBroker) { - var interval = DataRate.MiB.of(maxTrafficMiBInterval).perSecond().byteRate(); + var interval = DataRate.MiB.of((long) maxTrafficMiBInterval).perSecond().byteRate(); // get partitions' cost var partitionsTraffic = replicaLeaderLocation(clusterInfo).entrySet().stream() diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 0198cb27a5..07db05b879 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -23,8 +23,11 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.DataRate; +import org.astraea.common.Utils; import org.astraea.common.admin.ClusterBean; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoBuilder; @@ -124,6 +127,115 @@ void testWrapCostBaseOnNode() { }); } + @Test + void testGroupPartitionWithInterval() { + var assignor = new CostAwareAssignor(); + var assignorModifyInterval = new CostAwareAssignor(); + assignorModifyInterval.configure(Map.of("max.traffic.mib.interval", 12)); + var assignorModifyUpperBound = new CostAwareAssignor(); + assignorModifyUpperBound.configure(Map.of("max.upper.bound.mib", 25)); + var testPartitionCost = partitionCost(600); + var interval = ThreadLocalRandom.current().nextDouble(0.15); + var result = assignor.groupPartitionWithInterval(testPartitionCost, interval); + var resultWithModifyInterval = + assignorModifyInterval.groupPartitionWithInterval(testPartitionCost, interval); + var resultWithModifyUpperBound = + assignorModifyUpperBound.groupPartitionWithInterval(testPartitionCost, interval); + + Assertions.assertEquals( + (int) Math.ceil(assignor.maxUpperBoundMiB / assignor.maxTrafficMiBInterval), result.size()); + Assertions.assertEquals( + (int) + Math.ceil( + assignorModifyInterval.maxUpperBoundMiB + / assignorModifyInterval.maxTrafficMiBInterval), + resultWithModifyInterval.size()); + Assertions.assertEquals( + (int) + Math.ceil( + assignorModifyUpperBound.maxUpperBoundMiB + / assignorModifyUpperBound.maxTrafficMiBInterval), + resultWithModifyUpperBound.size()); + + var list = result.keySet().stream().sorted().collect(Collectors.toUnmodifiableList()); + var listWithModifyInterval = + resultWithModifyInterval.keySet().stream() + .sorted() + .collect(Collectors.toUnmodifiableList()); + var listWithModifyUpperBound = + resultWithModifyUpperBound.keySet().stream() + .sorted() + .collect(Collectors.toUnmodifiableList()); + + for (Double bound : list) { + Assertions.assertTrue( + result.get(bound).values().stream() + .allMatch(cost -> cost < bound && cost > bound - interval)); + } + for (Double bound : listWithModifyInterval) { + Assertions.assertTrue( + resultWithModifyInterval.get(bound).values().stream() + .allMatch(cost -> cost < bound && cost > bound - interval)); + } + for (Double bound : listWithModifyUpperBound) { + Assertions.assertTrue( + resultWithModifyUpperBound.get(bound).values().stream() + .allMatch(cost -> cost < bound && cost > bound - interval)); + } + } + + @Test + void testGroupPartitionWithoutInterval() { + var assignor = new CostAwareAssignor(); + var interval = ThreadLocalRandom.current().nextDouble(0.005); + var upperBound = (assignor.maxUpperBoundMiB / assignor.maxTrafficMiBInterval) * interval; + var testPartitionCost = partitionCost(600); + System.out.println(testPartitionCost); + var singleConsumer = assignor.groupPartitionWithoutInterval(testPartitionCost, upperBound, 1); + var twoConsumers = assignor.groupPartitionWithoutInterval(testPartitionCost, upperBound, 2); + Assertions.assertEquals(1, singleConsumer.size()); + singleConsumer.forEach( + (ignore, costs) -> { + var value = costs.values(); + Assertions.assertTrue(value.stream().allMatch(v -> v >= upperBound)); + }); + Assertions.assertEquals(2, twoConsumers.size()); + twoConsumers.forEach( + (ignore, costs) -> { + var value = costs.values(); + Assertions.assertTrue(value.stream().allMatch(v -> v >= upperBound)); + }); + } + + @Test + void testNodeAssignment() { + var assignor = new CostAwareAssignor(); + var interval = ThreadLocalRandom.current().nextDouble(0.001); + var partitionCost = + IntStream.range(0, 2) + .mapToObj(i -> Map.entry(i, partitionCost(ThreadLocalRandom.current().nextInt(250)))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var consumerCost = + IntStream.range(0, 5) + .mapToObj( + i -> Map.entry(Utils.randomString(5), ThreadLocalRandom.current().nextDouble(0.1))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + partitionCost.forEach( + (groupId, subAssignment) -> { + assignor.nodeAssignment(subAssignment, consumerCost, interval); + }); + } + + static Map partitionCost(int number) { + return IntStream.range(0, number) + .mapToObj( + i -> + Map.entry( + TopicPartition.of(Utils.randomString(4), ThreadLocalRandom.current().nextInt()), + ThreadLocalRandom.current().nextDouble(0.3))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + static ClusterInfo buildClusterInfo() { return ClusterInfoBuilder.builder() .addNode(Set.of(1, 2, 3)) From f1ec8824ef959ae6cace493da8a9a1a83d2d3b4a Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 11 Mar 2023 21:21:22 +0800 Subject: [PATCH 19/50] add comment and modify greedyAssign --- .../common/assignor/CostAwareAssignor.java | 79 ++++++++++--------- .../assignor/CostAwareAssignorTest.java | 10 +-- 2 files changed, 45 insertions(+), 44 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index b2498c7c80..6510e741da 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -22,7 +22,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -102,17 +101,19 @@ private void registerUnregisterNode(ClusterInfo clusterInfo) { } /** - * perform assign algorithm to get balanced assignment and ensure that similar loads within a node - * would be assigned to the same consumer. + * perform assign algorithm ensure that similar loads within a node would be assigned to the same + * consumer. * * @param costs the tp and their cost within a node * @param subscription All subscription for consumers - * @return the assignment + * @param intervalPerBroker Transforming the traffic of each node into cost + * @return the final assignment */ Map> greedyAssign( Map> costs, Map subscription, - Map limitedPerBroker) { + Map intervalPerBroker) { + // TODO: need detect consumer with different subscription // initial var assignment = new HashMap>(); var consumers = subscription.keySet(); @@ -123,41 +124,23 @@ Map> greedyAssign( assignment.keySet().stream() .map(c -> Map.entry(c, (double) 0)) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - Function, String> largestLoadConsumer = - (consumerCost) -> - Collections.max(consumerCost.entrySet(), Map.Entry.comparingByValue()).getKey(); costs.forEach( - (id, tpsCost) -> { - // let networkIngress cost be ascending order - var sortedCost = new LinkedHashMap(); - tpsCost.entrySet().stream() - .sorted(Map.Entry.comparingByValue()) - .forEach(entry -> sortedCost.put(entry.getKey(), entry.getValue())); - // maintain the temp cost of the consumer - var tmpCostPerConsumer = new HashMap<>(costPerConsumer); - // get the consumer with the largest load - var consumer = largestLoadConsumer.apply(tmpCostPerConsumer); - var lastValue = Collections.min(sortedCost.values()); - - for (var e : sortedCost.entrySet()) { - var tp = e.getKey(); - var cost = e.getValue(); - - if (cost - lastValue > limitedPerBroker.get(id)) { - tmpCostPerConsumer.remove(consumer); - consumer = largestLoadConsumer.apply(tmpCostPerConsumer); - lastValue = cost; - } - - assignment.get(consumer).add(tp); - costPerConsumer.computeIfPresent(consumer, (ignore, c) -> c + cost); - } - }); + (brokerId, cost) -> + assignPerNode(cost, costPerConsumer, intervalPerBroker.get(brokerId)) + .forEach((consumer, result) -> assignment.get(consumer).addAll(result))); return assignment; } - Map> nodeAssignment( + /** + * aggregate the assignment of with interval and without interval + * + * @param partitionCost partition cost + * @param consumerCost the consumer with its total cost + * @param interval the config of `max.traffic.mib.interval` + * @return the assignment of a node + */ + Map> assignPerNode( Map partitionCost, Map consumerCost, Double interval) { @@ -216,6 +199,16 @@ Map> nodeAssignment( .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } + /** + * Assign the partitions which cost is larger than `max.upper.bound.mib`. For costs that exceed + * the upper bound, they should be evenly distributed as much as possible. + * + * @param partitionCost partition cost based on the cost function user used + * @param interval the config of `max.traffic.mib.interval` + * @param groupNumber the group number must equal to the (number of consumers - number of group + * within interval) + * @return the part of assignment + */ protected Map> groupPartitionWithoutInterval( Map partitionCost, Double interval, int groupNumber) { var upperBound = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); @@ -226,7 +219,7 @@ protected Map> groupPartitionWithoutInte if (groupNumber == 1) return Map.of(1.0, new HashMap<>(dontCareSimilarCost)); var result = - IntStream.range(0, groupNumber) + IntStream.range(1, groupNumber + 1) .mapToObj(i -> Map.entry((double) i, new HashMap())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); var tmpCost = result.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> 0.0)); @@ -243,12 +236,22 @@ protected Map> groupPartitionWithoutInte .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } + /** + * Assign the partitions which cost is less than `max.upper.bound.mib`. The reason for needing to + * look at the upper bound is to assign partitions with similar amounts of traffic to the same + * consumer. + * + * @param partitionCost the partition cost + * @param interval the config of `max.traffic.mib.interval`, Distinguishing the interval of + * traffic + * @return the part of assignment + */ protected Map> groupPartitionWithInterval( Map partitionCost, Double interval) { // upper = 50, interval = 10 - // 0~10, 10~20, 20~30, 30~40, 40~50 + // range: 0~10, 10~20, 20~30, 30~40, 40~50 // upper = 35, interval = 10 - // 0~10, 10~20, 20~30, 30~35 + // range: 0~10, 10~20, 20~30, 30~35 var upperBoundCost = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); var groupNumbers = (int) Math.ceil(maxUpperBoundMiB / maxTrafficMiBInterval); var intervals = diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 07db05b879..698c072a3c 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -95,7 +95,7 @@ void testEstimateIntervalTraffic() { assignor.estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); var _10MiBCost = costPerBroker.get(1).get(TopicPartition.of("b-0")); var format = new DecimalFormat("#.####"); - // Assertions.assertEquals(_10MiBCost, resultOf10MiBCost.get(1)); + Assertions.assertEquals( Double.parseDouble(format.format(_10MiBCost)), Double.parseDouble(format.format(resultOf10MiBCost.get(1)))); @@ -190,9 +190,9 @@ void testGroupPartitionWithoutInterval() { var interval = ThreadLocalRandom.current().nextDouble(0.005); var upperBound = (assignor.maxUpperBoundMiB / assignor.maxTrafficMiBInterval) * interval; var testPartitionCost = partitionCost(600); - System.out.println(testPartitionCost); var singleConsumer = assignor.groupPartitionWithoutInterval(testPartitionCost, upperBound, 1); var twoConsumers = assignor.groupPartitionWithoutInterval(testPartitionCost, upperBound, 2); + Assertions.assertEquals(1, singleConsumer.size()); singleConsumer.forEach( (ignore, costs) -> { @@ -208,7 +208,7 @@ void testGroupPartitionWithoutInterval() { } @Test - void testNodeAssignment() { + void testAssignPerNode() { var assignor = new CostAwareAssignor(); var interval = ThreadLocalRandom.current().nextDouble(0.001); var partitionCost = @@ -221,9 +221,7 @@ void testNodeAssignment() { i -> Map.entry(Utils.randomString(5), ThreadLocalRandom.current().nextDouble(0.1))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); partitionCost.forEach( - (groupId, subAssignment) -> { - assignor.nodeAssignment(subAssignment, consumerCost, interval); - }); + (groupId, subAssignment) -> assignor.assignPerNode(subAssignment, consumerCost, interval)); } static Map partitionCost(int number) { From 84b6ed4b32376f760eaa5292d630f64a3fa4a940 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 11 Mar 2023 21:24:20 +0800 Subject: [PATCH 20/50] spotless --- .../src/main/java/org/astraea/common/assignor/Assignor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index c4e6f7e6bc..f54d6aaa12 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -79,8 +79,8 @@ protected void configure(Configuration config) { if (maxUpperBoundMiB < maxTrafficMiBInterval) throw new IllegalArgumentException("max traffic interval cannot larger than max upperbound"); var costFunctions = - Utils.costFunctions( - config.filteredPrefixConfigs(COST_PREFIX).raw(), HasPartitionCost.class, config); + Utils.costFunctions( + config.filteredPrefixConfigs(COST_PREFIX).raw(), HasPartitionCost.class, config); var customJMXPort = PartitionerUtils.parseIdJMXPort(config); var defaultJMXPort = config.integer(JMX_PORT); this.costFunction = From 3e4146d086edefcd6466e0ba26acec3d55040ac7 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 12 Mar 2023 14:44:55 +0800 Subject: [PATCH 21/50] Add throw exception and change field type --- .../src/main/java/org/astraea/common/assignor/Assignor.java | 5 +++-- .../java/org/astraea/common/assignor/CostAwareAssignor.java | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index f54d6aaa12..7de2d1cd29 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -46,7 +46,7 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurabl public static final String MAX_WAIT_BEAN = "max.wait.bean"; public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; public static final String MAX_UPPER_BOUND_MIB = "max.upper.bound.mib"; - int maxWaitBean = 3; + Duration maxWaitBean = Duration.ofSeconds(3); double maxTrafficMiBInterval = 10; double maxUpperBoundMiB = 40; Function> jmxPortGetter = (id) -> Optional.empty(); @@ -73,7 +73,8 @@ protected abstract Map> assign( */ protected void configure(Configuration config) { config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); - config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = value); + if (bootstrap.isEmpty()) throw new NoSuchFieldError("cannot find bootstrap"); + config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = Duration.ofSeconds(value)); config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); config.integer(MAX_UPPER_BOUND_MIB).ifPresent(value -> this.maxUpperBoundMiB = value); if (maxUpperBoundMiB < maxTrafficMiBInterval) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 6510e741da..7f3e5c78b9 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -80,7 +80,7 @@ protected Map> assign( () -> !metricCollector.clusterBean().all().isEmpty() && metricCollector.clusterBean().topics().containsAll(subscribedTopics), - Duration.ofSeconds(maxWaitBean)); + maxWaitBean); var clusterBean = metricCollector.clusterBean(); var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); From e0ccc020a7d7885b7073258687b2997a9ba1ae81 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 12 Mar 2023 14:56:10 +0800 Subject: [PATCH 22/50] Move the fields to sub-class --- .../org/astraea/common/assignor/Assignor.java | 49 ++++++++----------- .../common/assignor/CostAwareAssignor.java | 13 +++++ 2 files changed, 33 insertions(+), 29 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 7de2d1cd29..85a3b581a8 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -44,11 +44,7 @@ public abstract class Assignor implements ConsumerPartitionAssignor, Configurabl public static final String COST_PREFIX = "assignor.cost"; public static final String JMX_PORT = "jmx.port"; public static final String MAX_WAIT_BEAN = "max.wait.bean"; - public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; - public static final String MAX_UPPER_BOUND_MIB = "max.upper.bound.mib"; Duration maxWaitBean = Duration.ofSeconds(3); - double maxTrafficMiBInterval = 10; - double maxUpperBoundMiB = 40; Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -71,31 +67,7 @@ protected abstract Map> assign( * * @param config configuration */ - protected void configure(Configuration config) { - config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); - if (bootstrap.isEmpty()) throw new NoSuchFieldError("cannot find bootstrap"); - config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = Duration.ofSeconds(value)); - config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); - config.integer(MAX_UPPER_BOUND_MIB).ifPresent(value -> this.maxUpperBoundMiB = value); - if (maxUpperBoundMiB < maxTrafficMiBInterval) - throw new IllegalArgumentException("max traffic interval cannot larger than max upperbound"); - var costFunctions = - Utils.costFunctions( - config.filteredPrefixConfigs(COST_PREFIX).raw(), HasPartitionCost.class, config); - var customJMXPort = PartitionerUtils.parseIdJMXPort(config); - var defaultJMXPort = config.integer(JMX_PORT); - this.costFunction = - costFunctions.isEmpty() - ? HasPartitionCost.of(Map.of(new NetworkIngressCost(), 1D)) - : HasPartitionCost.of(costFunctions); - this.jmxPortGetter = id -> Optional.ofNullable(customJMXPort.get(id)).or(() -> defaultJMXPort); - metricCollector = - MetricCollector.local() - .interval(Duration.ofSeconds(1)) - .expiration(Duration.ofSeconds(15)) - .addMetricSensors(this.costFunction.metricSensor().stream().collect(Collectors.toSet())) - .build(); - } + protected void configure(Configuration config) {} // -----------------------[helper]-----------------------// @@ -168,6 +140,25 @@ public final void configure(Map configs) { Configuration.of( configs.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); + config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); + if (bootstrap.isEmpty()) throw new NoSuchFieldError("cannot find bootstrap"); + config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = Duration.ofSeconds(value)); + var costFunctions = + Utils.costFunctions( + config.filteredPrefixConfigs(COST_PREFIX).raw(), HasPartitionCost.class, config); + var customJMXPort = PartitionerUtils.parseIdJMXPort(config); + var defaultJMXPort = config.integer(JMX_PORT); + this.costFunction = + costFunctions.isEmpty() + ? HasPartitionCost.of(Map.of(new NetworkIngressCost(), 1D)) + : HasPartitionCost.of(costFunctions); + this.jmxPortGetter = id -> Optional.ofNullable(customJMXPort.get(id)).or(() -> defaultJMXPort); + metricCollector = + MetricCollector.local() + .interval(Duration.ofSeconds(1)) + .expiration(Duration.ofSeconds(15)) + .addMetricSensors(this.costFunction.metricSensor().stream().collect(Collectors.toSet())) + .build(); configure(config); } } diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 7f3e5c78b9..4f0e7ab113 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -30,6 +30,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import org.astraea.common.Configuration; import org.astraea.common.DataRate; import org.astraea.common.Utils; import org.astraea.common.admin.BrokerTopic; @@ -63,6 +64,10 @@ * config by `max.wait.bean=10` or `max.traffic.mib.interval=15` */ public class CostAwareAssignor extends Assignor { + public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; + public static final String MAX_UPPER_BOUND_MIB = "max.upper.bound.mib"; + double maxTrafficMiBInterval = 10; + double maxUpperBoundMiB = 40; @Override protected Map> assign( @@ -398,6 +403,14 @@ private Map> replicaLeaderLocation(ClusterInfo cluste Collectors.mapping(Map.Entry::getValue, Collectors.toUnmodifiableList()))); } + @Override + protected void configure(Configuration config) { + config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); + config.integer(MAX_UPPER_BOUND_MIB).ifPresent(value -> this.maxUpperBoundMiB = value); + if (maxUpperBoundMiB < maxTrafficMiBInterval) + throw new IllegalArgumentException("max traffic interval cannot larger than max upperbound"); + } + @Override public String name() { return "networkIngress"; From 93162671a9d39e82d488389ff5f30e6aefb1bb6f Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Wed, 15 Mar 2023 17:54:08 +0800 Subject: [PATCH 23/50] modify retry machanism --- .../common/assignor/CostAwareAssignor.java | 23 ++++++++++++++----- 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 4f0e7ab113..c04842ade7 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -81,13 +81,9 @@ protected Map> assign( .collect(Collectors.toUnmodifiableSet()); // wait for clusterBean - Utils.waitFor( - () -> - !metricCollector.clusterBean().all().isEmpty() - && metricCollector.clusterBean().topics().containsAll(subscribedTopics), - maxWaitBean); - var clusterBean = metricCollector.clusterBean(); + retry(clusterInfo); + var clusterBean = metricCollector.clusterBean(); var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); var costPerBroker = wrapCostBaseOnNode(clusterInfo, subscribedTopics, partitionCost); var intervalPerBroker = estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); @@ -403,6 +399,21 @@ private Map> replicaLeaderLocation(ClusterInfo cluste Collectors.mapping(Map.Entry::getValue, Collectors.toUnmodifiableList()))); } + private void retry(ClusterInfo clusterInfo) { + var timeoutMs = System.currentTimeMillis() + maxWaitBean.toMillis(); + while (System.currentTimeMillis() < timeoutMs) { + try { + var clusterBean = metricCollector.clusterBean(); + var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean); + if (partitionCost.value().values().stream().noneMatch(v -> Double.isNaN(v))) return; + } catch (NoSufficientMetricsException e) { + e.printStackTrace(); + Utils.sleep(Duration.ofSeconds(1)); + } + } + throw new RuntimeException("Failed to fetch clusterBean due to timeout"); + } + @Override protected void configure(Configuration config) { config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); From 574abf30ebd321a2403e5b09a26450e7f4e38026 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Wed, 15 Mar 2023 19:03:36 +0800 Subject: [PATCH 24/50] Fix coding style --- .../org/astraea/common/assignor/Assignor.java | 15 +++++++++------ .../common/assignor/CostAwareAssignor.java | 19 +++++-------------- 2 files changed, 14 insertions(+), 20 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 85a3b581a8..1d637b1130 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -41,10 +41,10 @@ /** Abstract assignor implementation which does some common work (e.g., configuration). */ public abstract class Assignor implements ConsumerPartitionAssignor, Configurable { - public static final String COST_PREFIX = "assignor.cost"; - public static final String JMX_PORT = "jmx.port"; - public static final String MAX_WAIT_BEAN = "max.wait.bean"; - Duration maxWaitBean = Duration.ofSeconds(3); + private static final String COST_PREFIX = "assignor.cost"; + private static final String JMX_PORT = "jmx.port"; + private static final String MAX_WAIT_BEAN = "max.wait.bean"; + protected Duration maxWaitBean = Duration.ofSeconds(3); Function> jmxPortGetter = (id) -> Optional.empty(); private String bootstrap; HasPartitionCost costFunction = HasPartitionCost.EMPTY; @@ -140,8 +140,11 @@ public final void configure(Map configs) { Configuration.of( configs.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); - config.string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> bootstrap = s); - if (bootstrap.isEmpty()) throw new NoSuchFieldError("cannot find bootstrap"); + bootstrap = + config + .string(ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG) + .filter(s -> !s.isEmpty()) + .orElseThrow(() -> new NoSuchFieldError("cannot find bootstrap")); config.integer(MAX_WAIT_BEAN).ifPresent(value -> this.maxWaitBean = Duration.ofSeconds(value)); var costFunctions = Utils.costFunctions( diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index c04842ade7..0f1062d754 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -297,20 +297,11 @@ Map> wrapCostBaseOnNode( .filter(Replica::isLeader) .filter(Replica::isOnline) .filter(replica -> topics.contains(replica.topic())) - .collect(Collectors.groupingBy(replica -> replica.nodeInfo().id())) - .entrySet() - .stream() - .map( - e -> - Map.entry( - e.getKey(), - e.getValue().stream() - .map( - replica -> - Map.entry( - replica.topicPartition(), cost.get(replica.topicPartition()))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .collect( + Collectors.groupingBy( + replica -> replica.nodeInfo().id(), + Collectors.toUnmodifiableMap( + Replica::topicPartition, r -> cost.get(r.topicPartition())))); } // visible for test From c2d73b5f9cec57ef2143791beb0fc1a217958354 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 15 Apr 2023 21:24:48 +0800 Subject: [PATCH 25/50] Reference feedback to assign partitions --- .../common/assignor/CostAwareAssignor.java | 385 +++--------------- .../assignor/CostAwareAssignorTest.java | 325 +-------------- 2 files changed, 68 insertions(+), 642 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 22af80be87..33a52aaf21 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -20,29 +20,17 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.Configuration; -import org.astraea.common.DataRate; import org.astraea.common.Utils; -import org.astraea.common.admin.BrokerTopic; -import org.astraea.common.admin.ClusterBean; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.cost.NoSufficientMetricsException; -import org.astraea.common.metrics.HasBeanObject; -import org.astraea.common.metrics.MBeanClient; -import org.astraea.common.metrics.broker.HasRate; -import org.astraea.common.metrics.broker.ServerMetrics; /** * This assignor scores the partitions by cost function(s) that user given. Each cost function @@ -65,16 +53,14 @@ * config by `max.wait.bean=10` or `max.traffic.mib.interval=15` */ public class CostAwareAssignor extends Assignor { - public static final String MAX_TRAFFIC_MIB_INTERVAL = "max.traffic.mib.interval"; - public static final String MAX_UPPER_BOUND_MIB = "max.upper.bound.mib"; - double maxTrafficMiBInterval = 10; - double maxUpperBoundMiB = 40; + protected static final String MAX_RETRY_TIME = "max.retry.time"; + Duration maxRetryTime = Duration.ofSeconds(30); @Override protected Map> assign( Map subscriptions, ClusterInfo clusterInfo) { - registerUnregisterNode(clusterInfo); + // TODO: Detect Unregister node and register them if any var subscribedTopics = subscriptions.values().stream() .map(org.astraea.common.assignor.Subscription::topics) @@ -85,317 +71,83 @@ protected Map> assign( retry(clusterInfo); var clusterBean = metricStore.clusterBean(); - var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean).value(); - var costPerBroker = wrapCostBaseOnNode(clusterInfo, subscribedTopics, partitionCost); - var intervalPerBroker = estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); - return greedyAssign(costPerBroker, subscriptions, intervalPerBroker); - } - - /** - * register unregistered nodes if present. if we didn't register unregistered nodes, we would miss - * the beanObjects from the nodes - * - * @param clusterInfo Currently cluster information. - */ - private void registerUnregisterNode(ClusterInfo clusterInfo) { - var unregister = checkUnregister(clusterInfo.nodes()); - if (!unregister.isEmpty()) { - unregister.forEach((id, host) -> MBeanClient.jndi(host, jmxPortGetter.apply(id))); - } - } - - /** - * perform assign algorithm ensure that similar loads within a node would be assigned to the same - * consumer. - * - * @param costs the tp and their cost within a node - * @param subscription All subscription for consumers - * @param intervalPerBroker Transforming the traffic of each node into cost - * @return the final assignment - */ - Map> greedyAssign( - Map> costs, - Map subscription, - Map intervalPerBroker) { - // TODO: need detect consumer with different subscription - // initial - var assignment = new HashMap>(); - var consumers = subscription.keySet(); - for (var consumer : consumers) { - assignment.put(consumer, new ArrayList<>()); - } - var costPerConsumer = - assignment.keySet().stream() - .map(c -> Map.entry(c, (double) 0)) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean); + var cost = + partitionCost.value().entrySet().stream() + .filter(e -> subscribedTopics.contains(e.getKey().topic())) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + var incompatiblePartition = partitionCost.incompatibility(); - costs.forEach( - (brokerId, cost) -> - assignPerNode(cost, costPerConsumer, intervalPerBroker.get(brokerId)) - .forEach((consumer, result) -> assignment.get(consumer).addAll(result))); - return assignment; + return greedyAssign(subscriptions, cost, incompatiblePartition); } - /** - * aggregate the assignment of with interval and without interval - * - * @param partitionCost partition cost - * @param consumerCost the consumer with its total cost - * @param interval the config of `max.traffic.mib.interval` - * @return the assignment of a node - */ - Map> assignPerNode( - Map partitionCost, - Map consumerCost, - Double interval) { - // TODO: avoid numberOfConsumer < intervalAssignment.size() - var intervalAssignment = groupPartitionWithInterval(partitionCost, interval); - var groupNumberOfNonInterval = consumerCost.size() - intervalAssignment.size(); - Map> result; - - if (groupNumberOfNonInterval == 0) { - var upperBound = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); - var dontCareSimilarCost = - partitionCost.entrySet().stream() - .filter(e -> e.getValue() >= upperBound) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - intervalAssignment.get(upperBound).putAll(dontCareSimilarCost); - result = intervalAssignment; - } else { - var dontCareSimilar = - groupPartitionWithoutInterval(partitionCost, interval, groupNumberOfNonInterval); - result = - Stream.concat(intervalAssignment.entrySet().stream(), dontCareSimilar.entrySet().stream()) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - - var assignOrder = - result.entrySet().stream() - .map( - e -> { - var id = e.getKey(); - var total = e.getValue().values().stream().mapToDouble(x -> x).sum(); - return Map.entry(id, total); - }) - .sorted(Map.Entry.comparingByValue()) - .map(Map.Entry::getKey) - .collect(Collectors.toList()); - Collections.reverse(assignOrder); - - var tmpConsumerCost = new HashMap<>(consumerCost); - Supplier lowestCostConsumer = - () -> Collections.min(tmpConsumerCost.entrySet(), Map.Entry.comparingByValue()).getKey(); - - return assignOrder.stream() - .map( - id -> { - var consumer = lowestCostConsumer.get(); - - tmpConsumerCost.remove(consumer); - consumerCost.compute( - consumer, - (ignore, cost) -> - cost + result.get(id).values().stream().mapToDouble(x -> x).sum()); - return Map.entry( - consumer, - result.get(id).keySet().stream().collect(Collectors.toUnmodifiableList())); - }) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - - /** - * Assign the partitions which cost is larger than `max.upper.bound.mib`. For costs that exceed - * the upper bound, they should be evenly distributed as much as possible. - * - * @param partitionCost partition cost based on the cost function user used - * @param interval the config of `max.traffic.mib.interval` - * @param groupNumber the group number must equal to the (number of consumers - number of group - * within interval) - * @return the part of assignment - */ - protected Map> groupPartitionWithoutInterval( - Map partitionCost, Double interval, int groupNumber) { - var upperBound = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); - var dontCareSimilarCost = - partitionCost.entrySet().stream() - .filter(e -> e.getValue() >= upperBound) + protected Map> greedyAssign( + Map subscriptions, + Map costs, + Map> incompatiblePartition) { + var tmpConsumerCost = + subscriptions.keySet().stream() + .map(c -> Map.entry(c, 0.0)) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - if (groupNumber == 1) return Map.of(1.0, new HashMap<>(dontCareSimilarCost)); - - var result = - IntStream.range(1, groupNumber + 1) - .mapToObj(i -> Map.entry((double) i, new HashMap())) + var tmpAssignment = + subscriptions.keySet().stream() + .map(v -> Map.entry(v, new ArrayList())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - var tmpCost = result.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> 0.0)); - Supplier minCost = - () -> Collections.min(tmpCost.entrySet(), Map.Entry.comparingByValue()).getKey(); - dontCareSimilarCost.forEach( - (tp, cost) -> { - var min = minCost.get(); - result.get(min).put(tp, cost); - tmpCost.computeIfPresent(min, (ignore, costValue) -> costValue + cost); - }); - return result.entrySet().stream() - .filter(e -> !e.getValue().isEmpty()) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - /** - * Assign the partitions which cost is less than `max.upper.bound.mib`. The reason for needing to - * look at the upper bound is to assign partitions with similar amounts of traffic to the same - * consumer. - * - * @param partitionCost the partition cost - * @param interval the config of `max.traffic.mib.interval`, Distinguishing the interval of - * traffic - * @return the part of assignment - */ - protected Map> groupPartitionWithInterval( - Map partitionCost, Double interval) { - // upper = 50, interval = 10 - // range: 0~10, 10~20, 20~30, 30~40, 40~50 - // upper = 35, interval = 10 - // range: 0~10, 10~20, 20~30, 30~35 - var upperBoundCost = interval * (maxUpperBoundMiB / maxTrafficMiBInterval); - var groupNumbers = (int) Math.ceil(maxUpperBoundMiB / maxTrafficMiBInterval); - var intervals = - IntStream.range(1, groupNumbers + 1) - .mapToObj( - i -> - Map.entry( - Math.min(interval * i, upperBoundCost), - new HashMap())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - var orderedList = intervals.keySet().stream().sorted().collect(Collectors.toUnmodifiableList()); - // Aggregate similar traffic to the same key of intervals. - // If cost is larger than upperBound cost, put it into dontCareSimilarCost - partitionCost.entrySet().stream() - .filter(e -> e.getValue() < upperBoundCost) - .forEach( - e -> { - for (var i : orderedList) { - if (e.getValue() < i) { - intervals.get(i).put(e.getKey(), e.getValue()); - break; - } - } - }); - return intervals.entrySet().stream() - .filter(e -> !e.getValue().isEmpty()) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - /** - * Wrap the partition and cost based on nodes. This method is used to process special cost, e.g., - * `LogSizeCost` and `NetworkIngressCost` - * - * @param clusterInfo the cluster information that admin fetch - * @param topics total topics that consumers subscribed - * @param cost partition cost calculated by cost function - * @return Map from each broker id to partitions' cost - */ - Map> wrapCostBaseOnNode( - ClusterInfo clusterInfo, Set topics, Map cost) { - return clusterInfo - .replicaStream() - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .filter(replica -> topics.contains(replica.topic())) - .collect( - Collectors.groupingBy( - replica -> replica.nodeInfo().id(), - Collectors.toUnmodifiableMap( - Replica::topicPartition, r -> cost.get(r.topicPartition())))); - } - - // visible for test - /** - * For all nodes, estimate the cost of given traffic. The assignor would use the interval cost to - * assign the partition with similar cost to the same consumer. - * - * @param clusterInfo the clusterInfo - * @param clusterBean the clusterBean - * @param tpCostPerBroker the partition cost of every broker - * @return Map from broker id to the cost of given traffic - */ - Map estimateIntervalTraffic( - ClusterInfo clusterInfo, - ClusterBean clusterBean, - Map> tpCostPerBroker) { - var interval = DataRate.MiB.of((long) maxTrafficMiBInterval).perSecond().byteRate(); - // get partitions' cost - var partitionsTraffic = - replicaLeaderLocation(clusterInfo).entrySet().stream() - .flatMap( - e -> { - var bt = e.getKey(); - var totalReplicaSize = e.getValue().stream().mapToLong(Replica::size).sum(); - var totalShare = - (double) - clusterBean - .brokerTopicMetrics(bt, ServerMetrics.Topic.Meter.class) + var lowestCostConsumer = + (Function) + (tp) -> { + var subscribeConsumers = + subscriptions.entrySet().stream() + .filter(e -> e.getValue().topics().contains(tp.topic())) + .map(Map.Entry::getKey) + .collect(Collectors.toUnmodifiableSet()); + var suitableConsumers = + incompatiblePartition.isEmpty() + ? subscribeConsumers + : incompatiblePartition.get(tp).isEmpty() + ? subscribeConsumers + : subscribeConsumers.stream() .filter( - bean -> bean.type().equals(ServerMetrics.Topic.BYTES_IN_PER_SEC)) - .max(Comparator.comparingLong(HasBeanObject::createdTimestamp)) - .map(HasRate::fifteenMinuteRate) - .orElse(0.0); - - if (Double.isNaN(totalShare) || totalShare < 0.0 || totalReplicaSize < 0) { - throw new NoSufficientMetricsException( - costFunction, - Duration.ofSeconds(1), - "no enough metric to calculate traffic"); - } - var calculateShare = - (Function) - (replica) -> - totalReplicaSize > 0 - ? (long) ((totalShare * replica.size()) / totalReplicaSize) - : 0L; - return e.getValue().stream() - .map(r -> Map.entry(r.topicPartition(), calculateShare.apply(r))); - }) - .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); - - return tpCostPerBroker.entrySet().stream() - .map( + c -> + tmpAssignment.get(c).stream() + .noneMatch( + p -> incompatiblePartition.get(tp).contains(p))) + .collect(Collectors.toUnmodifiableSet()); + + return suitableConsumers.isEmpty() + ? tmpConsumerCost.entrySet().stream() + .filter(e -> subscribeConsumers.contains(e.getKey())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey() + : tmpConsumerCost.entrySet().stream() + .filter(e -> suitableConsumers.contains(e.getKey())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey(); + }; + + return costs.entrySet().stream() + .flatMap( e -> { - // select a partition with its network ingress cost - var tpCost = - e.getValue().entrySet().stream() - .filter(entry -> entry.getValue() > 0.0) - .findFirst() - .orElseThrow(); - var traffic = partitionsTraffic.get(tpCost.getKey()); - var normalizedCost = tpCost.getValue(); - // convert the interval value to cost - var result = normalizedCost / (traffic / interval); - return Map.entry(e.getKey(), result); + var tp = e.getKey(); + var cost = e.getValue(); + var consumer = lowestCostConsumer.apply(tp); + tmpConsumerCost.compute(consumer, (ignore, totalCost) -> cost + totalCost); + tmpAssignment.get(consumer).add(tp); + return Stream.of(Map.entry(consumer, tp)); }) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - - /** - * the helper method that estimate the interval traffic - * - * @param clusterInfo cluster info - * @return Map from BrokerTopic to Replica - */ - private Map> replicaLeaderLocation(ClusterInfo clusterInfo) { - return clusterInfo - .replicaStream() - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .map( - replica -> Map.entry(BrokerTopic.of(replica.nodeInfo().id(), replica.topic()), replica)) .collect( - Collectors.groupingBy( + Collectors.toMap( Map.Entry::getKey, - Collectors.mapping(Map.Entry::getValue, Collectors.toUnmodifiableList()))); + entry -> Collections.singletonList(entry.getValue()), + (l1, l2) -> + Stream.of(l1, l2).flatMap(Collection::stream).collect(Collectors.toList()))); } private void retry(ClusterInfo clusterInfo) { - // temp - var timeoutMs = System.currentTimeMillis() + 20000; + var timeoutMs = System.currentTimeMillis() + maxRetryTime.toMillis(); while (System.currentTimeMillis() < timeoutMs) { try { var clusterBean = metricStore.clusterBean(); @@ -411,10 +163,7 @@ private void retry(ClusterInfo clusterInfo) { @Override protected void configure(Configuration config) { - config.integer(MAX_TRAFFIC_MIB_INTERVAL).ifPresent(value -> this.maxTrafficMiBInterval = value); - config.integer(MAX_UPPER_BOUND_MIB).ifPresent(value -> this.maxUpperBoundMiB = value); - if (maxUpperBoundMiB < maxTrafficMiBInterval) - throw new IllegalArgumentException("max traffic interval cannot larger than max upperbound"); + config.duration(MAX_RETRY_TIME).ifPresent(v -> this.maxRetryTime = v); } @Override diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 698c072a3c..7b7bb3df45 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -16,327 +16,4 @@ */ package org.astraea.common.assignor; -import java.text.DecimalFormat; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.astraea.common.DataRate; -import org.astraea.common.Utils; -import org.astraea.common.admin.ClusterBean; -import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.ClusterInfoBuilder; -import org.astraea.common.admin.Replica; -import org.astraea.common.admin.TopicPartition; -import org.astraea.common.cost.NetworkIngressCost; -import org.astraea.common.metrics.BeanObject; -import org.astraea.common.metrics.broker.ServerMetrics; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class CostAwareAssignorTest { - - @Test - void testEstimateIntervalTraffic() { - var assignor = new CostAwareAssignor(); - var cost = new NetworkIngressCost(); - var aFactorList = new ArrayList(); - IntStream.range(0, 3) - .forEach( - i -> { - var rand = new Random(); - aFactorList.add(i, rand.nextDouble()); - }); - var clusterInfo = - ClusterInfoBuilder.builder() - .addNode(Set.of(1)) - .addFolders(Map.of(1, Set.of("/folder0", "/folder1"))) - .addTopic( - "a", - 3, - (short) 1, - replica -> { - var factor = aFactorList.get(replica.partition()); - return Replica.builder(replica) - .size((long) (factor * DataRate.MiB.of(100).perSecond().byteRate())) - .build(); - }) - .addTopic( - "b", - 1, - (short) 1, - replica -> - Replica.builder(replica) - .size((long) DataRate.MiB.of(10).perSecond().byteRate()) - .build()) - .build(); - var clusterBean = - ClusterBean.of( - Map.of( - 1, - List.of( - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "a", - DataRate.MiB.of(100).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "b", - DataRate.MiB.of(10).perSecond().byteRate())))); - var costPerBroker = - assignor.wrapCostBaseOnNode( - clusterInfo, Set.of("a", "b"), cost.partitionCost(clusterInfo, clusterBean).value()); - var resultOf10MiBCost = - assignor.estimateIntervalTraffic(clusterInfo, clusterBean, costPerBroker); - var _10MiBCost = costPerBroker.get(1).get(TopicPartition.of("b-0")); - var format = new DecimalFormat("#.####"); - - Assertions.assertEquals( - Double.parseDouble(format.format(_10MiBCost)), - Double.parseDouble(format.format(resultOf10MiBCost.get(1)))); - } - - @Test - void testWrapCostBaseOnNode() { - var assignor = new CostAwareAssignor(); - var clusterInfo = buildClusterInfo(); - var topics = Set.of("a", "b", "c"); - var cost = new HashMap(); - var rand = new Random(); - IntStream.range(0, 9) - .forEach( - i -> { - cost.put(TopicPartition.of("a", i), rand.nextDouble()); - cost.put(TopicPartition.of("b", i), rand.nextDouble()); - cost.put(TopicPartition.of("c", i), rand.nextDouble()); - }); - - var brokerTp = assignor.wrapCostBaseOnNode(clusterInfo, topics, cost); - brokerTp.forEach((id, tps) -> Assertions.assertEquals(9, tps.size())); - clusterInfo - .replicaStream() - .forEach( - r -> { - var tps = brokerTp.get(r.nodeInfo().id()); - Assertions.assertTrue(tps.containsKey(r.topicPartition())); - }); - } - - @Test - void testGroupPartitionWithInterval() { - var assignor = new CostAwareAssignor(); - var assignorModifyInterval = new CostAwareAssignor(); - assignorModifyInterval.configure(Map.of("max.traffic.mib.interval", 12)); - var assignorModifyUpperBound = new CostAwareAssignor(); - assignorModifyUpperBound.configure(Map.of("max.upper.bound.mib", 25)); - var testPartitionCost = partitionCost(600); - var interval = ThreadLocalRandom.current().nextDouble(0.15); - var result = assignor.groupPartitionWithInterval(testPartitionCost, interval); - var resultWithModifyInterval = - assignorModifyInterval.groupPartitionWithInterval(testPartitionCost, interval); - var resultWithModifyUpperBound = - assignorModifyUpperBound.groupPartitionWithInterval(testPartitionCost, interval); - - Assertions.assertEquals( - (int) Math.ceil(assignor.maxUpperBoundMiB / assignor.maxTrafficMiBInterval), result.size()); - Assertions.assertEquals( - (int) - Math.ceil( - assignorModifyInterval.maxUpperBoundMiB - / assignorModifyInterval.maxTrafficMiBInterval), - resultWithModifyInterval.size()); - Assertions.assertEquals( - (int) - Math.ceil( - assignorModifyUpperBound.maxUpperBoundMiB - / assignorModifyUpperBound.maxTrafficMiBInterval), - resultWithModifyUpperBound.size()); - - var list = result.keySet().stream().sorted().collect(Collectors.toUnmodifiableList()); - var listWithModifyInterval = - resultWithModifyInterval.keySet().stream() - .sorted() - .collect(Collectors.toUnmodifiableList()); - var listWithModifyUpperBound = - resultWithModifyUpperBound.keySet().stream() - .sorted() - .collect(Collectors.toUnmodifiableList()); - - for (Double bound : list) { - Assertions.assertTrue( - result.get(bound).values().stream() - .allMatch(cost -> cost < bound && cost > bound - interval)); - } - for (Double bound : listWithModifyInterval) { - Assertions.assertTrue( - resultWithModifyInterval.get(bound).values().stream() - .allMatch(cost -> cost < bound && cost > bound - interval)); - } - for (Double bound : listWithModifyUpperBound) { - Assertions.assertTrue( - resultWithModifyUpperBound.get(bound).values().stream() - .allMatch(cost -> cost < bound && cost > bound - interval)); - } - } - - @Test - void testGroupPartitionWithoutInterval() { - var assignor = new CostAwareAssignor(); - var interval = ThreadLocalRandom.current().nextDouble(0.005); - var upperBound = (assignor.maxUpperBoundMiB / assignor.maxTrafficMiBInterval) * interval; - var testPartitionCost = partitionCost(600); - var singleConsumer = assignor.groupPartitionWithoutInterval(testPartitionCost, upperBound, 1); - var twoConsumers = assignor.groupPartitionWithoutInterval(testPartitionCost, upperBound, 2); - - Assertions.assertEquals(1, singleConsumer.size()); - singleConsumer.forEach( - (ignore, costs) -> { - var value = costs.values(); - Assertions.assertTrue(value.stream().allMatch(v -> v >= upperBound)); - }); - Assertions.assertEquals(2, twoConsumers.size()); - twoConsumers.forEach( - (ignore, costs) -> { - var value = costs.values(); - Assertions.assertTrue(value.stream().allMatch(v -> v >= upperBound)); - }); - } - - @Test - void testAssignPerNode() { - var assignor = new CostAwareAssignor(); - var interval = ThreadLocalRandom.current().nextDouble(0.001); - var partitionCost = - IntStream.range(0, 2) - .mapToObj(i -> Map.entry(i, partitionCost(ThreadLocalRandom.current().nextInt(250)))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - var consumerCost = - IntStream.range(0, 5) - .mapToObj( - i -> Map.entry(Utils.randomString(5), ThreadLocalRandom.current().nextDouble(0.1))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - partitionCost.forEach( - (groupId, subAssignment) -> assignor.assignPerNode(subAssignment, consumerCost, interval)); - } - - static Map partitionCost(int number) { - return IntStream.range(0, number) - .mapToObj( - i -> - Map.entry( - TopicPartition.of(Utils.randomString(4), ThreadLocalRandom.current().nextInt()), - ThreadLocalRandom.current().nextDouble(0.3))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - - static ClusterInfo buildClusterInfo() { - return ClusterInfoBuilder.builder() - .addNode(Set.of(1, 2, 3)) - .addFolders( - Map.of( - 1, - Set.of("/folder0", "/folder1"), - 2, - Set.of("/folder0", "/folder1"), - 3, - Set.of("/folder0", "/folder1"))) - .addTopic( - "a", - 9, - (short) 1, - replica -> { - var factor = getFactor(replica.partition()); - return Replica.builder(replica) - .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) - .build(); - }) - .addTopic( - "b", - 9, - (short) 1, - replica -> { - var factor = getFactor(replica.partition()); - return Replica.builder(replica) - .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) - .build(); - }) - .addTopic( - "c", - 9, - (short) 1, - replica -> { - var factor = getFactor(replica.partition()); - return Replica.builder(replica) - .size((long) (factor * DataRate.MB.of(10).perSecond().byteRate())) - .build(); - }) - .build(); - } - - static ClusterBean buildClusterBean() { - return ClusterBean.of( - Map.of( - 1, - List.of( - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "a", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "b", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "c", - DataRate.MB.of(90).perSecond().byteRate())), - 2, - List.of( - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "a", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "b", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "c", - DataRate.MB.of(90).perSecond().byteRate())), - 3, - List.of( - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "a", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "b", - DataRate.MB.of(90).perSecond().byteRate()), - bandwidth( - ServerMetrics.Topic.BYTES_IN_PER_SEC, - "c", - DataRate.MB.of(90).perSecond().byteRate())))); - } - - static ServerMetrics.Topic.Meter bandwidth( - ServerMetrics.Topic metric, String topic, double fifteenRate) { - var domainName = "kafka.server"; - var properties = - Map.of("type", "BrokerTopicMetric", "topic", topic, "name", metric.metricName()); - var attributes = Map.of("FifteenMinuteRate", fifteenRate); - return new ServerMetrics.Topic.Meter(new BeanObject(domainName, properties, attributes)); - } - - private static int getFactor(int partition) { - if (partition > 2 && partition < 6) return 3; - else if (partition >= 6) return 5; - return 1; - } -} +public class CostAwareAssignorTest {} From 4d4f0b4a12d33ed5104806011a488be465d21afb Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sat, 15 Apr 2023 21:26:04 +0800 Subject: [PATCH 26/50] Add test for greedyAssign --- .../assignor/CostAwareAssignorTest.java | 74 ++++++++++++++++++- 1 file changed, 73 insertions(+), 1 deletion(-) diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 7b7bb3df45..b97595342b 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -16,4 +16,76 @@ */ package org.astraea.common.assignor; -public class CostAwareAssignorTest {} +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.astraea.common.admin.TopicPartition; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class CostAwareAssignorTest { + + @Test + void testGreedyAssign() { + var assignor = new CostAwareAssignor(); + var subscription = + Map.of( + "c1", + new Subscription(List.of("t1", "t2", "t3"), null), + "c2", + new Subscription(List.of("t1", "t2", "t3"), null)); + var cost1 = + Map.of( + TopicPartition.of("t1", 0), + 0.2, + TopicPartition.of("t1", 1), + 0.4, + TopicPartition.of("t2", 0), + 0.2, + TopicPartition.of("t3", 0), + 0.4); + + var allSuitableAssignment = assignor.greedyAssign(subscription, cost1, Map.of()); + Assertions.assertEquals(2, allSuitableAssignment.size()); + allSuitableAssignment.forEach((c, assign) -> Assertions.assertEquals(2, assign.size())); + + var incompatibility = + Map.of( + TopicPartition.of("t1", 0), + Set.of(TopicPartition.of("t3", 0)), + TopicPartition.of("t1", 1), + Set.of(TopicPartition.of("t2", 0)), + TopicPartition.of("t2", 0), + Set.of(TopicPartition.of("t1", 1)), + TopicPartition.of("t3", 0), + Set.of(TopicPartition.of("t1", 0))); + var incompatibleAssignment = assignor.greedyAssign(subscription, cost1, incompatibility); + Assertions.assertEquals(2, incompatibleAssignment.size()); + incompatibleAssignment.forEach( + (consumer, assignment) -> { + assignment.forEach( + tp -> { + incompatibility + .get(tp) + .forEach(itp -> Assertions.assertFalse(assignment.contains(itp))); + }); + }); + + Function create = TopicPartition::of; + var allIncompatible = + Map.of( + create.apply("t1-0"), + Set.of(create.apply("t1-1"), create.apply("t2-0"), create.apply("t3-0")), + create.apply("t1-1"), + Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t3-0")), + create.apply("t2-0"), + Set.of(create.apply("t1-0"), create.apply("t1-1"), create.apply("t3-0")), + create.apply("t3-0"), + Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t1-1"))); + + var allIncompatibleAssignment = assignor.greedyAssign(subscription, cost1, allIncompatible); + allIncompatibleAssignment.forEach( + (c, assignment) -> Assertions.assertEquals(2, assignment.size())); + } +} From 1407b6d1154cfd191b8ab83bbbaf2d4b5ef261d4 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 16 Apr 2023 01:13:10 +0800 Subject: [PATCH 27/50] Pass config into NetworkIngressCost --- common/src/main/java/org/astraea/common/assignor/Assignor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index cb908f1809..ac3b078b7d 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -143,7 +143,7 @@ public final void configure(Map configs) { var defaultJMXPort = config.integer(JMX_PORT); this.costFunction = costFunctions.isEmpty() - ? HasPartitionCost.of(Map.of(new NetworkIngressCost(Configuration.EMPTY), 1D)) + ? HasPartitionCost.of(Map.of(new NetworkIngressCost(config), 1D)) : HasPartitionCost.of(costFunctions); this.jmxPortGetter = id -> From 52553a5e1963662149efeb47f164fdc3b69eea2e Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 16 Apr 2023 10:31:26 +0800 Subject: [PATCH 28/50] Replace flatMap to map --- .../java/org/astraea/common/assignor/CostAwareAssignor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 33a52aaf21..5c62489827 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -129,14 +129,14 @@ protected Map> greedyAssign( }; return costs.entrySet().stream() - .flatMap( + .map( e -> { var tp = e.getKey(); var cost = e.getValue(); var consumer = lowestCostConsumer.apply(tp); tmpConsumerCost.compute(consumer, (ignore, totalCost) -> cost + totalCost); tmpAssignment.get(consumer).add(tp); - return Stream.of(Map.entry(consumer, tp)); + return Map.entry(consumer, tp); }) .collect( Collectors.toMap( From 5f2eb0dfa9f86a2e2ba70b88e916f9f6fd90eea4 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 16 Apr 2023 16:34:30 +0800 Subject: [PATCH 29/50] Revise lambda to avoid creating unnecessary object --- .../org/astraea/common/assignor/CostAwareAssignor.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 5c62489827..f86642f360 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -87,12 +87,11 @@ protected Map> greedyAssign( Map> incompatiblePartition) { var tmpConsumerCost = subscriptions.keySet().stream() - .map(c -> Map.entry(c, 0.0)) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .collect(Collectors.toMap(Function.identity(), ignored -> 0.0D)); var tmpAssignment = subscriptions.keySet().stream() - .map(v -> Map.entry(v, new ArrayList())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .collect( + Collectors.toMap(Function.identity(), ignored -> new ArrayList())); var lowestCostConsumer = (Function) From e594ce814e55355bfd7233610b4b08dbc1133ef2 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 16 Apr 2023 22:05:11 +0800 Subject: [PATCH 30/50] Add comment for greedyAssign --- .../common/assignor/CostAwareAssignor.java | 38 ++++++++++++------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index f86642f360..aa9eb3aeec 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -38,19 +38,11 @@ * default cost function ranks partitions that are in the same node by NetworkIngressCost{@link * org.astraea.common.cost.NetworkIngressCost} * - *

When get the partitions' cost of each node, assignor would assign partitions to consumers base - * on node. Each consumer would get the partitions with "the similar cost" from same node. - * - *

The important configs are JMX port, MAX_WAIT_BEAN, MAX_TRAFFIC_MiB_INTERVAL. Most cost - * function need the JMX metrics to score partitions. Normally, all brokers use the same JMX port, - * so you could just define the `jmx.port=12345`. If one of brokers uses different JMX client port, - * you can define `broker.1001.jmx.port=3456` (`1001` is the broker id) to replace the value of - * `jmx.port`. If the jmx port is undefined, only local mbean client is created for each cost - * function. - * - *

MAX_WAIT_BEAN is the config of setting the amount of time waiting for fetch ClusterBean. - * MAX_TRAFFIC_MiB_INTERVAL is the config of setting how traffic similar is. You can define these - * config by `max.wait.bean=10` or `max.traffic.mib.interval=15` + *

The important configs are JMX port. Most cost function need the JMX metrics to score + * partitions. Normally, all brokers use the same JMX port, so you could just define the + * `jmx.port=12345`. If one of brokers uses different JMX client port, you can define + * `broker.1001.jmx.port=3456` (`1001` is the broker id) to replace the value of `jmx.port`. If the + * jmx port is undefined, only local mbean client is created for each cost function. */ public class CostAwareAssignor extends Assignor { protected static final String MAX_RETRY_TIME = "max.retry.time"; @@ -60,7 +52,6 @@ public class CostAwareAssignor extends Assignor { protected Map> assign( Map subscriptions, ClusterInfo clusterInfo) { - // TODO: Detect Unregister node and register them if any var subscribedTopics = subscriptions.values().stream() .map(org.astraea.common.assignor.Subscription::topics) @@ -81,6 +72,25 @@ protected Map> assign( return greedyAssign(subscriptions, cost, incompatiblePartition); } + /** + * Using the greedy strategy to assign partitions to consumers. The partitions would be evaluated + * based on the following steps to identify a suitable consumer for assigning. + * + *

1. Filter out the consumers without subscribing. + * + *

2. Filter out consumers that are not suitable to be assigned together with the + * topic-partition based on its incompatibility. + * + *

3. If there are consumers that are suitable for the topic-partition after filtering, the + * consumer with the lowest cost would be assigned the topic-partition. Otherwise, if there are no + * suitable consumers for the topic-partition, identify the consumers subscribed to that topic and + * assign the topic-partition to the consumer with the lowest cost. + * + * @param subscriptions the subscription of consumers + * @param costs partition cost evaluated by cost function + * @param incompatiblePartition the incompatibility of the partition + * @return the assignment calculated by greedy + */ protected Map> greedyAssign( Map subscriptions, Map costs, From e5aaa8ac76a0009de4a8ce21cf012b1d3689618a Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 17 Apr 2023 15:00:44 +0800 Subject: [PATCH 31/50] Modify object name --- .../org/astraea/common/assignor/CostAwareAssignor.java | 7 +++---- .../org/astraea/common/assignor/CostAwareAssignorTest.java | 4 ++-- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index aa9eb3aeec..419a7723f5 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -50,11 +50,10 @@ public class CostAwareAssignor extends Assignor { @Override protected Map> assign( - Map subscriptions, - ClusterInfo clusterInfo) { + Map subscriptions, ClusterInfo clusterInfo) { var subscribedTopics = subscriptions.values().stream() - .map(org.astraea.common.assignor.Subscription::topics) + .map(SubscriptionInfo::topics) .flatMap(Collection::stream) .collect(Collectors.toUnmodifiableSet()); @@ -92,7 +91,7 @@ protected Map> assign( * @return the assignment calculated by greedy */ protected Map> greedyAssign( - Map subscriptions, + Map subscriptions, Map costs, Map> incompatiblePartition) { var tmpConsumerCost = diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index b97595342b..ef6789ff3a 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -32,9 +32,9 @@ void testGreedyAssign() { var subscription = Map.of( "c1", - new Subscription(List.of("t1", "t2", "t3"), null), + new SubscriptionInfo(List.of("t1", "t2", "t3"), null), "c2", - new Subscription(List.of("t1", "t2", "t3"), null)); + new SubscriptionInfo(List.of("t1", "t2", "t3"), null)); var cost1 = Map.of( TopicPartition.of("t1", 0), From 279128eb5b86ba7a545ccac02177f4146e294076 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 23 Apr 2023 17:48:04 +0800 Subject: [PATCH 32/50] Separate assign and check incompatibility --- .../common/assignor/CostAwareAssignor.java | 193 +++++++++++------- .../assignor/CostAwareAssignorTest.java | 32 +-- 2 files changed, 141 insertions(+), 84 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 419a7723f5..0a7f6f58a8 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -17,15 +17,12 @@ package org.astraea.common.assignor; import java.time.Duration; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; @@ -67,91 +64,147 @@ protected Map> assign( .filter(e -> subscribedTopics.contains(e.getKey().topic())) .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); var incompatiblePartition = partitionCost.incompatibility(); + var greedyAssignment = greedyAssign(subscriptions, cost); + var adjustedAssignment = + checkIncompatibility(subscriptions, greedyAssignment, incompatiblePartition, cost); - return greedyAssign(subscriptions, cost, incompatiblePartition); + return adjustedAssignment; } /** - * Using the greedy strategy to assign partitions to consumers. The partitions would be evaluated - * based on the following steps to identify a suitable consumer for assigning. - * - *

1. Filter out the consumers without subscribing. - * - *

2. Filter out consumers that are not suitable to be assigned together with the - * topic-partition based on its incompatibility. - * - *

3. If there are consumers that are suitable for the topic-partition after filtering, the - * consumer with the lowest cost would be assigned the topic-partition. Otherwise, if there are no - * suitable consumers for the topic-partition, identify the consumers subscribed to that topic and - * assign the topic-partition to the consumer with the lowest cost. + * Using a greedy strategy to assign partitions to consumers, selecting the consumer with the + * lowest cost each time to assign. * * @param subscriptions the subscription of consumers - * @param costs partition cost evaluated by cost function - * @param incompatiblePartition the incompatibility of the partition - * @return the assignment calculated by greedy + * @param costs partition cost + * @return the assignment by greedyAssign */ protected Map> greedyAssign( - Map subscriptions, - Map costs, - Map> incompatiblePartition) { + Map subscriptions, Map costs) { var tmpConsumerCost = subscriptions.keySet().stream() - .collect(Collectors.toMap(Function.identity(), ignored -> 0.0D)); - var tmpAssignment = - subscriptions.keySet().stream() - .collect( - Collectors.toMap(Function.identity(), ignored -> new ArrayList())); + .collect(Collectors.toMap(Function.identity(), ignore -> 0.0D)); - var lowestCostConsumer = + var lightWeightConsumer = (Function) - (tp) -> { - var subscribeConsumers = - subscriptions.entrySet().stream() - .filter(e -> e.getValue().topics().contains(tp.topic())) - .map(Map.Entry::getKey) - .collect(Collectors.toUnmodifiableSet()); - var suitableConsumers = - incompatiblePartition.isEmpty() - ? subscribeConsumers - : incompatiblePartition.get(tp).isEmpty() - ? subscribeConsumers - : subscribeConsumers.stream() - .filter( - c -> - tmpAssignment.get(c).stream() - .noneMatch( - p -> incompatiblePartition.get(tp).contains(p))) - .collect(Collectors.toUnmodifiableSet()); - - return suitableConsumers.isEmpty() - ? tmpConsumerCost.entrySet().stream() - .filter(e -> subscribeConsumers.contains(e.getKey())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey() - : tmpConsumerCost.entrySet().stream() - .filter(e -> suitableConsumers.contains(e.getKey())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey(); - }; + (tp) -> + tmpConsumerCost.entrySet().stream() + .filter(e -> subscriptions.get(e.getKey()).topics().contains(tp.topic())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey(); return costs.entrySet().stream() .map( e -> { - var tp = e.getKey(); - var cost = e.getValue(); - var consumer = lowestCostConsumer.apply(tp); - tmpConsumerCost.compute(consumer, (ignore, totalCost) -> cost + totalCost); - tmpAssignment.get(consumer).add(tp); - return Map.entry(consumer, tp); + var consumer = lightWeightConsumer.apply(e.getKey()); + tmpConsumerCost.compute(consumer, (ignore, totalCost) -> totalCost + e.getValue()); + return Map.entry(consumer, e.getKey()); }) .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> Collections.singletonList(entry.getValue()), - (l1, l2) -> - Stream.of(l1, l2).flatMap(Collection::stream).collect(Collectors.toList()))); + Collectors.groupingBy( + Map.Entry::getKey, Collectors.mapping(Map.Entry::getValue, Collectors.toList()))); + } + + /** + * Try to avoid putting incompatible partitions on the same consumer. + * + * @param subscriptions the subscription of consumers + * @param assignment assignment + * @param incompatible incompatible partition calculated by cost function + * @param costs partition cost + * @return assignment that filter out most incompatible partitions + */ + protected Map> checkIncompatibility( + Map subscriptions, + Map> assignment, + Map> incompatible, + Map costs) { + // if there is no incompatible, just return the assignment + if (incompatible.isEmpty()) return assignment; + // check the assignment if there are incompatible partitions were put together + var unsuitable = + assignment.entrySet().stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .flatMap(tp -> incompatible.get(tp).stream()) + .collect(Collectors.toUnmodifiableSet()))) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + // if there is no incompatible partition put together, just return the assignment + if (unsuitable.values().stream().allMatch(Set::isEmpty)) return assignment; + + // filter incompatible partitions from assignment to get remaining assignment + var remaining = + assignment.keySet().stream() + .map( + consumer -> + Map.entry( + consumer, + assignment.get(consumer).stream() + .filter(tp -> !unsuitable.get(consumer).contains(tp)) + .collect(Collectors.toList()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + // calculate remaining cost for further assign + var remainingCost = + remaining.entrySet().stream() + .map(e -> Map.entry(e.getKey(), e.getValue().stream().mapToDouble(costs::get).sum())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + var assigned = + remaining.values().stream().flatMap(List::stream).collect(Collectors.toUnmodifiableSet()); + var unassigned = + assignment.values().stream() + .flatMap(Collection::stream) + .filter(tp -> !assigned.contains(tp)) + .collect(Collectors.toSet()); + + for (var tp : unassigned) { + String minConsumer; + // find the consumers that subscribe the topic which we assign now + var subscribedConsumer = + subscriptions.entrySet().stream() + .filter(e -> e.getValue().topics().contains(tp.topic())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + // find the consumers that are suitable with the tp + var suitableConsumer = + remaining.entrySet().stream() + .filter(e -> subscribedConsumer.contains(e.getKey())) + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .flatMap(p -> incompatible.get(p).stream()) + .collect(Collectors.toSet()))) + .filter(e -> !e.getValue().contains(tp)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + + // if there is no suitable consumer, choose the lowest cost consumer that subscribed topic and + // assign the tp to it + // Otherwise, choose the lowest cost consumer from suitable consumers and assign the tp to it + minConsumer = + suitableConsumer.isEmpty() + ? remainingCost.entrySet().stream() + .filter(e -> subscribedConsumer.contains(e.getKey())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey() + : remainingCost.entrySet().stream() + .filter(e -> suitableConsumer.contains(e.getKey())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey(); + + remaining.get(minConsumer).add(tp); + remainingCost.compute(minConsumer, (ignore, totalCost) -> totalCost + costs.get(tp)); + } + + return remaining; } private void retry(ClusterInfo clusterInfo) { diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index ef6789ff3a..1626c1a808 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -46,9 +46,11 @@ void testGreedyAssign() { TopicPartition.of("t3", 0), 0.4); - var allSuitableAssignment = assignor.greedyAssign(subscription, cost1, Map.of()); - Assertions.assertEquals(2, allSuitableAssignment.size()); - allSuitableAssignment.forEach((c, assign) -> Assertions.assertEquals(2, assign.size())); + var assignment = assignor.greedyAssign(subscription, cost1); + Assertions.assertEquals(2, assignment.size()); + assignment.forEach((c, assign) -> Assertions.assertEquals(2, assign.size())); + var finalAssignment = assignor.checkIncompatibility(subscription, assignment, Map.of(), cost1); + Assertions.assertEquals(assignment, finalAssignment); var incompatibility = Map.of( @@ -60,15 +62,16 @@ void testGreedyAssign() { Set.of(TopicPartition.of("t1", 1)), TopicPartition.of("t3", 0), Set.of(TopicPartition.of("t1", 0))); - var incompatibleAssignment = assignor.greedyAssign(subscription, cost1, incompatibility); - Assertions.assertEquals(2, incompatibleAssignment.size()); - incompatibleAssignment.forEach( - (consumer, assignment) -> { - assignment.forEach( + + assignment = assignor.greedyAssign(subscription, cost1); + Assertions.assertEquals(2, assignment.size()); + finalAssignment = + assignor.checkIncompatibility(subscription, assignment, incompatibility, cost1); + finalAssignment.forEach( + (c, a) -> { + a.forEach( tp -> { - incompatibility - .get(tp) - .forEach(itp -> Assertions.assertFalse(assignment.contains(itp))); + incompatibility.get(tp).forEach(itp -> Assertions.assertFalse(a.contains(itp))); }); }); @@ -84,8 +87,9 @@ void testGreedyAssign() { create.apply("t3-0"), Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t1-1"))); - var allIncompatibleAssignment = assignor.greedyAssign(subscription, cost1, allIncompatible); - allIncompatibleAssignment.forEach( - (c, assignment) -> Assertions.assertEquals(2, assignment.size())); + assignment = assignor.greedyAssign(subscription, cost1); + finalAssignment = + assignor.checkIncompatibility(subscription, assignment, incompatibility, cost1); + finalAssignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); } } From 03517c7a796e18e22d67d4911e39a60eede9c456 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 23 Apr 2023 20:36:57 +0800 Subject: [PATCH 33/50] Fix style --- .../java/org/astraea/common/assignor/CostAwareAssignor.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 0a7f6f58a8..270f2e330d 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -65,10 +65,8 @@ protected Map> assign( .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); var incompatiblePartition = partitionCost.incompatibility(); var greedyAssignment = greedyAssign(subscriptions, cost); - var adjustedAssignment = - checkIncompatibility(subscriptions, greedyAssignment, incompatiblePartition, cost); - return adjustedAssignment; + return checkIncompatibility(subscriptions, greedyAssignment, incompatiblePartition, cost); } /** From 8406bf60b18c2d35a7cc6b3db72f6854bd33dc81 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Sun, 23 Apr 2023 22:32:53 +0800 Subject: [PATCH 34/50] Add Assign interface to move greedy impl to it --- .../org/astraea/common/assignor/Assign.java | 61 +++++++++++++++++++ .../common/assignor/CostAwareAssignor.java | 29 +-------- 2 files changed, 64 insertions(+), 26 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/assignor/Assign.java diff --git a/common/src/main/java/org/astraea/common/assignor/Assign.java b/common/src/main/java/org/astraea/common/assignor/Assign.java new file mode 100644 index 0000000000..a76770f42d --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/Assign.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.astraea.common.assignor; + +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.astraea.common.admin.TopicPartition; + +@FunctionalInterface +public interface Assign { + Map> strategy( + Map subscriptions, Map costs); + + /** + * implement the greedy assign strategy + * @return the assignment by greedy strategy + */ + static Assign greedy() { + return (subscriptions, costs) -> { + var tmpConsumerCost = + subscriptions.keySet().stream() + .collect(Collectors.toMap(Function.identity(), ignore -> 0.0D)); + + var lowestCostConsumer = + (Function) + (tp) -> + tmpConsumerCost.entrySet().stream() + .filter(e -> subscriptions.get(e.getKey()).topics().contains(tp.topic())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey(); + + return costs.entrySet().stream() + .map( + e -> { + var consumer = lowestCostConsumer.apply(e.getKey()); + tmpConsumerCost.compute(consumer, (ignore, totalCost) -> totalCost + e.getValue()); + return Map.entry(consumer, e.getKey()); + }) + .collect( + Collectors.groupingBy( + Map.Entry::getKey, Collectors.mapping(Map.Entry::getValue, Collectors.toList()))); + }; + } +} diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 270f2e330d..b6605b0293 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Function; import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.Utils; @@ -79,29 +78,7 @@ protected Map> assign( */ protected Map> greedyAssign( Map subscriptions, Map costs) { - var tmpConsumerCost = - subscriptions.keySet().stream() - .collect(Collectors.toMap(Function.identity(), ignore -> 0.0D)); - - var lightWeightConsumer = - (Function) - (tp) -> - tmpConsumerCost.entrySet().stream() - .filter(e -> subscriptions.get(e.getKey()).topics().contains(tp.topic())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey(); - - return costs.entrySet().stream() - .map( - e -> { - var consumer = lightWeightConsumer.apply(e.getKey()); - tmpConsumerCost.compute(consumer, (ignore, totalCost) -> totalCost + e.getValue()); - return Map.entry(consumer, e.getKey()); - }) - .collect( - Collectors.groupingBy( - Map.Entry::getKey, Collectors.mapping(Map.Entry::getValue, Collectors.toList()))); + return Assign.greedy().strategy(subscriptions, costs); } /** @@ -120,7 +97,7 @@ protected Map> checkIncompatibility( Map costs) { // if there is no incompatible, just return the assignment if (incompatible.isEmpty()) return assignment; - // check the assignment if there are incompatible partitions were put together + // get all consumers' incompatible partitions var unsuitable = assignment.entrySet().stream() .map( @@ -134,6 +111,7 @@ protected Map> checkIncompatibility( // if there is no incompatible partition put together, just return the assignment if (unsuitable.values().stream().allMatch(Set::isEmpty)) return assignment; + String minConsumer; // filter incompatible partitions from assignment to get remaining assignment var remaining = assignment.keySet().stream() @@ -160,7 +138,6 @@ protected Map> checkIncompatibility( .collect(Collectors.toSet()); for (var tp : unassigned) { - String minConsumer; // find the consumers that subscribe the topic which we assign now var subscribedConsumer = subscriptions.entrySet().stream() From 534d7cfc6b86e688c90845682097a0aa8a4f5fc5 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 24 Apr 2023 00:31:45 +0800 Subject: [PATCH 35/50] Add the interface to reassign based on incompatible --- .../org/astraea/common/assignor/Assign.java | 36 +++-- .../common/assignor/CostAwareAssignor.java | 115 ++------------- .../org/astraea/common/assignor/Reassign.java | 134 ++++++++++++++++++ 3 files changed, 166 insertions(+), 119 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/assignor/Reassign.java diff --git a/common/src/main/java/org/astraea/common/assignor/Assign.java b/common/src/main/java/org/astraea/common/assignor/Assign.java index a76770f42d..00751a4053 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assign.java +++ b/common/src/main/java/org/astraea/common/assignor/Assign.java @@ -24,13 +24,14 @@ @FunctionalInterface public interface Assign { - Map> strategy( + Map> result( Map subscriptions, Map costs); - /** - * implement the greedy assign strategy - * @return the assignment by greedy strategy - */ + /** + * implement the greedy assign strategy + * + * @return the assignment by greedy strategy + */ static Assign greedy() { return (subscriptions, costs) -> { var tmpConsumerCost = @@ -46,16 +47,21 @@ static Assign greedy() { .get() .getKey(); - return costs.entrySet().stream() - .map( - e -> { - var consumer = lowestCostConsumer.apply(e.getKey()); - tmpConsumerCost.compute(consumer, (ignore, totalCost) -> totalCost + e.getValue()); - return Map.entry(consumer, e.getKey()); - }) - .collect( - Collectors.groupingBy( - Map.Entry::getKey, Collectors.mapping(Map.Entry::getValue, Collectors.toList()))); + var result = + costs.entrySet().stream() + .map( + e -> { + var consumer = lowestCostConsumer.apply(e.getKey()); + tmpConsumerCost.compute( + consumer, (ignore, totalCost) -> totalCost + e.getValue()); + return Map.entry(consumer, e.getKey()); + }) + .collect( + Collectors.groupingBy( + Map.Entry::getKey, + Collectors.mapping(Map.Entry::getValue, Collectors.toList()))); + + return result; }; } } diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index b6605b0293..5de7a60f28 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -63,123 +63,30 @@ protected Map> assign( .filter(e -> subscribedTopics.contains(e.getKey().topic())) .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); var incompatiblePartition = partitionCost.incompatibility(); - var greedyAssignment = greedyAssign(subscriptions, cost); - return checkIncompatibility(subscriptions, greedyAssignment, incompatiblePartition, cost); + return greedyAssign(subscriptions, cost, incompatiblePartition); } /** * Using a greedy strategy to assign partitions to consumers, selecting the consumer with the * lowest cost each time to assign. * + *

If there are incompatible partitions assigned to the same consumer, perform the reassigning + * to avoid assigning incompatible partitions to the same consumer. + * * @param subscriptions the subscription of consumers * @param costs partition cost + * @param incompatible incompatible partitions calculated by cost function * @return the assignment by greedyAssign */ protected Map> greedyAssign( - Map subscriptions, Map costs) { - return Assign.greedy().strategy(subscriptions, costs); - } - - /** - * Try to avoid putting incompatible partitions on the same consumer. - * - * @param subscriptions the subscription of consumers - * @param assignment assignment - * @param incompatible incompatible partition calculated by cost function - * @param costs partition cost - * @return assignment that filter out most incompatible partitions - */ - protected Map> checkIncompatibility( Map subscriptions, - Map> assignment, - Map> incompatible, - Map costs) { - // if there is no incompatible, just return the assignment - if (incompatible.isEmpty()) return assignment; - // get all consumers' incompatible partitions - var unsuitable = - assignment.entrySet().stream() - .map( - e -> - Map.entry( - e.getKey(), - e.getValue().stream() - .flatMap(tp -> incompatible.get(tp).stream()) - .collect(Collectors.toUnmodifiableSet()))) - .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); - // if there is no incompatible partition put together, just return the assignment - if (unsuitable.values().stream().allMatch(Set::isEmpty)) return assignment; - - String minConsumer; - // filter incompatible partitions from assignment to get remaining assignment - var remaining = - assignment.keySet().stream() - .map( - consumer -> - Map.entry( - consumer, - assignment.get(consumer).stream() - .filter(tp -> !unsuitable.get(consumer).contains(tp)) - .collect(Collectors.toList()))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - // calculate remaining cost for further assign - var remainingCost = - remaining.entrySet().stream() - .map(e -> Map.entry(e.getKey(), e.getValue().stream().mapToDouble(costs::get).sum())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - var assigned = - remaining.values().stream().flatMap(List::stream).collect(Collectors.toUnmodifiableSet()); - var unassigned = - assignment.values().stream() - .flatMap(Collection::stream) - .filter(tp -> !assigned.contains(tp)) - .collect(Collectors.toSet()); - - for (var tp : unassigned) { - // find the consumers that subscribe the topic which we assign now - var subscribedConsumer = - subscriptions.entrySet().stream() - .filter(e -> e.getValue().topics().contains(tp.topic())) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); - // find the consumers that are suitable with the tp - var suitableConsumer = - remaining.entrySet().stream() - .filter(e -> subscribedConsumer.contains(e.getKey())) - .map( - e -> - Map.entry( - e.getKey(), - e.getValue().stream() - .flatMap(p -> incompatible.get(p).stream()) - .collect(Collectors.toSet()))) - .filter(e -> !e.getValue().contains(tp)) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); - - // if there is no suitable consumer, choose the lowest cost consumer that subscribed topic and - // assign the tp to it - // Otherwise, choose the lowest cost consumer from suitable consumers and assign the tp to it - minConsumer = - suitableConsumer.isEmpty() - ? remainingCost.entrySet().stream() - .filter(e -> subscribedConsumer.contains(e.getKey())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey() - : remainingCost.entrySet().stream() - .filter(e -> suitableConsumer.contains(e.getKey())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey(); - - remaining.get(minConsumer).add(tp); - remainingCost.compute(minConsumer, (ignore, totalCost) -> totalCost + costs.get(tp)); - } - - return remaining; + Map costs, + Map> incompatible) { + var assignment = Assign.greedy().result(subscriptions, costs); + return incompatible.isEmpty() + ? assignment + : Reassign.incompatible().result(subscriptions, assignment, incompatible, costs); } private void retry(ClusterInfo clusterInfo) { diff --git a/common/src/main/java/org/astraea/common/assignor/Reassign.java b/common/src/main/java/org/astraea/common/assignor/Reassign.java new file mode 100644 index 0000000000..2d81a42b58 --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/Reassign.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.astraea.common.assignor; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.astraea.common.admin.TopicPartition; + +@FunctionalInterface +public interface Reassign { + /** + * Try to avoid putting incompatible partitions on the same consumer. + * + * @param subscriptions the subscription of consumers + * @param assignment assignment + * @param incompatible incompatible partition calculated by cost function + * @param costs partition cost + * @return assignment that filter out most incompatible partitions + */ + Map> result( + Map subscriptions, + Map> assignment, + Map> incompatible, + Map costs); + + static Reassign incompatible() { + return (subscriptions, assignment, incompatible, costs) -> { + // get the incompatible partitions of each consumer from consumer assignment + var unsuitable = + assignment.entrySet().stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .flatMap(tp -> incompatible.get(tp).stream()) + .collect(Collectors.toUnmodifiableSet()))) + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + + // filter incompatible partitions from assignment to get remaining assignment + var remaining = + assignment.keySet().stream() + .map( + consumer -> + Map.entry( + consumer, + assignment.get(consumer).stream() + .filter(tp -> !unsuitable.get(consumer).contains(tp)) + .collect(Collectors.toList()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // calculate remaining cost for further assign + var remainingCost = + remaining.entrySet().stream() + .map(e -> Map.entry(e.getKey(), e.getValue().stream().mapToDouble(costs::get).sum())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // the partitions have been assigned + var assigned = + remaining.values().stream().flatMap(List::stream).collect(Collectors.toUnmodifiableSet()); + // the partitions need to be reassigned + var unassigned = + assignment.values().stream() + .flatMap(Collection::stream) + .filter(tp -> !assigned.contains(tp)) + .collect(Collectors.toSet()); + + if (unassigned.isEmpty()) return assignment; + + String minConsumer; + for (var tp : unassigned) { + // find the consumers that subscribe the topic which we assign now + var subscribedConsumer = + subscriptions.entrySet().stream() + .filter(e -> e.getValue().topics().contains(tp.topic())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + // find the consumers that are suitable with the tp + var suitableConsumer = + remaining.entrySet().stream() + .filter(e -> subscribedConsumer.contains(e.getKey())) + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .flatMap(p -> incompatible.get(p).stream()) + .collect(Collectors.toSet()))) + .filter(e -> !e.getValue().contains(tp)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + + // if there is no suitable consumer, choose the lowest cost consumer that subscribed topic + // and assign the tp to it. + // Otherwise, choose the lowest cost consumer from suitable consumers and assign the tp to + // it + minConsumer = + suitableConsumer.isEmpty() + ? remainingCost.entrySet().stream() + .filter(e -> subscribedConsumer.contains(e.getKey())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey() + : remainingCost.entrySet().stream() + .filter(e -> suitableConsumer.contains(e.getKey())) + .min(Map.Entry.comparingByValue()) + .get() + .getKey(); + + remaining.get(minConsumer).add(tp); + remainingCost.compute(minConsumer, (ignore, totalCost) -> totalCost + costs.get(tp)); + } + + return remaining; + }; + } +} From cc6582f5a852cfa765f79bb8cd64960cafe01a81 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 24 Apr 2023 00:33:01 +0800 Subject: [PATCH 36/50] Change name --- .../java/org/astraea/common/assignor/CostAwareAssignor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 5de7a60f28..173e73ccb5 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -111,6 +111,6 @@ protected void configure(Configuration config) { @Override public String name() { - return "networkIngress"; + return "costAware"; } } From 10a87dee6993026dedc594b2ade89226d162863f Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 24 Apr 2023 01:22:22 +0800 Subject: [PATCH 37/50] Fix test --- .../common/assignor/CostAwareAssignor.java | 6 ++---- .../org/astraea/common/assignor/Reassign.java | 1 + .../assignor/CostAwareAssignorTest.java | 19 +++++++++---------- 3 files changed, 12 insertions(+), 14 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 173e73ccb5..1f70053897 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -79,14 +79,12 @@ protected Map> assign( * @param incompatible incompatible partitions calculated by cost function * @return the assignment by greedyAssign */ - protected Map> greedyAssign( + private Map> greedyAssign( Map subscriptions, Map costs, Map> incompatible) { var assignment = Assign.greedy().result(subscriptions, costs); - return incompatible.isEmpty() - ? assignment - : Reassign.incompatible().result(subscriptions, assignment, incompatible, costs); + return Reassign.incompatible().result(subscriptions, assignment, incompatible, costs); } private void retry(ClusterInfo clusterInfo) { diff --git a/common/src/main/java/org/astraea/common/assignor/Reassign.java b/common/src/main/java/org/astraea/common/assignor/Reassign.java index 2d81a42b58..525df6a908 100644 --- a/common/src/main/java/org/astraea/common/assignor/Reassign.java +++ b/common/src/main/java/org/astraea/common/assignor/Reassign.java @@ -42,6 +42,7 @@ Map> result( static Reassign incompatible() { return (subscriptions, assignment, incompatible, costs) -> { + if (incompatible.isEmpty()) return assignment; // get the incompatible partitions of each consumer from consumer assignment var unsuitable = assignment.entrySet().stream() diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 1626c1a808..fcda1502f0 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -28,7 +28,8 @@ public class CostAwareAssignorTest { @Test void testGreedyAssign() { - var assignor = new CostAwareAssignor(); + var assign = Assign.greedy(); + var reassign = Reassign.incompatible(); var subscription = Map.of( "c1", @@ -46,10 +47,10 @@ void testGreedyAssign() { TopicPartition.of("t3", 0), 0.4); - var assignment = assignor.greedyAssign(subscription, cost1); + var assignment = assign.result(subscription, cost1); Assertions.assertEquals(2, assignment.size()); - assignment.forEach((c, assign) -> Assertions.assertEquals(2, assign.size())); - var finalAssignment = assignor.checkIncompatibility(subscription, assignment, Map.of(), cost1); + assignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); + var finalAssignment = reassign.result(subscription, assignment, Map.of(), cost1); Assertions.assertEquals(assignment, finalAssignment); var incompatibility = @@ -63,10 +64,9 @@ void testGreedyAssign() { TopicPartition.of("t3", 0), Set.of(TopicPartition.of("t1", 0))); - assignment = assignor.greedyAssign(subscription, cost1); + assignment = assign.result(subscription, cost1); Assertions.assertEquals(2, assignment.size()); - finalAssignment = - assignor.checkIncompatibility(subscription, assignment, incompatibility, cost1); + finalAssignment = reassign.result(subscription, assignment, incompatibility, cost1); finalAssignment.forEach( (c, a) -> { a.forEach( @@ -87,9 +87,8 @@ void testGreedyAssign() { create.apply("t3-0"), Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t1-1"))); - assignment = assignor.greedyAssign(subscription, cost1); - finalAssignment = - assignor.checkIncompatibility(subscription, assignment, incompatibility, cost1); + assignment = assign.result(subscription, cost1); + finalAssignment = reassign.result(subscription, assignment, incompatibility, cost1); finalAssignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); } } From 8e925cc880a77e8b7fda17b7000a7d13b022357f Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 25 Apr 2023 08:57:03 +0800 Subject: [PATCH 38/50] Rename interfaces --- .../assignor/{Assign.java => Combinator.java} | 6 +++--- .../common/assignor/CostAwareAssignor.java | 4 ++-- .../assignor/{Reassign.java => Shuffler.java} | 6 +++--- .../common/assignor/CostAwareAssignorTest.java | 16 ++++++++-------- 4 files changed, 16 insertions(+), 16 deletions(-) rename common/src/main/java/org/astraea/common/assignor/{Assign.java => Combinator.java} (95%) rename common/src/main/java/org/astraea/common/assignor/{Reassign.java => Shuffler.java} (98%) diff --git a/common/src/main/java/org/astraea/common/assignor/Assign.java b/common/src/main/java/org/astraea/common/assignor/Combinator.java similarity index 95% rename from common/src/main/java/org/astraea/common/assignor/Assign.java rename to common/src/main/java/org/astraea/common/assignor/Combinator.java index 00751a4053..885ec4931d 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assign.java +++ b/common/src/main/java/org/astraea/common/assignor/Combinator.java @@ -23,8 +23,8 @@ import org.astraea.common.admin.TopicPartition; @FunctionalInterface -public interface Assign { - Map> result( +public interface Combinator { + Map> combine( Map subscriptions, Map costs); /** @@ -32,7 +32,7 @@ Map> result( * * @return the assignment by greedy strategy */ - static Assign greedy() { + static Combinator greedy() { return (subscriptions, costs) -> { var tmpConsumerCost = subscriptions.keySet().stream() diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 1f70053897..b59e1e41a5 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -83,8 +83,8 @@ private Map> greedyAssign( Map subscriptions, Map costs, Map> incompatible) { - var assignment = Assign.greedy().result(subscriptions, costs); - return Reassign.incompatible().result(subscriptions, assignment, incompatible, costs); + var assignment = Combinator.greedy().combine(subscriptions, costs); + return Shuffler.incompatible().shuffle(subscriptions, assignment, incompatible, costs); } private void retry(ClusterInfo clusterInfo) { diff --git a/common/src/main/java/org/astraea/common/assignor/Reassign.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java similarity index 98% rename from common/src/main/java/org/astraea/common/assignor/Reassign.java rename to common/src/main/java/org/astraea/common/assignor/Shuffler.java index 525df6a908..a16d9c04ed 100644 --- a/common/src/main/java/org/astraea/common/assignor/Reassign.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -24,7 +24,7 @@ import org.astraea.common.admin.TopicPartition; @FunctionalInterface -public interface Reassign { +public interface Shuffler { /** * Try to avoid putting incompatible partitions on the same consumer. * @@ -34,13 +34,13 @@ public interface Reassign { * @param costs partition cost * @return assignment that filter out most incompatible partitions */ - Map> result( + Map> shuffle( Map subscriptions, Map> assignment, Map> incompatible, Map costs); - static Reassign incompatible() { + static Shuffler incompatible() { return (subscriptions, assignment, incompatible, costs) -> { if (incompatible.isEmpty()) return assignment; // get the incompatible partitions of each consumer from consumer assignment diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index fcda1502f0..bcd9adbea6 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -28,8 +28,8 @@ public class CostAwareAssignorTest { @Test void testGreedyAssign() { - var assign = Assign.greedy(); - var reassign = Reassign.incompatible(); + var assign = Combinator.greedy(); + var reassign = Shuffler.incompatible(); var subscription = Map.of( "c1", @@ -47,10 +47,10 @@ void testGreedyAssign() { TopicPartition.of("t3", 0), 0.4); - var assignment = assign.result(subscription, cost1); + var assignment = assign.combine(subscription, cost1); Assertions.assertEquals(2, assignment.size()); assignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); - var finalAssignment = reassign.result(subscription, assignment, Map.of(), cost1); + var finalAssignment = reassign.shuffle(subscription, assignment, Map.of(), cost1); Assertions.assertEquals(assignment, finalAssignment); var incompatibility = @@ -64,9 +64,9 @@ void testGreedyAssign() { TopicPartition.of("t3", 0), Set.of(TopicPartition.of("t1", 0))); - assignment = assign.result(subscription, cost1); + assignment = assign.combine(subscription, cost1); Assertions.assertEquals(2, assignment.size()); - finalAssignment = reassign.result(subscription, assignment, incompatibility, cost1); + finalAssignment = reassign.shuffle(subscription, assignment, incompatibility, cost1); finalAssignment.forEach( (c, a) -> { a.forEach( @@ -87,8 +87,8 @@ void testGreedyAssign() { create.apply("t3-0"), Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t1-1"))); - assignment = assign.result(subscription, cost1); - finalAssignment = reassign.result(subscription, assignment, incompatibility, cost1); + assignment = assign.combine(subscription, cost1); + finalAssignment = reassign.shuffle(subscription, assignment, incompatibility, cost1); finalAssignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); } } From 9db68d2fe0d9a733590f693cc923ee8797891d6f Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Mon, 1 May 2023 13:02:46 +0800 Subject: [PATCH 39/50] revise shuffle --- .../common/assignor/CostAwareAssignor.java | 2 +- .../org/astraea/common/assignor/Shuffler.java | 174 +++++++++++------- .../assignor/CostAwareAssignorTest.java | 2 +- 3 files changed, 107 insertions(+), 71 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index b59e1e41a5..59d6c079fd 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -84,7 +84,7 @@ private Map> greedyAssign( Map costs, Map> incompatible) { var assignment = Combinator.greedy().combine(subscriptions, costs); - return Shuffler.incompatible().shuffle(subscriptions, assignment, incompatible, costs); + return Shuffler.incompatible(4000).shuffle(subscriptions, assignment, incompatible, costs); } private void retry(ClusterInfo clusterInfo) { diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index a16d9c04ed..386f03b128 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -16,10 +16,15 @@ */ package org.astraea.common.assignor; -import java.util.Collection; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Function; import java.util.stream.Collectors; import org.astraea.common.admin.TopicPartition; @@ -40,7 +45,7 @@ Map> shuffle( Map> incompatible, Map costs); - static Shuffler incompatible() { + static Shuffler incompatible(long maxTime) { return (subscriptions, assignment, incompatible, costs) -> { if (incompatible.isEmpty()) return assignment; // get the incompatible partitions of each consumer from consumer assignment @@ -54,82 +59,113 @@ static Shuffler incompatible() { .flatMap(tp -> incompatible.get(tp).stream()) .collect(Collectors.toUnmodifiableSet()))) .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); + System.out.println("unsuitable = " + unsuitable); + System.out.println("assignment = " + assignment); + if (assignment.entrySet().stream() + .noneMatch( + e -> e.getValue().stream().anyMatch(tp -> unsuitable.get(e.getKey()).contains(tp)))) { + System.out.println("none match"); + return assignment; + } - // filter incompatible partitions from assignment to get remaining assignment - var remaining = - assignment.keySet().stream() - .map( - consumer -> - Map.entry( - consumer, - assignment.get(consumer).stream() - .filter(tp -> !unsuitable.get(consumer).contains(tp)) - .collect(Collectors.toList()))) + var tmpCost = new HashMap<>(costs); + var submitHeavyCost = + (Function) + (c) -> { + var tpCost = + tmpCost.entrySet().stream() + .filter(tc -> subscriptions.get(c).topics().contains(tc.getKey().topic())) + .max(Map.Entry.comparingByValue()) + .get(); + tmpCost.remove(tpCost); + return tpCost.getKey(); + }; + var result = + subscriptions.keySet().stream() + .map(c -> Map.entry(c, new ArrayList())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - // calculate remaining cost for further assign - var remainingCost = - remaining.entrySet().stream() - .map(e -> Map.entry(e.getKey(), e.getValue().stream().mapToDouble(costs::get).sum())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + result.forEach((c, r) -> r.add(submitHeavyCost.apply(c))); + + var possibleAssignments = new HashSet>>(); + var randomAssign = + (Function) + (tp) -> { + var subsConsumer = + subscriptions.entrySet().stream() + .filter(e -> e.getValue().topics().contains(tp.topic())) + .collect(Collectors.toUnmodifiableList()); + return subsConsumer + .get(ThreadLocalRandom.current().nextInt(subsConsumer.size())) + .getKey(); + }; - // the partitions have been assigned - var assigned = - remaining.values().stream().flatMap(List::stream).collect(Collectors.toUnmodifiableSet()); - // the partitions need to be reassigned - var unassigned = - assignment.values().stream() - .flatMap(Collection::stream) - .filter(tp -> !assigned.contains(tp)) - .collect(Collectors.toSet()); + var start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < maxTime) { + possibleAssignments.add( + tmpCost.keySet().stream() + .map(tp -> Map.entry(randomAssign.apply(tp), tp)) + .collect( + Collectors.groupingBy( + Map.Entry::getKey, + Collectors.mapping(Map.Entry::getValue, Collectors.toUnmodifiableList())))); + } - if (unassigned.isEmpty()) return assignment; + var incompatibility = + (Function>, Long>) + (possibleAssignment) -> { + var unsuit = + possibleAssignment.entrySet().stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .flatMap(tp -> incompatible.get(tp).stream()) + .collect(Collectors.toUnmodifiableSet()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - String minConsumer; - for (var tp : unassigned) { - // find the consumers that subscribe the topic which we assign now - var subscribedConsumer = - subscriptions.entrySet().stream() - .filter(e -> e.getValue().topics().contains(tp.topic())) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); - // find the consumers that are suitable with the tp - var suitableConsumer = - remaining.entrySet().stream() - .filter(e -> subscribedConsumer.contains(e.getKey())) - .map( - e -> - Map.entry( - e.getKey(), + return possibleAssignment.entrySet().stream() + .mapToLong( + e -> e.getValue().stream() - .flatMap(p -> incompatible.get(p).stream()) - .collect(Collectors.toSet()))) - .filter(e -> !e.getValue().contains(tp)) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); + .filter(tp -> unsuit.get(e.getKey()).contains(tp)) + .count() + / 2) + .sum(); + }; - // if there is no suitable consumer, choose the lowest cost consumer that subscribed topic - // and assign the tp to it. - // Otherwise, choose the lowest cost consumer from suitable consumers and assign the tp to - // it - minConsumer = - suitableConsumer.isEmpty() - ? remainingCost.entrySet().stream() - .filter(e -> subscribedConsumer.contains(e.getKey())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey() - : remainingCost.entrySet().stream() - .filter(e -> suitableConsumer.contains(e.getKey())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey(); - - remaining.get(minConsumer).add(tp); - remainingCost.compute(minConsumer, (ignore, totalCost) -> totalCost + costs.get(tp)); - } + var sigma = + (Function>, Double>) + (r) -> { + var totalCost = + r.entrySet().stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream().mapToDouble(costs::get).sum())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var avg = totalCost.values().stream().mapToDouble(d -> d).average().getAsDouble(); - return remaining; + return Math.sqrt( + totalCost.values().stream().mapToDouble(c -> Math.pow(avg - c, 2)).sum() + / totalCost.size()); + }; + var w = + possibleAssignments.stream() + .map(r -> Map.entry(incompatibility.apply(r), r)) + .collect( + Collectors.groupingBy( + Map.Entry::getKey, + Collectors.mapping(Map.Entry::getValue, Collectors.toSet()))); + System.out.println("w = " + w); + var resu = + w.entrySet().stream().min(Map.Entry.comparingByKey()).get().getValue().stream() + .min(Comparator.comparingDouble(sigma::apply)) + .get(); + System.out.println("resu = " + resu); + return resu; }; } } diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index bcd9adbea6..18f88cb78b 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -29,7 +29,7 @@ public class CostAwareAssignorTest { @Test void testGreedyAssign() { var assign = Combinator.greedy(); - var reassign = Shuffler.incompatible(); + var reassign = Shuffler.incompatible(1000); var subscription = Map.of( "c1", From 74629669664f19414102e2c9aa9a65d2b2c0ee6a Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 2 May 2023 16:25:05 +0800 Subject: [PATCH 40/50] Reduce the complexity of shuffle --- .../org/astraea/common/assignor/Shuffler.java | 96 +++++++------------ 1 file changed, 32 insertions(+), 64 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index 386f03b128..f012b6ce3f 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -16,9 +16,7 @@ */ package org.astraea.common.assignor; -import java.util.ArrayList; import java.util.Comparator; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -59,33 +57,10 @@ static Shuffler incompatible(long maxTime) { .flatMap(tp -> incompatible.get(tp).stream()) .collect(Collectors.toUnmodifiableSet()))) .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); - System.out.println("unsuitable = " + unsuitable); - System.out.println("assignment = " + assignment); if (assignment.entrySet().stream() .noneMatch( - e -> e.getValue().stream().anyMatch(tp -> unsuitable.get(e.getKey()).contains(tp)))) { - System.out.println("none match"); + e -> e.getValue().stream().anyMatch(tp -> unsuitable.get(e.getKey()).contains(tp)))) return assignment; - } - - var tmpCost = new HashMap<>(costs); - var submitHeavyCost = - (Function) - (c) -> { - var tpCost = - tmpCost.entrySet().stream() - .filter(tc -> subscriptions.get(c).topics().contains(tc.getKey().topic())) - .max(Map.Entry.comparingByValue()) - .get(); - tmpCost.remove(tpCost); - return tpCost.getKey(); - }; - var result = - subscriptions.keySet().stream() - .map(c -> Map.entry(c, new ArrayList())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - result.forEach((c, r) -> r.add(submitHeavyCost.apply(c))); var possibleAssignments = new HashSet>>(); var randomAssign = @@ -103,7 +78,7 @@ static Shuffler incompatible(long maxTime) { var start = System.currentTimeMillis(); while (System.currentTimeMillis() - start < maxTime) { possibleAssignments.add( - tmpCost.keySet().stream() + costs.keySet().stream() .map(tp -> Map.entry(randomAssign.apply(tp), tp)) .collect( Collectors.groupingBy( @@ -111,8 +86,25 @@ static Shuffler incompatible(long maxTime) { Collectors.mapping(Map.Entry::getValue, Collectors.toUnmodifiableList())))); } - var incompatibility = - (Function>, Long>) + var standardSigma = + (Function>, Double>) + (r) -> { + var totalCost = + r.entrySet().stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream().mapToDouble(costs::get).sum())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var avg = totalCost.values().stream().mapToDouble(d -> d).average().getAsDouble(); + + return Math.sqrt( + totalCost.values().stream().mapToDouble(c -> Math.pow(avg - c, 2)).sum() + / totalCost.size()); + }; + var numberOfIncompatibility = + (Function>, Integer>) (possibleAssignment) -> { var unsuit = possibleAssignment.entrySet().stream() @@ -126,46 +118,22 @@ static Shuffler incompatible(long maxTime) { .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); return possibleAssignment.entrySet().stream() - .mapToLong( + .mapToInt( e -> - e.getValue().stream() + (int) + e.getValue().stream() .filter(tp -> unsuit.get(e.getKey()).contains(tp)) - .count() - / 2) + .count()) .sum(); }; - var sigma = - (Function>, Double>) - (r) -> { - var totalCost = - r.entrySet().stream() - .map( - e -> - Map.entry( - e.getKey(), - e.getValue().stream().mapToDouble(costs::get).sum())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - var avg = totalCost.values().stream().mapToDouble(d -> d).average().getAsDouble(); - - return Math.sqrt( - totalCost.values().stream().mapToDouble(c -> Math.pow(avg - c, 2)).sum() - / totalCost.size()); - }; - var w = - possibleAssignments.stream() - .map(r -> Map.entry(incompatibility.apply(r), r)) - .collect( - Collectors.groupingBy( - Map.Entry::getKey, - Collectors.mapping(Map.Entry::getValue, Collectors.toSet()))); - System.out.println("w = " + w); - var resu = - w.entrySet().stream().min(Map.Entry.comparingByKey()).get().getValue().stream() - .min(Comparator.comparingDouble(sigma::apply)) - .get(); - System.out.println("resu = " + resu); - return resu; + return possibleAssignments.stream() + .map(e -> Map.entry(e, standardSigma.apply(e))) + .sorted(Map.Entry.comparingByValue()) + .map(Map.Entry::getKey) + .limit((int) Math.floor((double) possibleAssignments.size() / 10)) + .min(Comparator.comparingLong(numberOfIncompatibility::apply)) + .get(); }; } } From 2fade96067bbbaae11d04522240bcf54bdb34f62 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 2 May 2023 17:44:37 +0800 Subject: [PATCH 41/50] Add test --- .../org/astraea/common/assignor/Shuffler.java | 1 + .../assignor/CostAwareAssignorTest.java | 58 +++++-------------- 2 files changed, 17 insertions(+), 42 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index f012b6ce3f..2cd786d3c0 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -54,6 +54,7 @@ static Shuffler incompatible(long maxTime) { Map.entry( e.getKey(), e.getValue().stream() + .filter(incompatible::containsKey) .flatMap(tp -> incompatible.get(tp).stream()) .collect(Collectors.toUnmodifiableSet()))) .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 18f88cb78b..36a2b8c13d 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Function; import org.astraea.common.admin.TopicPartition; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -27,16 +26,16 @@ public class CostAwareAssignorTest { @Test - void testGreedyAssign() { - var assign = Combinator.greedy(); - var reassign = Shuffler.incompatible(1000); + void testCombineAndShuffle() { + var combinator = Combinator.greedy(); + var shuffler = Shuffler.incompatible(3000); var subscription = Map.of( "c1", new SubscriptionInfo(List.of("t1", "t2", "t3"), null), "c2", new SubscriptionInfo(List.of("t1", "t2", "t3"), null)); - var cost1 = + var cost = Map.of( TopicPartition.of("t1", 0), 0.2, @@ -47,48 +46,23 @@ void testGreedyAssign() { TopicPartition.of("t3", 0), 0.4); - var assignment = assign.combine(subscription, cost1); - Assertions.assertEquals(2, assignment.size()); - assignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); - var finalAssignment = reassign.shuffle(subscription, assignment, Map.of(), cost1); - Assertions.assertEquals(assignment, finalAssignment); + var assignment = combinator.combine(subscription, cost); + var shuffledAssignment = shuffler.shuffle(subscription, assignment, Map.of(), cost); + Assertions.assertEquals(assignment, shuffledAssignment); var incompatibility = Map.of( - TopicPartition.of("t1", 0), - Set.of(TopicPartition.of("t3", 0)), TopicPartition.of("t1", 1), - Set.of(TopicPartition.of("t2", 0)), - TopicPartition.of("t2", 0), - Set.of(TopicPartition.of("t1", 1)), + Set.of(TopicPartition.of("t3", 0)), TopicPartition.of("t3", 0), - Set.of(TopicPartition.of("t1", 0))); - - assignment = assign.combine(subscription, cost1); - Assertions.assertEquals(2, assignment.size()); - finalAssignment = reassign.shuffle(subscription, assignment, incompatibility, cost1); - finalAssignment.forEach( - (c, a) -> { - a.forEach( - tp -> { - incompatibility.get(tp).forEach(itp -> Assertions.assertFalse(a.contains(itp))); - }); - }); - - Function create = TopicPartition::of; - var allIncompatible = + Set.of(TopicPartition.of("t1", 1))); + assignment = Map.of( - create.apply("t1-0"), - Set.of(create.apply("t1-1"), create.apply("t2-0"), create.apply("t3-0")), - create.apply("t1-1"), - Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t3-0")), - create.apply("t2-0"), - Set.of(create.apply("t1-0"), create.apply("t1-1"), create.apply("t3-0")), - create.apply("t3-0"), - Set.of(create.apply("t1-0"), create.apply("t2-0"), create.apply("t1-1"))); - - assignment = assign.combine(subscription, cost1); - finalAssignment = reassign.shuffle(subscription, assignment, incompatibility, cost1); - finalAssignment.forEach((c, a) -> Assertions.assertEquals(2, a.size())); + "c1", + List.of(TopicPartition.of("t1-0"), TopicPartition.of("t1-1")), + "c2", + List.of(TopicPartition.of("t2-0"), TopicPartition.of("t3-0"))); + shuffledAssignment = shuffler.shuffle(subscription, assignment, incompatibility, cost); + Assertions.assertEquals(assignment, shuffledAssignment); } } From af3503199c704e0488bd37b21c570745ed726203 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 2 May 2023 18:21:40 +0800 Subject: [PATCH 42/50] Spotless --- common/src/main/java/org/astraea/common/assignor/Assignor.java | 3 +-- common/src/main/java/org/astraea/common/assignor/Shuffler.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 50bd5f505e..14dd051299 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -38,9 +38,8 @@ import org.astraea.common.admin.TopicPartition; import org.astraea.common.consumer.ConsumerConfigs; import org.astraea.common.cost.HasPartitionCost; -import org.astraea.common.cost.ReplicaLeaderSizeCost; -import org.astraea.common.metrics.JndiClient; import org.astraea.common.cost.NetworkIngressCost; +import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.MBeanClient; import org.astraea.common.metrics.collector.MetricStore; import org.astraea.common.partitioner.PartitionerUtils; diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index 2cd786d3c0..79b77733b7 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -70,7 +70,7 @@ static Shuffler incompatible(long maxTime) { var subsConsumer = subscriptions.entrySet().stream() .filter(e -> e.getValue().topics().contains(tp.topic())) - .collect(Collectors.toUnmodifiableList()); + .toList(); return subsConsumer .get(ThreadLocalRandom.current().nextInt(subsConsumer.size())) .getKey(); From 3e4b57cc9aa19519ef97a4907372dc8d2e630f26 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 2 May 2023 20:36:58 +0800 Subject: [PATCH 43/50] Add wait --- .../astraea/common/assignor/Combinator.java | 3 +- .../common/assignor/CostAwareAssignor.java | 38 +++++++------------ .../org/astraea/common/assignor/Shuffler.java | 5 ++- 3 files changed, 19 insertions(+), 27 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Combinator.java b/common/src/main/java/org/astraea/common/assignor/Combinator.java index 885ec4931d..bbdebe2264 100644 --- a/common/src/main/java/org/astraea/common/assignor/Combinator.java +++ b/common/src/main/java/org/astraea/common/assignor/Combinator.java @@ -28,7 +28,8 @@ Map> combine( Map subscriptions, Map costs); /** - * implement the greedy assign strategy + * Using a greedy strategy to assign partitions to consumers, selecting the consumer with the + * lowest cost each time to assign. * * @return the assignment by greedy strategy */ diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 59d6c079fd..15575f44b9 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -20,7 +20,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.Utils; @@ -42,7 +41,9 @@ */ public class CostAwareAssignor extends Assignor { protected static final String MAX_RETRY_TIME = "max.retry.time"; + protected static final String SHUFFLE_TIME = "shuffle.time"; Duration maxRetryTime = Duration.ofSeconds(30); + Duration shuffleTime = Duration.ofSeconds(5); @Override protected Map> assign( @@ -53,8 +54,14 @@ protected Map> assign( .flatMap(Collection::stream) .collect(Collectors.toUnmodifiableSet()); - // wait for clusterBean - retry(clusterInfo); + metricStore.wait( + (ignore) -> { + var bean = metricStore.clusterBean(); + var c = costFunction.partitionCost(clusterInfo, bean); + if (c.value().values().stream().noneMatch(v -> Double.isNaN(v))) return true; + return false; + }, + shuffleTime); var clusterBean = metricStore.clusterBean(); var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean); @@ -64,27 +71,9 @@ protected Map> assign( .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); var incompatiblePartition = partitionCost.incompatibility(); - return greedyAssign(subscriptions, cost, incompatiblePartition); - } - - /** - * Using a greedy strategy to assign partitions to consumers, selecting the consumer with the - * lowest cost each time to assign. - * - *

If there are incompatible partitions assigned to the same consumer, perform the reassigning - * to avoid assigning incompatible partitions to the same consumer. - * - * @param subscriptions the subscription of consumers - * @param costs partition cost - * @param incompatible incompatible partitions calculated by cost function - * @return the assignment by greedyAssign - */ - private Map> greedyAssign( - Map subscriptions, - Map costs, - Map> incompatible) { - var assignment = Combinator.greedy().combine(subscriptions, costs); - return Shuffler.incompatible(4000).shuffle(subscriptions, assignment, incompatible, costs); + var assignment = Combinator.greedy().combine(subscriptions, cost); + return Shuffler.incompatible(shuffleTime) + .shuffle(subscriptions, assignment, incompatiblePartition, cost); } private void retry(ClusterInfo clusterInfo) { @@ -105,6 +94,7 @@ private void retry(ClusterInfo clusterInfo) { @Override protected void configure(Configuration config) { config.duration(MAX_RETRY_TIME).ifPresent(v -> this.maxRetryTime = v); + config.duration(SHUFFLE_TIME).ifPresent(v -> this.shuffleTime = v); } @Override diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index 79b77733b7..9da112526d 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -16,6 +16,7 @@ */ package org.astraea.common.assignor; +import java.time.Duration; import java.util.Comparator; import java.util.HashSet; import java.util.List; @@ -43,7 +44,7 @@ Map> shuffle( Map> incompatible, Map costs); - static Shuffler incompatible(long maxTime) { + static Shuffler incompatible(Duration maxTime) { return (subscriptions, assignment, incompatible, costs) -> { if (incompatible.isEmpty()) return assignment; // get the incompatible partitions of each consumer from consumer assignment @@ -77,7 +78,7 @@ static Shuffler incompatible(long maxTime) { }; var start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < maxTime) { + while (System.currentTimeMillis() - start < maxTime.toMillis()) { possibleAssignments.add( costs.keySet().stream() .map(tp -> Map.entry(randomAssign.apply(tp), tp)) From f842f00f4303898ff720e0a4e352722d5e483e0b Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 2 May 2023 22:36:15 +0800 Subject: [PATCH 44/50] Revise wait --- .../org/astraea/common/assignor/CostAwareAssignor.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 15575f44b9..89859027ad 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -55,11 +55,9 @@ protected Map> assign( .collect(Collectors.toUnmodifiableSet()); metricStore.wait( - (ignore) -> { - var bean = metricStore.clusterBean(); - var c = costFunction.partitionCost(clusterInfo, bean); - if (c.value().values().stream().noneMatch(v -> Double.isNaN(v))) return true; - return false; + (clusterBean) -> { + return costFunction.partitionCost(clusterInfo, clusterBean).value().values().stream() + .noneMatch(v -> Double.isNaN(v)); }, shuffleTime); From 3b0258ac6c584ffd0eb453a92a31f82b9389db08 Mon Sep 17 00:00:00 2001 From: harryteng9527 Date: Tue, 2 May 2023 23:08:28 +0800 Subject: [PATCH 45/50] Remove retry and test --- .../common/assignor/CostAwareAssignor.java | 24 +++---------------- .../assignor/CostAwareAssignorTest.java | 3 ++- 2 files changed, 5 insertions(+), 22 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 89859027ad..dcae2e09d7 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -22,10 +22,8 @@ import java.util.Map; import java.util.stream.Collectors; import org.astraea.common.Configuration; -import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.TopicPartition; -import org.astraea.common.cost.NoSufficientMetricsException; /** * This assignor scores the partitions by cost function(s) that user given. Each cost function @@ -55,10 +53,9 @@ protected Map> assign( .collect(Collectors.toUnmodifiableSet()); metricStore.wait( - (clusterBean) -> { - return costFunction.partitionCost(clusterInfo, clusterBean).value().values().stream() - .noneMatch(v -> Double.isNaN(v)); - }, + (clusterBean) -> + costFunction.partitionCost(clusterInfo, clusterBean).value().values().stream() + .noneMatch(v -> Double.isNaN(v)), shuffleTime); var clusterBean = metricStore.clusterBean(); @@ -74,21 +71,6 @@ protected Map> assign( .shuffle(subscriptions, assignment, incompatiblePartition, cost); } - private void retry(ClusterInfo clusterInfo) { - var timeoutMs = System.currentTimeMillis() + maxRetryTime.toMillis(); - while (System.currentTimeMillis() < timeoutMs) { - try { - var clusterBean = metricStore.clusterBean(); - var partitionCost = costFunction.partitionCost(clusterInfo, clusterBean); - if (partitionCost.value().values().stream().noneMatch(v -> Double.isNaN(v))) return; - } catch (NoSufficientMetricsException e) { - e.printStackTrace(); - Utils.sleep(Duration.ofSeconds(1)); - } - } - throw new RuntimeException("Failed to fetch clusterBean due to timeout"); - } - @Override protected void configure(Configuration config) { config.duration(MAX_RETRY_TIME).ifPresent(v -> this.maxRetryTime = v); diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 36a2b8c13d..1d62fe3beb 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -16,6 +16,7 @@ */ package org.astraea.common.assignor; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.Set; @@ -28,7 +29,7 @@ public class CostAwareAssignorTest { @Test void testCombineAndShuffle() { var combinator = Combinator.greedy(); - var shuffler = Shuffler.incompatible(3000); + var shuffler = Shuffler.incompatible(Duration.ofSeconds(1)); var subscription = Map.of( "c1", From 0d5820ead1c6ff165d2533b5de9a9b49f00da2c0 Mon Sep 17 00:00:00 2001 From: Zhi-Mao Teng Date: Tue, 23 May 2023 13:55:54 +0800 Subject: [PATCH 46/50] Use new shuffler --- .../org/astraea/common/assignor/Assignor.java | 2 +- .../astraea/common/assignor/Combinator.java | 68 ------------------- .../common/assignor/CostAwareAssignor.java | 5 +- .../assignor/CostAwareAssignorTest.java | 52 +------------- 4 files changed, 4 insertions(+), 123 deletions(-) delete mode 100644 common/src/main/java/org/astraea/common/assignor/Combinator.java diff --git a/common/src/main/java/org/astraea/common/assignor/Assignor.java b/common/src/main/java/org/astraea/common/assignor/Assignor.java index 32a3a73741..b0912a09e5 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -44,7 +44,7 @@ /** Abstract assignor implementation which does some common work (e.g., configuration). */ public abstract class Assignor implements ConsumerPartitionAssignor, Configurable { - private Configuration config; + protected Configuration config; public static final String COST_PREFIX = "assignor.cost"; public static final String JMX_PORT = "jmx.port"; Function jmxPortGetter = diff --git a/common/src/main/java/org/astraea/common/assignor/Combinator.java b/common/src/main/java/org/astraea/common/assignor/Combinator.java deleted file mode 100644 index bbdebe2264..0000000000 --- a/common/src/main/java/org/astraea/common/assignor/Combinator.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.astraea.common.assignor; - -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; -import org.astraea.common.admin.TopicPartition; - -@FunctionalInterface -public interface Combinator { - Map> combine( - Map subscriptions, Map costs); - - /** - * Using a greedy strategy to assign partitions to consumers, selecting the consumer with the - * lowest cost each time to assign. - * - * @return the assignment by greedy strategy - */ - static Combinator greedy() { - return (subscriptions, costs) -> { - var tmpConsumerCost = - subscriptions.keySet().stream() - .collect(Collectors.toMap(Function.identity(), ignore -> 0.0D)); - - var lowestCostConsumer = - (Function) - (tp) -> - tmpConsumerCost.entrySet().stream() - .filter(e -> subscriptions.get(e.getKey()).topics().contains(tp.topic())) - .min(Map.Entry.comparingByValue()) - .get() - .getKey(); - - var result = - costs.entrySet().stream() - .map( - e -> { - var consumer = lowestCostConsumer.apply(e.getKey()); - tmpConsumerCost.compute( - consumer, (ignore, totalCost) -> totalCost + e.getValue()); - return Map.entry(consumer, e.getKey()); - }) - .collect( - Collectors.groupingBy( - Map.Entry::getKey, - Collectors.mapping(Map.Entry::getValue, Collectors.toList()))); - - return result; - }; - } -} diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index dcae2e09d7..3629b9840f 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -66,9 +66,8 @@ protected Map> assign( .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); var incompatiblePartition = partitionCost.incompatibility(); - var assignment = Combinator.greedy().combine(subscriptions, cost); - return Shuffler.incompatible(shuffleTime) - .shuffle(subscriptions, assignment, incompatiblePartition, cost); + var shuffler = Shuffler.randomShuffler(subscriptions, cost, incompatiblePartition, config); + return shuffler.shuffle(); } @Override diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java index 1d62fe3beb..7b7bb3df45 100644 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java @@ -16,54 +16,4 @@ */ package org.astraea.common.assignor; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.astraea.common.admin.TopicPartition; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class CostAwareAssignorTest { - - @Test - void testCombineAndShuffle() { - var combinator = Combinator.greedy(); - var shuffler = Shuffler.incompatible(Duration.ofSeconds(1)); - var subscription = - Map.of( - "c1", - new SubscriptionInfo(List.of("t1", "t2", "t3"), null), - "c2", - new SubscriptionInfo(List.of("t1", "t2", "t3"), null)); - var cost = - Map.of( - TopicPartition.of("t1", 0), - 0.2, - TopicPartition.of("t1", 1), - 0.4, - TopicPartition.of("t2", 0), - 0.2, - TopicPartition.of("t3", 0), - 0.4); - - var assignment = combinator.combine(subscription, cost); - var shuffledAssignment = shuffler.shuffle(subscription, assignment, Map.of(), cost); - Assertions.assertEquals(assignment, shuffledAssignment); - - var incompatibility = - Map.of( - TopicPartition.of("t1", 1), - Set.of(TopicPartition.of("t3", 0)), - TopicPartition.of("t3", 0), - Set.of(TopicPartition.of("t1", 1))); - assignment = - Map.of( - "c1", - List.of(TopicPartition.of("t1-0"), TopicPartition.of("t1-1")), - "c2", - List.of(TopicPartition.of("t2-0"), TopicPartition.of("t3-0"))); - shuffledAssignment = shuffler.shuffle(subscription, assignment, incompatibility, cost); - Assertions.assertEquals(assignment, shuffledAssignment); - } -} +public class CostAwareAssignorTest {} From 7e5aac6e4a1321061c0aeb40e93d4a06a9438bb3 Mon Sep 17 00:00:00 2001 From: Zhi-Mao Teng Date: Tue, 23 May 2023 14:03:03 +0800 Subject: [PATCH 47/50] Add filter to avoid Null pointer and make skewCostLimiter more strict --- .../org/astraea/common/assignor/Hint.java | 4 +++- .../org/astraea/common/assignor/Limiter.java | 19 ++----------------- .../org/astraea/common/assignor/Shuffler.java | 4 +--- 3 files changed, 6 insertions(+), 21 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Hint.java b/common/src/main/java/org/astraea/common/assignor/Hint.java index 87a7e41061..24bb27ff67 100644 --- a/common/src/main/java/org/astraea/common/assignor/Hint.java +++ b/common/src/main/java/org/astraea/common/assignor/Hint.java @@ -61,7 +61,8 @@ static Hint incompatibleHint( .filter(e -> e.getValue().topics().contains(tp.topic())) .map(Map.Entry::getKey) .toList(); - if (incompatibilities.get(tp).isEmpty()) return subscriber; + if (incompatibilities.containsKey(tp) && incompatibilities.get(tp).isEmpty()) + return subscriber; var candidates = currentAssignment.entrySet().stream() @@ -71,6 +72,7 @@ static Hint incompatibleHint( Map.entry( e.getKey(), e.getValue().stream() + .filter(incompatibilities::containsKey) .filter(p -> incompatibilities.get(p).contains(tp)) .count())) .collect( diff --git a/common/src/main/java/org/astraea/common/assignor/Limiter.java b/common/src/main/java/org/astraea/common/assignor/Limiter.java index a782eaaa6f..4cd0910d87 100644 --- a/common/src/main/java/org/astraea/common/assignor/Limiter.java +++ b/common/src/main/java/org/astraea/common/assignor/Limiter.java @@ -53,21 +53,7 @@ static Limiter incompatibleLimiter(Map> inco == 0; } - static Limiter skewCostLimiter( - Map partitionCost, Map subscriptions) { - var tmpConsumerCost = - subscriptions.keySet().stream() - .map(c -> Map.entry(c, 0.0)) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - partitionCost.entrySet().stream() - .sorted(Map.Entry.comparingByValue()) - .map(Map.Entry::getKey) - .forEach( - tp -> { - var minCostConsumer = - tmpConsumerCost.entrySet().stream().min(Map.Entry.comparingByValue()).get(); - minCostConsumer.setValue(minCostConsumer.getValue() + partitionCost.get(tp)); - }); + static Limiter skewCostLimiter(Map partitionCost) { var standardDeviation = (Function, Double>) (vs) -> { @@ -75,7 +61,6 @@ static Limiter skewCostLimiter( return Math.sqrt( vs.stream().mapToDouble(v -> Math.pow(v - average, 2)).average().getAsDouble()); }; - var limit = standardDeviation.apply(tmpConsumerCost.values()); return (combinator) -> { var sd = @@ -84,7 +69,7 @@ static Limiter skewCostLimiter( .map(tps -> tps.stream().mapToDouble(partitionCost::get).sum()) .collect(Collectors.toSet())); - return sd < limit; + return sd == 0.0; }; } } diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index 19ff5c7260..4123cdd505 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -36,8 +36,7 @@ static Shuffler randomShuffler( var limiters = Limiter.of( Set.of( - Limiter.skewCostLimiter(partitionCost, subscriptions), - Limiter.incompatibleLimiter(incompatible))); + Limiter.skewCostLimiter(partitionCost), Limiter.incompatibleLimiter(incompatible))); var hints = Hint.of( Set.of( @@ -79,7 +78,6 @@ static Shuffler randomShuffler( } rejectedCombinators.add(combinator); } - return result == null ? rejectedCombinators.stream() .map(c -> Map.entry(c, standardDeviation.apply(c))) From 98be5eeb057289429b156bc69a1295e104cfcb0a Mon Sep 17 00:00:00 2001 From: Zhi-Mao Teng Date: Tue, 23 May 2023 14:14:22 +0800 Subject: [PATCH 48/50] Modify randomShuffler signature, replace config to shuffleTime --- .../org/astraea/common/assignor/CostAwareAssignor.java | 7 ++++++- .../main/java/org/astraea/common/assignor/Shuffler.java | 4 +--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java index 3629b9840f..67d8d4bd91 100644 --- a/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java +++ b/common/src/main/java/org/astraea/common/assignor/CostAwareAssignor.java @@ -66,7 +66,12 @@ protected Map> assign( .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue)); var incompatiblePartition = partitionCost.incompatibility(); - var shuffler = Shuffler.randomShuffler(subscriptions, cost, incompatiblePartition, config); + var shuffler = + Shuffler.randomShuffler( + subscriptions, + cost, + incompatiblePartition, + config.duration(SHUFFLE_TIME).orElse(shuffleTime).toMillis()); return shuffler.shuffle(); } diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java index 4123cdd505..9bf7a69900 100644 --- a/common/src/main/java/org/astraea/common/assignor/Shuffler.java +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -22,7 +22,6 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import org.astraea.common.Configuration; import org.astraea.common.admin.TopicPartition; public interface Shuffler { @@ -32,7 +31,7 @@ static Shuffler randomShuffler( Map subscriptions, Map partitionCost, Map> incompatible, - Configuration config) { + long shuffleTime) { var limiters = Limiter.of( Set.of( @@ -43,7 +42,6 @@ static Shuffler randomShuffler( Hint.lowCostHint(subscriptions, partitionCost), Hint.incompatibleHint(subscriptions, incompatible))); var generator = Generator.randomGenerator(subscriptions, partitionCost, hints); - var shuffleTime = config.duration("shuffle.time").get().toMillis(); var standardDeviation = (Function>, Double>) (combinator) -> { From 9fb2b119638dc85393489cd9f0939b42441603be Mon Sep 17 00:00:00 2001 From: Zhi-Mao Teng Date: Tue, 23 May 2023 16:58:26 +0800 Subject: [PATCH 49/50] Add GeneratorTest and modify Hint --- .../org/astraea/common/assignor/Hint.java | 15 +- .../assignor/CostAwareAssignorTest.java | 19 --- .../common/assignor/GeneratorTest.java | 130 ++++++++++++++++++ 3 files changed, 141 insertions(+), 23 deletions(-) delete mode 100644 common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java create mode 100644 common/src/test/java/org/astraea/common/assignor/GeneratorTest.java diff --git a/common/src/main/java/org/astraea/common/assignor/Hint.java b/common/src/main/java/org/astraea/common/assignor/Hint.java index 24bb27ff67..87fd0d1769 100644 --- a/common/src/main/java/org/astraea/common/assignor/Hint.java +++ b/common/src/main/java/org/astraea/common/assignor/Hint.java @@ -37,16 +37,23 @@ static Hint of(Set hints) { static Hint lowCostHint( Map subscriptions, Map partitionCost) { return (currentAssignment, tp) -> { - var candidates = + var consumerPerCost = currentAssignment.entrySet().stream() .filter(e -> subscriptions.get(e.getKey()).topics().contains(tp.topic())) .map( e -> Map.entry( e.getKey(), e.getValue().stream().mapToDouble(partitionCost::get).sum())) - .sorted(Map.Entry.comparingByValue()) - .map(Map.Entry::getKey) - .toList(); + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + List candidates; + + if (consumerPerCost.containsValue(0.0)) + candidates = + consumerPerCost.entrySet().stream() + .filter(e -> e.getValue() == 0.0) + .map(Map.Entry::getKey) + .toList(); + else candidates = consumerPerCost.entrySet().stream().map(Map.Entry::getKey).toList(); return candidates.stream().limit((long) Math.ceil(candidates.size() / 2.0)).toList(); }; diff --git a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java b/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java deleted file mode 100644 index 7b7bb3df45..0000000000 --- a/common/src/test/java/org/astraea/common/assignor/CostAwareAssignorTest.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.astraea.common.assignor; - -public class CostAwareAssignorTest {} diff --git a/common/src/test/java/org/astraea/common/assignor/GeneratorTest.java b/common/src/test/java/org/astraea/common/assignor/GeneratorTest.java new file mode 100644 index 0000000000..bf67e1192e --- /dev/null +++ b/common/src/test/java/org/astraea/common/assignor/GeneratorTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.astraea.common.assignor; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import org.astraea.common.admin.TopicPartition; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class GeneratorTest { + @Test + void testCombinator() { + var checkSubscriptionAndAssignment = + (BiFunction>, Map, Boolean>) + (assignment, subscription) -> + assignment.entrySet().stream() + .allMatch( + e -> + e.getValue().stream() + .allMatch( + tp -> + subscription + .get(e.getKey()) + .topics() + .contains(tp.topic()))); + + // test same subscription + var subscription = + Map.of( + "c1", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null), + "c2", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null)); + var cost = + Map.of( + TopicPartition.of("t1", 0), + 0.2, + TopicPartition.of("t1", 1), + 0.4, + TopicPartition.of("t2", 0), + 0.2, + TopicPartition.of("t3", 0), + 0.4); + var randomGenerator = + Generator.randomGenerator(subscription, cost, Hint.lowCostHint(subscription, cost)); + var result = randomGenerator.get(); + var partitionsInResult = result.values().stream().flatMap(Collection::stream).toList(); + var partitionInCluster = cost.keySet(); + Assertions.assertEquals(partitionInCluster.size(), partitionsInResult.size()); + Assertions.assertTrue(partitionsInResult.containsAll(partitionInCluster)); + Assertions.assertTrue(checkSubscriptionAndAssignment.apply(result, subscription)); + result.forEach((c, tps) -> Assertions.assertFalse(tps.isEmpty())); + + // Validate that unsubscribed partitions are not be assigned to the consumer. + subscription = + Map.of( + "c1", + new SubscriptionInfo(List.of("t1"), null), + "c2", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null)); + randomGenerator = + Generator.randomGenerator(subscription, cost, Hint.lowCostHint(subscription, cost)); + result = randomGenerator.get(); + partitionsInResult = result.values().stream().flatMap(Collection::stream).toList(); + + Assertions.assertEquals(partitionInCluster.size(), partitionsInResult.size()); + Assertions.assertTrue(partitionsInResult.containsAll(partitionInCluster)); + Assertions.assertTrue(checkSubscriptionAndAssignment.apply(result, subscription)); + + // Every consumer get at least one partition + subscription = + Map.of( + "c1", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null), + "c2", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null), + "c3", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null), + "c4", + new SubscriptionInfo(List.of("t1", "t2", "t3"), null)); + randomGenerator = + Generator.randomGenerator(subscription, cost, Hint.lowCostHint(subscription, cost)); + result = randomGenerator.get(); + result.forEach((c, tps) -> Assertions.assertEquals(1, tps.size())); + + // Validate there is no consumer was assigned more than 1 partition + cost = + Map.of( + TopicPartition.of("t1", 0), + 0.2, + TopicPartition.of("t2", 0), + 0.2, + TopicPartition.of("t3", 0), + 0.4); + randomGenerator = + Generator.randomGenerator(subscription, cost, Hint.lowCostHint(subscription, cost)); + result = randomGenerator.get(); + result.forEach((c, tps) -> Assertions.assertTrue(tps.size() <= 1)); + + subscription = + Map.of( + "c1", + new SubscriptionInfo(List.of("t1"), null), + "c2", + new SubscriptionInfo(List.of("t2"), null), + "c3", + new SubscriptionInfo(List.of("t3"), null)); + randomGenerator = + Generator.randomGenerator(subscription, cost, Hint.lowCostHint(subscription, cost)); + result = randomGenerator.get(); + Assertions.assertTrue(checkSubscriptionAndAssignment.apply(result, subscription)); + } +} From 43a25f9aad79b4af7214d85c6bd38aad0f8266c0 Mon Sep 17 00:00:00 2001 From: Zhi-Mao Teng Date: Tue, 23 May 2023 21:21:48 +0800 Subject: [PATCH 50/50] Fix Hint --- common/src/main/java/org/astraea/common/assignor/Hint.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/common/src/main/java/org/astraea/common/assignor/Hint.java b/common/src/main/java/org/astraea/common/assignor/Hint.java index 87fd0d1769..3aefe41c84 100644 --- a/common/src/main/java/org/astraea/common/assignor/Hint.java +++ b/common/src/main/java/org/astraea/common/assignor/Hint.java @@ -53,7 +53,12 @@ static Hint lowCostHint( .filter(e -> e.getValue() == 0.0) .map(Map.Entry::getKey) .toList(); - else candidates = consumerPerCost.entrySet().stream().map(Map.Entry::getKey).toList(); + else + candidates = + consumerPerCost.entrySet().stream() + .sorted(Map.Entry.comparingByValue()) + .map(Map.Entry::getKey) + .toList(); return candidates.stream().limit((long) Math.ceil(candidates.size() / 2.0)).toList(); };