-
Notifications
You must be signed in to change notification settings - Fork 61
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[DISPATCHER] Maintenance of smooth weight round robin #1396
Changes from 7 commits
2acb2ce
78a8b74
aac1d83
b575b70
34a1c29
010c9fb
0de2bef
e64842f
410a6ec
0ca7b04
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.partitioner; | ||
|
||
import java.time.Duration; | ||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.stream.Collectors; | ||
import java.util.stream.IntStream; | ||
import org.astraea.common.Lazy; | ||
import org.astraea.common.admin.ClusterInfo; | ||
import org.astraea.common.admin.NodeInfo; | ||
import org.astraea.common.admin.ReplicaInfo; | ||
|
||
public class PreArrangementSmoothRR { | ||
private final int preLength; | ||
private final AtomicInteger next = new AtomicInteger(0); | ||
final int[] roundRobin; | ||
final Duration roundRobinLease; | ||
volatile long timeToUpdateRoundRobin = -1; | ||
|
||
private PreArrangementSmoothRR(int preLength, Duration roundRobinLease) { | ||
this.preLength = preLength; | ||
this.roundRobin = new int[preLength]; | ||
this.roundRobinLease = roundRobinLease; | ||
} | ||
|
||
public static PreArrangementSmoothRR of(int preLength, Duration roundRobinLease) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 這個類別不錯,將統一的邏輯拉出來 這邊可以改成 package-private |
||
return new PreArrangementSmoothRR(preLength, roundRobinLease); | ||
} | ||
|
||
synchronized void tryToUpdateRoundRobin( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
ClusterInfo<ReplicaInfo> clusterInfo, Lazy<Map<Integer, Double>> costToScore) { | ||
if (System.currentTimeMillis() >= timeToUpdateRoundRobin) { | ||
var roundRobin = RoundRobin.smooth(costToScore.get()); | ||
var ids = | ||
clusterInfo.nodes().stream().map(NodeInfo::id).collect(Collectors.toUnmodifiableSet()); | ||
// TODO: make ROUND_ROBIN_LENGTH configurable ??? | ||
IntStream.range(0, preLength) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
.forEach(index -> this.roundRobin[index] = roundRobin.next(ids).orElse(-1)); | ||
timeToUpdateRoundRobin = System.currentTimeMillis() + roundRobinLease.toMillis(); | ||
} | ||
} | ||
|
||
int next() { | ||
return roundRobin[ | ||
next.getAndUpdate(previous -> previous >= roundRobin.length - 1 ? 0 : previous + 1)]; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -42,6 +42,30 @@ static <E> RoundRobin<E> smooth(Map<E, Double> scores) { | |
*/ | ||
Optional<E> next(Set<E> availableTargets); | ||
|
||
/** | ||
* Given initial key-score pair, it will output a preferred key with the highest current weight. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 這個註解很棒 |
||
* The current weight of the chosen key will decrease the sum of effective weight. And all current | ||
* weight will increment by its effective weight. It may result in "higher score with higher | ||
* chosen rate". For example: | ||
* | ||
* <p>||========================================||============================================|| | ||
* ||------------ Broker in cluster ------------||------------- Effective weight -------------|| | ||
* ||------------------Broker1------------------||-------------------- 5 ---------------------|| | ||
* ||------------------Broker2------------------||-------------------- 1 ---------------------|| | ||
* ||------------------Broker3------------------||-------------------- 1 ---------------------|| | ||
* ||===========================================||============================================|| | ||
* | ||
* <p>||===================||=======================||===============||======================|| | ||
* ||--- Request Number ---|| Before current weight || Target Broker || After current weight || | ||
* ||----------1-----------||------ {5, 1, 1} ------||----Broker1----||----- {-2, 1, 1} -----|| | ||
* ||----------2-----------||------ {3, 2, 2} ------||----Broker1----||----- {-4, 2, 2} -----|| | ||
* ||----------3-----------||------ {1, 3, 3} ------||----Broker2----||----- { 1,-4, 3} -----|| | ||
* ||----------4-----------||------ {6,-3, 4} ------||----Broker1----||----- {-1,-3, 4} -----|| | ||
* ||----------5-----------||------ {4,-2, 5} ------||----Broker3----||----- { 4,-2,-2} -----|| | ||
* ||----------6-----------||------ {9,-1,-1} ------||----Broker1----||----- { 2,-1,-1} -----|| | ||
* ||----------7-----------||------ {7, 0, 0} ------||----Broker1----||----- { 0, 0, 0} -----|| | ||
* ||======================||=======================||===============||======================|| | ||
*/ | ||
class SmoothRoundRobin<E> implements RoundRobin<E> { | ||
|
||
private final Map<E, Double> effectiveScores; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,87 @@ | ||
/* | ||
* 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.partitioner; | ||
|
||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import java.util.function.Supplier; | ||
import java.util.stream.Collectors; | ||
import org.astraea.common.Lazy; | ||
|
||
public final class SmoothWeightCal<E> { | ||
private final double UPPER_LIMIT_OFFSET_RATIO = 0.1; | ||
private Map<E, Double> currentEffectiveWeightResult; | ||
Lazy<Map<E, Double>> effectiveWeightResult = Lazy.of(); | ||
|
||
public SmoothWeightCal(Map<E, Double> effectiveWeight) { | ||
this.effectiveWeightResult.get( | ||
() -> | ||
effectiveWeight.entrySet().stream() | ||
.collect(Collectors.toMap(Map.Entry::getKey, ignored -> 1.0))); | ||
this.currentEffectiveWeightResult = effectiveWeightResult.get(); | ||
} | ||
|
||
/** | ||
* Update effective weight. | ||
* | ||
* @param brokerScore Broker Score. | ||
*/ | ||
synchronized void refresh(Supplier<Map<E, Double>> brokerScore) { | ||
this.effectiveWeightResult = | ||
Lazy.of( | ||
() -> { | ||
var score = brokerScore.get(); | ||
var avgScore = score.values().stream().mapToDouble(i -> i).average().orElse(1.0); | ||
var offsetRatioOfBroker = | ||
score.entrySet().stream() | ||
.collect( | ||
Collectors.toMap( | ||
Map.Entry::getKey, | ||
entry -> (entry.getValue() - avgScore) / avgScore)); | ||
// If the average offset of all brokers from the cluster is greater than 0.1, it is | ||
// unbalanced. | ||
var balance = | ||
standardDeviationImperative(avgScore, score) | ||
> UPPER_LIMIT_OFFSET_RATIO * avgScore; | ||
this.currentEffectiveWeightResult = | ||
this.currentEffectiveWeightResult.entrySet().stream() | ||
.collect( | ||
Collectors.toUnmodifiableMap( | ||
Map.Entry::getKey, | ||
entry -> { | ||
var offsetRatio = offsetRatioOfBroker.get(entry.getKey()); | ||
var weight = | ||
balance | ||
? entry.getValue() * (1 - offsetRatio) | ||
: entry.getValue(); | ||
return Math.max(weight, 0.1); | ||
})); | ||
|
||
return this.currentEffectiveWeightResult; | ||
}); | ||
} | ||
|
||
private double standardDeviationImperative(double avgMetrics, Map<E, Double> metrics) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 這段邏輯可否整合到 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 已經整合過去了。 |
||
var variance = new AtomicReference<>(0.0); | ||
metrics | ||
.values() | ||
.forEach( | ||
metric -> | ||
variance.updateAndGet(v -> v + (metric - avgMetrics) * (metric - avgMetrics))); | ||
return Math.sqrt(variance.get() / metrics.size()); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.partitioner; | ||
|
||
import java.net.InetSocketAddress; | ||
import java.time.Duration; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.concurrent.ConcurrentLinkedDeque; | ||
import java.util.function.Function; | ||
import java.util.function.Supplier; | ||
import java.util.stream.Collectors; | ||
import org.apache.kafka.common.Cluster; | ||
import org.astraea.common.Configuration; | ||
import org.astraea.common.Utils; | ||
import org.astraea.common.admin.BrokerTopic; | ||
import org.astraea.common.admin.ClusterInfo; | ||
import org.astraea.common.admin.ReplicaInfo; | ||
import org.astraea.common.cost.NeutralIntegratedCost; | ||
import org.astraea.common.metrics.collector.MetricCollector; | ||
|
||
public class SmoothWeightRoundRobinDispatcher implements Dispatcher { | ||
private static final int ROUND_ROBIN_LENGTH = 400; | ||
private static final String JMX_PORT = "jmx.port"; | ||
public static final String ROUND_ROBIN_LEASE_KEY = "round.robin.lease"; | ||
private final ConcurrentLinkedDeque<Integer> unusedPartitions = new ConcurrentLinkedDeque<>(); | ||
private final MetricCollector metricCollector = | ||
MetricCollector.builder().interval(Duration.ofMillis(1500)).build(); | ||
private final NeutralIntegratedCost neutralIntegratedCost = new NeutralIntegratedCost(); | ||
private SmoothWeightCal<Integer> smoothWeightCal; | ||
private PreArrangementSmoothRR preArrangementSmoothRR; | ||
private Function<Integer, Optional<Integer>> jmxPortGetter = (id) -> Optional.empty(); | ||
|
||
@Override | ||
public int partition( | ||
String topic, byte[] key, byte[] value, ClusterInfo<ReplicaInfo> clusterInfo) { | ||
var partitionLeaders = clusterInfo.replicaLeaders(topic); | ||
// just return first partition if there is no available partitions | ||
if (partitionLeaders.isEmpty()) return 0; | ||
|
||
// just return the only one available partition | ||
if (partitionLeaders.size() == 1) return partitionLeaders.get(0).partition(); | ||
|
||
var targetPartition = unusedPartitions.poll(); | ||
refreshPartitionMetaData(clusterInfo, topic); | ||
Supplier<Map<Integer, Double>> supplier = | ||
() -> | ||
// fetch the latest beans for each node | ||
neutralIntegratedCost.brokerCost(clusterInfo, metricCollector.clusterBean()).value(); | ||
|
||
smoothWeightCal.refresh(supplier); | ||
|
||
if (targetPartition == null) { | ||
preArrangementSmoothRR.tryToUpdateRoundRobin( | ||
clusterInfo, smoothWeightCal.effectiveWeightResult); | ||
var target = preArrangementSmoothRR.next(); | ||
|
||
var candidate = | ||
target < 0 ? partitionLeaders : clusterInfo.replicaLeaders(BrokerTopic.of(target, topic)); | ||
candidate = candidate.isEmpty() ? partitionLeaders : candidate; | ||
|
||
targetPartition = candidate.get((int) (Math.random() * candidate.size())).partition(); | ||
} | ||
|
||
return targetPartition; | ||
} | ||
|
||
@Override | ||
public void doClose() { | ||
metricCollector.close(); | ||
} | ||
|
||
@Override | ||
public void configure(Configuration configuration) { | ||
configure( | ||
configuration.integer(JMX_PORT), | ||
PartitionerUtils.parseIdJMXPort(configuration), | ||
configuration | ||
.string(ROUND_ROBIN_LEASE_KEY) | ||
.map(Utils::toDuration) | ||
// The duration of updating beans is 4 seconds, so | ||
// the default duration of updating RR is 4 seconds. | ||
.orElse(Duration.ofSeconds(4))); | ||
} | ||
|
||
void configure( | ||
Optional<Integer> jmxPortDefault, | ||
Map<Integer, Integer> customJmxPort, | ||
Duration roundRobinLease) { | ||
this.jmxPortGetter = id -> Optional.ofNullable(customJmxPort.get(id)).or(() -> jmxPortDefault); | ||
this.neutralIntegratedCost.fetcher().ifPresent(metricCollector::addFetcher); | ||
this.preArrangementSmoothRR = PreArrangementSmoothRR.of(ROUND_ROBIN_LENGTH, roundRobinLease); | ||
this.smoothWeightCal = | ||
new SmoothWeightCal<>( | ||
customJmxPort.entrySet().stream() | ||
.collect(Collectors.toMap(Map.Entry::getKey, ignore -> 1.0))); | ||
} | ||
|
||
@Override | ||
public void onNewBatch(String topic, Cluster cluster, int prevPartition) { | ||
unusedPartitions.add(prevPartition); | ||
} | ||
|
||
private void refreshPartitionMetaData(ClusterInfo<ReplicaInfo> clusterInfo, String topic) { | ||
clusterInfo.availableReplicas(topic).stream() | ||
.filter(p -> !metricCollector.listIdentities().contains(p.nodeInfo().id())) | ||
.forEach( | ||
node -> { | ||
if (!metricCollector.listIdentities().contains(node.nodeInfo().id())) { | ||
jmxPortGetter | ||
.apply(node.nodeInfo().id()) | ||
.ifPresent( | ||
port -> | ||
metricCollector.registerJmx( | ||
node.nodeInfo().id(), | ||
InetSocketAddress.createUnresolved(node.nodeInfo().host(), port))); | ||
} | ||
}); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
可否改叫做
RoundRobinKeeper