Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion common/src/main/java/org/astraea/common/Configuration.java
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public Optional<String> string(String key) {

@Override
public List<String> list(String key, String separator) {
return Arrays.asList(requireString(key).split(separator));
return string(key).map(s -> Arrays.asList(s.split(separator))).orElseGet(List::of);
}
};
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,8 @@ public class BalancerProblemFormat {
"org.astraea.common.cost.ReplicaLeaderCost",
"org.astraea.common.cost.RecordSizeCost",
"org.astraea.common.cost.ReplicaNumberCost",
"org.astraea.common.cost.ReplicaLeaderSizeCost");
"org.astraea.common.cost.ReplicaLeaderSizeCost",
"org.astraea.common.cost.BrokerDiskSpaceCost");

public AlgorithmConfig parse() {
return AlgorithmConfig.builder()
Expand Down
131 changes: 131 additions & 0 deletions common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.cost;

import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import org.astraea.common.Configuration;
import org.astraea.common.DataSize;
import org.astraea.common.admin.ClusterInfo;
import org.astraea.common.admin.NodeInfo;
import org.astraea.common.admin.Replica;
import org.astraea.common.metrics.ClusterBean;

public class BrokerDiskSpaceCost implements HasMoveCost {

public static final String BROKER_COST_LIMIT_KEY = "max.broker.total.disk.space";
public static final String BROKER_PATH_COST_LIMIT_KEY = "max.broker.path.disk.space";
private final Map<Integer, DataSize> brokerMoveCostLimit;
private final Map<BrokerPath, DataSize> diskMoveCostLimit;

public BrokerDiskSpaceCost(Configuration configuration) {
this.diskMoveCostLimit = diskMoveCostLimit(configuration);
this.brokerMoveCostLimit = brokerMoveCostLimit(configuration);
}

@Override
public MoveCost moveCost(ClusterInfo before, ClusterInfo after, ClusterBean clusterBean) {
if (brokerDiskUsageSizeOverflow(before, after, brokerMoveCostLimit)) return () -> true;
if (brokerPathDiskUsageSizeOverflow(before, after, diskMoveCostLimit)) return () -> true;
return () -> false;
}

private static Map<BrokerPath, DataSize> diskMoveCostLimit(Configuration configuration) {
return configuration.list(BROKER_PATH_COST_LIMIT_KEY, ",").stream()
.collect(
Collectors.toMap(
idAndPath -> {
var brokerPath = idAndPath.split(":")[0].split("-");
return new BrokerPath(
Integer.parseInt(brokerPath[0]),
IntStream.range(1, brokerPath.length)
.boxed()
.map(x -> brokerPath[x])
.collect(Collectors.joining("-")));
},
idAndPath -> DataSize.of(idAndPath.split(":")[1])));
}

private Map<Integer, DataSize> brokerMoveCostLimit(Configuration configuration) {
return configuration.list(BROKER_COST_LIMIT_KEY, ",").stream()
.collect(
Collectors.toMap(
idAndPath -> Integer.parseInt(idAndPath.split(":")[0]),
idAndPath -> DataSize.of(idAndPath.split(":")[1])));
}

static boolean brokerDiskUsageSizeOverflow(
ClusterInfo before, ClusterInfo after, Map<Integer, DataSize> brokerMoveCostLimit) {
for (var id :
Stream.concat(before.nodes().stream(), after.nodes().stream())
.map(NodeInfo::id)
.parallel()
.collect(Collectors.toSet())) {

var beforeSize = (Long) before.replicaStream(id).map(Replica::size).mapToLong(y -> y).sum();
var addedSize =
(Long)
after
.replicaStream(id)
.filter(r -> before.replicaStream(id).noneMatch(r::equals))
.map(Replica::size)
.mapToLong(y -> y)
.sum();
if ((beforeSize + addedSize)
> brokerMoveCostLimit.getOrDefault(id, DataSize.Byte.of(Long.MAX_VALUE)).bytes())
return true;
}
return false;
}

static boolean brokerPathDiskUsageSizeOverflow(
ClusterInfo before,
ClusterInfo after,
Map<BrokerDiskSpaceCost.BrokerPath, DataSize> diskMoveCostLimit) {
for (var brokerPaths :
Stream.concat(
before.brokerFolders().entrySet().stream(),
after.brokerFolders().entrySet().stream())
.collect(Collectors.toSet())) {
for (var path : brokerPaths.getValue()) {
var brokerPath = new BrokerDiskSpaceCost.BrokerPath(brokerPaths.getKey(), path);
var beforeSize =
before
.replicaStream(brokerPaths.getKey())
.filter(r -> r.path().equals(path))
.mapToLong(Replica::size)
.sum();
var addedSize =
(Long)
after
.replicaStream(brokerPaths.getKey())
.filter(r -> before.replicaStream(brokerPaths.getKey()).noneMatch(r::equals))
.map(Replica::size)
.mapToLong(y -> y)
.sum();
if ((beforeSize + addedSize)
> diskMoveCostLimit.getOrDefault(brokerPath, DataSize.Byte.of(Long.MAX_VALUE)).bytes())
return true;
}
}
return false;
}

record BrokerPath(int broker, String path) {}
}
62 changes: 62 additions & 0 deletions common/src/main/java/org/astraea/common/cost/CostUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.cost;

import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.astraea.common.admin.ClusterInfo;
import org.astraea.common.admin.NodeInfo;
import org.astraea.common.admin.Replica;

final class CostUtils {

private CostUtils() {}

static boolean changedRecordSizeOverflow(
ClusterInfo before, ClusterInfo after, Predicate<Replica> predicate, long limit) {
var totalRemovedSize = 0L;
var totalAddedSize = 0L;
for (var id :
Stream.concat(before.nodes().stream(), after.nodes().stream())
.map(NodeInfo::id)
.parallel()
.collect(Collectors.toSet())) {
var removed =
(int)
before
.replicaStream(id)
.filter(predicate)
.filter(r -> !after.replicas(r.topicPartition()).contains(r))
.mapToLong(Replica::size)
.sum();
var added =
(int)
after
.replicaStream(id)
.filter(predicate)
.filter(r -> !before.replicas(r.topicPartition()).contains(r))
.mapToLong(Replica::size)
.sum();
totalRemovedSize = totalRemovedSize + removed;
totalAddedSize = totalAddedSize + added;
// if migrate cost overflow, leave early and return true
if (totalRemovedSize > limit || totalAddedSize > limit) return true;
}
return Math.max(totalRemovedSize, totalAddedSize) > limit;
}
}
33 changes: 0 additions & 33 deletions common/src/main/java/org/astraea/common/cost/MigrationCost.java
Original file line number Diff line number Diff line change
Expand Up @@ -117,39 +117,6 @@ private static Map<Integer, Long> migratedChanged(
.collect(Collectors.toMap(Function.identity(), n -> cost.getOrDefault(n, 0L)));
}

static boolean changedRecordSizeOverflow(
ClusterInfo before, ClusterInfo after, Predicate<Replica> predicate, long limit) {
var totalRemovedSize = 0L;
var totalAddedSize = 0L;
for (var id :
Stream.concat(before.nodes().stream(), after.nodes().stream())
.map(NodeInfo::id)
.parallel()
.collect(Collectors.toSet())) {
var removed =
(int)
before
.replicaStream(id)
.filter(predicate)
.filter(r -> !after.replicas(r.topicPartition()).contains(r))
.mapToLong(Replica::size)
.sum();
var added =
(int)
after
.replicaStream(id)
.filter(predicate)
.filter(r -> !before.replicas(r.topicPartition()).contains(r))
.mapToLong(Replica::size)
.sum();
totalRemovedSize = totalRemovedSize + removed;
totalAddedSize = totalAddedSize + added;
// if migrate cost overflow, leave early and return true
if (totalRemovedSize > limit || totalAddedSize > limit) return true;
}
return Math.max(totalRemovedSize, totalAddedSize) > limit;
}

private static Map<Integer, Long> changedReplicaNumber(ClusterInfo before, ClusterInfo after) {
return Stream.concat(before.nodes().stream(), after.nodes().stream())
.map(NodeInfo::id)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.astraea.common.cost;

import static org.astraea.common.cost.MigrationCost.changedRecordSizeOverflow;
import static org.astraea.common.cost.CostUtils.changedRecordSizeOverflow;

import java.util.Map;
import java.util.stream.Collectors;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.astraea.common.cost;

import static org.astraea.common.cost.MigrationCost.changedRecordSizeOverflow;
import static org.astraea.common.cost.CostUtils.changedRecordSizeOverflow;

import java.util.Map;
import java.util.stream.Collectors;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ void testString() {
void testList() {
var config = Configuration.of(Map.of("key", "v0,v1"));
Assertions.assertEquals(List.of("v0", "v1"), config.list("key", ","));
Assertions.assertEquals(List.of(), config.list("nonExistKey", ","));
}

@Test
Expand Down
Loading