From 4f46847f28000dd1bf15eaa1230bcbeabb1e81f2 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 15 May 2023 19:42:54 +0800 Subject: [PATCH 01/77] [METRICS] rewrite ControllerMetrics.Gauge by java 17 record (#1744) --- .../metrics/broker/ControllerMetrics.java | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/common/src/main/java/org/astraea/common/metrics/broker/ControllerMetrics.java b/common/src/main/java/org/astraea/common/metrics/broker/ControllerMetrics.java index b7ae0f8313..88599b7e82 100644 --- a/common/src/main/java/org/astraea/common/metrics/broker/ControllerMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/broker/ControllerMetrics.java @@ -20,7 +20,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.function.Function; import java.util.stream.Collectors; import org.astraea.common.EnumInfo; @@ -81,30 +80,15 @@ public Gauge fetch(MBeanClient mBeanClient) { return new Gauge(mBeanClient.bean(ALL.get(this))); } - public static class Gauge implements HasGauge { - private final BeanObject beanObject; - - public Gauge(BeanObject beanObject) { - this.beanObject = beanObject; - } + public record Gauge(BeanObject beanObject) implements HasGauge { public String metricsName() { return beanObject().properties().get("name"); } - @Override - public Integer value() { - return (int) Objects.requireNonNull(beanObject().attributes().get("Value")); - } - public Controller type() { return Controller.ofAlias(metricsName()); } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } } From bae3ca317418a3d34a3af354c53a4cecb604bcc6 Mon Sep 17 00:00:00 2001 From: Haser Date: Mon, 15 May 2023 20:49:03 +0800 Subject: [PATCH 02/77] [CONNECT] Wrap the kafka SinkTaskContext (#1726) * wrap the kafka SinkTaskContext * fix format violations * move SinkTaskContext to package connector, also remote builder. * remove useless methods in TaskContext * remove SinkTaskContext and replace with anonymous function * replace TopicPartition... with Collection for method pause * add javadoc for each method in TaskContext * move method comment to interface, also remove the usage in Exporter --- .../org/astraea/connector/TaskContext.java | 84 +++++++++++++++++++ 1 file changed, 84 insertions(+) create mode 100644 connector/src/main/java/org/astraea/connector/TaskContext.java diff --git a/connector/src/main/java/org/astraea/connector/TaskContext.java b/connector/src/main/java/org/astraea/connector/TaskContext.java new file mode 100644 index 0000000000..6e2e69171b --- /dev/null +++ b/connector/src/main/java/org/astraea/connector/TaskContext.java @@ -0,0 +1,84 @@ +/* + * 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.connector; + +import java.util.Collection; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.astraea.common.admin.TopicPartition; + +public interface TaskContext { + + /** + * Reset the consumer offsets for the specified partitions. + * + * @param offsets The map of offsets to commit. The key is the {@link TopicPartition} and the + * value is the offset to reset to. + */ + void offset(Map offsets); + + /** + * Reset the consumer offset for the specified partition. + * + * @param topicPartition The {@link TopicPartition} to reset. + * @param offset The offset to reset to. + */ + void offset(TopicPartition topicPartition, long offset); + + /** + * Pause the specified partitions for consuming messages. + * + * @param partitions The collection of partitions should be paused. + */ + void pause(Collection partitions); + + /** + * Request an offset commit. This is asynchronous and may not be complete when the method returns. + */ + void requestCommit(); + + static TaskContext of(SinkTaskContext context) { + return new TaskContext() { + @Override + public void offset(Map offsets) { + context.offset( + offsets.entrySet().stream() + .collect( + Collectors.toMap(e -> TopicPartition.to(e.getKey()), Map.Entry::getValue))); + } + + @Override + public void offset(TopicPartition topicPartition, long offset) { + context.offset(TopicPartition.to(topicPartition), offset); + } + + @Override + public void pause(Collection partitions) { + context.pause( + partitions.stream() + .map(TopicPartition::to) + .toArray(org.apache.kafka.common.TopicPartition[]::new)); + } + + @Override + public void requestCommit() { + context.requestCommit(); + } + }; + } +} From 63235ae4608a41d280650004bda67a73b9b184b4 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 15 May 2023 23:35:21 +0800 Subject: [PATCH 03/77] [METRICS] rewrite LogMetrics by java 17 record (#1748) --- .../common/metrics/broker/LogMetrics.java | 82 ++----------------- .../common/cost/LogMetricsGaugeBuilder.java | 59 +++++++++++++ .../astraea/common/cost/NetworkCostTest.java | 36 +++++--- .../cost/utils/ClusterInfoSensorTest.java | 13 ++- 4 files changed, 96 insertions(+), 94 deletions(-) create mode 100644 common/src/test/java/org/astraea/common/cost/LogMetricsGaugeBuilder.java diff --git a/common/src/main/java/org/astraea/common/metrics/broker/LogMetrics.java b/common/src/main/java/org/astraea/common/metrics/broker/LogMetrics.java index 7c4af1cbb7..d4c2c6d37c 100644 --- a/common/src/main/java/org/astraea/common/metrics/broker/LogMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/broker/LogMetrics.java @@ -36,7 +36,7 @@ public final class LogMetrics { public static final Collection QUERIES = Stream.of(LogCleanerManager.ALL.values().stream(), Log.ALL.values().stream()) .flatMap(f -> f) - .collect(Collectors.toUnmodifiableList()); + .toList(); public enum LogCleanerManager implements EnumInfo { UNCLEANABLE_BYTES("uncleanable-bytes"), @@ -80,17 +80,10 @@ public String toString() { } public List fetch(MBeanClient mBeanClient) { - return mBeanClient.beans(ALL.get(this)).stream() - .map(Gauge::new) - .collect(Collectors.toUnmodifiableList()); + return mBeanClient.beans(ALL.get(this)).stream().map(Gauge::new).toList(); } - public static class Gauge implements HasGauge { - private final BeanObject beanObject; - - public Gauge(BeanObject beanObject) { - this.beanObject = beanObject; - } + public record Gauge(BeanObject beanObject) implements HasGauge { public String path() { var path = beanObject().properties().get("logDirectory"); @@ -106,11 +99,6 @@ public String metricsName() { public LogCleanerManager type() { return ofAlias(metricsName()); } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } @@ -171,70 +159,21 @@ public static Collection gauges(Collection beans, Log type .filter(m -> m instanceof Gauge) .map(m -> (Gauge) m) .filter(m -> m.type() == type) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public List fetch(MBeanClient mBeanClient) { - return mBeanClient.beans(ALL.get(this)).stream() - .map(Gauge::new) - .collect(Collectors.toUnmodifiableList()); - } - - public Builder builder() { - return new Builder(this); - } - - public static class Builder { - private final LogMetrics.Log metric; - private String topic; - private int partition; - private long value; - - public Builder(Log metric) { - this.metric = metric; - } - - public Builder topic(String topic) { - this.topic = topic; - return this; - } - - public Builder partition(int partition) { - this.partition = partition; - return this; - } - - public Builder logSize(long value) { - this.value = value; - return this; - } - - public Gauge build() { - return new Gauge( - new BeanObject( - LogMetrics.DOMAIN_NAME, - Map.ofEntries( - Map.entry("type", LOG_TYPE), - Map.entry("name", metric.metricName()), - Map.entry("topic", topic), - Map.entry("partition", String.valueOf(partition))), - Map.of("Value", value))); - } + return mBeanClient.beans(ALL.get(this)).stream().map(Gauge::new).toList(); } - public static class Gauge implements HasGauge { - private final BeanObject beanObject; - - public Gauge(BeanObject beanObject) { - this.beanObject = beanObject; - } + public record Gauge(BeanObject beanObject) implements HasGauge { public String topic() { - return beanObject().properties().get("topic"); + return topicIndex().get(); } public int partition() { - return Integer.parseInt(beanObject().properties().get("partition")); + return partitionIndex().get().partition(); } public String metricsName() { @@ -244,11 +183,6 @@ public String metricsName() { public Log type() { return ofAlias(metricsName()); } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } diff --git a/common/src/test/java/org/astraea/common/cost/LogMetricsGaugeBuilder.java b/common/src/test/java/org/astraea/common/cost/LogMetricsGaugeBuilder.java new file mode 100644 index 0000000000..9ea1c4faf0 --- /dev/null +++ b/common/src/test/java/org/astraea/common/cost/LogMetricsGaugeBuilder.java @@ -0,0 +1,59 @@ +/* + * 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 org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.broker.LogMetrics; + +public class LogMetricsGaugeBuilder { + private final LogMetrics.Log metric; + private String topic; + private int partition; + private long value; + + public LogMetricsGaugeBuilder(LogMetrics.Log metric) { + this.metric = metric; + } + + public LogMetricsGaugeBuilder topic(String topic) { + this.topic = topic; + return this; + } + + public LogMetricsGaugeBuilder partition(int partition) { + this.partition = partition; + return this; + } + + public LogMetricsGaugeBuilder logSize(long value) { + this.value = value; + return this; + } + + public LogMetrics.Log.Gauge build() { + return new LogMetrics.Log.Gauge( + new BeanObject( + LogMetrics.DOMAIN_NAME, + Map.ofEntries( + Map.entry("type", LogMetrics.LOG_TYPE), + Map.entry("name", metric.metricName()), + Map.entry("topic", topic), + Map.entry("partition", String.valueOf(partition))), + Map.of("Value", value))); + } +} diff --git a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java index cba78a7ad9..da9b785f53 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java @@ -208,19 +208,35 @@ void testReplicationAware() { List.of( MetricFactory.ofPartitionMetric("Rain", 0, 2), MetricFactory.ofPartitionMetric("Drop", 0, 0), - LogMetrics.Log.SIZE.builder().topic("Rain").partition(0).logSize(1).build(), - LogMetrics.Log.SIZE.builder().topic("Drop").partition(0).logSize(1).build(), + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) + .topic("Rain") + .partition(0) + .logSize(1) + .build(), + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) + .topic("Drop") + .partition(0) + .logSize(1) + .build(), bandwidth(ServerMetrics.Topic.BYTES_IN_PER_SEC, "Rain", 100), bandwidth(ServerMetrics.Topic.BYTES_OUT_PER_SEC, "Rain", 300)), 2, List.of( MetricFactory.ofPartitionMetric("Rain", 0, 0), - LogMetrics.Log.SIZE.builder().topic("Rain").partition(0).logSize(1).build(), + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) + .topic("Rain") + .partition(0) + .logSize(1) + .build(), noise(5566)), 3, List.of( MetricFactory.ofPartitionMetric("Drop", 0, 2), - LogMetrics.Log.SIZE.builder().topic("Drop").partition(0).logSize(1).build(), + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) + .topic("Drop") + .partition(0) + .logSize(1) + .build(), bandwidth(ServerMetrics.Topic.BYTES_IN_PER_SEC, "Drop", 80), bandwidth(ServerMetrics.Topic.BYTES_OUT_PER_SEC, "Drop", 800)))); @@ -274,8 +290,7 @@ void testZeroBandwidth() { 1, List.of( MetricFactory.ofPartitionMetric("Pipeline", 0, 2), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("Pipeline") .partition(0) .logSize(0) @@ -284,8 +299,7 @@ void testZeroBandwidth() { 2, List.of( MetricFactory.ofPartitionMetric("Pipeline", 0, 0), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("Pipeline") .partition(0) .logSize(0) @@ -294,8 +308,7 @@ void testZeroBandwidth() { 3, List.of( MetricFactory.ofPartitionMetric("Pipeline", 0, 0), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("Pipeline") .partition(0) .logSize(0) @@ -695,8 +708,7 @@ public LargeTestCase(int brokers, int partitions, int seed) { .build())) .seriesByBrokerReplica( (time, broker, replica) -> - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic(replica.topic()) .partition(replica.partition()) .logSize(replica.size()) diff --git a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java index be39d52cca..639d8872d1 100644 --- a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java +++ b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java @@ -26,6 +26,7 @@ import org.astraea.common.admin.Admin; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; +import org.astraea.common.cost.LogMetricsGaugeBuilder; import org.astraea.common.metrics.BeanObject; import org.astraea.common.metrics.ClusterBean; import org.astraea.common.metrics.HasBeanObject; @@ -132,15 +133,13 @@ void testFollower() { 1, List.of( MetricFactory.ofPartitionMetric("TwoReplica", 0, 2), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("TwoReplica") .partition(0) .logSize(200) .build(), MetricFactory.ofPartitionMetric("OneReplica", 0, 1), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("OneReplica") .partition(0) .logSize(100) @@ -149,8 +148,7 @@ void testFollower() { 2, List.of( MetricFactory.ofPartitionMetric("TwoReplica", 0, 0), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("TwoReplica") .partition(0) .logSize(150) @@ -177,8 +175,7 @@ void testVariousReplicaFactor() { Stream partition(int partition, int replica) { return Stream.of( MetricFactory.ofPartitionMetric("topic", partition, replica), - LogMetrics.Log.SIZE - .builder() + new LogMetricsGaugeBuilder(LogMetrics.Log.SIZE) .topic("topic") .partition(partition) .logSize(0) From 6fec319226a955e6775be8ee7b312bee4a7d6dc7 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 16 May 2023 01:15:54 +0800 Subject: [PATCH 04/77] [METRICS] rewrite NetworkMetrics by java 17 record (#1749) --- .../common/metrics/broker/NetworkMetrics.java | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/common/src/main/java/org/astraea/common/metrics/broker/NetworkMetrics.java b/common/src/main/java/org/astraea/common/metrics/broker/NetworkMetrics.java index 70437bc975..45c6f48afe 100644 --- a/common/src/main/java/org/astraea/common/metrics/broker/NetworkMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/broker/NetworkMetrics.java @@ -132,27 +132,11 @@ public Histogram fetch(MBeanClient mBeanClient) { return new Histogram(mBeanClient.bean(ALL.get(this))); } - public static class Histogram implements HasHistogram { - - private final BeanObject beanObject; - - public Histogram(BeanObject beanObject) { - this.beanObject = beanObject; - } + public record Histogram(BeanObject beanObject) implements HasHistogram { public Request type() { return ofAlias(beanObject.properties().get("request")); } - - @Override - public String toString() { - return beanObject().toString(); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } From 913eca33479a1a61bdb7eddc6988e8e298dc0a38 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 16 May 2023 18:27:52 +0800 Subject: [PATCH 05/77] [METRICS] rewrite ServerMetrics by java 17 record (#1750) --- .../common/metrics/broker/ServerMetrics.java | 228 ++---------------- .../astraea/common/cost/NetworkCostTest.java | 10 +- .../cost/ServerMetricsMeterBuilder.java | 85 +++++++ 3 files changed, 105 insertions(+), 218 deletions(-) create mode 100644 common/src/test/java/org/astraea/common/cost/ServerMetricsMeterBuilder.java diff --git a/common/src/main/java/org/astraea/common/metrics/broker/ServerMetrics.java b/common/src/main/java/org/astraea/common/metrics/broker/ServerMetrics.java index 7823d01937..989c4d79bb 100644 --- a/common/src/main/java/org/astraea/common/metrics/broker/ServerMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/broker/ServerMetrics.java @@ -18,11 +18,8 @@ import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -54,7 +51,7 @@ public final class ServerMetrics { ReplicaManager.ALL.values().stream(), Socket.QUERIES.stream()) .flatMap(f -> f) - .collect(Collectors.toUnmodifiableList()); + .toList(); public static List appInfo(MBeanClient client) { return client.beans(APP_INFO_QUERY).stream() @@ -93,12 +90,10 @@ public String metricName() { } public HasBeanObject fetch(MBeanClient mBeanClient) { - switch (this) { - case CLUSTER_ID: - return new ClusterIdGauge(mBeanClient.bean(ALL.get(this))); - default: - return new Gauge(mBeanClient.bean(ALL.get(this))); - } + if (this == KafkaServer.CLUSTER_ID) + return new ClusterIdGauge(mBeanClient.bean(ALL.get(this))); + + return new Gauge(mBeanClient.bean(ALL.get(this))); } public static KafkaServer ofAlias(String alias) { @@ -115,12 +110,7 @@ public String toString() { return alias(); } - public static class Gauge implements HasGauge { - private final BeanObject beanObject; - - public Gauge(BeanObject beanObject) { - this.beanObject = beanObject; - } + public record Gauge(BeanObject beanObject) implements HasGauge { public String metricsName() { return beanObject().properties().get("name"); @@ -129,25 +119,9 @@ public String metricsName() { public KafkaServer type() { return ofAlias(metricsName()); } - - @Override - public BeanObject beanObject() { - return beanObject; - } } - public static class ClusterIdGauge implements HasGauge { - private final BeanObject beanObject; - - public ClusterIdGauge(BeanObject beanObject) { - this.beanObject = beanObject; - } - - @Override - public BeanObject beanObject() { - return beanObject; - } - } + public record ClusterIdGauge(BeanObject beanObject) implements HasGauge {} } public enum DelayedOperationPurgatory implements EnumInfo { @@ -200,12 +174,7 @@ public Gauge fetch(MBeanClient mBeanClient) { return new Gauge(mBeanClient.bean(ALL.get(this))); } - public static class Gauge implements HasGauge { - private final BeanObject beanObject; - - public Gauge(BeanObject beanObject) { - this.beanObject = beanObject; - } + public record Gauge(BeanObject beanObject) implements HasGauge { public String metricsName() { return beanObject().properties().get("delayedOperation"); @@ -214,11 +183,6 @@ public String metricsName() { public DelayedOperationPurgatory type() { return ofAlias(metricsName()); } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } @@ -272,81 +236,7 @@ public List fetch(MBeanClient mBeanClient) { .collect(Collectors.toList()); } - public Builder builder() { - return new Builder(this); - } - - public static class Builder { - - private final ServerMetrics.Topic metric; - private String topic; - private long time; - private final Map attributes = new HashMap<>(); - - public Builder(Topic metric) { - this.metric = metric; - } - - public Builder topic(String topic) { - this.topic = topic; - return this; - } - - public Builder time(long time) { - this.time = time; - return this; - } - - public Builder meanRate(double value) { - this.attributes.put("MeanRate", value); - return this; - } - - public Builder oneMinuteRate(double value) { - this.attributes.put("OneMinuteRate", value); - return this; - } - - public Builder fiveMinuteRate(double value) { - this.attributes.put("FiveMinuteRate", value); - return this; - } - - public Builder fifteenMinuteRate(double value) { - this.attributes.put("FifteenMinuteRate", value); - return this; - } - - public Builder rateUnit(TimeUnit timeUnit) { - this.attributes.put("RateUnit", timeUnit); - return this; - } - - public Builder count(long count) { - this.attributes.put("Count", count); - return this; - } - - public Meter build() { - return new Meter( - new BeanObject( - ServerMetrics.DOMAIN_NAME, - Map.ofEntries( - Map.entry("type", "BrokerTopicMetrics"), - Map.entry("topic", topic), - Map.entry("name", metric.metricName())), - Map.copyOf(attributes))); - } - } - - public static class Meter implements HasMeter { - - private final BeanObject beanObject; - - public Meter(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } - + public record Meter(BeanObject beanObject) implements HasMeter { public String metricsName() { return beanObject().properties().get("name"); } @@ -358,16 +248,6 @@ public String topic() { public Topic type() { return ofAlias(metricsName()); } - - @Override - public String toString() { - return beanObject().toString(); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } @@ -467,20 +347,14 @@ public Collection of(Collection objects) { .filter(o -> o instanceof Meter) .filter(o -> metricName().equals(o.beanObject().properties().get("name"))) .map(o -> (Meter) o) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public Meter fetch(MBeanClient mBeanClient) { return new Meter(mBeanClient.bean(ALL.get(this))); } - public static class Meter implements HasMeter { - - private final BeanObject beanObject; - - public Meter(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } + public record Meter(BeanObject beanObject) implements HasMeter { public String metricsName() { return beanObject().properties().get("name"); @@ -489,16 +363,6 @@ public String metricsName() { public BrokerTopic type() { return ofAlias(metricsName()); } - - @Override - public String toString() { - return beanObject().toString(); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } @@ -551,14 +415,7 @@ public String toString() { return alias(); } - public static class Gauge implements HasGauge { - - private final BeanObject beanObject; - - public Gauge(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } - + public record Gauge(BeanObject beanObject) implements HasGauge { public String metricsName() { return beanObject().properties().get("name"); } @@ -566,16 +423,6 @@ public String metricsName() { public ReplicaManager type() { return ReplicaManager.ofAlias(metricsName()); } - - @Override - public String toString() { - return beanObject().toString(); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } } } @@ -638,22 +485,11 @@ public static List client(MBeanClient mBeanClient) { return mBeanClient.beans(CLIENT_QUERY).stream().map(Client::new).collect(Collectors.toList()); } - public static class SocketMetric implements HasBeanObject { + public record SocketMetric(BeanObject beanObject) implements HasBeanObject { private static final String MEMORY_POOL_DEPLETED_TIME_TOTAL = "MemoryPoolDepletedTimeTotal"; private static final String MEMORY_POOL_AVG_DEPLETED_PERCENT = "MemoryPoolAvgDepletedPercent"; private static final String BROKER_CONNECTION_ACCEPT_RATE = "broker-connection-accept-rate"; - private final BeanObject beanObject; - - public SocketMetric(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } - public double memoryPoolDepletedTimeTotal() { return (double) beanObject().attributes().get(MEMORY_POOL_DEPLETED_TIME_TOTAL); } @@ -668,24 +504,13 @@ public double brokerConnectionAcceptRate() { } /** property : listener */ - public static class SocketListenerMetric implements HasBeanObject { + public record SocketListenerMetric(BeanObject beanObject) implements HasBeanObject { private static final String CONNECTION_ACCEPT_THROTTLE_TIME = "connection-accept-throttle-time"; private static final String CONNECTION_ACCEPT_RATE = "connection-accept-rate"; private static final String IP_CONNECTION_ACCEPT_THROTTLE_TIME = "ip-connection-accept-throttle-time"; - private final BeanObject beanObject; - - public SocketListenerMetric(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } - public String listener() { return beanObject().properties().get(PROP_LISTENER); } @@ -704,7 +529,7 @@ public double ipConnectionAcceptThrottleTime() { } /** property : listener and networkProcessor */ - public static class SocketNetworkProcessorMetric implements HasBeanObject { + public record SocketNetworkProcessorMetric(BeanObject beanObject) implements HasBeanObject { private static final String INCOMING_BYTE_TOTAL = "incoming-byte-total"; private static final String SELECT_TOTAL = "select-total"; private static final String SUCCESSFUL_AUTHENTICATION_RATE = "successful-authentication-rate"; @@ -750,17 +575,6 @@ public static class SocketNetworkProcessorMetric implements HasBeanObject { private static final String REQUEST_TOTAL = "request-total"; private static final String IO_WAITTIME_TOTAL = "io-waittime-total"; - private final BeanObject beanObject; - - public SocketNetworkProcessorMetric(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } - public String listener() { return beanObject().properties().get(PROP_LISTENER); } @@ -927,19 +741,9 @@ public double ioWaittimeTotal() { } /** property : listener and networkProcessor and clientSoftwareName */ - public static class Client implements HasBeanObject { + public record Client(BeanObject beanObject) implements HasBeanObject { private static final String CONNECTIONS = "connections"; - private final BeanObject beanObject; - - public Client(BeanObject beanObject) { - this.beanObject = Objects.requireNonNull(beanObject); - } - - @Override - public BeanObject beanObject() { - return beanObject; - } public String listener() { return beanObject().properties().get(PROP_LISTENER); diff --git a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java index da9b785f53..7e7daaab04 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java @@ -633,8 +633,7 @@ public LargeTestCase(int brokers, int partitions, int seed) { .timeRange(LocalDateTime.now(), Duration.ZERO) .seriesByBrokerTopic( (time, broker, topic) -> - ServerMetrics.Topic.BYTES_IN_PER_SEC - .builder() + new ServerMetricsMeterBuilder(ServerMetrics.Topic.BYTES_IN_PER_SEC) .topic(topic) .time(time.toEpochSecond(ZoneOffset.UTC)) .oneMinuteRate( @@ -661,8 +660,7 @@ public LargeTestCase(int brokers, int partitions, int seed) { .build()) .seriesByBrokerTopic( (time, broker, topic) -> - ServerMetrics.Topic.BYTES_OUT_PER_SEC - .builder() + new ServerMetricsMeterBuilder(ServerMetrics.Topic.BYTES_OUT_PER_SEC) .topic(topic) .time(time.toEpochSecond(ZoneOffset.UTC)) .oneMinuteRate( @@ -701,8 +699,8 @@ public LargeTestCase(int brokers, int partitions, int seed) { IntStream.range(0, 10) .mapToObj( i -> - ServerMetrics.Topic.TOTAL_FETCH_REQUESTS_PER_SEC - .builder() + new ServerMetricsMeterBuilder( + ServerMetrics.Topic.TOTAL_FETCH_REQUESTS_PER_SEC) .topic("Noise_" + i) .time(time.toEpochSecond(ZoneOffset.UTC)) .build())) diff --git a/common/src/test/java/org/astraea/common/cost/ServerMetricsMeterBuilder.java b/common/src/test/java/org/astraea/common/cost/ServerMetricsMeterBuilder.java new file mode 100644 index 0000000000..b0945243e1 --- /dev/null +++ b/common/src/test/java/org/astraea/common/cost/ServerMetricsMeterBuilder.java @@ -0,0 +1,85 @@ +/* + * 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.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.broker.ServerMetrics; + +public class ServerMetricsMeterBuilder { + private final ServerMetrics.Topic metric; + private String topic; + private long time; + private final Map attributes = new HashMap<>(); + + public ServerMetricsMeterBuilder(ServerMetrics.Topic metric) { + this.metric = metric; + } + + public ServerMetricsMeterBuilder topic(String topic) { + this.topic = topic; + return this; + } + + public ServerMetricsMeterBuilder time(long time) { + this.time = time; + return this; + } + + public ServerMetricsMeterBuilder meanRate(double value) { + this.attributes.put("MeanRate", value); + return this; + } + + public ServerMetricsMeterBuilder oneMinuteRate(double value) { + this.attributes.put("OneMinuteRate", value); + return this; + } + + public ServerMetricsMeterBuilder fiveMinuteRate(double value) { + this.attributes.put("FiveMinuteRate", value); + return this; + } + + public ServerMetricsMeterBuilder fifteenMinuteRate(double value) { + this.attributes.put("FifteenMinuteRate", value); + return this; + } + + public ServerMetricsMeterBuilder rateUnit(TimeUnit timeUnit) { + this.attributes.put("RateUnit", timeUnit); + return this; + } + + public ServerMetricsMeterBuilder count(long count) { + this.attributes.put("Count", count); + return this; + } + + public ServerMetrics.Topic.Meter build() { + return new ServerMetrics.Topic.Meter( + new BeanObject( + ServerMetrics.DOMAIN_NAME, + Map.ofEntries( + Map.entry("type", "BrokerTopicMetrics"), + Map.entry("topic", topic), + Map.entry("name", metric.metricName())), + Map.copyOf(attributes))); + } +} From a43b582fd0a1cab68b862ed8e427e64047409104 Mon Sep 17 00:00:00 2001 From: Yang-Ming Lin Date: Wed, 17 May 2023 00:56:00 +0800 Subject: [PATCH 06/77] [METRICS] rewrite JvmMemory toString by remove StringBuilder (#1746) --- .../common/metrics/platform/JvmMemory.java | 43 +------------------ 1 file changed, 1 insertion(+), 42 deletions(-) diff --git a/common/src/main/java/org/astraea/common/metrics/platform/JvmMemory.java b/common/src/main/java/org/astraea/common/metrics/platform/JvmMemory.java index 8fae8e9488..f3d40e7355 100644 --- a/common/src/main/java/org/astraea/common/metrics/platform/JvmMemory.java +++ b/common/src/main/java/org/astraea/common/metrics/platform/JvmMemory.java @@ -16,47 +16,6 @@ */ package org.astraea.common.metrics.platform; -import java.lang.management.MemoryUsage; import org.astraea.common.metrics.BeanObject; -public class JvmMemory implements HasJvmMemory { - - private final BeanObject beanObject; - private MemoryUsage heapMemoryUsage; - private MemoryUsage nonHeapMemoryUsage; - - @Override - public MemoryUsage heapMemoryUsage() { - // override the default implementation to avoid creating excessive objects - if (heapMemoryUsage == null) heapMemoryUsage = HasJvmMemory.super.heapMemoryUsage(); - return heapMemoryUsage; - } - - @Override - public MemoryUsage nonHeapMemoryUsage() { - // override the default implementation to avoid creating excessive objects - if (nonHeapMemoryUsage == null) nonHeapMemoryUsage = HasJvmMemory.super.nonHeapMemoryUsage(); - return nonHeapMemoryUsage; - } - - public JvmMemory(BeanObject beanObject) { - this.beanObject = beanObject; - } - - @Override - public BeanObject beanObject() { - return beanObject; - } - - @Override - public String toString() { - StringBuilder sb = - new StringBuilder() - .append(" HeapMemoryUsage: ") - .append(heapMemoryUsage()) - .append(System.lineSeparator()) - .append(" NonHeapMemoryUsage") - .append(nonHeapMemoryUsage()); - return "JvmMemory {\n" + sb + "\n}"; - } -} +public record JvmMemory(BeanObject beanObject) implements HasJvmMemory {} From 47bb2ce8c4c2749f2b1310c9635893dd637ae180 Mon Sep 17 00:00:00 2001 From: Xiang-Jun Sun Date: Wed, 17 May 2023 14:25:04 +0800 Subject: [PATCH 07/77] [Cost] Add broker disk space cost (#1604) --- .../org/astraea/common/Configuration.java | 2 +- .../balancer/BalancerProblemFormat.java | 3 +- .../common/cost/BrokerDiskSpaceCost.java | 131 ++++++ .../org/astraea/common/cost/CostUtils.java | 62 +++ .../astraea/common/cost/MigrationCost.java | 33 -- .../astraea/common/cost/RecordSizeCost.java | 2 +- .../common/cost/ReplicaLeaderSizeCost.java | 2 +- .../org/astraea/common/ConfigurationTest.java | 1 + .../common/cost/BrokerDiskSpaceCostTest.java | 428 ++++++++++++++++++ .../astraea/common/cost/CostUtilsTest.java | 236 ++++++++++ .../common/cost/MigrationCostTest.java | 143 ------ docs/web_server/web_api_balancer_chinese.md | 53 ++- 12 files changed, 893 insertions(+), 203 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java create mode 100644 common/src/main/java/org/astraea/common/cost/CostUtils.java create mode 100644 common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java create mode 100644 common/src/test/java/org/astraea/common/cost/CostUtilsTest.java diff --git a/common/src/main/java/org/astraea/common/Configuration.java b/common/src/main/java/org/astraea/common/Configuration.java index bdde8c7f6c..1dc62648ab 100644 --- a/common/src/main/java/org/astraea/common/Configuration.java +++ b/common/src/main/java/org/astraea/common/Configuration.java @@ -44,7 +44,7 @@ public Optional string(String key) { @Override public List 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); } }; } diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java b/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java index b3d9e92a22..7cb4771841 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java @@ -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() diff --git a/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java b/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java new file mode 100644 index 0000000000..e524e1a31f --- /dev/null +++ b/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java @@ -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 brokerMoveCostLimit; + private final Map 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 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 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 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 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) {} +} diff --git a/common/src/main/java/org/astraea/common/cost/CostUtils.java b/common/src/main/java/org/astraea/common/cost/CostUtils.java new file mode 100644 index 0000000000..b23985d657 --- /dev/null +++ b/common/src/main/java/org/astraea/common/cost/CostUtils.java @@ -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 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; + } +} diff --git a/common/src/main/java/org/astraea/common/cost/MigrationCost.java b/common/src/main/java/org/astraea/common/cost/MigrationCost.java index b3cdafddec..f580d6ee71 100644 --- a/common/src/main/java/org/astraea/common/cost/MigrationCost.java +++ b/common/src/main/java/org/astraea/common/cost/MigrationCost.java @@ -117,39 +117,6 @@ private static Map migratedChanged( .collect(Collectors.toMap(Function.identity(), n -> cost.getOrDefault(n, 0L))); } - static boolean changedRecordSizeOverflow( - ClusterInfo before, ClusterInfo after, Predicate 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 changedReplicaNumber(ClusterInfo before, ClusterInfo after) { return Stream.concat(before.nodes().stream(), after.nodes().stream()) .map(NodeInfo::id) diff --git a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java index 19f0f58ff6..9cbf965952 100644 --- a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java @@ -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; diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index 79cdf070c7..d489834fa8 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -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; diff --git a/common/src/test/java/org/astraea/common/ConfigurationTest.java b/common/src/test/java/org/astraea/common/ConfigurationTest.java index 05fff83cf6..f4bc920bbe 100644 --- a/common/src/test/java/org/astraea/common/ConfigurationTest.java +++ b/common/src/test/java/org/astraea/common/ConfigurationTest.java @@ -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 diff --git a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java new file mode 100644 index 0000000000..0e4c998737 --- /dev/null +++ b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java @@ -0,0 +1,428 @@ +/* + * 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 static org.astraea.common.cost.BrokerDiskSpaceCost.brokerDiskUsageSizeOverflow; +import static org.astraea.common.cost.BrokerDiskSpaceCost.brokerPathDiskUsageSizeOverflow; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.requests.DescribeLogDirsResponse; +import org.astraea.common.Configuration; +import org.astraea.common.DataSize; +import org.astraea.common.admin.Broker; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Replica; +import org.astraea.common.metrics.ClusterBean; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class BrokerDiskSpaceCostTest { + + @Test + void testMoveCosts() { + var dataSize = DataSize.of("500MB"); + /* + replica distribution: + p0: 0,1 -> 2,1 + p1: 0,1 -> 0,2 + p2: 0,2 -> 0,2 + replicas during migrated per broker: + 0: p0,p1,p2 + 1: p0,p1 + 2: p0,p1,p2 + */ + var before = + List.of( + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build()); + var after = + List.of( + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(dataSize.bytes()) + .path("/path0") + .build()); + var beforeClusterInfo = of(before); + var afterClusterInfo = of(after); + var brokerConfig = + Configuration.of( + Map.of(BrokerDiskSpaceCost.BROKER_COST_LIMIT_KEY, "0:1500MB,1:1000MB,2:1500MB")); + var brokerOverflowConfig = + Configuration.of( + Map.of(BrokerDiskSpaceCost.BROKER_COST_LIMIT_KEY, "0:1300MB,1:1000MB,2:1500MB")); + var pathConfig = + Configuration.of( + Map.of( + BrokerDiskSpaceCost.BROKER_PATH_COST_LIMIT_KEY, + "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:1000MB")); + var pathOverflowConfig = + Configuration.of( + Map.of( + BrokerDiskSpaceCost.BROKER_PATH_COST_LIMIT_KEY, + "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:900MB")); + // set broker limit no overflow + var cf0 = new BrokerDiskSpaceCost(brokerConfig); + var moveCost0 = cf0.moveCost(beforeClusterInfo, afterClusterInfo, ClusterBean.EMPTY); + // set broker limit and overflow + var cf1 = new BrokerDiskSpaceCost(brokerOverflowConfig); + var moveCost1 = cf1.moveCost(beforeClusterInfo, afterClusterInfo, ClusterBean.EMPTY); + // set path limit no overflow + var cf2 = new BrokerDiskSpaceCost(pathConfig); + var moveCost2 = cf2.moveCost(beforeClusterInfo, afterClusterInfo, ClusterBean.EMPTY); + // set path limit and overflow + var cf3 = new BrokerDiskSpaceCost(pathOverflowConfig); + var moveCost3 = cf3.moveCost(beforeClusterInfo, afterClusterInfo, ClusterBean.EMPTY); + + Assertions.assertFalse(moveCost0.overflow()); + Assertions.assertTrue(moveCost1.overflow()); + Assertions.assertFalse(moveCost2.overflow()); + Assertions.assertTrue(moveCost3.overflow()); + } + + @Test + void testBrokerDiskUsageSizeOverflow() { + var limit = + Map.of( + 0, DataSize.Byte.of(1600), + 1, DataSize.Byte.of(1598), + 2, DataSize.Byte.of(1600)); + var overFlowLimit = + Map.of( + 0, DataSize.Byte.of(1600), + 1, DataSize.Byte.of(1598), + 2, DataSize.Byte.of(1500)); + var totalResult = brokerDiskUsageSizeOverflow(beforeClusterInfo(), afterClusterInfo(), limit); + var overflowResult = + brokerDiskUsageSizeOverflow(beforeClusterInfo(), afterClusterInfo(), overFlowLimit); + Assertions.assertFalse(totalResult); + Assertions.assertTrue(overflowResult); + } + + @Test + void testBrokerPathDiskUsageSizeOverflow() { + var limit = + Map.of( + new BrokerDiskSpaceCost.BrokerPath(0, "/path0"), + DataSize.Byte.of(1600), + new BrokerDiskSpaceCost.BrokerPath(1, "/path0"), + DataSize.Byte.of(1598), + new BrokerDiskSpaceCost.BrokerPath(2, "/path0"), + DataSize.Byte.of(1600), + new BrokerDiskSpaceCost.BrokerPath(2, "/path1"), + DataSize.Byte.of(600)); + var overFlowLimit = + Map.of( + new BrokerDiskSpaceCost.BrokerPath(0, "/path0"), DataSize.Byte.of(1600), + new BrokerDiskSpaceCost.BrokerPath(1, "/path0"), DataSize.Byte.of(1598), + new BrokerDiskSpaceCost.BrokerPath(2, "/path0"), DataSize.Byte.of(1600), + new BrokerDiskSpaceCost.BrokerPath(2, "/path1"), DataSize.Byte.of(500)); + var totalResult = + brokerPathDiskUsageSizeOverflow(beforeClusterInfo(), afterClusterInfo(), limit); + var overflowResult = + brokerPathDiskUsageSizeOverflow(beforeClusterInfo(), afterClusterInfo(), overFlowLimit); + Assertions.assertFalse(totalResult); + Assertions.assertTrue(overflowResult); + } + + public static ClusterInfo of(List replicas) { + var dataPath = + Map.of( + 0, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 1, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 2, + Map.of( + "/path0", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), + "/path1", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + return ClusterInfo.of( + "fake", + replicas.stream() + .map(Replica::nodeInfo) + .distinct() + .map( + nodeInfo -> + Broker.of( + false, + new Node(nodeInfo.id(), "", nodeInfo.port()), + Map.of(), + dataPath.get(nodeInfo.id()), + List.of())) + .collect(Collectors.toList()), + Map.of(), + replicas); + } + + /* + before distribution: + p0: 0,1 + p1: 0,1 + p2: 2,0 + after distribution: + p0: 2,1 + p1: 0,2 + p2: 1,0 + leader log size: + p0: 100 + p1: 500 + p2 1000 + */ + private static ClusterInfo beforeClusterInfo() { + var dataPath = + Map.of( + 0, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 1, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 2, + Map.of( + "/path0", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), + "/path1", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + var replicas = + List.of( + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(100) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(99) + .isLeader(false) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(500) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(499) + .isLeader(false) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(1000) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(1000) + .isLeader(false) + .path("/path0") + .build()); + return ClusterInfo.of( + "fake", + replicas.stream() + .map(Replica::nodeInfo) + .distinct() + .map( + nodeInfo -> + Broker.of( + false, + new Node(nodeInfo.id(), "", nodeInfo.port()), + Map.of(), + dataPath.get(nodeInfo.id()), + List.of())) + .collect(Collectors.toList()), + Map.of(), + replicas); + } + + private static ClusterInfo afterClusterInfo() { + var dataPath = + Map.of( + 0, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 1, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 2, + Map.of( + "/path0", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), + "/path1", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + var replicas = + List.of( + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(100) + .isLeader(true) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(99) + .isLeader(false) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(500) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(500) + .isLeader(false) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(1000) + .isLeader(true) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(1000) + .isLeader(false) + .path("/path0") + .build()); + return ClusterInfo.of( + "fake", + replicas.stream() + .map(Replica::nodeInfo) + .distinct() + .map( + nodeInfo -> + Broker.of( + false, + new Node(nodeInfo.id(), "", nodeInfo.port()), + Map.of(), + dataPath.get(nodeInfo.id()), + List.of())) + .collect(Collectors.toList()), + Map.of(), + replicas); + } +} diff --git a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java new file mode 100644 index 0000000000..e086242e57 --- /dev/null +++ b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java @@ -0,0 +1,236 @@ +/* + * 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 static org.astraea.common.cost.CostUtils.changedRecordSizeOverflow; +import static org.astraea.common.cost.MigrationCost.recordSizeToFetch; +import static org.astraea.common.cost.MigrationCost.recordSizeToSync; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.requests.DescribeLogDirsResponse; +import org.astraea.common.admin.Broker; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Replica; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class CostUtilsTest { + + @Test + void testChangedRecordSizeOverflow() { + var limit = 1600; + var moveInResult = recordSizeToSync(beforeClusterInfo(), afterClusterInfo()); + Assertions.assertEquals(3, moveInResult.size()); + Assertions.assertEquals(0, moveInResult.get(0)); + Assertions.assertEquals(1000, moveInResult.get(1)); + Assertions.assertEquals(100 + 500, moveInResult.get(2)); + + var moveOutResult = recordSizeToFetch(beforeClusterInfo(), afterClusterInfo()); + Assertions.assertEquals(3, moveOutResult.size()); + Assertions.assertEquals(100 + 500, moveOutResult.get(0)); + Assertions.assertEquals(0, moveOutResult.get(1)); + Assertions.assertEquals(1000, moveOutResult.get(2)); + + var totalResult = + changedRecordSizeOverflow(beforeClusterInfo(), afterClusterInfo(), ignored -> true, limit); + var overflowResult = + changedRecordSizeOverflow( + beforeClusterInfo(), afterClusterInfo(), ignored -> true, limit - 100); + Assertions.assertFalse(totalResult); + Assertions.assertTrue(overflowResult); + } + + /* + before distribution: + p0: 0,1 + p1: 0,1 + p2: 2,0 + after distribution: + p0: 2,1 + p1: 0,2 + p2: 1,0 + leader log size: + p0: 100 + p1: 500 + p2 1000 + */ + private static ClusterInfo beforeClusterInfo() { + var dataPath = + Map.of( + 0, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 1, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 2, + Map.of( + "/path0", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), + "/path1", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + var replicas = + List.of( + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(100) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(99) + .isLeader(false) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(500) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(499) + .isLeader(false) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(1000) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(1000) + .isLeader(false) + .path("/path0") + .build()); + return ClusterInfo.of( + "fake", + replicas.stream() + .map(Replica::nodeInfo) + .distinct() + .map( + nodeInfo -> + Broker.of( + false, + new Node(nodeInfo.id(), "", nodeInfo.port()), + Map.of(), + dataPath.get(nodeInfo.id()), + List.of())) + .collect(Collectors.toList()), + Map.of(), + replicas); + } + + private static ClusterInfo afterClusterInfo() { + var dataPath = + Map.of( + 0, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 1, + Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 2, + Map.of( + "/path0", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), + "/path1", + new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + var replicas = + List.of( + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(100) + .isLeader(true) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(0) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(99) + .isLeader(false) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(500) + .isLeader(true) + .path("/path0") + .build(), + Replica.builder() + .topic("topic1") + .partition(1) + .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .size(500) + .isLeader(false) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .size(1000) + .isLeader(true) + .path("/path1") + .build(), + Replica.builder() + .topic("topic1") + .partition(2) + .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .size(1000) + .isLeader(false) + .path("/path0") + .build()); + return ClusterInfo.of( + "fake", + replicas.stream() + .map(Replica::nodeInfo) + .distinct() + .map( + nodeInfo -> + Broker.of( + false, + new Node(nodeInfo.id(), "", nodeInfo.port()), + Map.of(), + dataPath.get(nodeInfo.id()), + List.of())) + .collect(Collectors.toList()), + Map.of(), + replicas); + } +} diff --git a/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java b/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java index f30297ce7b..89ddd813bf 100644 --- a/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java @@ -16,16 +16,11 @@ */ package org.astraea.common.cost; -import static org.astraea.common.cost.MigrationCost.changedRecordSizeOverflow; -import static org.astraea.common.cost.MigrationCost.recordSizeToFetch; -import static org.astraea.common.cost.MigrationCost.recordSizeToSync; import static org.astraea.common.cost.MigrationCost.replicaLeaderToAdd; import static org.astraea.common.cost.MigrationCost.replicaLeaderToRemove; import static org.astraea.common.cost.MigrationCost.replicaNumChanged; import java.util.List; -import java.util.Map; -import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoTest; import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; @@ -295,142 +290,4 @@ void testChangedReplicaNumber() { Assertions.assertEquals(-1, changedReplicaCount.get(1)); Assertions.assertEquals(2, changedReplicaCount.get(2)); } - - @Test - void testChangedRecordSizeOverflow() { - var limit = 1600; - var moveInResult = recordSizeToSync(beforeClusterInfo(), afterClusterInfo()); - Assertions.assertEquals(3, moveInResult.size()); - Assertions.assertEquals(0, moveInResult.get(0)); - Assertions.assertEquals(1000, moveInResult.get(1)); - Assertions.assertEquals(100 + 500, moveInResult.get(2)); - - var moveOutResult = recordSizeToFetch(beforeClusterInfo(), afterClusterInfo()); - Assertions.assertEquals(3, moveOutResult.size()); - Assertions.assertEquals(100 + 500, moveOutResult.get(0)); - Assertions.assertEquals(0, moveOutResult.get(1)); - Assertions.assertEquals(1000, moveOutResult.get(2)); - - var totalResult = - changedRecordSizeOverflow(beforeClusterInfo(), afterClusterInfo(), ignored -> true, limit); - var overflowResult = - changedRecordSizeOverflow( - beforeClusterInfo(), afterClusterInfo(), ignored -> true, limit - 100); - Assertions.assertFalse(totalResult); - Assertions.assertTrue(overflowResult); - } - - /* - before distribution: - p0: 0,1 - p1: 0,1 - p2: 2,0 - after distribution: - p0: 2,1 - p1: 0,2 - p2: 1,0 - leader log size: - p0: 100 - p1: 500 - p2 1000 - */ - private static ClusterInfo beforeClusterInfo() { - return ClusterInfo.of( - "fake", - List.of(NodeInfo.of(0, "aa", 22), NodeInfo.of(1, "aa", 22), NodeInfo.of(2, "aa", 22)), - Map.of(), - List.of( - Replica.builder() - .topic("topic1") - .partition(0) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) - .size(100) - .isLeader(true) - .build(), - Replica.builder() - .topic("topic1") - .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) - .size(99) - .isLeader(false) - .build(), - Replica.builder() - .topic("topic1") - .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) - .size(500) - .isLeader(true) - .build(), - Replica.builder() - .topic("topic1") - .partition(1) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) - .size(499) - .isLeader(false) - .build(), - Replica.builder() - .topic("topic1") - .partition(2) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) - .size(1000) - .isLeader(true) - .build(), - Replica.builder() - .topic("topic1") - .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) - .size(1000) - .isLeader(false) - .build())); - } - - private static ClusterInfo afterClusterInfo() { - return ClusterInfo.of( - "fake", - List.of(NodeInfo.of(0, "aa", 22), NodeInfo.of(1, "aa", 22), NodeInfo.of(2, "aa", 22)), - Map.of(), - List.of( - Replica.builder() - .topic("topic1") - .partition(0) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) - .size(100) - .isLeader(true) - .build(), - Replica.builder() - .topic("topic1") - .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) - .size(99) - .isLeader(false) - .build(), - Replica.builder() - .topic("topic1") - .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) - .size(500) - .isLeader(true) - .build(), - Replica.builder() - .topic("topic1") - .partition(1) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) - .size(500) - .isLeader(false) - .build(), - Replica.builder() - .topic("topic1") - .partition(2) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) - .size(1000) - .isLeader(true) - .build(), - Replica.builder() - .topic("topic1") - .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) - .size(1000) - .isLeader(false) - .build())); - } } diff --git a/docs/web_server/web_api_balancer_chinese.md b/docs/web_server/web_api_balancer_chinese.md index f458409c91..ba5568aa7a 100644 --- a/docs/web_server/web_api_balancer_chinese.md +++ b/docs/web_server/web_api_balancer_chinese.md @@ -33,12 +33,14 @@ POST /balancer costConfig: -| config key | config value | -|-----------------------------|--------------------| -| max.migrated.size | 設定最大可搬移的資料量 | -| max.migrated.leader.number | 設定最大可搬移的leader 數量 | -| max.migrated.replica.number | 設定最大可搬移的replica 數量 | -| max.migrated.leader.size | 設定最大可搬移的leader 資料量 | +| config key | config value | value format | +| --------------------------- | ------------------------------------------------------------ | ------------------------------------------------------------ | +| max.migrated.size | 設定最大可搬移的資料量 | "`data size` + `unit`" ex.100KB, 500MB, 3GB | +| max.migrated.leader.number | 設定最大可搬移的leader 數量 | "`limit number`" ex. 1,2,3,100 | +| max.migrated.replica.number | 設定最大可搬移的replica 數量 | "`limit number`" ex. 1,2,3,100 | +| max.migrated.leader.size | 設定最大可搬移的leader 資料量 | "`data size` + `unit`" ex.100KB, 500MB, 3GB | +| max.broker.total.disk.space | 設定搬移過程中broker最大可以佔用的replica 資料量 | "`broker Id` + `:` + `data size` " ex. "0:1500MB ,1:1000MB ,2:1500MB" | +| max.broker.path.disk.space | 設定搬移過程中broker上的data folder最大可以佔用的replica 資料量 | "`broker Id` + `-` + `data path` + `:` + `data size` " ex. "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:900MB" | 目前支援的 Cost Function @@ -61,23 +63,28 @@ cURL 範例 curl -X POST http://localhost:8001/balancer \ -H "Content-Type: application/json" \ -d '{ - "timeout": "5s", - "balancer": "org.astraea.common.balancer.algorithms.GreedyBalancer", - "balancerConfig": { - "shuffle.tweaker.min.step": 1, - "shuffle.tweaker.max.step": 10 - }, - "clusterCosts": [ - { "cost": "org.astraea.common.cost.ReplicaLeaderCost", "weight": 1 } - ], - "moveCosts": [ - "org.astraea.common.cost.ReplicaLeaderCost", - "org.astraea.common.cost.RecordSizeCost" - ], - "costConfig": { - "max.migrated.size": "500MB", - "max.migrated.leader.number": 5 - } + "timeout": "5s", + "balancer": "org.astraea.common.balancer.algorithms.GreedyBalancer", + "balancerConfig": { + "shuffle.tweaker.min.step": "1", + "shuffle.tweaker.max.step": "10" + }, + "clusterCosts": [ + { + "cost": "org.astraea.common.cost.ReplicaLeaderCost", + "weight": 1 + } + ], + "moveCosts": [ + "org.astraea.common.cost.ReplicaLeaderCost", + "org.astraea.common.cost.RecordSizeCost" + ], + "costConfig": { + "max.migrated.size": "500MB", + "max.migrated.leader.number": 5, + "max.broker.total.disk.space": "0:1500MB,1:1000MB,2:1500MB", + "max.broker.path.disk.space": "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:1000MB" + } }' ``` From 6ac86453b61d1979d799b592cbfbcb1f7dc5322f Mon Sep 17 00:00:00 2001 From: Xiang-Jun Sun Date: Wed, 17 May 2023 15:14:40 +0800 Subject: [PATCH 08/77] [DOC] Reformat `web_api_balancer_docs` (#1752) --- docs/web_server/web_api_balancer_chinese.md | 43 ++++++++++----------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/docs/web_server/web_api_balancer_chinese.md b/docs/web_server/web_api_balancer_chinese.md index ba5568aa7a..91b044d47a 100644 --- a/docs/web_server/web_api_balancer_chinese.md +++ b/docs/web_server/web_api_balancer_chinese.md @@ -63,29 +63,26 @@ cURL 範例 curl -X POST http://localhost:8001/balancer \ -H "Content-Type: application/json" \ -d '{ - "timeout": "5s", - "balancer": "org.astraea.common.balancer.algorithms.GreedyBalancer", - "balancerConfig": { - "shuffle.tweaker.min.step": "1", - "shuffle.tweaker.max.step": "10" - }, - "clusterCosts": [ - { - "cost": "org.astraea.common.cost.ReplicaLeaderCost", - "weight": 1 - } - ], - "moveCosts": [ - "org.astraea.common.cost.ReplicaLeaderCost", - "org.astraea.common.cost.RecordSizeCost" - ], - "costConfig": { - "max.migrated.size": "500MB", - "max.migrated.leader.number": 5, - "max.broker.total.disk.space": "0:1500MB,1:1000MB,2:1500MB", - "max.broker.path.disk.space": "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:1000MB" - } - }' + "timeout": "5s", + "balancer": "org.astraea.common.balancer.algorithms.GreedyBalancer", + "balancerConfig": { + "shuffle.tweaker.min.step": 1, + "shuffle.tweaker.max.step": 10 + }, + "clusterCosts": [ + { "cost": "org.astraea.common.cost.ReplicaLeaderCost", "weight": 1 } + ], + "moveCosts": [ + "org.astraea.common.cost.ReplicaLeaderCost", + "org.astraea.common.cost.RecordSizeCost" + ], + "costConfig": { + "max.migrated.size": "500MB", + "max.migrated.leader.number": 5, + "max.broker.total.disk.space": "0:1500MB,1:1000MB,2:1500MB", + "max.broker.path.disk.space": "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:1000MB" + } + }' ``` JSON Response 範例 From a69f7d1484ff6993325b0a4dbd2071bf6723a4b6 Mon Sep 17 00:00:00 2001 From: Xiang-Jun Sun Date: Wed, 17 May 2023 17:04:23 +0800 Subject: [PATCH 09/77] [BUG] fix `ClusterCost` normalize (#1729) --- .../astraea/common/cost/BrokerInputCost.java | 2 +- .../astraea/common/cost/BrokerOutputCost.java | 2 +- .../org/astraea/common/cost/Dispersion.java | 38 +++++++------------ .../common/cost/ReplicaLeaderCost.java | 4 +- .../common/cost/ReplicaLeaderSizeCost.java | 2 +- .../common/cost/BrokerInputCostTest.java | 5 ++- .../common/cost/BrokerOutputCostTest.java | 5 ++- .../astraea/common/cost/DispersionTest.java | 33 ++++++++++++---- .../common/cost/ReplicaLeaderCostTest.java | 19 +++++++--- .../cost/ReplicaLeaderSizeCostTest.java | 2 +- 10 files changed, 66 insertions(+), 46 deletions(-) diff --git a/common/src/main/java/org/astraea/common/cost/BrokerInputCost.java b/common/src/main/java/org/astraea/common/cost/BrokerInputCost.java index 7a1dc8ff74..dc6260cce1 100644 --- a/common/src/main/java/org/astraea/common/cost/BrokerInputCost.java +++ b/common/src/main/java/org/astraea/common/cost/BrokerInputCost.java @@ -26,7 +26,7 @@ import org.astraea.common.metrics.collector.MetricSensor; public class BrokerInputCost implements HasBrokerCost, HasClusterCost { - private final Dispersion dispersion = Dispersion.cov(); + private final Dispersion dispersion = Dispersion.normalizedStandardDeviation(); @Override public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { diff --git a/common/src/main/java/org/astraea/common/cost/BrokerOutputCost.java b/common/src/main/java/org/astraea/common/cost/BrokerOutputCost.java index 355cb90be4..f91f6b91cd 100644 --- a/common/src/main/java/org/astraea/common/cost/BrokerOutputCost.java +++ b/common/src/main/java/org/astraea/common/cost/BrokerOutputCost.java @@ -25,7 +25,7 @@ import org.astraea.common.metrics.collector.MetricSensor; public class BrokerOutputCost implements HasBrokerCost, HasClusterCost { - private final Dispersion dispersion = Dispersion.cov(); + private final Dispersion dispersion = Dispersion.normalizedStandardDeviation(); @Override public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { diff --git a/common/src/main/java/org/astraea/common/cost/Dispersion.java b/common/src/main/java/org/astraea/common/cost/Dispersion.java index bc4c87e170..3d29b5ab72 100644 --- a/common/src/main/java/org/astraea/common/cost/Dispersion.java +++ b/common/src/main/java/org/astraea/common/cost/Dispersion.java @@ -21,58 +21,48 @@ /** Aggregate a sequence into a number */ @FunctionalInterface public interface Dispersion { + /** - * Apply coefficient of variation to a series of values. - * - *

This implementation come with some assumption: + * Obtain standard deviation from a series of values. * *

    - *
  • If no number was given, then the cov is zero. - *
  • If all numbers are zero, then the cov is zero. + *
  • If no number was given, then the standard deviation is zero. *
*/ - static Dispersion cov() { + static Dispersion standardDeviation() { return numbers -> { // special case: no number if (numbers.isEmpty()) return 0; var numSummary = numbers.stream().mapToDouble(Number::doubleValue).summaryStatistics(); - // special case: all value zero - if (numSummary.getMax() == 0 && numSummary.getMin() == 0) return 0; - // special case: zero average, no cov defined here - if (numSummary.getAverage() == 0) - throw new ArithmeticException( - "Coefficient of variation has no definition with zero average"); var numVariance = numbers.stream() .mapToDouble(Number::doubleValue) .map(score -> score - numSummary.getAverage()) .map(score -> score * score) .summaryStatistics() - .getAverage(); - return Math.sqrt(numVariance) / numSummary.getAverage(); + .getSum(); + return Math.sqrt(numVariance / numbers.size()); }; } /** - * Obtain standard deviation from a series of values. + * Obtain standard deviation from a series of values after normalizing to [0,1]. * *
    *
  • If no number was given, then the standard deviation is zero. *
*/ - static Dispersion standardDeviation() { + static Dispersion normalizedStandardDeviation() { return numbers -> { // special case: no number if (numbers.isEmpty()) return 0; - var numSummary = numbers.stream().mapToDouble(Number::doubleValue).summaryStatistics(); - var numVariance = + var totalNumber = numbers.stream().mapToInt(Number::intValue).sum(); + var standardDeviation = standardDeviation(); + var normalized = numbers.stream() - .mapToDouble(Number::doubleValue) - .map(score -> score - numSummary.getAverage()) - .map(score -> score * score) - .summaryStatistics() - .getSum(); - return Math.sqrt(numVariance / numbers.size()); + .map(score -> totalNumber == 0 ? 0 : score.doubleValue() / totalNumber) + .toList(); + return standardDeviation.calculate(normalized); }; } diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java index b89ce63272..fee24f5c6e 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java @@ -29,7 +29,7 @@ /** more replica leaders -> higher cost */ public class ReplicaLeaderCost implements HasBrokerCost, HasClusterCost, HasMoveCost { - private final Dispersion dispersion = Dispersion.cov(); + private final Dispersion dispersion = Dispersion.normalizedStandardDeviation(); private final Configuration config; public static final String MAX_MIGRATE_LEADER_KEY = "max.migrated.leader.number"; @@ -52,7 +52,7 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { @Override public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { var brokerScore = leaderCount(clusterInfo); - var value = dispersion.calculate(brokerScore.values()); + var value = dispersion.calculate(brokerScore.values()) * 2; return ClusterCost.of( value, () -> diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index d489834fa8..00acb46537 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -46,7 +46,7 @@ public ReplicaLeaderSizeCost(Configuration config) { this.config = config; } - private final Dispersion dispersion = Dispersion.cov(); + private final Dispersion dispersion = Dispersion.standardDeviation(); @Override public MoveCost moveCost(ClusterInfo before, ClusterInfo after, ClusterBean clusterBean) { diff --git a/common/src/test/java/org/astraea/common/cost/BrokerInputCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerInputCostTest.java index 13727c757c..8b2cc007e5 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerInputCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerInputCostTest.java @@ -53,8 +53,11 @@ void testCost() { Assertions.assertEquals(5000D, scores.get(3)); // testClusterCost + var total = scores.values().stream().mapToDouble(x -> x).sum(); + var sd = scores.values().stream().map(x -> x / total).toList(); + var dispersion = Dispersion.standardDeviation(); var clusterCost = brokerInputCost.clusterCost(ClusterInfo.empty(), clusterBean).value(); - Assertions.assertEquals(0.535, Math.round(clusterCost * 1000.0) / 1000.0); + Assertions.assertEquals(dispersion.calculate(sd), clusterCost); } @Test diff --git a/common/src/test/java/org/astraea/common/cost/BrokerOutputCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerOutputCostTest.java index 29d08043bf..ba0cb2ac65 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerOutputCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerOutputCostTest.java @@ -57,8 +57,11 @@ void testCost() { Assertions.assertEquals(5000D, scores.get(3)); // testClusterCost + var total = scores.values().stream().mapToDouble(x -> x).sum(); + var sd = scores.values().stream().map(x -> x / total).toList(); + var dispersion = Dispersion.standardDeviation(); var clusterCost = brokerOutputCost.clusterCost(ClusterInfo.empty(), clusterBean).value(); - Assertions.assertEquals(0.535, Math.round(clusterCost * 1000.0) / 1000.0); + Assertions.assertEquals(dispersion.calculate(sd), clusterCost); } @Test diff --git a/common/src/test/java/org/astraea/common/cost/DispersionTest.java b/common/src/test/java/org/astraea/common/cost/DispersionTest.java index 2bcb030cad..e2045c77fc 100644 --- a/common/src/test/java/org/astraea/common/cost/DispersionTest.java +++ b/common/src/test/java/org/astraea/common/cost/DispersionTest.java @@ -23,10 +23,10 @@ class DispersionTest { @Test - void testCorrelationCoefficient() { - var dispersion = Dispersion.cov(); - var scores = List.of(0.2, 0.4, 0.7); - Assertions.assertEquals(0.47418569253607507, dispersion.calculate(scores)); + void testStandardDeviation() { + var dispersion = Dispersion.standardDeviation(); + var scores = List.of(8, 8, 4, 4); + Assertions.assertEquals(2, dispersion.calculate(scores)); var zeroScores = List.of(0.0, 0.0, 0.0); var score = dispersion.calculate(zeroScores); @@ -35,14 +35,31 @@ void testCorrelationCoefficient() { } @Test - void standardDeviation() { - var dispersion = Dispersion.standardDeviation(); + void testNormalizedStandardDeviation() { + // test calculate var scores = List.of(8, 8, 4, 4); - Assertions.assertEquals(2, dispersion.calculate(scores)); + var normalizedSD = Dispersion.normalizedStandardDeviation(); + var standardDeviation = Dispersion.standardDeviation(); + var total = scores.stream().mapToDouble(x -> x).sum(); + var sd = scores.stream().map(x -> x / total).toList(); + Assertions.assertEquals(standardDeviation.calculate(sd), normalizedSD.calculate(scores)); + // test zero var zeroScores = List.of(0.0, 0.0, 0.0); - var score = dispersion.calculate(zeroScores); + var score = normalizedSD.calculate(zeroScores); Assertions.assertFalse(Double.isNaN(score)); Assertions.assertEquals(0.0, score); + + // the test standard deviation interval is between [0,1] + var dispersion = Dispersion.normalizedStandardDeviation(); + var list1 = List.of(0, Integer.MAX_VALUE); + var list2 = List.of(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, Integer.MAX_VALUE); + var list3 = List.of(-1 * Integer.MAX_VALUE, Integer.MAX_VALUE); + var cost1 = dispersion.calculate(list1); + var cost2 = dispersion.calculate(list2); + var cost3 = dispersion.calculate(list3); + Assertions.assertTrue(cost1 <= 1 && cost1 >= 0); + Assertions.assertTrue(cost2 <= 1 && cost2 >= 0); + Assertions.assertTrue(cost3 <= 1 && cost3 >= 0); } } diff --git a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java index 90f0a2db01..fb9be0a48f 100644 --- a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java @@ -29,7 +29,7 @@ import org.junit.jupiter.api.Test; public class ReplicaLeaderCostTest { - private final Dispersion dispersion = Dispersion.cov(); + private final Dispersion dispersion = Dispersion.standardDeviation(); @Test void testLeaderCount() { @@ -87,12 +87,14 @@ void testNoMetrics() { Replica.builder() .topic("topic") .partition(0) + .isLeader(true) .nodeInfo(NodeInfo.of(10, "broker0", 1111)) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") - .partition(0) + .partition(1) + .isLeader(true) .nodeInfo(NodeInfo.of(10, "broker0", 1111)) .path("/tmp/aa") .buildLeader(), @@ -100,6 +102,7 @@ void testNoMetrics() { .topic("topic") .partition(0) .nodeInfo(NodeInfo.of(11, "broker1", 1111)) + .isLeader(true) .path("/tmp/aa") .buildLeader()); var clusterInfo = @@ -112,14 +115,18 @@ void testNoMetrics() { Map.of(), replicas); var brokerCost = ReplicaLeaderCost.leaderCount(clusterInfo); - var clusterCost = - dispersion.calculate( - brokerCost.values().stream().map(x -> (double) x).collect(Collectors.toSet())); + var cf = new ReplicaLeaderCost(); + var leaderNum = brokerCost.values().stream().mapToInt(x -> x).sum(); + var normalizedScore = + brokerCost.values().stream() + .map(score -> (double) score / leaderNum) + .collect(Collectors.toList()); + var clusterCost = dispersion.calculate(normalizedScore) * 2; Assertions.assertTrue(brokerCost.containsKey(10)); Assertions.assertTrue(brokerCost.containsKey(11)); Assertions.assertEquals(3, brokerCost.size()); Assertions.assertTrue(brokerCost.get(10) > brokerCost.get(11)); Assertions.assertEquals(brokerCost.get(12), 0); - Assertions.assertEquals(clusterCost, 0.816496580927726); + Assertions.assertEquals(clusterCost, cf.clusterCost(clusterInfo, ClusterBean.EMPTY).value()); } } diff --git a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java index fe140a0d26..8826579dd1 100644 --- a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java @@ -39,7 +39,7 @@ static void closeService() { @Test void testClusterCost() { - final Dispersion dispersion = Dispersion.cov(); + final Dispersion dispersion = Dispersion.standardDeviation(); var loadCostFunction = new ReplicaLeaderSizeCost(); var brokerLoad = loadCostFunction.brokerCost(clusterInfo(), ClusterBean.EMPTY).value(); var clusterCost = loadCostFunction.clusterCost(clusterInfo(), ClusterBean.EMPTY).value(); From 4d0c726ccaf23984c7fdb9a8fc48c3c57fa5d16f Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 17 May 2023 18:45:21 +0800 Subject: [PATCH 10/77] [PRODUCER] rewrite proudcer.Metadata by java 17 record (#1751) --- .../org/astraea/common/producer/Metadata.java | 94 +++++-------------- 1 file changed, 26 insertions(+), 68 deletions(-) diff --git a/common/src/main/java/org/astraea/common/producer/Metadata.java b/common/src/main/java/org/astraea/common/producer/Metadata.java index ad013a3a85..2f2fb5afc7 100644 --- a/common/src/main/java/org/astraea/common/producer/Metadata.java +++ b/common/src/main/java/org/astraea/common/producer/Metadata.java @@ -19,73 +19,31 @@ import java.util.Optional; import org.apache.kafka.clients.producer.RecordMetadata; -public interface Metadata { - - static Metadata of(RecordMetadata metadata) { - return new Metadata() { - @Override - public long offset() { - return metadata.offset(); - } - - @Override - public int serializedKeySize() { - return metadata.serializedKeySize(); - } - - @Override - public int serializedValueSize() { - return metadata.serializedValueSize(); - } - - @Override - public Optional timestamp() { - return metadata.hasTimestamp() ? Optional.of(metadata.timestamp()) : Optional.empty(); - } - - @Override - public String topic() { - return metadata.topic(); - } - - @Override - public int partition() { - return metadata.partition(); - } - }; +/** + * @param topic The topic the record was appended to + * @param partition The partition the record was sent to + * @param offset The offset of the record in the topic/partition. + * @param serializedKeySize The size of the serialized, uncompressed key in bytes. If key is null, + * the returned size is -1. + * @param serializedValueSize The size of the serialized, uncompressed value in bytes. If value is + * null, the returned size is -1. + * @param timestamp the timestamp of the record + */ +public record Metadata( + String topic, + int partition, + long offset, + int serializedKeySize, + int serializedValueSize, + Optional timestamp) { + + public static Metadata of(RecordMetadata metadata) { + return new Metadata( + metadata.topic(), + metadata.partition(), + metadata.offset(), + metadata.serializedKeySize(), + metadata.serializedValueSize(), + metadata.hasTimestamp() ? Optional.of(metadata.timestamp()) : Optional.empty()); } - - /** - * The offset of the record in the topic/partition. - * - * @return the offset of the record, or -1 - */ - long offset(); - - /** - * @return The size of the serialized, uncompressed key in bytes. If key is null, the returned - * size is -1. - */ - int serializedKeySize(); - - /** - * @return The size of the serialized, uncompressed value in bytes. If value is null, the returned - * size is -1. - */ - int serializedValueSize(); - - /** - * @return the timestamp of the record - */ - Optional timestamp(); - - /** - * @return The topic the record was appended to - */ - String topic(); - - /** - * @return The partition the record was sent to - */ - int partition(); } From 94aa36a30402ad172bd3809ed6686f1a30fd64ce Mon Sep 17 00:00:00 2001 From: Haser Date: Wed, 17 May 2023 23:37:40 +0800 Subject: [PATCH 11/77] [CONNECTOR] rewrite sink connector by java 17 toList (#1754) --- .../src/main/java/org/astraea/connector/SinkConnector.java | 5 +---- connector/src/main/java/org/astraea/connector/SinkTask.java | 3 +-- .../src/main/java/org/astraea/connector/backup/Exporter.java | 3 +-- .../src/main/java/org/astraea/connector/perf/PerfSink.java | 3 +-- 4 files changed, 4 insertions(+), 10 deletions(-) diff --git a/connector/src/main/java/org/astraea/connector/SinkConnector.java b/connector/src/main/java/org/astraea/connector/SinkConnector.java index e5c30be7c8..15b76dbe9d 100644 --- a/connector/src/main/java/org/astraea/connector/SinkConnector.java +++ b/connector/src/main/java/org/astraea/connector/SinkConnector.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import org.apache.kafka.common.config.ConfigDef; import org.astraea.common.Configuration; import org.astraea.common.VersionUtils; @@ -53,9 +52,7 @@ public final Class taskClass() @Override public List> taskConfigs(int maxTasks) { - return takeConfiguration(maxTasks).stream() - .map(Configuration::raw) - .collect(Collectors.toList()); + return takeConfiguration(maxTasks).stream().map(Configuration::raw).toList(); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SinkTask.java b/connector/src/main/java/org/astraea/connector/SinkTask.java index 9d83dc4578..711d7d9a1c 100644 --- a/connector/src/main/java/org/astraea/connector/SinkTask.java +++ b/connector/src/main/java/org/astraea/connector/SinkTask.java @@ -21,7 +21,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.header.Headers; @@ -57,7 +56,7 @@ public final void start(Map props) { @Override public final void put(Collection records) { if (records != null && !records.isEmpty()) - put(records.stream().map(SinkTask::toRecord).collect(Collectors.toList())); + put(records.stream().map(SinkTask::toRecord).toList()); } private static byte[] toBytes(Schema schema, Object value) { diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 776dd15a1a..6d4128d633 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -26,7 +26,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.LongAdder; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.Configuration; @@ -128,7 +127,7 @@ protected Class task() { @Override protected List takeConfiguration(int maxTasks) { - return IntStream.range(0, maxTasks).mapToObj(ignored -> configs).collect(Collectors.toList()); + return IntStream.range(0, maxTasks).mapToObj(ignored -> configs).toList(); } @Override diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java index 88603f25af..6ad2760ed6 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java @@ -18,7 +18,6 @@ import java.time.Duration; import java.util.List; -import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.Configuration; import org.astraea.common.Utils; @@ -50,7 +49,7 @@ protected Class task() { @Override protected List takeConfiguration(int maxTasks) { - return IntStream.range(0, maxTasks).mapToObj(i -> config).collect(Collectors.toList()); + return IntStream.range(0, maxTasks).mapToObj(i -> config).toList(); } @Override From 12bcb0ba8b85a173b72a416acc23c84dc9293732 Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Thu, 18 May 2023 02:18:11 +0800 Subject: [PATCH 12/77] apply toList under app/src/main. (#1755) --- .../java/org/astraea/app/backup/Backup.java | 2 +- .../app/performance/ConsumerThread.java | 4 +--- .../app/performance/DataGenerator.java | 7 ++----- .../astraea/app/performance/MonkeyThread.java | 3 +-- .../astraea/app/performance/Performance.java | 19 ++++++------------- .../app/performance/ProducerThread.java | 4 +--- .../org/astraea/app/web/BalancerHandler.java | 2 +- .../java/org/astraea/app/web/BeanHandler.java | 11 ++++------- .../org/astraea/app/web/BrokerHandler.java | 4 ++-- .../java/org/astraea/app/web/Channel.java | 2 +- .../org/astraea/app/web/GroupHandler.java | 7 +++---- .../org/astraea/app/web/ProducerHandler.java | 5 ++--- .../org/astraea/app/web/QuotaHandler.java | 2 +- .../astraea/app/web/ReassignmentHandler.java | 8 +++----- .../org/astraea/app/web/TopicHandler.java | 6 +++--- .../astraea/app/web/TransactionHandler.java | 4 +--- 16 files changed, 33 insertions(+), 57 deletions(-) diff --git a/app/src/main/java/org/astraea/app/backup/Backup.java b/app/src/main/java/org/astraea/app/backup/Backup.java index 0becc9f4cd..4b92ee152c 100644 --- a/app/src/main/java/org/astraea/app/backup/Backup.java +++ b/app/src/main/java/org/astraea/app/backup/Backup.java @@ -51,7 +51,7 @@ public void restoreDistribution(ClusterInfo clusterInfo, String bootstrapServers Comparator.comparing( replica -> !replica.isLeader())) .map(replica -> replica.nodeInfo().id()) - .collect(Collectors.toUnmodifiableList())))))) + .toList()))))) .configs(topic.config().raw()) .run() .toCompletableFuture() diff --git a/app/src/main/java/org/astraea/app/performance/ConsumerThread.java b/app/src/main/java/org/astraea/app/performance/ConsumerThread.java index 3b65aec33b..2f268adb80 100644 --- a/app/src/main/java/org/astraea/app/performance/ConsumerThread.java +++ b/app/src/main/java/org/astraea/app/performance/ConsumerThread.java @@ -73,9 +73,7 @@ static List create( consumerSupplier) { if (consumers == 0) return List.of(); var closeLatches = - IntStream.range(0, consumers) - .mapToObj(ignored -> new CountDownLatch(1)) - .collect(Collectors.toUnmodifiableList()); + IntStream.range(0, consumers).mapToObj(ignored -> new CountDownLatch(1)).toList(); var executors = Executors.newFixedThreadPool(consumers); // monitor CompletableFuture.runAsync( diff --git a/app/src/main/java/org/astraea/app/performance/DataGenerator.java b/app/src/main/java/org/astraea/app/performance/DataGenerator.java index 7b73729513..1e28b650e2 100644 --- a/app/src/main/java/org/astraea/app/performance/DataGenerator.java +++ b/app/src/main/java/org/astraea/app/performance/DataGenerator.java @@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; -import java.util.stream.Collectors; import java.util.stream.LongStream; import org.astraea.common.Configuration; import org.astraea.common.DataUnit; @@ -49,15 +48,13 @@ static DataGenerator of( RecordGenerator.builder() .batchSize(argument.transactionSize) .keyTableSeed(argument.recordKeyTableSeed) - .keyRange( - LongStream.rangeClosed(0, 10000).boxed().collect(Collectors.toUnmodifiableList())) + .keyRange(LongStream.rangeClosed(0, 10000).boxed().toList()) .keyDistribution(argument.keyDistributionType.create(10000, keyDistConfig)) .keySizeDistribution( argument.keySizeDistributionType.create( (int) argument.keySize.bytes(), keySizeDistConfig)) .valueTableSeed(argument.recordValueTableSeed) - .valueRange( - LongStream.rangeClosed(0, 10000).boxed().collect(Collectors.toUnmodifiableList())) + .valueRange(LongStream.rangeClosed(0, 10000).boxed().toList()) .valueDistribution(argument.valueDistributionType.create(10000, valueDistConfig)) .valueSizeDistribution( argument.valueDistributionType.create( diff --git a/app/src/main/java/org/astraea/app/performance/MonkeyThread.java b/app/src/main/java/org/astraea/app/performance/MonkeyThread.java index d7908ec6e7..5a7d573615 100644 --- a/app/src/main/java/org/astraea/app/performance/MonkeyThread.java +++ b/app/src/main/java/org/astraea/app/performance/MonkeyThread.java @@ -22,7 +22,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.consumer.Consumer; import org.astraea.common.consumer.ConsumerConfigs; @@ -52,7 +51,7 @@ static List play(List consumerThreads, Performance return unsubscribeMonkey(consumerThreads, entry.getValue()); } }) - .collect(Collectors.toUnmodifiableList()); + .toList(); } private static MonkeyThread killMonkey(List consumerThreads, Duration frequency) { diff --git a/app/src/main/java/org/astraea/app/performance/Performance.java b/app/src/main/java/org/astraea/app/performance/Performance.java index 07067472e0..f3a8b3f6d1 100644 --- a/app/src/main/java/org/astraea/app/performance/Performance.java +++ b/app/src/main/java/org/astraea/app/performance/Performance.java @@ -76,7 +76,7 @@ public static List execute(final Argument param) { var blockingQueues = IntStream.range(0, param.producers) .mapToObj(i -> new ArrayBlockingQueue>>(3000)) - .collect(Collectors.toUnmodifiableList()); + .toList(); // ensure topics are existent System.out.println("checking topics: " + String.join(",", param.topics)); param.checkTopics(); @@ -123,10 +123,7 @@ public static List execute(final Argument param) { var current = Report.recordsConsumedTotal(); if (blockingQueues.stream().allMatch(Collection::isEmpty)) { - var unfinishedProducers = - producerThreads.stream() - .filter(p -> !p.closed()) - .collect(Collectors.toUnmodifiableList()); + var unfinishedProducers = producerThreads.stream().filter(p -> !p.closed()).toList(); unfinishedProducers.forEach(AbstractThread::close); } @@ -388,7 +385,7 @@ else if (specifiedByBroker) { .filter(replica -> specifyBrokers.contains(replica.nodeInfo().id())) .map(replica -> TopicPartition.of(replica.topic(), replica.partition())) .distinct() - .collect(Collectors.toUnmodifiableList()); + .toList(); if (selections.isEmpty()) throw new IllegalArgumentException( "No partition match the specify.brokers requirement"); @@ -426,8 +423,7 @@ else if (specifiedByBroker) { "The following topic/partitions are nonexistent in the cluster: " + notExist); } - final var selection = - specifyPartitions.stream().distinct().collect(Collectors.toUnmodifiableList()); + final var selection = specifyPartitions.stream().distinct().toList(); return () -> selection.get(ThreadLocalRandom.current().nextInt(selection.size())); } else if (throttle) { // TODO: The functions of throttle and select partitioner should not conflict with each @@ -444,15 +440,12 @@ else if (specifiedByBroker) { .replicaStream() .map(Replica::topicPartition) .distinct() - .collect(Collectors.toUnmodifiableList()); + .toList(); return () -> selection.get(ThreadLocalRandom.current().nextInt(selection.size())); } } else { final var selection = - topics.stream() - .map(topic -> TopicPartition.of(topic, -1)) - .distinct() - .collect(Collectors.toUnmodifiableList()); + topics.stream().map(topic -> TopicPartition.of(topic, -1)).distinct().toList(); return () -> selection.get(ThreadLocalRandom.current().nextInt(selection.size())); } } diff --git a/app/src/main/java/org/astraea/app/performance/ProducerThread.java b/app/src/main/java/org/astraea/app/performance/ProducerThread.java index 1218c99055..777e50aace 100644 --- a/app/src/main/java/org/astraea/app/performance/ProducerThread.java +++ b/app/src/main/java/org/astraea/app/performance/ProducerThread.java @@ -51,9 +51,7 @@ static List create( var producers = queues.size(); if (producers <= 0) return List.of(); var closeLatches = - IntStream.range(0, producers) - .mapToObj(ignored -> new CountDownLatch(1)) - .collect(Collectors.toUnmodifiableList()); + IntStream.range(0, producers).mapToObj(ignored -> new CountDownLatch(1)).toList(); var executors = Executors.newFixedThreadPool(producers); // monitor CompletableFuture.runAsync( diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 9fbd533f36..ae12056b8e 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -179,7 +179,7 @@ private PlanExecutionProgress progress(String taskId) { tp -> Change.from( contextCluster.replicas(tp), solution.proposal().replicas(tp))) - .collect(Collectors.toUnmodifiableList()); + .toList(); var report = (Supplier) () -> diff --git a/app/src/main/java/org/astraea/app/web/BeanHandler.java b/app/src/main/java/org/astraea/app/web/BeanHandler.java index ed88814c60..473640c648 100644 --- a/app/src/main/java/org/astraea/app/web/BeanHandler.java +++ b/app/src/main/java/org/astraea/app/web/BeanHandler.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.concurrent.CompletionStage; import java.util.function.Function; -import java.util.stream.Collectors; import org.astraea.common.admin.Admin; import org.astraea.common.metrics.BeanObject; import org.astraea.common.metrics.BeanQuery; @@ -48,12 +47,10 @@ public CompletionStage get(Channel channel) { try (var client = JndiClient.of(b.host(), jmxPorts.apply(b.id()))) { return new NodeBean( b.host(), - client.beans(builder.build()).stream() - .map(Bean::new) - .collect(Collectors.toUnmodifiableList())); + client.beans(builder.build()).stream().map(Bean::new).toList()); } }) - .collect(Collectors.toUnmodifiableList()))); + .toList())); } static class Property implements Response { @@ -86,11 +83,11 @@ static class Bean implements Response { this.properties = obj.properties().entrySet().stream() .map(e -> new Property(e.getKey(), e.getValue())) - .collect(Collectors.toUnmodifiableList()); + .toList(); this.attributes = obj.attributes().entrySet().stream() .map(e -> new Attribute(e.getKey(), e.getValue().toString())) - .collect(Collectors.toUnmodifiableList()); + .toList(); } } diff --git a/app/src/main/java/org/astraea/app/web/BrokerHandler.java b/app/src/main/java/org/astraea/app/web/BrokerHandler.java index af92a5b7b6..849fb4d333 100644 --- a/app/src/main/java/org/astraea/app/web/BrokerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BrokerHandler.java @@ -64,7 +64,7 @@ public CompletionStage get(Channel channel) { brokers.stream() .filter(b -> ids.contains(b.id())) .map(Broker::new) - .collect(Collectors.toList()))) + .toList())) .thenApply( brokers -> { if (brokers.isEmpty()) throw new NoSuchElementException("no brokers are found"); @@ -96,7 +96,7 @@ static class Broker implements Response { .entrySet() .stream() .map(e -> new Topic(e.getKey(), e.getValue().size())) - .collect(Collectors.toUnmodifiableList()); + .toList(); this.configs = broker.config().raw(); } } diff --git a/app/src/main/java/org/astraea/app/web/Channel.java b/app/src/main/java/org/astraea/app/web/Channel.java index e772b303ab..c0b071fd40 100644 --- a/app/src/main/java/org/astraea/app/web/Channel.java +++ b/app/src/main/java/org/astraea/app/web/Channel.java @@ -159,7 +159,7 @@ static Channel of(HttpExchange exchange) { Arrays.stream(uri.getPath().split("/")) .map(String::trim) .filter(s -> !s.isEmpty()) - .collect(Collectors.toUnmodifiableList()); + .toList(); // form: /resource/target if (allPaths.size() == 1) return Optional.empty(); else if (allPaths.size() == 2) return Optional.of(allPaths.get(1)); diff --git a/app/src/main/java/org/astraea/app/web/GroupHandler.java b/app/src/main/java/org/astraea/app/web/GroupHandler.java index acbe3ec7cb..c5ec35757f 100644 --- a/app/src/main/java/org/astraea/app/web/GroupHandler.java +++ b/app/src/main/java/org/astraea/app/web/GroupHandler.java @@ -144,10 +144,9 @@ public CompletionStage get(Channel channel) { empty()) .filter(Optional::isPresent) .map(Optional::get) - .collect( - Collectors.toUnmodifiableList()))) - .collect(Collectors.toUnmodifiableList()))) - .collect(Collectors.toUnmodifiableList()))) + .toList())) + .toList())) + .toList())) .thenApply( groups -> { if (channel.target().isPresent() && groups.size() == 1) return groups.get(0); diff --git a/app/src/main/java/org/astraea/app/web/ProducerHandler.java b/app/src/main/java/org/astraea/app/web/ProducerHandler.java index ab6620418c..0419bbb1c5 100644 --- a/app/src/main/java/org/astraea/app/web/ProducerHandler.java +++ b/app/src/main/java/org/astraea/app/web/ProducerHandler.java @@ -60,7 +60,7 @@ public CompletionStage get(Channel channel) { .entrySet() .stream() .map(e -> new Partition(e.getKey(), e.getValue())) - .collect(Collectors.toUnmodifiableList()))); + .toList())); } static class ProducerState implements Response { @@ -88,8 +88,7 @@ static class Partition implements Response { Collection states) { this.topic = tp.topic(); this.partition = tp.partition(); - this.states = - states.stream().map(ProducerState::new).collect(Collectors.toUnmodifiableList()); + this.states = states.stream().map(ProducerState::new).toList(); } } diff --git a/app/src/main/java/org/astraea/app/web/QuotaHandler.java b/app/src/main/java/org/astraea/app/web/QuotaHandler.java index 3c292d492c..828dda20d0 100644 --- a/app/src/main/java/org/astraea/app/web/QuotaHandler.java +++ b/app/src/main/java/org/astraea/app/web/QuotaHandler.java @@ -167,7 +167,7 @@ static class Quotas implements Response { final List quotas; Quotas(Collection quotas) { - this.quotas = quotas.stream().map(Quota::new).collect(Collectors.toUnmodifiableList()); + this.quotas = quotas.stream().map(Quota::new).toList(); } } diff --git a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java index 05312ee909..fee8c4823e 100644 --- a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java +++ b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java @@ -90,9 +90,7 @@ public CompletionStage post(Channel channel) { if (excludedBroker.isEmpty()) return CompletableFuture.completedFuture(Response.BAD_REQUEST); var availableBrokers = - brokers.stream() - .filter(b -> b.id() != exclude) - .collect(Collectors.toList()); + brokers.stream().filter(b -> b.id() != exclude).toList(); var partitions = excludedBroker.get().topicPartitions().stream() .filter( @@ -131,7 +129,7 @@ public CompletionStage post(Channel channel) { Stream.of(process2Folders, process2Nodes, processExclude) .flatMap(Function.identity()) .map(CompletionStage::toCompletableFuture) - .collect(Collectors.toUnmodifiableList())) + .toList()) .thenApply( rs -> { if (!rs.isEmpty() && rs.stream().allMatch(r -> r == Response.ACCEPT)) @@ -161,7 +159,7 @@ public CompletionStage get(Channel channel) { r -> new AddingReplica( r, leaderSizes.getOrDefault(r.topicPartition(), 0L))) - .collect(Collectors.toUnmodifiableList())); + .toList()); }); } diff --git a/app/src/main/java/org/astraea/app/web/TopicHandler.java b/app/src/main/java/org/astraea/app/web/TopicHandler.java index d4320a3033..34243bcb17 100644 --- a/app/src/main/java/org/astraea/app/web/TopicHandler.java +++ b/app/src/main/java/org/astraea/app/web/TopicHandler.java @@ -116,7 +116,7 @@ private CompletionStage get( .filter(replica -> replica.topic().equals(p.topic())) .filter(replica -> replica.partition() == p.partition()) .map(Replica::new) - .collect(Collectors.toUnmodifiableList())), + .toList()), Collectors.toList()))); // topic name -> group ids var gs = @@ -149,7 +149,7 @@ private CompletionStage get( gs.getOrDefault(topic.name(), Set.of()), ps.get(topic.name()), topic.config().raw())) - .collect(Collectors.toUnmodifiableList())); + .toList()); }); } @@ -197,7 +197,7 @@ public CompletionStage post(Channel channel) { new Topics( topicNames.stream() .map(t -> new TopicInfo(t, Set.of(), List.of(), Map.of())) - .collect(Collectors.toUnmodifiableList()))); + .toList())); } @Override diff --git a/app/src/main/java/org/astraea/app/web/TransactionHandler.java b/app/src/main/java/org/astraea/app/web/TransactionHandler.java index 837ea0c62b..ec45a81fca 100644 --- a/app/src/main/java/org/astraea/app/web/TransactionHandler.java +++ b/app/src/main/java/org/astraea/app/web/TransactionHandler.java @@ -49,9 +49,7 @@ public CompletionStage get(Channel channel) { .thenCompose(admin::transactions) .thenApply( transactions -> - transactions.stream() - .map(t -> new Transaction(t.transactionId(), t)) - .collect(Collectors.toUnmodifiableList())) + transactions.stream().map(t -> new Transaction(t.transactionId(), t)).toList()) .thenApply( transactions -> { if (channel.target().isPresent() && transactions.size() == 1) From 126c5e6f2118139857614876d340a8a423f88307 Mon Sep 17 00:00:00 2001 From: Haser Date: Thu, 18 May 2023 06:52:56 +0800 Subject: [PATCH 13/77] [CONNECTOR] rewrite connect source by java 17 record and toList (#1756) --- .../astraea/connector/SourceConnector.java | 5 +--- .../org/astraea/connector/SourceTask.java | 30 ++----------------- .../astraea/connector/backup/Importer.java | 3 +- 3 files changed, 5 insertions(+), 33 deletions(-) diff --git a/connector/src/main/java/org/astraea/connector/SourceConnector.java b/connector/src/main/java/org/astraea/connector/SourceConnector.java index 51de0da065..e2cd3cc9df 100644 --- a/connector/src/main/java/org/astraea/connector/SourceConnector.java +++ b/connector/src/main/java/org/astraea/connector/SourceConnector.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import org.apache.kafka.common.config.ConfigDef; import org.astraea.common.Configuration; import org.astraea.common.VersionUtils; @@ -51,9 +50,7 @@ public final Class taskCla @Override public List> taskConfigs(int maxTasks) { - return takeConfiguration(maxTasks).stream() - .map(Configuration::raw) - .collect(Collectors.toList()); + return takeConfiguration(maxTasks).stream().map(Configuration::raw).toList(); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SourceTask.java b/connector/src/main/java/org/astraea/connector/SourceTask.java index a2de195483..f82316cb68 100644 --- a/connector/src/main/java/org/astraea/connector/SourceTask.java +++ b/connector/src/main/java/org/astraea/connector/SourceTask.java @@ -76,7 +76,7 @@ public final List poll() r.headers().stream() .map(h -> new HeaderImpl(h.key(), null, h.value())) .collect(Collectors.toList()))) - .collect(Collectors.toList()); + .toList(); } @Override @@ -92,32 +92,8 @@ public final void commitRecord( commit(Metadata.of(metadata)); } - private static class HeaderImpl implements org.apache.kafka.connect.header.Header { - - private final String key; - private final Schema schema; - private final Object value; - - private HeaderImpl(String key, Schema schema, Object value) { - this.key = key; - this.schema = schema; - this.value = value; - } - - @Override - public String key() { - return key; - } - - @Override - public Schema schema() { - return schema; - } - - @Override - public Object value() { - return value; - } + private record HeaderImpl(String key, Schema schema, Object value) + implements org.apache.kafka.connect.header.Header { @Override public org.apache.kafka.connect.header.Header with(Schema schema, Object value) { diff --git a/connector/src/main/java/org/astraea/connector/backup/Importer.java b/connector/src/main/java/org/astraea/connector/backup/Importer.java index 31ee09d507..f8d34cb8da 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Importer.java +++ b/connector/src/main/java/org/astraea/connector/backup/Importer.java @@ -26,7 +26,6 @@ import java.util.Optional; import java.util.Set; import java.util.function.Predicate; -import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.Configuration; import org.astraea.common.Utils; @@ -117,7 +116,7 @@ protected List takeConfiguration(int maxTasks) { taskMap.put(TASKS_COUNT_KEY, String.valueOf(maxTasks)); return Configuration.of(taskMap); }) - .collect(Collectors.toList()); + .toList(); } @Override From 63e5ee407a2ddda69e581193b4d7f81bb8de2b17 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 18 May 2023 06:53:11 +0800 Subject: [PATCH 14/77] [CONNECTOR] rewrite connector.Definition by java 17 record (#1753) --- .../org/astraea/connector/Definition.java | 96 ++++++++----------- .../astraea/connector/backup/Exporter.java | 34 ++++--- .../astraea/connector/backup/Importer.java | 8 +- .../org/astraea/connector/perf/PerfSink.java | 7 +- .../astraea/connector/perf/PerfSource.java | 89 +++++++++-------- .../org/astraea/connector/DefinitionTest.java | 1 + 6 files changed, 113 insertions(+), 122 deletions(-) diff --git a/connector/src/main/java/org/astraea/connector/Definition.java b/connector/src/main/java/org/astraea/connector/Definition.java index 800b3a36c0..97cf2ce086 100644 --- a/connector/src/main/java/org/astraea/connector/Definition.java +++ b/connector/src/main/java/org/astraea/connector/Definition.java @@ -18,13 +18,32 @@ import java.util.Collection; import java.util.Objects; +import java.util.Optional; import java.util.function.BiConsumer; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; -public interface Definition { +public record Definition( + String name, + Optional defaultValue, + String documentation, + Type type, + BiConsumer validator) { + + @Override + public Optional defaultValue() { + // ConfigDef.NO_DEFAULT_VALUE is a placeholder used to represent the lack of a default value. + return defaultValue.filter(v -> v != ConfigDef.NO_DEFAULT_VALUE); + } + + /** + * @return true if the configuration is required, and it has no default value. + */ + public boolean required() { + return defaultValue.filter(v -> v == ConfigDef.NO_DEFAULT_VALUE).isPresent(); + } - static Builder builder() { + public static Builder builder() { return new Builder(); } @@ -35,32 +54,20 @@ static ConfigDef toConfigDef(Collection defs) { def.define( d.name(), ConfigDef.Type.valueOf(d.type().name()), - d.defaultValue(), - d.validator() == null - ? null - : (n, o) -> { - try { - d.validator().accept(n, o); - } catch (Exception e) { - throw new ConfigException(n, o, e.getMessage()); - } - }, + d.required() ? ConfigDef.NO_DEFAULT_VALUE : d.defaultValue().orElse(null), + (n, o) -> { + try { + d.validator().accept(n, o); + } catch (Exception e) { + throw new ConfigException(n, o, e.getMessage()); + } + }, ConfigDef.Importance.MEDIUM, d.documentation())); return def; } - String name(); - - Object defaultValue(); - - String documentation(); - - Type type(); - - BiConsumer validator(); - - enum Type { + public enum Type { BOOLEAN, STRING, INT, @@ -72,14 +79,14 @@ enum Type { PASSWORD } - class Builder { + public static class Builder { private String name; private Object defaultValue; private String documentation = ""; private Type type = Type.STRING; - private BiConsumer validator; + private BiConsumer validator = (l, h) -> {}; private Builder() {} @@ -113,39 +120,12 @@ public Builder validator(BiConsumer validator) { } public Definition build() { - return new Definition() { - private final String name = Objects.requireNonNull(Builder.this.name); - private final String documentation = Objects.requireNonNull(Builder.this.documentation); - private final Object defaultValue = Builder.this.defaultValue; - private final Type type = Objects.requireNonNull(Builder.this.type); - - private final BiConsumer validator = Builder.this.validator; - - @Override - public String name() { - return name; - } - - @Override - public Object defaultValue() { - return defaultValue; - } - - @Override - public String documentation() { - return documentation; - } - - @Override - public Type type() { - return type; - } - - @Override - public BiConsumer validator() { - return validator; - } - }; + return new Definition( + Objects.requireNonNull(name), + Optional.ofNullable(defaultValue), + Objects.requireNonNull(documentation), + Objects.requireNonNull(type), + validator); } } } diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 6d4128d633..4448e750b9 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -16,6 +16,7 @@ */ package org.astraea.connector.backup; +import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -79,21 +80,25 @@ public class Exporter extends SinkConnector { .documentation("the path required for file storage.") .required() .build(); + + static DataSize SIZE_DEFAULT = DataSize.MB.of(100); static Definition SIZE_KEY = Definition.builder() .name("size") .type(Definition.Type.STRING) .validator((name, obj) -> DataSize.of(obj.toString())) - .defaultValue("100MB") + .defaultValue(SIZE_DEFAULT.toString()) .documentation("is the maximum number of the size will be included in each file.") .build(); + static Duration TIME_DEFAULT = Duration.ofSeconds(3); + static Definition TIME_KEY = Definition.builder() .name("roll.duration") .type(Definition.Type.STRING) .validator((name, obj) -> Utils.toDuration(obj.toString())) - .defaultValue("3s") + .defaultValue(TIME_DEFAULT.toSeconds() + "s") .documentation("the maximum time before a new archive file is rolling out.") .build(); @@ -104,6 +109,8 @@ public class Exporter extends SinkConnector { .documentation("a value that needs to be overridden in the file system.") .build(); + static DataSize BUFFER_SIZE_DEFAULT = DataSize.MB.of(300); + static Definition BUFFER_SIZE_KEY = Definition.builder() .name("writer.buffer.size") @@ -111,7 +118,7 @@ public class Exporter extends SinkConnector { .validator((name, obj) -> DataSize.of(obj.toString())) .documentation( "a value that represents the capacity of a blocking queue from which the writer can take records.") - .defaultValue("300MB") + .defaultValue(BUFFER_SIZE_DEFAULT.toString()) .build(); private Configuration configs; @@ -254,23 +261,20 @@ List> recordsFromBuffer() { protected void init(Configuration configuration) { this.topicName = configuration.requireString(TOPICS_KEY); this.path = configuration.requireString(PATH_KEY.name()); - this.size = - DataSize.of( - configuration.string(SIZE_KEY.name()).orElse(SIZE_KEY.defaultValue().toString())); + this.size = configuration.string(SIZE_KEY.name()).map(DataSize::of).orElse(SIZE_DEFAULT); this.interval = - Utils.toDuration( - configuration.string(TIME_KEY.name()).orElse(TIME_KEY.defaultValue().toString())) + configuration + .string(TIME_KEY.name()) + .map(Utils::toDuration) + .orElse(TIME_DEFAULT) .toMillis(); - this.bufferSize.reset(); - this.bufferSizeLimit = - DataSize.of( - configuration - .string(BUFFER_SIZE_KEY.name()) - .orElse(BUFFER_SIZE_KEY.defaultValue().toString())) + configuration + .string(BUFFER_SIZE_KEY.name()) + .map(DataSize::of) + .orElse(BUFFER_SIZE_DEFAULT) .bytes(); - this.fs = FileSystem.of(configuration.requireString(SCHEMA_KEY.name()), configuration); this.writerFuture = CompletableFuture.runAsync(createWriter()); } diff --git a/connector/src/main/java/org/astraea/connector/backup/Importer.java b/connector/src/main/java/org/astraea/connector/backup/Importer.java index f8d34cb8da..7f3741655a 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Importer.java +++ b/connector/src/main/java/org/astraea/connector/backup/Importer.java @@ -78,11 +78,12 @@ public class Importer extends SourceConnector { .documentation("The root directory of the file that needs to be imported.") .required() .build(); + static String CLEAN_SOURCE_DEFAULT = "off"; static Definition CLEAN_SOURCE_KEY = Definition.builder() .name("clean.source") .type(Definition.Type.STRING) - .defaultValue("off") + .defaultValue(CLEAN_SOURCE_DEFAULT) .documentation( "Clean source policy. Available policies: \"off\", \"delete\", \"archive\". Default: off") .build(); @@ -149,10 +150,7 @@ protected void init(Configuration configuration, MetadataStorage storage) { this.rootDir = configuration.requireString(PATH_KEY.name()); this.tasksCount = configuration.requireInteger(TASKS_COUNT_KEY); this.paths = new LinkedList<>(); - this.cleanSource = - configuration - .string(CLEAN_SOURCE_KEY.name()) - .orElse(CLEAN_SOURCE_KEY.defaultValue().toString()); + this.cleanSource = configuration.string(CLEAN_SOURCE_KEY.name()).orElse(CLEAN_SOURCE_DEFAULT); this.archiveDir = configuration.string(ARCHIVE_DIR_KEY.name()); } diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java index 6ad2760ed6..898b1b3f78 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java @@ -27,11 +27,14 @@ import org.astraea.connector.SinkTask; public class PerfSink extends SinkConnector { + + static Duration FREQUENCY_DEFAULT = Duration.ofMillis(300); + static Definition FREQUENCY_DEF = Definition.builder() .name("frequency") .type(Definition.Type.STRING) - .defaultValue("300ms") + .defaultValue(FREQUENCY_DEFAULT.toMillis() + "ms") .validator((name, value) -> Utils.toDuration(value.toString())) .build(); @@ -59,7 +62,7 @@ protected List definitions() { public static class Task extends SinkTask { - private Duration frequency = Utils.toDuration(FREQUENCY_DEF.defaultValue().toString()); + private Duration frequency = FREQUENCY_DEFAULT; private volatile long lastPut = System.currentTimeMillis(); diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java index a788ad4b79..0a5ff86f00 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java @@ -37,79 +37,97 @@ import org.astraea.connector.SourceTask; public class PerfSource extends SourceConnector { + + static DataSize THROUGHPUT_DEFAULT = DataSize.GB.of(100); + static Definition THROUGHPUT_DEF = Definition.builder() .name("throughput") .type(Definition.Type.STRING) - .defaultValue("100GB") + .defaultValue(THROUGHPUT_DEFAULT.toString()) .validator((name, value) -> DataSize.of(value.toString())) .documentation("the data rate (in second) of sending records") .build(); + static DistributionType KEY_DISTRIBUTION_DEFAULT = DistributionType.UNIFORM; + static Definition KEY_DISTRIBUTION_DEF = Definition.builder() .name("key.distribution") .type(Definition.Type.STRING) .validator((name, obj) -> DistributionType.ofAlias(obj.toString())) - .defaultValue(DistributionType.UNIFORM.alias()) + .defaultValue(KEY_DISTRIBUTION_DEFAULT.alias()) .documentation( "Distribution name for key and key size. Available distribution names: \"fixed\" \"uniform\", \"zipfian\", \"latest\". Default: uniform") .build(); + + static DistributionType KEY_SIZE_DISTRIBUTION_DEFAULT = DistributionType.FIXED; static Definition KEY_SIZE_DISTRIBUTION_DEF = Definition.builder() .name("key.size.distribution") .type(Definition.Type.STRING) .validator((name, obj) -> DistributionType.ofAlias(obj.toString())) - .defaultValue(DistributionType.FIXED.alias()) + .defaultValue(KEY_SIZE_DISTRIBUTION_DEFAULT.alias()) .documentation( "Distribution name for key size. Available distribution names: \"fixed\" \"uniform\", \"zipfian\", \"latest\". Default: fixed") .build(); + + static DataSize KEY_SIZE_DEFAULT = DataSize.Byte.of(50); + static Definition KEY_SIZE_DEF = Definition.builder() .name("key.size") .type(Definition.Type.STRING) .validator((name, obj) -> DataSize.of(obj.toString())) - .defaultValue(DataSize.Byte.of(50).toString()) + .defaultValue(KEY_SIZE_DEFAULT.toString()) .documentation( "the max length of key. The distribution of length is defined by " + KEY_DISTRIBUTION_DEF.name()) .build(); + static DistributionType VALUE_DISTRIBUTION_DEFAULT = DistributionType.UNIFORM; + static Definition VALUE_DISTRIBUTION_DEF = Definition.builder() .name("value.distribution") .type(Definition.Type.STRING) .validator((name, obj) -> DistributionType.ofAlias(obj.toString())) - .defaultValue(DistributionType.UNIFORM.alias()) + .defaultValue(VALUE_DISTRIBUTION_DEFAULT.alias()) .documentation( "Distribution name for value and value size. Available distribution names: \"fixed\" \"uniform\", \"zipfian\", \"latest\". Default: uniform") .build(); + + static DataSize VALUE_SIZE_DEFAULT = DataSize.KB.of(1); + static Definition VALUE_SIZE_DEF = Definition.builder() .name("value.size") .type(Definition.Type.STRING) .validator((name, obj) -> DataSize.of(obj.toString())) - .defaultValue(DataSize.KB.of(1).toString()) + .defaultValue(VALUE_SIZE_DEFAULT.toString()) .documentation( "the max length of value. The distribution of length is defined by " + VALUE_DISTRIBUTION_DEF.name()) .build(); + static DistributionType VALUE_SIZE_DISTRIBUTION_DEFAULT = DistributionType.FIXED; static Definition VALUE_SIZE_DISTRIBUTION_DEF = Definition.builder() .name("value.size.distribution") .type(Definition.Type.STRING) .validator((name, obj) -> DistributionType.ofAlias(obj.toString())) - .defaultValue(DistributionType.FIXED.alias()) + .defaultValue(VALUE_SIZE_DISTRIBUTION_DEFAULT.alias()) .documentation( "Distribution name for value size. Available distribution names: \"fixed\" \"uniform\", \"zipfian\", \"latest\". Default: fixed") .build(); + static int BATCH_SIZE_DEFAULT = 1; static Definition BATCH_SIZE_DEF = Definition.builder() .name("batch.size") .type(Definition.Type.INT) - .defaultValue(1) + .defaultValue(BATCH_SIZE_DEFAULT) .documentation("the max length of batching messages.") .build(); + static Definition KEY_TABLE_SEED = Definition.builder() .name("key.table.seed") @@ -182,53 +200,40 @@ public static class Task extends SourceTask { @Override protected void init(Configuration configuration, MetadataStorage storage) { var throughput = - DataSize.of( - configuration - .string(THROUGHPUT_DEF.name()) - .orElse(THROUGHPUT_DEF.defaultValue().toString())); + configuration.string(THROUGHPUT_DEF.name()).map(DataSize::of).orElse(THROUGHPUT_DEFAULT); var KeySize = - DataSize.of( - configuration - .string(KEY_SIZE_DEF.name()) - .orElse(KEY_SIZE_DEF.defaultValue().toString())); + configuration.string(KEY_SIZE_DEF.name()).map(DataSize::of).orElse(KEY_SIZE_DEFAULT); var keyDistribution = - DistributionType.ofAlias( - configuration - .string(KEY_DISTRIBUTION_DEF.name()) - .orElse(KEY_DISTRIBUTION_DEF.defaultValue().toString())); + configuration + .string(KEY_DISTRIBUTION_DEF.name()) + .map(DistributionType::ofAlias) + .orElse(KEY_DISTRIBUTION_DEFAULT); var keySizeDistribution = - DistributionType.ofAlias( - configuration - .string(KEY_SIZE_DISTRIBUTION_DEF.name()) - .orElse(KEY_SIZE_DISTRIBUTION_DEF.defaultValue().toString())); + configuration + .string(KEY_SIZE_DISTRIBUTION_DEF.name()) + .map(DistributionType::ofAlias) + .orElse(KEY_SIZE_DISTRIBUTION_DEFAULT); var valueSize = - DataSize.of( - configuration - .string(VALUE_SIZE_DEF.name()) - .orElse(VALUE_SIZE_DEF.defaultValue().toString())); + configuration.string(VALUE_SIZE_DEF.name()).map(DataSize::of).orElse(VALUE_SIZE_DEFAULT); var valueDistribution = - DistributionType.ofAlias( - configuration - .string(VALUE_DISTRIBUTION_DEF.name()) - .orElse(VALUE_DISTRIBUTION_DEF.defaultValue().toString())); + configuration + .string(VALUE_DISTRIBUTION_DEF.name()) + .map(DistributionType::ofAlias) + .orElse(VALUE_DISTRIBUTION_DEFAULT); var valueSizeDistribution = - DistributionType.ofAlias( - configuration - .string(VALUE_SIZE_DISTRIBUTION_DEF.name()) - .orElse(VALUE_SIZE_DISTRIBUTION_DEF.defaultValue().toString())); - - var batchSize = configuration - .integer(BATCH_SIZE_DEF.name()) - .orElse((Integer) BATCH_SIZE_DEF.defaultValue()); + .string(VALUE_SIZE_DISTRIBUTION_DEF.name()) + .map(DistributionType::ofAlias) + .orElse(VALUE_SIZE_DISTRIBUTION_DEFAULT); + var batchSize = configuration.integer(BATCH_SIZE_DEF.name()).orElse(BATCH_SIZE_DEFAULT); var keyTableSeed = configuration .longInteger(KEY_TABLE_SEED.name()) - .orElse((Long) KEY_TABLE_SEED.defaultValue()); + .orElse(ThreadLocalRandom.current().nextLong()); var valueTableSeed = configuration .longInteger(VALUE_TABLE_SEED.name()) - .orElse((Long) VALUE_TABLE_SEED.defaultValue()); + .orElse(ThreadLocalRandom.current().nextLong()); specifyPartitions = configuration.list(SourceConnector.TOPICS_KEY, ",").stream() diff --git a/connector/src/test/java/org/astraea/connector/DefinitionTest.java b/connector/src/test/java/org/astraea/connector/DefinitionTest.java index 2f1d593265..ded4139054 100644 --- a/connector/src/test/java/org/astraea/connector/DefinitionTest.java +++ b/connector/src/test/java/org/astraea/connector/DefinitionTest.java @@ -29,6 +29,7 @@ void testRequired() { var kafkaConf = Definition.toConfigDef(List.of(def)); var kafkaDef = kafkaConf.configKeys().entrySet().iterator().next().getValue(); Assertions.assertEquals(ConfigDef.NO_DEFAULT_VALUE, kafkaDef.defaultValue); + Assertions.assertTrue(def.required()); } @Test From a43d47a994e8f0a05c4fa821247485fae91aab1e Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 18 May 2023 08:12:50 +0800 Subject: [PATCH 15/77] [COMMON] rewrite Configuration by java 17 record (#1758) --- .../app/performance/DataGenerator.java | 6 +-- .../org/astraea/app/web/BalancerHandler.java | 4 +- .../org/astraea/common/Configuration.java | 53 +++++++------------ .../org/astraea/common/assignor/Assignor.java | 2 +- .../common/balancer/AlgorithmConfig.java | 2 +- .../balancer/BalancerProblemFormat.java | 4 +- .../astraea/common/cost/RecordSizeCost.java | 2 +- .../common/cost/ReplicaLeaderCost.java | 2 +- .../common/cost/ReplicaLeaderSizeCost.java | 2 +- .../common/cost/ReplicaNumberCost.java | 2 +- .../common/partitioner/Partitioner.java | 2 +- .../org/astraea/common/ConfigurationTest.java | 12 ++--- .../astraea/common/DistributionTypeTest.java | 14 +++-- .../java/org/astraea/common/UtilsTest.java | 10 ++-- .../astraea/common/balancer/BalancerTest.java | 4 +- .../algorithms/GreedyBalancerTest.java | 5 +- .../algorithms/SingleStepBalancerTest.java | 2 +- .../executor/StraightPlanExecutorTest.java | 4 +- .../common/cost/BrokerDiskSpaceCostTest.java | 8 +-- .../astraea/common/cost/NetworkCostTest.java | 6 +-- .../common/cost/NetworkIngressCostTest.java | 4 +- .../common/cost/ReplicaLeaderCostTest.java | 4 +- .../partitioner/PartitionerUtilsTest.java | 6 +-- .../StrictCostPartitionerPerfTest.java | 2 +- .../StrictCostPartitionerTest.java | 20 +++---- .../org/astraea/connector/SinkConnector.java | 2 +- .../java/org/astraea/connector/SinkTask.java | 2 +- .../astraea/connector/SourceConnector.java | 2 +- .../org/astraea/connector/SourceTask.java | 2 +- .../astraea/connector/backup/Importer.java | 2 +- .../astraea/connector/perf/PerfSource.java | 4 +- .../org/astraea/connector/ConnectorTest.java | 4 +- .../connector/backup/ExporterTest.java | 16 +++--- .../connector/backup/ImporterTest.java | 4 +- .../astraea/connector/perf/PerfSinkTest.java | 2 +- .../connector/perf/PerfSourceTest.java | 10 ++-- .../java/org/astraea/fs/FileSystemTest.java | 2 +- .../org/astraea/fs/ftp/FtpFileSystemTest.java | 2 +- .../astraea/fs/hdfs/HdfsFileSystemTest.java | 4 +- .../astraea/fs/local/LocalFileSystemTest.java | 2 +- .../astraea/gui/tab/health/BalancerNode.java | 2 +- 41 files changed, 116 insertions(+), 128 deletions(-) diff --git a/app/src/main/java/org/astraea/app/performance/DataGenerator.java b/app/src/main/java/org/astraea/app/performance/DataGenerator.java index 1e28b650e2..e5afa45d22 100644 --- a/app/src/main/java/org/astraea/app/performance/DataGenerator.java +++ b/app/src/main/java/org/astraea/app/performance/DataGenerator.java @@ -41,9 +41,9 @@ static DataGenerator of( Performance.Argument argument) { if (queues.size() == 0) return terminatedGenerator(); - var keyDistConfig = Configuration.of(argument.keyDistributionConfig); - var keySizeDistConfig = Configuration.of(argument.keySizeDistributionConfig); - var valueDistConfig = Configuration.of(argument.valueDistributionConfig); + var keyDistConfig = new Configuration(argument.keyDistributionConfig); + var keySizeDistConfig = new Configuration(argument.keySizeDistributionConfig); + var valueDistConfig = new Configuration(argument.valueDistributionConfig); var dataSupplier = RecordGenerator.builder() .batchSize(argument.transactionSize) diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index ae12056b8e..5e0c6038da 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -113,7 +113,7 @@ public CompletionStage put(Channel channel) { final var request = channel.request(TypeRef.of(BalancerPutRequest.class)); final var taskId = request.id; final var taskPhase = balancerConsole.taskPhase(taskId); - final var executorConfig = Configuration.of(request.executorConfig); + final var executorConfig = new Configuration(request.executorConfig); final var executor = Utils.construct(request.executor, RebalancePlanExecutor.class, executorConfig); @@ -216,7 +216,7 @@ static PostRequestWrapper parsePostRequestWrapper( return new PostRequestWrapper( balancerPostRequest.balancer, - Configuration.of(balancerPostRequest.balancerConfig), + new Configuration(balancerPostRequest.balancerConfig), balancerPostRequest.parse(), currentClusterInfo); } diff --git a/common/src/main/java/org/astraea/common/Configuration.java b/common/src/main/java/org/astraea/common/Configuration.java index 1dc62648ab..d1e27ff877 100644 --- a/common/src/main/java/org/astraea/common/Configuration.java +++ b/common/src/main/java/org/astraea/common/Configuration.java @@ -18,50 +18,31 @@ import java.time.Duration; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Collectors; -public interface Configuration { +public record Configuration(Map raw) { - Configuration EMPTY = Configuration.of(Map.of()); - - static Configuration of(Map configs) { - return new Configuration() { - @Override - public Map raw() { - return Collections.unmodifiableMap(configs); - } - - @Override - public Optional string(String key) { - return Optional.ofNullable(configs.get(key)).map(Object::toString); - } - - @Override - public List list(String key, String separator) { - return string(key).map(s -> Arrays.asList(s.split(separator))).orElseGet(List::of); - } - }; - } - - Map raw(); + public static final Configuration EMPTY = new Configuration(Map.of()); /** * @param key the key whose associated value is to be returned * @return string value. never null */ - Optional string(String key); + public Optional string(String key) { + return Optional.ofNullable(raw.get(Objects.requireNonNull(key))); + } /** * @param key the key whose associated value is to be returned * @return optional {@link Pattern} compiled from the string associated with the key. never null */ - default Optional regexString(String key) { + public Optional regexString(String key) { return string(key).map(Pattern::compile); } @@ -69,11 +50,11 @@ default Optional regexString(String key) { * @param key the key whose associated value is to be returned * @return integer value. never null */ - default Optional integer(String key) { + public Optional integer(String key) { return string(key).map(Integer::parseInt); } - default Optional longInteger(String key) { + public Optional longInteger(String key) { return string(key).map(Long::parseLong); } @@ -81,7 +62,7 @@ default Optional longInteger(String key) { * @param key the key whose associated value is to be returned * @return duration value. If there is no key, return Optional.Empty */ - default Optional duration(String key) { + public Optional duration(String key) { return string(key).map(Utils::toDuration); } @@ -89,11 +70,11 @@ default Optional duration(String key) { * @param key the key whose associated value is to be returned * @return DataSize value. If there is no key, return Optional.Empty */ - default Optional dataSize(String key) { + public Optional dataSize(String key) { return string(key).map(DataSize::of); } - default int requireInteger(String key) { + public int requireInteger(String key) { return integer(key).orElseThrow(() -> new NoSuchElementException(key + " is nonexistent")); } @@ -101,7 +82,7 @@ default int requireInteger(String key) { * @param key the key whose associated value is to be returned * @return string value. never null */ - default String requireString(String key) { + public String requireString(String key) { return string(key).orElseThrow(() -> new NoSuchElementException(key + " is nonexistent")); } @@ -110,8 +91,8 @@ default String requireString(String key) { * @return new Configuration only contains which the key value starts with the prefix, and the * prefix string and the following dot will be removed from the key */ - default Configuration filteredPrefixConfigs(String prefix) { - return of( + public Configuration filteredPrefixConfigs(String prefix) { + return new Configuration( raw().entrySet().stream() .filter(k -> k.getKey().startsWith(prefix)) .collect( @@ -124,5 +105,7 @@ default Configuration filteredPrefixConfigs(String prefix) { * @param separator to split string to multiple strings * @return string list. never null */ - List list(String key, String separator); + public List list(String key, String separator) { + return string(key).map(s -> Arrays.asList(s.split(separator))).orElseGet(List::of); + } } 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 3cfe207c46..4d75189b22 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -161,7 +161,7 @@ public final GroupAssignment assign(Cluster metadata, GroupSubscription groupSub @Override public final void configure(Map configs) { this.config = - Configuration.of( + new Configuration( configs.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); var costFunctions = diff --git a/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java b/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java index c1ca72f121..00f73572ef 100644 --- a/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java +++ b/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java @@ -184,7 +184,7 @@ public Builder timeout(Duration timeout) { } public AlgorithmConfig build() { - var config = Configuration.of(balancerConfig); + var config = new Configuration(balancerConfig); return new AlgorithmConfig() { @Override diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java b/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java index 7cb4771841..27abca9c9f 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerProblemFormat.java @@ -55,7 +55,7 @@ public AlgorithmConfig parse() { private HasClusterCost clusterCost() { if (clusterCosts.isEmpty()) throw new IllegalArgumentException("clusterCosts is not specified"); - var config = Configuration.of(costConfig); + var config = new Configuration(costConfig); return HasClusterCost.of( Utils.costFunctions( clusterCosts.stream() @@ -65,7 +65,7 @@ private HasClusterCost clusterCost() { } private HasMoveCost moveCost() { - var config = Configuration.of(costConfig); + var config = new Configuration(costConfig); var cf = Utils.costFunctions(moveCosts, HasMoveCost.class, config); return HasMoveCost.of(cf); } diff --git a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java index 9cbf965952..6b1b4c6cab 100644 --- a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java @@ -33,7 +33,7 @@ public class RecordSizeCost public static final String MAX_MIGRATE_SIZE_KEY = "max.migrated.size"; public RecordSizeCost() { - this.config = Configuration.of(Map.of()); + this.config = new Configuration(Map.of()); } public RecordSizeCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java index fee24f5c6e..7d4f4346e6 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java @@ -34,7 +34,7 @@ public class ReplicaLeaderCost implements HasBrokerCost, HasClusterCost, HasMove public static final String MAX_MIGRATE_LEADER_KEY = "max.migrated.leader.number"; public ReplicaLeaderCost() { - this.config = Configuration.of(Map.of()); + this.config = new Configuration(Map.of()); } public ReplicaLeaderCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index 00acb46537..6d9c72451f 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -39,7 +39,7 @@ public class ReplicaLeaderSizeCost public static final String MOVED_LEADER_SIZE = "moved leader size (bytes)"; public ReplicaLeaderSizeCost() { - this.config = Configuration.of(Map.of()); + this.config = new Configuration(Map.of()); } public ReplicaLeaderSizeCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java index 2460f17766..7307c7e998 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java @@ -32,7 +32,7 @@ public class ReplicaNumberCost implements HasClusterCost, HasMoveCost { private final Configuration config; public ReplicaNumberCost() { - this.config = Configuration.of(Map.of()); + this.config = new Configuration(Map.of()); } public ReplicaNumberCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/partitioner/Partitioner.java b/common/src/main/java/org/astraea/common/partitioner/Partitioner.java index c43d6b5a0d..ab5be8282a 100644 --- a/common/src/main/java/org/astraea/common/partitioner/Partitioner.java +++ b/common/src/main/java/org/astraea/common/partitioner/Partitioner.java @@ -141,7 +141,7 @@ private static class Interdependent { @Override public final void configure(Map configs) { var config = - Configuration.of( + new Configuration( configs.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); config.string(ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG).ifPresent(s -> admin = Admin.of(s)); diff --git a/common/src/test/java/org/astraea/common/ConfigurationTest.java b/common/src/test/java/org/astraea/common/ConfigurationTest.java index f4bc920bbe..668668a225 100644 --- a/common/src/test/java/org/astraea/common/ConfigurationTest.java +++ b/common/src/test/java/org/astraea/common/ConfigurationTest.java @@ -26,27 +26,27 @@ public class ConfigurationTest { @Test void testString() { - var config = Configuration.of(Map.of("key", "value")); + var config = new Configuration(Map.of("key", "value")); Assertions.assertEquals(Optional.of("value"), config.string("key")); Assertions.assertEquals("value", config.requireString("key")); } @Test void testList() { - var config = Configuration.of(Map.of("key", "v0,v1")); + var config = new Configuration(Map.of("key", "v0,v1")); Assertions.assertEquals(List.of("v0", "v1"), config.list("key", ",")); Assertions.assertEquals(List.of(), config.list("nonExistKey", ",")); } @Test void testFilteredConfigs() { - var config = Configuration.of(Map.of("key", "v1", "filtered.key", "v2", "key.filtered", "v3")); + var config = new Configuration(Map.of("key", "v1", "filtered.key", "v2", "key.filtered", "v3")); Assertions.assertEquals(Map.of("key", "v2"), config.filteredPrefixConfigs("filtered").raw()); } @Test void testDuration() { - var config = Configuration.of(Map.of("wait.time", "15ms", "response", "3s")); + var config = new Configuration(Map.of("wait.time", "15ms", "response", "3s")); var waitTime = config.duration("wait.time"); var response = config.duration("response"); var empty = config.duration("walala"); @@ -57,13 +57,13 @@ void testDuration() { @Test void testLong() { - var config = Configuration.of(Map.of("long.value", "2147483648")); + var config = new Configuration(Map.of("long.value", "2147483648")); Assertions.assertEquals(2147483648L, config.longInteger("long.value").orElse(0L)); } @Test void testDataSize() { - var config = Configuration.of(Map.of("upper.bound", "30MiB", "traffic.interval", "5MB")); + var config = new Configuration(Map.of("upper.bound", "30MiB", "traffic.interval", "5MB")); var upper = config.dataSize("upper.bound"); var interval = config.dataSize("traffic.interval"); var empty = config.dataSize("kekw"); diff --git a/common/src/test/java/org/astraea/common/DistributionTypeTest.java b/common/src/test/java/org/astraea/common/DistributionTypeTest.java index fd55fda25c..a7c18b5472 100644 --- a/common/src/test/java/org/astraea/common/DistributionTypeTest.java +++ b/common/src/test/java/org/astraea/common/DistributionTypeTest.java @@ -88,7 +88,7 @@ void testZipfianConfig() { // fixed seed var zip100 = DistributionType.ZIPFIAN.create( - 10000, Configuration.of(Map.of(DistributionType.ZIPFIAN_SEED, "100"))); + 10000, new Configuration(Map.of(DistributionType.ZIPFIAN_SEED, "100"))); Assertions.assertEquals( List.of(11, 18, 0, 1126, 12), IntStream.range(0, 5) @@ -101,10 +101,12 @@ void testZipfianConfig() { var seed = ThreadLocalRandom.current().nextInt(); var zipA = DistributionType.ZIPFIAN.create( - 10000, Configuration.of(Map.of(DistributionType.ZIPFIAN_SEED, Integer.toString(seed)))); + 10000, + new Configuration(Map.of(DistributionType.ZIPFIAN_SEED, Integer.toString(seed)))); var zipB = DistributionType.ZIPFIAN.create( - 10000, Configuration.of(Map.of(DistributionType.ZIPFIAN_SEED, Integer.toString(seed)))); + 10000, + new Configuration(Map.of(DistributionType.ZIPFIAN_SEED, Integer.toString(seed)))); var sequenceA = IntStream.range(0, 1000).map(i -> zipA.get().intValue()).toArray(); var sequenceB = IntStream.range(0, 1000).map(i -> zipB.get().intValue()).toArray(); Assertions.assertArrayEquals(sequenceA, sequenceB); @@ -112,10 +114,12 @@ void testZipfianConfig() { // high exponent come with high skewness var zip1 = DistributionType.ZIPFIAN.create( - 100, Configuration.of(Map.of(DistributionType.ZIPFIAN_EXPONENT, Double.toString(1.0)))); + 100, + new Configuration(Map.of(DistributionType.ZIPFIAN_EXPONENT, Double.toString(1.0)))); var zip2 = DistributionType.ZIPFIAN.create( - 100, Configuration.of(Map.of(DistributionType.ZIPFIAN_EXPONENT, Double.toString(2.0)))); + 100, + new Configuration(Map.of(DistributionType.ZIPFIAN_EXPONENT, Double.toString(2.0)))); var counting1 = IntStream.range(0, 10000) .map(x -> zip1.get().intValue()) diff --git a/common/src/test/java/org/astraea/common/UtilsTest.java b/common/src/test/java/org/astraea/common/UtilsTest.java index 437856ece3..dc3ba07d45 100644 --- a/common/src/test/java/org/astraea/common/UtilsTest.java +++ b/common/src/test/java/org/astraea/common/UtilsTest.java @@ -206,7 +206,7 @@ public int value() { @ParameterizedTest @ValueSource(classes = {TestCostFunction.class, TestConfigCostFunction.class}) void testConstruct(Class aClass) { - var config = Configuration.of(Map.of()); + var config = new Configuration(Map.of()); var costFunction = Utils.construct(aClass, config); Assertions.assertInstanceOf(CostFunction.class, costFunction); @@ -226,7 +226,7 @@ void testConstruct(Class aClass) { void testConstructException() { // arrange var aClass = TestBadCostFunction.class; - var config = Configuration.of(Map.of()); + var config = new Configuration(Map.of()); // act, assert Assertions.assertThrows(RuntimeException.class, () -> Utils.construct(aClass, config)); @@ -283,7 +283,7 @@ public TestBadCostFunction(int value) {} @Test void testCostFunctions() { var config = - Configuration.of( + new Configuration( Map.of( "org.astraea.common.cost.BrokerInputCost", "20", @@ -307,7 +307,7 @@ void testCostFunctions() { // test negative weight var config2 = - Configuration.of( + new Configuration( Map.of( "org.astraea.common.cost.BrokerInputCost", "-20", @@ -321,7 +321,7 @@ void testCostFunctions() { var cf = Set.of( "org.astraea.common.cost.RecordSizeCost", "org.astraea.common.cost.ReplicaLeaderCost"); - var mConfig = Configuration.of(Map.of("maxMigratedSize", "50MB", "maxMigratedLeader", "5")); + var mConfig = new Configuration(Map.of("maxMigratedSize", "50MB", "maxMigratedLeader", "5")); var mAns = Utils.costFunctions(cf, HasMoveCost.class, mConfig); Assertions.assertEquals(2, mAns.size()); diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerTest.java index 2956b0f111..5fcb74dc10 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerTest.java @@ -168,7 +168,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) .toCompletableFuture() .join(); var newAllocation = - Utils.construct(theClass, Configuration.of(Map.of("iteration", "500"))) + Utils.construct(theClass, new Configuration(Map.of("iteration", "500"))) .offer( AlgorithmConfig.builder() .clusterInfo(clusterInfo) @@ -285,7 +285,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) return () -> 0; } }; - Utils.construct(theClass, Configuration.of(Map.of("iteration", "500"))) + Utils.construct(theClass, new Configuration(Map.of("iteration", "500"))) .offer( AlgorithmConfig.builder() .clusterInfo(ClusterInfo.empty()) diff --git a/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java b/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java index 3f9b617391..3cc236f67c 100644 --- a/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java @@ -58,12 +58,13 @@ void testConfig() { @Test void testJmx() { - var cost = new DecreasingCost(Configuration.of(Map.of())); + var cost = new DecreasingCost(new Configuration(Map.of())); var id = "TestJmx-" + UUID.randomUUID(); var clusterInfo = FakeClusterInfo.of(5, 5, 5, 2); var balancer = Utils.construct( - GreedyBalancer.class, Configuration.of(Map.of(GreedyBalancer.ITERATION_CONFIG, "100"))); + GreedyBalancer.class, + new Configuration(Map.of(GreedyBalancer.ITERATION_CONFIG, "100"))); try (JndiClient client = JndiClient.local()) { IntStream.range(0, 10) diff --git a/common/src/test/java/org/astraea/common/balancer/algorithms/SingleStepBalancerTest.java b/common/src/test/java/org/astraea/common/balancer/algorithms/SingleStepBalancerTest.java index 74e70acd87..29dd3dd1c5 100644 --- a/common/src/test/java/org/astraea/common/balancer/algorithms/SingleStepBalancerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/algorithms/SingleStepBalancerTest.java @@ -28,7 +28,7 @@ class SingleStepBalancerTest extends BalancerConfigTestSuite { public SingleStepBalancerTest() { super( SingleStepBalancer.class, - Configuration.of( + new Configuration( Map.of( "shuffle.tweaker.min.step", "1000", "shuffle.tweaker.max.step", "2000"))); diff --git a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java index 2db29c6073..511eac0e22 100644 --- a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java +++ b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java @@ -114,7 +114,7 @@ void testAsyncRun() { var execute = new StraightPlanExecutor( - Configuration.of( + new Configuration( Map.of(StraightPlanExecutor.CONFIG_ENABLE_DATA_DIRECTORY_MIGRATION, "true"))) .run(admin, expectedAllocation, Duration.ofSeconds(10)); @@ -173,7 +173,7 @@ void testDisableDataDirMigration() { var spiedAdmin = Mockito.spy(admin); var executor = new StraightPlanExecutor( - Configuration.of( + new Configuration( Map.of(StraightPlanExecutor.CONFIG_ENABLE_DATA_DIRECTORY_MIGRATION, "false"))); executor.run(spiedAdmin, target, Duration.ofSeconds(30)).toCompletableFuture().join(); diff --git a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java index 0e4c998737..f7f7dc9b82 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java @@ -140,18 +140,18 @@ void testMoveCosts() { var beforeClusterInfo = of(before); var afterClusterInfo = of(after); var brokerConfig = - Configuration.of( + new Configuration( Map.of(BrokerDiskSpaceCost.BROKER_COST_LIMIT_KEY, "0:1500MB,1:1000MB,2:1500MB")); var brokerOverflowConfig = - Configuration.of( + new Configuration( Map.of(BrokerDiskSpaceCost.BROKER_COST_LIMIT_KEY, "0:1300MB,1:1000MB,2:1500MB")); var pathConfig = - Configuration.of( + new Configuration( Map.of( BrokerDiskSpaceCost.BROKER_PATH_COST_LIMIT_KEY, "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:1000MB")); var pathOverflowConfig = - Configuration.of( + new Configuration( Map.of( BrokerDiskSpaceCost.BROKER_PATH_COST_LIMIT_KEY, "0-/path0:1500MB,1-/path0:1000MB,2-/path0:1500MB,2-/path1:900MB")); diff --git a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java index 7e7daaab04..e85a4b0d76 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java @@ -536,19 +536,19 @@ void testEstimationMethod(ServerMetrics.Topic metric, Class PartitionerUtils.parseIdJMXPort(config3)); } diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java index b8c10b8d71..1fbaad6424 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java @@ -94,7 +94,7 @@ void test() { var value = "value".getBytes(StandardCharsets.UTF_8); try (var partitioner = new StrictCostPartitioner()) { partitioner.admin = admin; - partitioner.configure(Configuration.of(Map.of("round.robin.lease", "2s"))); + partitioner.configure(new Configuration(Map.of("round.robin.lease", "2s"))); Supplier>> resultSupplier = () -> { diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 24a7718be2..4912bb8517 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -45,10 +45,10 @@ public class StrictCostPartitionerTest { @Test void testJmxPort() { try (var partitioner = new StrictCostPartitioner()) { - partitioner.configure(Configuration.of(Map.of())); + partitioner.configure(new Configuration(Map.of())); Assertions.assertThrows( NoSuchElementException.class, () -> partitioner.jmxPortGetter.apply(0)); - partitioner.configure(Configuration.of(Map.of(StrictCostPartitioner.JMX_PORT, "12345"))); + partitioner.configure(new Configuration(Map.of(StrictCostPartitioner.JMX_PORT, "12345"))); Assertions.assertEquals(12345, partitioner.jmxPortGetter.apply(0)); } } @@ -60,14 +60,14 @@ void testNegativeWeight() { IllegalArgumentException.class, () -> partitioner.configure( - Configuration.of( + new Configuration( Map.of( Partitioner.COST_PREFIX + "." + ReplicaLeaderCost.class.getName(), "-1")))); // Test for cost functions configuring partitioner.configure( - Configuration.of( + new Configuration( Map.of( Partitioner.COST_PREFIX + "." + ReplicaLeaderCost.class.getName(), "0.1", @@ -83,7 +83,7 @@ void testNegativeWeight() { void testConfigureCostFunctions() { try (var partitioner = new StrictCostPartitioner()) { partitioner.configure( - Configuration.of( + new Configuration( Map.of( Partitioner.COST_PREFIX + "." + ReplicaLeaderCost.class.getName(), "0.1", @@ -149,7 +149,7 @@ void testCostFunctionWithoutSensor() { .buildLeader(); try (var partitioner = new StrictCostPartitioner()) { partitioner.configure( - Configuration.of( + new Configuration( (Map.of(Partitioner.COST_PREFIX + "." + DumbHasBrokerCost.class.getName(), "1")))); partitioner.partition( "topic", @@ -166,7 +166,7 @@ void testEmptyJmxPort() { // pass due to local mbean partitioner.configure( - Configuration.of( + new Configuration( Map.of(Partitioner.COST_PREFIX + "." + NodeThroughputCost.class.getName(), "1"))); } } @@ -184,7 +184,7 @@ void testReturnedPartition() { var partitionId = 123; try (var partitioner = new StrictCostPartitioner()) { partitioner.configure( - Configuration.of( + new Configuration( Map.of(Partitioner.COST_PREFIX + "." + MyFunction.class.getName(), "1"))); var replicaInfo0 = @@ -214,7 +214,7 @@ void testReturnedPartition() { @Test void testDefaultFunction() { try (var partitioner = new StrictCostPartitioner()) { - partitioner.configure(Configuration.of(Map.of())); + partitioner.configure(new Configuration(Map.of())); Assertions.assertNotEquals(HasBrokerCost.EMPTY, partitioner.costFunction); Utils.waitFor(() -> partitioner.metricStore.sensors().size() == 1); } @@ -244,7 +244,7 @@ void testInvalidCostToScore() { void testRoundRobinLease() { try (var partitioner = new StrictCostPartitioner()) { partitioner.configure( - Configuration.of(Map.of(StrictCostPartitioner.ROUND_ROBIN_LEASE_KEY, "2s"))); + new Configuration(Map.of(StrictCostPartitioner.ROUND_ROBIN_LEASE_KEY, "2s"))); Assertions.assertEquals(Duration.ofSeconds(2), partitioner.roundRobinKeeper.roundRobinLease); partitioner.roundRobinKeeper.tryToUpdate(ClusterInfo.empty(), Map::of); diff --git a/connector/src/main/java/org/astraea/connector/SinkConnector.java b/connector/src/main/java/org/astraea/connector/SinkConnector.java index 15b76dbe9d..662902de45 100644 --- a/connector/src/main/java/org/astraea/connector/SinkConnector.java +++ b/connector/src/main/java/org/astraea/connector/SinkConnector.java @@ -42,7 +42,7 @@ protected void close() { // -------------------------[final]-------------------------// @Override public final void start(Map props) { - init(Configuration.of(props)); + init(new Configuration(props)); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SinkTask.java b/connector/src/main/java/org/astraea/connector/SinkTask.java index 711d7d9a1c..6f9533aed0 100644 --- a/connector/src/main/java/org/astraea/connector/SinkTask.java +++ b/connector/src/main/java/org/astraea/connector/SinkTask.java @@ -50,7 +50,7 @@ public final String version() { @Override public final void start(Map props) { - init(Configuration.of(props)); + init(new Configuration(props)); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SourceConnector.java b/connector/src/main/java/org/astraea/connector/SourceConnector.java index e2cd3cc9df..89ac37531d 100644 --- a/connector/src/main/java/org/astraea/connector/SourceConnector.java +++ b/connector/src/main/java/org/astraea/connector/SourceConnector.java @@ -40,7 +40,7 @@ protected void close() { // -------------------------[final]-------------------------// @Override public final void start(Map props) { - init(Configuration.of(props), MetadataStorage.of(context().offsetStorageReader())); + init(new Configuration(props), MetadataStorage.of(context().offsetStorageReader())); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SourceTask.java b/connector/src/main/java/org/astraea/connector/SourceTask.java index f82316cb68..99ef9a8cb5 100644 --- a/connector/src/main/java/org/astraea/connector/SourceTask.java +++ b/connector/src/main/java/org/astraea/connector/SourceTask.java @@ -52,7 +52,7 @@ public final String version() { @Override public final void start(Map props) { - init(Configuration.of(props), MetadataStorage.of(context.offsetStorageReader())); + init(new Configuration(props), MetadataStorage.of(context.offsetStorageReader())); } @Override diff --git a/connector/src/main/java/org/astraea/connector/backup/Importer.java b/connector/src/main/java/org/astraea/connector/backup/Importer.java index 7f3741655a..880f50e71f 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Importer.java +++ b/connector/src/main/java/org/astraea/connector/backup/Importer.java @@ -115,7 +115,7 @@ protected List takeConfiguration(int maxTasks) { var taskMap = new HashMap<>(config.raw()); taskMap.put(FILE_SET_KEY, String.valueOf(i)); taskMap.put(TASKS_COUNT_KEY, String.valueOf(maxTasks)); - return Configuration.of(taskMap); + return new Configuration(taskMap); }) .toList(); } diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java index 0a5ff86f00..c3063b239f 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java @@ -164,7 +164,7 @@ protected List takeConfiguration(int maxTasks) { t -> { var copy = new HashMap<>(config.raw()); copy.put(SourceConnector.TOPICS_KEY, t); - return Configuration.of(copy); + return new Configuration(copy); }) .collect(Collectors.toUnmodifiableList()); return Utils.chunk(topics, maxTasks).stream() @@ -172,7 +172,7 @@ protected List takeConfiguration(int maxTasks) { tps -> { var copy = new HashMap<>(config.raw()); copy.put(SourceConnector.TOPICS_KEY, String.join(",", tps)); - return Configuration.of(copy); + return new Configuration(copy); }) .collect(Collectors.toUnmodifiableList()); } diff --git a/connector/src/test/java/org/astraea/connector/ConnectorTest.java b/connector/src/test/java/org/astraea/connector/ConnectorTest.java index 9845c58088..7e134bd8fb 100644 --- a/connector/src/test/java/org/astraea/connector/ConnectorTest.java +++ b/connector/src/test/java/org/astraea/connector/ConnectorTest.java @@ -126,7 +126,7 @@ protected Class task() { @Override protected List takeConfiguration(int maxTasks) { return IntStream.range(0, maxTasks) - .mapToObj(i -> Configuration.of(Map.of())) + .mapToObj(i -> new Configuration(Map.of())) .collect(Collectors.toList()); } @@ -166,7 +166,7 @@ protected Class task() { @Override protected List takeConfiguration(int maxTasks) { return IntStream.range(0, maxTasks) - .mapToObj(i -> Configuration.of(Map.of())) + .mapToObj(i -> new Configuration(Map.of())) .collect(Collectors.toList()); } diff --git a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java index 261a39ed66..dcfe2bb61b 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java @@ -189,7 +189,7 @@ void testFtpSinkTask() { "roll.duration", "100m"); - var fs = FileSystem.of("ftp", Configuration.of(configs)); + var fs = FileSystem.of("ftp", new Configuration(configs)); task.start(configs); @@ -276,7 +276,7 @@ void testFtpSinkTaskIntervalWith1File() { "roll.duration", "300ms"); - var fs = FileSystem.of("ftp", Configuration.of(configs)); + var fs = FileSystem.of("ftp", new Configuration(configs)); task.start(configs); @@ -351,7 +351,7 @@ void testFtpSinkTaskIntervalWith2Writers() { "roll.duration", "100ms"); - var fs = FileSystem.of("ftp", Configuration.of(configs)); + var fs = FileSystem.of("ftp", new Configuration(configs)); task.start(configs); @@ -485,7 +485,7 @@ void testHdfsSinkTask() { Assertions.assertTrue(task.isWriterDone()); - var fs = FileSystem.of("hdfs", Configuration.of(configs)); + var fs = FileSystem.of("hdfs", new Configuration(configs)); Assertions.assertEquals( 2, fs.listFolders("/" + String.join("/", fileSize, topicName)).size()); @@ -559,7 +559,7 @@ void testHdfsSinkTaskIntervalWith1File() { Utils.sleep(Duration.ofMillis(1000)); - var fs = FileSystem.of("hdfs", Configuration.of(configs)); + var fs = FileSystem.of("hdfs", new Configuration(configs)); Assertions.assertEquals( 1, fs.listFiles("/" + String.join("/", fileSize, topicName, "0")).size()); @@ -657,7 +657,7 @@ void testHdfsSinkTaskIntervalWith2Writers() { task.put(List.of(record3)); Utils.sleep(Duration.ofMillis(1000)); - var fs = FileSystem.of("hdfs", Configuration.of(configs)); + var fs = FileSystem.of("hdfs", new Configuration(configs)); Assertions.assertEquals( 2, fs.listFolders("/" + String.join("/", fileSize, topicName)).size()); @@ -761,7 +761,7 @@ void testCreateRecordWriter() { var writers = new HashMap(); var task = new Exporter.Task(); - task.fs = FileSystem.of("hdfs", Configuration.of(configs)); + task.fs = FileSystem.of("hdfs", new Configuration(configs)); task.interval = 1000; RecordWriter recordWriter = task.createRecordWriter(tp, offset); @@ -822,7 +822,7 @@ void testWriteRecords() { var writers = new HashMap(); var task = new Exporter.Task(); - task.fs = FileSystem.of("hdfs", Configuration.of(configs)); + task.fs = FileSystem.of("hdfs", new Configuration(configs)); task.size = DataSize.of("100MB"); task.bufferSize.reset(); task.recordsQueue.add( diff --git a/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java index 985f0cd618..fbea1aed67 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java @@ -126,7 +126,7 @@ void testFtpSourceTask() { "file.set", "0"); - var fs = FileSystem.of("ftp", Configuration.of(configs)); + var fs = FileSystem.of("ftp", new Configuration(configs)); var records = List.of( @@ -150,7 +150,7 @@ void testFtpSourceTask() { records.forEach(writer::append); writer.close(); - task.init(Configuration.of(configs), MetadataStorage.EMPTY); + task.init(new Configuration(configs), MetadataStorage.EMPTY); var returnRecords = new ArrayList<>(task.take()); for (int i = 0; i < records.size(); i++) { diff --git a/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java b/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java index 9cca7d93ec..9dad45cf99 100644 --- a/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java +++ b/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java @@ -103,7 +103,7 @@ void testFrequency() { @Test void testTask() { var task = new PerfSink.Task(); - task.init(Configuration.of(Map.of(PerfSink.FREQUENCY_DEF.name(), "1s"))); + task.init(new Configuration(Map.of(PerfSink.FREQUENCY_DEF.name(), "1s"))); var now = System.currentTimeMillis(); task.put(List.>of()); diff --git a/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java b/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java index 27c0f57b67..ef4f6dcfcb 100644 --- a/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java +++ b/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java @@ -50,7 +50,7 @@ static void closeService() { @Test void testDistributeConfigs() { var s = new PerfSource(); - var config = Configuration.of(Map.of(SourceConnector.TOPICS_KEY, "a,b,c,d")); + var config = new Configuration(Map.of(SourceConnector.TOPICS_KEY, "a,b,c,d")); s.init(config, MetadataStorage.EMPTY); var configs = s.takeConfiguration(10); Assertions.assertEquals(4, configs.size()); @@ -331,7 +331,7 @@ void testMetrics() { @Test void testInit() { var task = new PerfSource.Task(); - task.init(Configuration.of(Map.of(ConnectorConfigs.TOPICS_KEY, "a")), MetadataStorage.EMPTY); + task.init(new Configuration(Map.of(ConnectorConfigs.TOPICS_KEY, "a")), MetadataStorage.EMPTY); Assertions.assertNotNull(task.recordGenerator); Assertions.assertEquals(1, task.specifyPartitions.size()); } @@ -340,7 +340,7 @@ void testInit() { void testKeyAndValue() { var task = new PerfSource.Task(); task.init( - Configuration.of( + new Configuration( Map.of( ConnectorConfigs.TOPICS_KEY, "a", @@ -362,7 +362,7 @@ void testKeyAndValue() { void testZeroKeySize() { var task = new PerfSource.Task(); task.init( - Configuration.of( + new Configuration( Map.of(ConnectorConfigs.TOPICS_KEY, "a", PerfSource.KEY_SIZE_DEF.name(), "0Byte")), MetadataStorage.EMPTY); var records = task.take(); @@ -374,7 +374,7 @@ void testZeroKeySize() { void testZeroValueSize() { var task = new PerfSource.Task(); task.init( - Configuration.of( + new Configuration( Map.of(ConnectorConfigs.TOPICS_KEY, "a", PerfSource.VALUE_SIZE_DEF.name(), "0Byte")), MetadataStorage.EMPTY); var records = task.take(); diff --git a/fs/src/test/java/org/astraea/fs/FileSystemTest.java b/fs/src/test/java/org/astraea/fs/FileSystemTest.java index c9c1bce833..6ab079abfd 100644 --- a/fs/src/test/java/org/astraea/fs/FileSystemTest.java +++ b/fs/src/test/java/org/astraea/fs/FileSystemTest.java @@ -40,7 +40,7 @@ void testOf() { Assertions.assertThrows( IllegalArgumentException.class, () -> FileSystem.of("unknown", Configuration.EMPTY)); - var fs = FileSystem.of("local", Configuration.of(Map.of("local.impl", Tmp.class.getName()))); + var fs = FileSystem.of("local", new Configuration(Map.of("local.impl", Tmp.class.getName()))); Assertions.assertInstanceOf(Tmp.class, fs); } diff --git a/fs/src/test/java/org/astraea/fs/ftp/FtpFileSystemTest.java b/fs/src/test/java/org/astraea/fs/ftp/FtpFileSystemTest.java index 9a8034c8cf..1d4468f4dd 100644 --- a/fs/src/test/java/org/astraea/fs/ftp/FtpFileSystemTest.java +++ b/fs/src/test/java/org/astraea/fs/ftp/FtpFileSystemTest.java @@ -31,7 +31,7 @@ public class FtpFileSystemTest extends AbstractFileSystemTest { protected FileSystem fileSystem() { return FileSystem.of( "ftp", - Configuration.of( + new Configuration( Map.of( FtpFileSystem.HOSTNAME_KEY, server.hostname(), diff --git a/fs/src/test/java/org/astraea/fs/hdfs/HdfsFileSystemTest.java b/fs/src/test/java/org/astraea/fs/hdfs/HdfsFileSystemTest.java index 27f43b221e..1258817314 100644 --- a/fs/src/test/java/org/astraea/fs/hdfs/HdfsFileSystemTest.java +++ b/fs/src/test/java/org/astraea/fs/hdfs/HdfsFileSystemTest.java @@ -33,7 +33,7 @@ public class HdfsFileSystemTest extends AbstractFileSystemTest { void testCreate() { var fs = HdfsFileSystem.create( - Configuration.of( + new Configuration( Map.of( HdfsFileSystem.HOSTNAME_KEY, server.hostname(), @@ -50,7 +50,7 @@ void testCreate() { protected FileSystem fileSystem() { return FileSystem.of( "hdfs", - Configuration.of( + new Configuration( Map.of( HdfsFileSystem.HOSTNAME_KEY, server.hostname(), diff --git a/fs/src/test/java/org/astraea/fs/local/LocalFileSystemTest.java b/fs/src/test/java/org/astraea/fs/local/LocalFileSystemTest.java index 6ddd60f3ac..2d65b85b28 100644 --- a/fs/src/test/java/org/astraea/fs/local/LocalFileSystemTest.java +++ b/fs/src/test/java/org/astraea/fs/local/LocalFileSystemTest.java @@ -31,7 +31,7 @@ protected FileSystem fileSystem() { () -> { var tmp = Files.createTempDirectory("test_local_fs"); return FileSystem.of( - "local", Configuration.of(Map.of(LocalFileSystem.ROOT_KEY, tmp.toString()))); + "local", new Configuration(Map.of(LocalFileSystem.ROOT_KEY, tmp.toString()))); }); } } diff --git a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java index 0b02005579..2561971e49 100644 --- a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java @@ -219,7 +219,7 @@ static Map clusterCosts(List keys) { logger.log("searching better assignments ... "); return Utils.construct( GreedyBalancer.class, - Configuration.of(Map.of(GreedyBalancer.ITERATION_CONFIG, "10000"))) + new Configuration(Map.of(GreedyBalancer.ITERATION_CONFIG, "10000"))) .offer( AlgorithmConfig.builder() .clusterInfo(clusterInfo) From c75bcb411d524ba959d744ff565dc9f6d96eb76a Mon Sep 17 00:00:00 2001 From: Xiang-Jun Sun Date: Thu, 18 May 2023 09:44:54 +0800 Subject: [PATCH 16/77] [COST] fix `BrokerDiskSpaceCost` (#1757) --- .../org/astraea/common/cost/BrokerDiskSpaceCost.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java b/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java index e524e1a31f..c9925fa301 100644 --- a/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java +++ b/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java @@ -83,9 +83,8 @@ static boolean brokerDiskUsageSizeOverflow( (Long) after .replicaStream(id) - .filter(r -> before.replicaStream(id).noneMatch(r::equals)) - .map(Replica::size) - .mapToLong(y -> y) + .filter(r -> !before.replicas(r.topicPartition()).contains(r)) + .mapToLong(Replica::size) .sum(); if ((beforeSize + addedSize) > brokerMoveCostLimit.getOrDefault(id, DataSize.Byte.of(Long.MAX_VALUE)).bytes()) @@ -115,9 +114,8 @@ static boolean brokerPathDiskUsageSizeOverflow( (Long) after .replicaStream(brokerPaths.getKey()) - .filter(r -> before.replicaStream(brokerPaths.getKey()).noneMatch(r::equals)) - .map(Replica::size) - .mapToLong(y -> y) + .filter(r -> !before.replicas(r.topicPartition()).contains(r)) + .mapToLong(Replica::size) .sum(); if ((beforeSize + addedSize) > diskMoveCostLimit.getOrDefault(brokerPath, DataSize.Byte.of(Long.MAX_VALUE)).bytes()) From a446e69948bfc4ed02acca5702a843169e49c459 Mon Sep 17 00:00:00 2001 From: Haser Date: Thu, 18 May 2023 22:41:02 +0800 Subject: [PATCH 17/77] [APP] Rewrite app by java 17 toList and instanceof (#1760) --- app/src/main/java/org/astraea/app/App.java | 4 ++-- .../java/org/astraea/app/web/RecordHandler.java | 16 ++++++---------- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/app/src/main/java/org/astraea/app/App.java b/app/src/main/java/org/astraea/app/App.java index 481501bfff..118b048af1 100644 --- a/app/src/main/java/org/astraea/app/App.java +++ b/app/src/main/java/org/astraea/app/App.java @@ -70,8 +70,8 @@ static void execute(Map> mains, List args) throws Throw method.invoke(null, (Object) args.subList(1, args.size()).toArray(String[]::new)); } catch (InvocationTargetException targetException) { // Print out ParameterException, don't throw. - if (targetException.getTargetException() instanceof ParameterException) { - System.out.println(targetException.getTargetException().getMessage()); + if (targetException.getTargetException() instanceof ParameterException exception) { + System.out.println(exception.getMessage()); } else { throw targetException.getTargetException(); } diff --git a/app/src/main/java/org/astraea/app/web/RecordHandler.java b/app/src/main/java/org/astraea/app/web/RecordHandler.java index 8f3a70178f..a2eda70c95 100644 --- a/app/src/main/java/org/astraea/app/web/RecordHandler.java +++ b/app/src/main/java/org/astraea/app/web/RecordHandler.java @@ -17,7 +17,6 @@ package org.astraea.app.web; import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toList; import java.time.Duration; import java.util.Base64; @@ -163,8 +162,7 @@ public CompletionStage get(Channel channel) { // visible for testing GetResponse get(Consumer consumer, int limit, Duration timeout) { try { - return new GetResponse( - consumer, consumer.poll(timeout).stream().map(Record::new).collect(toList())); + return new GetResponse(consumer, consumer.poll(timeout).stream().map(Record::new).toList()); } catch (Exception e) { consumer.close(); throw e; @@ -190,9 +188,7 @@ public CompletionStage post(Channel channel) { () -> { try { return producer.send( - records.stream() - .map(record -> createRecord(producer, record)) - .collect(toList())); + records.stream().map(record -> createRecord(producer, record)).toList()); } finally { if (producer.transactional()) { producer.close(); @@ -214,7 +210,7 @@ public CompletionStage post(Channel channel) { return Response.for404("missing result"); })) .map(CompletionStage::toCompletableFuture) - .collect(toList()))); + .toList())); if (postRequest.async()) return CompletableFuture.completedFuture(Response.ACCEPT); return CompletableFuture.completedFuture( @@ -410,8 +406,8 @@ public String json() { @Override public void onComplete(Throwable error) { try { - if (error == null && consumer instanceof SubscribedConsumer) { - ((SubscribedConsumer) consumer).commitOffsets(Duration.ofSeconds(5)); + if (error == null && consumer instanceof SubscribedConsumer subscribedConsumer) { + subscribedConsumer.commitOffsets(Duration.ofSeconds(5)); } } finally { consumer.close(); @@ -438,7 +434,7 @@ static class Record { timestamp = record.timestamp(); serializedKeySize = record.serializedKeySize(); serializedValueSize = record.serializedValueSize(); - headers = record.headers().stream().map(Header::new).collect(toList()); + headers = record.headers().stream().map(Header::new).toList(); key = record.key(); value = record.value(); leaderEpoch = record.leaderEpoch().orElse(null); From e3a2a793833f47d685cbcd22c1d3993cbb9c331c Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Thu, 18 May 2023 22:41:27 +0800 Subject: [PATCH 18/77] [APP] rewrite app by java 17 toList. (#1759) --- .../app/argument/PositiveIntegerListField.java | 3 +-- .../app/argument/PositiveShortListField.java | 3 +-- .../org/astraea/app/argument/StringListField.java | 3 +-- .../java/org/astraea/app/web/BalancerHandler.java | 5 ++--- .../org/astraea/app/web/ReassignmentHandler.java | 2 +- .../astraea/app/web/SkewedPartitionScenario.java | 3 +-- .../main/java/org/astraea/app/web/TopicHandler.java | 2 +- app/src/test/java/org/astraea/app/EnumInfoTest.java | 6 ++---- app/src/test/java/org/astraea/app/TestUtils.java | 4 ++-- .../org/astraea/app/web/BalancerHandlerTest.java | 9 +++------ .../java/org/astraea/app/web/GroupHandlerTest.java | 4 +--- .../org/astraea/app/web/ProducerHandlerTest.java | 5 +---- .../java/org/astraea/app/web/RecordHandlerTest.java | 13 +++++-------- .../test/java/org/astraea/app/web/RequestTest.java | 3 +-- .../org/astraea/app/web/ThrottleHandlerTest.java | 8 ++------ .../app/web/TopicHandlerForProbabilityTest.java | 3 +-- .../java/org/astraea/app/web/TopicHandlerTest.java | 3 +-- 17 files changed, 27 insertions(+), 52 deletions(-) diff --git a/app/src/main/java/org/astraea/app/argument/PositiveIntegerListField.java b/app/src/main/java/org/astraea/app/argument/PositiveIntegerListField.java index 09dc5943b1..e2038ec71d 100644 --- a/app/src/main/java/org/astraea/app/argument/PositiveIntegerListField.java +++ b/app/src/main/java/org/astraea/app/argument/PositiveIntegerListField.java @@ -17,12 +17,11 @@ package org.astraea.app.argument; import java.util.List; -import java.util.stream.Collectors; import java.util.stream.Stream; public class PositiveIntegerListField extends PositiveNumberListField { @Override public List convert(String value) { - return Stream.of(value.split(SEPARATOR)).map(Integer::valueOf).collect(Collectors.toList()); + return Stream.of(value.split(SEPARATOR)).map(Integer::valueOf).toList(); } } diff --git a/app/src/main/java/org/astraea/app/argument/PositiveShortListField.java b/app/src/main/java/org/astraea/app/argument/PositiveShortListField.java index 6baf4f3592..93475db64c 100644 --- a/app/src/main/java/org/astraea/app/argument/PositiveShortListField.java +++ b/app/src/main/java/org/astraea/app/argument/PositiveShortListField.java @@ -17,12 +17,11 @@ package org.astraea.app.argument; import java.util.List; -import java.util.stream.Collectors; import java.util.stream.Stream; public class PositiveShortListField extends PositiveNumberListField { @Override public List convert(String value) { - return Stream.of(value.split(SEPARATOR)).map(Short::valueOf).collect(Collectors.toList()); + return Stream.of(value.split(SEPARATOR)).map(Short::valueOf).toList(); } } diff --git a/app/src/main/java/org/astraea/app/argument/StringListField.java b/app/src/main/java/org/astraea/app/argument/StringListField.java index 98b02621a6..731fb423b3 100644 --- a/app/src/main/java/org/astraea/app/argument/StringListField.java +++ b/app/src/main/java/org/astraea/app/argument/StringListField.java @@ -17,12 +17,11 @@ package org.astraea.app.argument; import java.util.List; -import java.util.stream.Collectors; import java.util.stream.Stream; public class StringListField extends ListField { @Override public List convert(String value) { - return Stream.of(value.split(SEPARATOR)).collect(Collectors.toList()); + return Stream.of(value.split(SEPARATOR)).toList(); } } diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 5e0c6038da..84889b19a3 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -29,7 +29,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; @@ -283,11 +282,11 @@ static Change from(Collection before, Collection after) { before.stream() .sorted(Comparator.comparing(Replica::isPreferredLeader).reversed()) .map(r -> new Placement(r, Optional.of(r.size()))) - .collect(Collectors.toList()), + .toList(), after.stream() .sorted(Comparator.comparing(Replica::isPreferredLeader).reversed()) .map(r -> new Placement(r, Optional.empty())) - .collect(Collectors.toList())); + .toList()); } Change(String topic, int partition, List before, List after) { diff --git a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java index fee8c4823e..f798c05362 100644 --- a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java +++ b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java @@ -111,7 +111,7 @@ public CompletionStage post(Channel channel) { .filter( b -> b.topicPartitions().contains(tp)) .map(NodeInfo::id) - .collect(Collectors.toList()); + .toList(); if (!ids.isEmpty()) return ids; return List.of( availableBrokers diff --git a/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java b/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java index 7e45b7700a..7fa6eb074c 100644 --- a/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java +++ b/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java @@ -62,8 +62,7 @@ public CompletionStage apply(Admin admin) { admin.waitPartitionLeaderSynced( Map.of(topicName, partitions), Duration.ofSeconds(4))) .thenCompose(ignored -> admin.brokers()) - .thenApply( - brokers -> brokers.stream().map(NodeInfo::id).sorted().collect(Collectors.toList())) + .thenApply(brokers -> brokers.stream().map(NodeInfo::id).sorted().toList()) .thenCompose( brokerIds -> { var distribution = diff --git a/app/src/main/java/org/astraea/app/web/TopicHandler.java b/app/src/main/java/org/astraea/app/web/TopicHandler.java index 34243bcb17..1e7beec0f2 100644 --- a/app/src/main/java/org/astraea/app/web/TopicHandler.java +++ b/app/src/main/java/org/astraea/app/web/TopicHandler.java @@ -190,7 +190,7 @@ public CompletionStage post(Channel channel) { .thenApply(ignored -> null) .toCompletableFuture(); }) - .collect(Collectors.toList())) + .toList()) .thenCompose(ignored -> get(topicNames, null, id -> true)) .exceptionally( ignored -> diff --git a/app/src/test/java/org/astraea/app/EnumInfoTest.java b/app/src/test/java/org/astraea/app/EnumInfoTest.java index 310ba1fcf1..82bdd57b3f 100644 --- a/app/src/test/java/org/astraea/app/EnumInfoTest.java +++ b/app/src/test/java/org/astraea/app/EnumInfoTest.java @@ -17,7 +17,6 @@ package org.astraea.app; import java.util.Arrays; -import java.util.stream.Collectors; import java.util.stream.Stream; import org.astraea.common.EnumInfo; import org.junit.jupiter.api.Assertions; @@ -104,14 +103,13 @@ void testProductionClass() { Assertions.assertTrue(productionClasses.size() > 100); Assertions.assertTrue( productionClasses.stream().allMatch(x -> x.getPackageName().startsWith("org.astraea"))); - System.out.println( - productionClasses.stream().filter(Class::isEnum).collect(Collectors.toList())); + System.out.println(productionClasses.stream().filter(Class::isEnum).toList()); } @Test void testEnumClassProvider() { var enumClassProvider = new EnumClassProvider(); - var enumCls = enumClassProvider.provideArguments(null).collect(Collectors.toList()); + var enumCls = enumClassProvider.provideArguments(null).toList(); Assertions.assertTrue(enumCls.size() > 0); Assertions.assertTrue(enumCls.stream().map(x -> (Class) x.get()[0]).allMatch(Class::isEnum)); } diff --git a/app/src/test/java/org/astraea/app/TestUtils.java b/app/src/test/java/org/astraea/app/TestUtils.java index 426cf62c52..26074ba091 100644 --- a/app/src/test/java/org/astraea/app/TestUtils.java +++ b/app/src/test/java/org/astraea/app/TestUtils.java @@ -46,14 +46,14 @@ public static List> getProductionClass() { FileUtils.listFiles(mainDir.toFile(), new String[] {"class"}, true).stream() .map(File::toPath) .map(mainDir::relativize) - .collect(Collectors.toList()); + .toList(); var classNames = dirFiles.stream() .map(Path::toString) .map(FilenameUtils::removeExtension) .map(x -> x.replace(File.separatorChar, '.')) - .collect(Collectors.toList()); + .toList(); return classNames.stream() .map(x -> Utils.packException(() -> Class.forName(x))) diff --git a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java index 73c9ba3aed..1e4d5d4085 100644 --- a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java @@ -193,10 +193,7 @@ private static Set createAndProduceTopic( .join(); if (skewed) { Utils.sleep(Duration.ofSeconds(1)); - var placement = - service.dataFolders().keySet().stream() - .limit(replicas) - .collect(Collectors.toUnmodifiableList()); + var placement = service.dataFolders().keySet().stream().limit(replicas).toList(); admin .moveToBrokers( admin.topicPartitions(Set.of(topic)).toCompletableFuture().join().stream() @@ -955,13 +952,13 @@ void testChangeOrder() { .mapToObj(partition -> Map.entry(ThreadLocalRandom.current().nextInt(), partition)) .sorted(Map.Entry.comparingByKey()) .map(Map.Entry::getValue) - .collect(Collectors.toUnmodifiableList()); + .toList(); var destPlacement = IntStream.range(0, 10) .mapToObj(partition -> Map.entry(ThreadLocalRandom.current().nextInt(), partition)) .sorted(Map.Entry.comparingByKey()) .map(Map.Entry::getValue) - .collect(Collectors.toUnmodifiableList()); + .toList(); var base = ClusterInfo.builder() .addNode(Set.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) diff --git a/app/src/test/java/org/astraea/app/web/GroupHandlerTest.java b/app/src/test/java/org/astraea/app/web/GroupHandlerTest.java index 5bc4c86639..de375d0c2b 100644 --- a/app/src/test/java/org/astraea/app/web/GroupHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/GroupHandlerTest.java @@ -22,7 +22,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletionException; -import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; @@ -237,8 +236,7 @@ void testDeleteGroup() { try (var admin = Admin.of(SERVICE.bootstrapServers())) { var handler = new GroupHandler(admin); - var groupIds = - IntStream.range(0, 3).mapToObj(x -> Utils.randomString(10)).collect(Collectors.toList()); + var groupIds = IntStream.range(0, 3).mapToObj(x -> Utils.randomString(10)).toList(); groupIds.forEach( groupId -> { try (var consumer = diff --git a/app/src/test/java/org/astraea/app/web/ProducerHandlerTest.java b/app/src/test/java/org/astraea/app/web/ProducerHandlerTest.java index 631b044733..ee3ed3e0a0 100644 --- a/app/src/test/java/org/astraea/app/web/ProducerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/ProducerHandlerTest.java @@ -55,10 +55,7 @@ void testListProducers() { handler.get(Channel.EMPTY).toCompletableFuture().join()); Assertions.assertNotEquals(0, result.partitions.size()); - var partitions = - result.partitions.stream() - .filter(t -> t.topic.equals(topicName)) - .collect(Collectors.toUnmodifiableList()); + var partitions = result.partitions.stream().filter(t -> t.topic.equals(topicName)).toList(); Assertions.assertEquals(1, partitions.size()); Assertions.assertEquals(topicName, partitions.iterator().next().topic); Assertions.assertEquals(0, partitions.iterator().next().partition); diff --git a/app/src/test/java/org/astraea/app/web/RecordHandlerTest.java b/app/src/test/java/org/astraea/app/web/RecordHandlerTest.java index ed44664d15..7673b035e1 100644 --- a/app/src/test/java/org/astraea/app/web/RecordHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/RecordHandlerTest.java @@ -38,7 +38,6 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.app.web.RecordHandler.Metadata; @@ -150,8 +149,7 @@ void testPostRawString() { .keyDeserializer(Deserializer.STRING) .valueDeserializer(Deserializer.STRING) .build()) { - var records = - consumer.poll(Duration.ofSeconds(5)).stream().collect(Collectors.toUnmodifiableList()); + var records = consumer.poll(Duration.ofSeconds(5)).stream().toList(); Assertions.assertEquals(1, records.size()); Assertions.assertEquals(0, records.get(0).partition()); Assertions.assertEquals("abc", records.get(0).key()); @@ -209,8 +207,7 @@ void testPost(boolean isTransaction) { .keyDeserializer(Deserializer.STRING) .valueDeserializer(Deserializer.INTEGER) .build()) { - var records = - consumer.poll(Duration.ofSeconds(10)).stream().collect(Collectors.toUnmodifiableList()); + var records = consumer.poll(Duration.ofSeconds(10)).stream().toList(); Assertions.assertEquals(2, records.size()); var record = records.get(0); @@ -820,7 +817,7 @@ void testDelete() { var records = Stream.of(0, 0, 1, 1, 1, 2, 2, 2, 2) .map(x -> Record.builder().topic(topicName).partition(x).value(new byte[100]).build()) - .collect(Collectors.toList()); + .toList(); producer.send(records); producer.flush(); @@ -904,7 +901,7 @@ void testDeleteOffset() { var records = Stream.of(0, 0, 1, 1, 1, 2, 2, 2, 2) .map(x -> Record.builder().topic(topicName).partition(x).value(new byte[100]).build()) - .collect(Collectors.toList()); + .toList(); producer.send(records); producer.flush(); @@ -961,7 +958,7 @@ void testDeletePartition() { var records = Stream.of(0, 0, 1, 1, 1, 2, 2, 2, 2) .map(x -> Record.builder().topic(topicName).partition(x).value(new byte[100]).build()) - .collect(Collectors.toList()); + .toList(); producer.send(records); producer.flush(); diff --git a/app/src/test/java/org/astraea/app/web/RequestTest.java b/app/src/test/java/org/astraea/app/web/RequestTest.java index 0aa3fd8b0e..74457b3e26 100644 --- a/app/src/test/java/org/astraea/app/web/RequestTest.java +++ b/app/src/test/java/org/astraea/app/web/RequestTest.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.stream.Collectors; import java.util.stream.Stream; import org.astraea.app.TestUtils; import org.astraea.app.web.RecordHandler.PostRecord; @@ -60,7 +59,7 @@ private static List> requestClasses() { return TestUtils.getProductionClass().stream() .filter(Request.class::isAssignableFrom) .filter(c -> !c.isInterface()) - .collect(Collectors.toList()); + .toList(); } public static class RequestClassProvider implements ArgumentsProvider { diff --git a/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java b/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java index c00a850fa5..35ebc92214 100644 --- a/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java @@ -176,9 +176,7 @@ void testThrottleSomeLogs() { handler.get(Channel.EMPTY).toCompletableFuture().join()); var topic = - throttleSetting.topics.stream() - .filter(t -> t.name.get().equals(topicName)) - .collect(Collectors.toList()); + throttleSetting.topics.stream().filter(t -> t.name.get().equals(topicName)).toList(); Assertions.assertEquals(2, topic.size()); var leader = @@ -228,9 +226,7 @@ void testThrottleEveryLog() { ThrottleHandler.ThrottleSetting.class, handler.get(Channel.EMPTY).toCompletableFuture().join()); var topic = - throttleSetting.topics.stream() - .filter(t -> t.name.get().equals(topicName)) - .collect(Collectors.toList()); + throttleSetting.topics.stream().filter(t -> t.name.get().equals(topicName)).toList(); Assertions.assertEquals(9, topic.size()); IntStream.range(0, 3) diff --git a/app/src/test/java/org/astraea/app/web/TopicHandlerForProbabilityTest.java b/app/src/test/java/org/astraea/app/web/TopicHandlerForProbabilityTest.java index 0617fa3757..71ca06ce5a 100644 --- a/app/src/test/java/org/astraea/app/web/TopicHandlerForProbabilityTest.java +++ b/app/src/test/java/org/astraea/app/web/TopicHandlerForProbabilityTest.java @@ -69,8 +69,7 @@ void testCreateTopicByProbability() { .partitions.stream() .flatMap(p -> p.replicas.stream()) .collect(Collectors.groupingBy(r -> r.broker)); - var numberOfReplicas = - groupByBroker.values().stream().map(List::size).collect(Collectors.toList()); + var numberOfReplicas = groupByBroker.values().stream().map(List::size).toList(); replica0 += numberOfReplicas.get(0); replica1 += numberOfReplicas.get(1); replica2 += numberOfReplicas.size() == 3 ? numberOfReplicas.get(2) : 0; diff --git a/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java b/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java index 6aa023deff..972394489e 100644 --- a/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java @@ -322,8 +322,7 @@ void testCreateTopicWithReplicas() { @Test void testDeleteTopic() { - var topicNames = - IntStream.range(0, 3).mapToObj(x -> Utils.randomString(10)).collect(Collectors.toList()); + var topicNames = IntStream.range(0, 3).mapToObj(x -> Utils.randomString(10)).toList(); try (var admin = Admin.of(SERVICE.bootstrapServers())) { var handler = new TopicHandler(admin); for (var name : topicNames) From c9a00b3e70942d7c606cc3f01c87815a609fc155 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 19 May 2023 00:11:50 +0800 Subject: [PATCH 19/77] [BALANCER] rewrite AlgorithmConfig by java 17 record (#1761) --- .../common/balancer/AlgorithmConfig.java | 114 +++++------------- 1 file changed, 33 insertions(+), 81 deletions(-) diff --git a/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java b/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java index 00f73572ef..cbfa9e4a19 100644 --- a/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java +++ b/common/src/main/java/org/astraea/common/balancer/AlgorithmConfig.java @@ -27,59 +27,41 @@ import org.astraea.common.cost.HasMoveCost; import org.astraea.common.metrics.ClusterBean; -/** The generic algorithm parameter for resolving the Kafka rebalance problem. */ -public interface AlgorithmConfig { - - static Builder builder() { +/** + * The generic algorithm parameter for resolving the Kafka rebalance problem. + * + * @param executionId a String indicate the name of this execution. This information is used for + * debug and logging usage. + * @param clusterCostFunction the cluster cost function for this problem. + * @param moveCostFunction the movement cost functions for this problem + * @param balancerConfig the configuration of this balancer run + * @param clusterInfo the initial cluster state of this optimization problem + * @param clusterBean the metrics of the associated cluster and optimization problem + * @param timeout the execution limit of this optimization problem + */ +public record AlgorithmConfig( + String executionId, + HasClusterCost clusterCostFunction, + HasMoveCost moveCostFunction, + Configuration balancerConfig, + ClusterInfo clusterInfo, + ClusterBean clusterBean, + Duration timeout) { + + public static Builder builder() { return new Builder(null); } - static Builder builder(AlgorithmConfig config) { + public static Builder builder(AlgorithmConfig config) { return new Builder(config); } - /** - * @return a String indicate the name of this execution. This information is used for debug and - * logging usage. - */ - String executionId(); - - /** - * @return the cluster cost function for this problem. - */ - HasClusterCost clusterCostFunction(); - - /** - * @return the movement cost functions for this problem - */ - HasMoveCost moveCostFunction(); - - /** - * @return the configuration of this balancer run - */ - Configuration balancerConfig(); - - /** - * @return the initial cluster state of this optimization problem - */ - ClusterInfo clusterInfo(); - - /** - * @return the metrics of the associated cluster and optimization problem - */ - ClusterBean clusterBean(); - - /** - * @return the execution limit of this optimization problem - */ - Duration timeout(); - - class Builder { + public static class Builder { private String executionId = "noname-" + UUID.randomUUID(); private HasClusterCost clusterCostFunction; private HasMoveCost moveCostFunction = HasMoveCost.EMPTY; - private Map balancerConfig = new HashMap<>(); + private final Map balancerConfig = new HashMap<>(); private ClusterInfo clusterInfo; private ClusterBean clusterBean = ClusterBean.EMPTY; @@ -184,44 +166,14 @@ public Builder timeout(Duration timeout) { } public AlgorithmConfig build() { - var config = new Configuration(balancerConfig); - - return new AlgorithmConfig() { - @Override - public String executionId() { - return executionId; - } - - @Override - public HasClusterCost clusterCostFunction() { - return clusterCostFunction; - } - - @Override - public HasMoveCost moveCostFunction() { - return moveCostFunction; - } - - @Override - public Configuration balancerConfig() { - return config; - } - - @Override - public ClusterInfo clusterInfo() { - return clusterInfo; - } - - @Override - public ClusterBean clusterBean() { - return clusterBean; - } - - @Override - public Duration timeout() { - return timeout; - } - }; + return new AlgorithmConfig( + executionId, + clusterCostFunction, + moveCostFunction, + new Configuration(balancerConfig), + clusterInfo, + clusterBean, + timeout); } } } From 3669c0a6aa03ab7b99bb19d1fbf8b98369470a7f Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Fri, 19 May 2023 21:46:45 +0800 Subject: [PATCH 20/77] [SERIALIZATION] Serialize `BeanObject#createdTimestampe` (#1762) --- common/src/main/java/org/astraea/common/ByteUtils.java | 9 ++++++++- .../proto/org/astraea/common/generated/BeanObject.proto | 2 ++ .../common/serializer/BeanObjectSerializerTest.java | 4 +++- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index 38a509fe52..29c304a041 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -17,6 +17,7 @@ package org.astraea.common; import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Timestamp; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; @@ -178,6 +179,10 @@ public static byte[] toBytes(BeanObject value) { // Bean attribute may contain non-primitive value. e.g. TimeUnit, Byte. } }); + beanBuilder.setCreatedTimestamp( + Timestamp.newBuilder() + .setSeconds(value.createdTimestamp() / 1000) + .setNanos((int) (value.createdTimestamp() % 1000) * 1000000)); return beanBuilder.build().toByteArray(); } @@ -313,7 +318,9 @@ public static BeanObject readBeanObject(byte[] bytes) throws SerializationExcept outerBean.getAttributesMap().entrySet().stream() .collect( Collectors.toUnmodifiableMap( - Map.Entry::getKey, e -> Objects.requireNonNull(toObject(e.getValue()))))); + Map.Entry::getKey, e -> Objects.requireNonNull(toObject(e.getValue())))), + outerBean.getCreatedTimestamp().getSeconds() * 1000 + + outerBean.getCreatedTimestamp().getNanos() / 1000000); } catch (InvalidProtocolBufferException ex) { // Pack exception thrown by protoBuf to Serialization exception. throw new SerializationException(ex); diff --git a/common/src/main/proto/org/astraea/common/generated/BeanObject.proto b/common/src/main/proto/org/astraea/common/generated/BeanObject.proto index c6797f2c25..a1669bb67b 100644 --- a/common/src/main/proto/org/astraea/common/generated/BeanObject.proto +++ b/common/src/main/proto/org/astraea/common/generated/BeanObject.proto @@ -2,10 +2,12 @@ syntax = "proto3"; package org.astraea.common.generated; +import "google/protobuf/timestamp.proto"; import "org/astraea/common/generated/Primitive.proto"; message BeanObject { string domain = 1; map properties = 2; map attributes = 3; + google.protobuf.Timestamp createdTimestamp = 4; } diff --git a/common/src/test/java/org/astraea/common/serializer/BeanObjectSerializerTest.java b/common/src/test/java/org/astraea/common/serializer/BeanObjectSerializerTest.java index a486ad4ace..f9546c2828 100644 --- a/common/src/test/java/org/astraea/common/serializer/BeanObjectSerializerTest.java +++ b/common/src/test/java/org/astraea/common/serializer/BeanObjectSerializerTest.java @@ -44,7 +44,8 @@ public void testSerializationDeserialization() { true, "String", "str"); - var bean = new BeanObject(domain, properties, attributes); + var time = System.currentTimeMillis(); + var bean = new BeanObject(domain, properties, attributes, time); // Valid arguments should not throw Assertions.assertDoesNotThrow( @@ -58,6 +59,7 @@ public void testSerializationDeserialization() { Assertions.assertEquals("domain", beanObj.domainName()); Assertions.assertEquals(properties, beanObj.properties()); Assertions.assertEquals(attributes, beanObj.attributes()); + Assertions.assertEquals(time, beanObj.createdTimestamp()); } @Test From 76d5f36d00c21c2f5d96513df329b1f53aef1335 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Sun, 21 May 2023 11:53:47 +0800 Subject: [PATCH 21/77] [COST] Enhance performance for `ClusterInfoSensor` (#1764) --- .../common/cost/utils/ClusterInfoSensor.java | 63 ++++++++++--------- 1 file changed, 34 insertions(+), 29 deletions(-) diff --git a/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java b/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java index 1e994b9f35..ac2db0446e 100644 --- a/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java +++ b/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java @@ -17,7 +17,6 @@ package org.astraea.common.cost.utils; import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -44,7 +43,7 @@ public List fetch(BeanObjectClient client, ClusterBean LogMetrics.Log.SIZE.fetch(client), ClusterMetrics.Partition.REPLICAS_COUNT.fetch(client)) .flatMap(Collection::stream) - .collect(Collectors.toUnmodifiableList()); + .toList(); } /** @@ -61,6 +60,22 @@ public static ClusterInfo metricViewCluster(ClusterBean clusterBean) { .filter(id -> id != -1) .map(id -> NodeInfo.of(id, "", -1)) .collect(Collectors.toUnmodifiableMap(NodeInfo::id, x -> x)); + var replicaMap = + clusterBean.brokerIds().stream() + .collect( + Collectors.toUnmodifiableMap( + broker -> broker, + broker -> + clusterBean + .brokerMetrics(broker, LogMetrics.Log.Gauge.class) + .filter(x -> LogMetrics.Log.SIZE.metricName().equals(x.metricsName())) + .filter(x -> x.partitionIndex().isPresent()) + .collect( + Collectors.toUnmodifiableMap( + x -> x.partitionIndex().orElseThrow(), + x -> x, + (a, b) -> + a.createdTimestamp() > b.createdTimestamp() ? a : b)))); var replicas = clusterBean.brokerTopics().stream() .filter(bt -> bt.broker() != -1) @@ -70,50 +85,40 @@ public static ClusterInfo metricViewCluster(ClusterBean clusterBean) { var partitions = clusterBean .brokerTopicMetrics(bt, ClusterMetrics.PartitionMetric.class) - .sorted( - Comparator.comparingLong(HasBeanObject::createdTimestamp).reversed()) + .collect( + Collectors.toUnmodifiableMap( + ClusterMetrics.PartitionMetric::topicPartition, + x -> x, + (a, b) -> a.createdTimestamp() > b.createdTimestamp() ? a : b)) + .values() + .stream() .collect( Collectors.toUnmodifiableMap( ClusterMetrics.PartitionMetric::topicPartition, m -> { var tp = m.topicPartition(); - var size = - clusterBean - .brokerMetrics(broker, LogMetrics.Log.Gauge.class) - .filter(x -> x.partition() == tp.partition()) - .filter(x -> x.topic().equals(tp.topic())) - .filter( - x -> - LogMetrics.Log.SIZE - .metricName() - .equals(x.metricsName())) - .max( - Comparator.comparingLong( - HasBeanObject::createdTimestamp)) - .orElseThrow( - () -> - new IllegalStateException( - "Partition " - + tp - + " detected, but its size metric doesn't exists. " - + "Maybe the given cluster bean is partially sampled")) - .value(); + var size = replicaMap.get(broker).get(tp); + if (size == null) + throw new IllegalStateException( + "Partition " + + tp + + " detected, but its size metric doesn't exists. " + + "Maybe the given cluster bean is partially sampled"); var build = Replica.builder() .topic(tp.topic()) .partition(tp.partition()) .nodeInfo(nodes.get(broker)) .path("") - .size(size); + .size(size.value()); var isLeader = m.value() != 0; return isLeader ? build.buildLeader() : build.buildInSyncFollower(); - }, - (latest, earlier) -> latest)); + })); return partitions.values().stream(); }) - .collect(Collectors.toUnmodifiableList()); + .toList(); var clusterId = clusterBean.all().entrySet().stream() .filter(e -> e.getKey() != -1) From 6db827e7b6d88835625bbf21f149c3896045f83c Mon Sep 17 00:00:00 2001 From: Zhi-Mao Teng Date: Sun, 21 May 2023 22:46:25 +0800 Subject: [PATCH 22/77] [ASSIGNOR] Implement randomly generate assignments (#1703) --- .../astraea/common/assignor/Generator.java | 54 +++++++++++ .../org/astraea/common/assignor/Hint.java | 89 ++++++++++++++++++ .../org/astraea/common/assignor/Limiter.java | 90 ++++++++++++++++++ .../org/astraea/common/assignor/Shuffler.java | 92 +++++++++++++++++++ 4 files changed, 325 insertions(+) create mode 100644 common/src/main/java/org/astraea/common/assignor/Generator.java create mode 100644 common/src/main/java/org/astraea/common/assignor/Hint.java create mode 100644 common/src/main/java/org/astraea/common/assignor/Limiter.java create mode 100644 common/src/main/java/org/astraea/common/assignor/Shuffler.java diff --git a/common/src/main/java/org/astraea/common/assignor/Generator.java b/common/src/main/java/org/astraea/common/assignor/Generator.java new file mode 100644 index 0000000000..1cda8bdbaf --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/Generator.java @@ -0,0 +1,54 @@ +/* + * 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.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.astraea.common.admin.TopicPartition; + +@FunctionalInterface +public interface Generator { + + Map> get(); + + static Generator randomGenerator( + Map subscription, + Map partitionCost, + Hint hints) { + return () -> { + Map> combinator = + subscription.keySet().stream() + .map(c -> Map.entry(c, new ArrayList())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + List candidates; + + for (var tp : partitionCost.keySet()) { + candidates = hints.get(combinator, tp); + if (candidates.isEmpty()) candidates = subscription.keySet().stream().toList(); + + combinator + .get(candidates.get(ThreadLocalRandom.current().nextInt(candidates.size()))) + .add(tp); + } + + return combinator; + }; + } +} diff --git a/common/src/main/java/org/astraea/common/assignor/Hint.java b/common/src/main/java/org/astraea/common/assignor/Hint.java new file mode 100644 index 0000000000..87a7e41061 --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/Hint.java @@ -0,0 +1,89 @@ +/* + * 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.Set; +import java.util.stream.Collectors; +import org.astraea.common.admin.TopicPartition; + +@FunctionalInterface +public interface Hint { + List get(Map> currentAssignment, TopicPartition tp); + + static Hint of(Set hints) { + return (currentAssignment, tp) -> + hints.stream() + .map(h -> h.get(currentAssignment, tp)) + .reduce((l1, l2) -> l1.stream().filter(l2::contains).toList()) + .get(); + } + + static Hint lowCostHint( + Map subscriptions, Map partitionCost) { + return (currentAssignment, tp) -> { + var candidates = + 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(); + + return candidates.stream().limit((long) Math.ceil(candidates.size() / 2.0)).toList(); + }; + } + + static Hint incompatibleHint( + Map subscriptions, + Map> incompatibilities) { + return (currentAssignment, tp) -> { + var subscriber = + subscriptions.entrySet().stream() + .filter(e -> e.getValue().topics().contains(tp.topic())) + .map(Map.Entry::getKey) + .toList(); + if (incompatibilities.get(tp).isEmpty()) return subscriber; + + var candidates = + currentAssignment.entrySet().stream() + .filter(e -> subscriber.contains(e.getKey())) + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream() + .filter(p -> incompatibilities.get(p).contains(tp)) + .count())) + .collect( + Collectors.groupingBy( + Map.Entry::getValue, + Collectors.mapping(Map.Entry::getKey, Collectors.toList()))) + .entrySet() + .stream() + .min(Map.Entry.comparingByKey()) + .get() + .getValue(); + + return candidates.isEmpty() ? List.of() : candidates; + }; + } +} diff --git a/common/src/main/java/org/astraea/common/assignor/Limiter.java b/common/src/main/java/org/astraea/common/assignor/Limiter.java new file mode 100644 index 0000000000..a782eaaa6f --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/Limiter.java @@ -0,0 +1,90 @@ +/* + * 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.function.Function; +import java.util.stream.Collectors; +import org.astraea.common.admin.TopicPartition; + +@FunctionalInterface +public interface Limiter { + + boolean check(Map> condition); + + static Limiter of(Set limiters) { + return (combinator) -> limiters.stream().allMatch(l -> l.check(combinator)); + } + + static Limiter incompatibleLimiter(Map> incompatible) { + return (combinator) -> + combinator.entrySet().stream() + .map( + e -> { + var consumer = e.getKey(); + var tps = e.getValue(); + + var unsuitable = + incompatible.entrySet().stream() + .filter(entry -> tps.contains(entry.getKey())) + .flatMap(entry -> entry.getValue().stream()) + .collect(Collectors.toUnmodifiableSet()); + return Map.entry(consumer, tps.stream().filter(unsuitable::contains).count()); + }) + .mapToDouble(Map.Entry::getValue) + .sum() + == 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)); + }); + var standardDeviation = + (Function, Double>) + (vs) -> { + var average = vs.stream().mapToDouble(c -> c).average().getAsDouble(); + return Math.sqrt( + vs.stream().mapToDouble(v -> Math.pow(v - average, 2)).average().getAsDouble()); + }; + var limit = standardDeviation.apply(tmpConsumerCost.values()); + + return (combinator) -> { + var sd = + standardDeviation.apply( + combinator.values().stream() + .map(tps -> tps.stream().mapToDouble(partitionCost::get).sum()) + .collect(Collectors.toSet())); + + return sd < limit; + }; + } +} diff --git a/common/src/main/java/org/astraea/common/assignor/Shuffler.java b/common/src/main/java/org/astraea/common/assignor/Shuffler.java new file mode 100644 index 0000000000..19ff5c7260 --- /dev/null +++ b/common/src/main/java/org/astraea/common/assignor/Shuffler.java @@ -0,0 +1,92 @@ +/* + * 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.HashSet; +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.admin.TopicPartition; + +public interface Shuffler { + Map> shuffle(); + + static Shuffler randomShuffler( + Map subscriptions, + Map partitionCost, + Map> incompatible, + Configuration config) { + var limiters = + Limiter.of( + Set.of( + Limiter.skewCostLimiter(partitionCost, subscriptions), + Limiter.incompatibleLimiter(incompatible))); + var hints = + Hint.of( + Set.of( + 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) -> { + var costPerConsumer = + combinator.entrySet().stream() + .map( + e -> + Map.entry( + e.getKey(), + e.getValue().stream().mapToDouble(partitionCost::get).sum())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var avg = + costPerConsumer.values().stream().mapToDouble(d -> d).average().getAsDouble(); + + return Math.sqrt( + costPerConsumer.values().stream() + .mapToDouble(d -> Math.pow(d - avg, 2)) + .average() + .getAsDouble()); + }; + var rejectedCombinators = new HashSet>>(); + + return () -> { + Map> result = null; + var start = System.currentTimeMillis(); + + while (System.currentTimeMillis() - start < shuffleTime) { + var combinator = generator.get(); + if (limiters.check(combinator)) { + result = combinator; + break; + } + rejectedCombinators.add(combinator); + } + + return result == null + ? rejectedCombinators.stream() + .map(c -> Map.entry(c, standardDeviation.apply(c))) + .min(Map.Entry.comparingByValue()) + .get() + .getKey() + : result; + }; + } +} From 9613d6b82e72bd0533e124f323f44fd2afa8d461 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Mon, 22 May 2023 15:46:47 +0800 Subject: [PATCH 23/77] [BALANCER] Implement balancer config `balancer.broker.balancing.mode` (#1737) --- .../astraea/app/web/BalancerHandlerTest.java | 30 +- .../common/admin/ClusterInfoBuilder.java | 13 + .../common/balancer/BalancerConfigs.java | 52 ++- .../common/balancer/BalancerUtils.java | 180 +++++++++ .../balancer/algorithms/GreedyBalancer.java | 61 ++- .../algorithms/SingleStepBalancer.java | 66 +++- .../common/admin/ClusterInfoBuilderTest.java | 19 + .../balancer/BalancerConfigTestSuite.java | 363 +++++++++++++++++- .../common/balancer/BalancerUtilsTest.java | 208 ++++++++++ docs/web_server/web_api_balancer_chinese.md | 8 +- 10 files changed, 938 insertions(+), 62 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/balancer/BalancerUtils.java create mode 100644 common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java diff --git a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java index 1e4d5d4085..f6727111cb 100644 --- a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java @@ -224,27 +224,19 @@ private static Set createAndProduceTopic( void testBestPlan() { try (var admin = Admin.of(SERVICE.bootstrapServers())) { var currentClusterInfo = - ClusterInfo.of( - "fake", - List.of(NodeInfo.of(10, "host", 22), NodeInfo.of(11, "host", 22)), - Map.of(), - List.of( - Replica.builder() - .topic("topic") - .partition(0) - .nodeInfo(NodeInfo.of(10, "host", 22)) - .lag(0) - .size(100) - .isLeader(true) - .isSync(true) - .isFuture(false) - .isOffline(false) - .isPreferredLeader(true) - .path("/tmp/aa") - .build())); + ClusterInfo.builder() + .addNode(Set.of(1, 2)) + .addFolders( + Map.ofEntries(Map.entry(1, Set.of("/folder")), Map.entry(2, Set.of("/folder")))) + .addTopic("topic", 1, (short) 1) + .build(); HasClusterCost clusterCostFunction = - (clusterInfo, clusterBean) -> () -> clusterInfo == currentClusterInfo ? 100D : 10D; + (clusterInfo, clusterBean) -> + () -> + ClusterInfo.findNonFulfilledAllocation(currentClusterInfo, clusterInfo).isEmpty() + ? 100D + : 10D; HasMoveCost moveCostFunction = HasMoveCost.EMPTY; HasMoveCost failMoveCostFunction = (before, after, clusterBean) -> () -> true; diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java index c06919747a..1aa93e1c40 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java @@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -92,6 +93,18 @@ public ClusterInfoBuilder addNode(Set brokerIds) { }); } + /** + * Remove specific brokers from the cluster state. + * + * @param toRemove id to remove + * @return this + */ + public ClusterInfoBuilder removeNodes(Predicate toRemove) { + return applyNodes( + (nodes, replicas) -> + nodes.stream().filter(node -> toRemove.negate().test(node.id())).toList()); + } + /** * Add some fake folders to a specific broker. * diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java index 01ebcd26ff..da74c14425 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java @@ -35,9 +35,53 @@ private BalancerConfigs() {} public static final String BALANCER_ALLOWED_TOPICS_REGEX = "balancer.allowed.topics.regex"; /** - * A regular expression indicates which brokers are eligible for moving loading. When specified, a - * broker with an id that doesn't match this expression cannot accept a partition from the other - * broker or move its partition to other brokers. + * This configuration indicates the balancing mode for each broker. + * + *

This configuration requires a string with a series of key-value pairs, each pair is + * separated by a comma, and the key and value are separated by a colon. + * (brokerId_A|"default"):(mode),(brokerId_B):(mode), ... The key indicates the integer id + * for a broker. And the value indicates the balancing mode for the associated broker. When the + * key is a string value "default"(without the double quotes), it indicates the + * associated balancing mode should be the default mode for the rest of the brokers that are not + * addressed in the configuration. By default, all the brokers use "balancing" mode. + * + *

Possible balancing modes

+ * + *
    + *
  • balancing: The broker will participate in the load balancing process. The + * replica assignment for this broker is eligible for changes. + *
  • demoted: The broker should become empty after the rebalance. This mode + * allows the user to clear all the loadings for certain brokers, enabling a graceful + * removal of those brokers. Note to the balancer implementation: A broker in this mode + * assumes it will be out of service after the balancing is finished. Therefore, when + * evaluating the cluster cost, the brokers to demote should be excluded. However, these + * brokers will be included in the move cost evaluation. Since these brokers are still part + * of the cluster right now, and move cost focusing on the cost associated during the + * ongoing balancing process itself. + *
  • excluded: The broker will not participate in the load balancing process. The + * replica assignment for this broker is not eligible for changes. It will neither accept + * replicas from other brokers nor reassign replicas to other brokers. + *
+ * + *

Flag Interaction:

+ * + *
    + *
  1. When this flag is used in conjunction with {@link + * BalancerConfigs#BALANCER_ALLOWED_TOPICS_REGEX}, if a demoted broker contains partition + * from those forbidden topics, an exception should be raised. + *
+ * + *

Limitation:

+ * + *
    + *
  1. Demoting a broker may be infeasible if there are not enough brokers to fit the required + * replica factor for a specific partition. This situation is more likely to occur if there + * are many excluded brokers that reject accepting new replicas. If such a case + * is detected, an exception should be raised. + *
  2. Any broker with ongoing replica-move-in, replica-move-out, or inter-folder movement + * cannot be the demoting target. An exception will be raised if any of the demoting brokers + * have such ongoing events. * + *
*/ - public static final String BALANCER_ALLOWED_BROKERS_REGEX = "balancer.allowed.brokers.regex"; + public static final String BALANCER_BROKER_BALANCING_MODE = "balancer.broker.balancing.mode"; } diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java new file mode 100644 index 0000000000..d846905f0b --- /dev/null +++ b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java @@ -0,0 +1,180 @@ +/* + * 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.balancer; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.astraea.common.EnumInfo; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Replica; + +public final class BalancerUtils { + + private BalancerUtils() {} + + public static Map balancingMode(ClusterInfo cluster, String config) { + var num = Pattern.compile("[0-9]+"); + + var map = + Arrays.stream(config.split(",")) + .filter(Predicate.not(String::isEmpty)) + .map(x -> x.split(":")) + .collect( + Collectors.toUnmodifiableMap( + s -> (Object) (num.matcher(s[0]).find() ? Integer.parseInt(s[0]) : s[0]), + s -> + switch (s[1]) { + case "balancing" -> BalancingModes.BALANCING; + case "demoted" -> BalancingModes.DEMOTED; + case "excluded" -> BalancingModes.EXCLUDED; + default -> throw new IllegalArgumentException( + "Unsupported balancing mode: " + s[1]); + })); + + Function mode = + (id) -> map.getOrDefault(id, map.getOrDefault("default", BalancingModes.BALANCING)); + + return cluster.brokers().stream() + .map(NodeInfo::id) + .collect(Collectors.toUnmodifiableMap(Function.identity(), mode)); + } + + /** + * Verify there is no logic conflict between {@link BalancerConfigs#BALANCER_ALLOWED_TOPICS_REGEX} + * and {@link BalancerConfigs#BALANCER_BROKER_BALANCING_MODE}. It also performs other common + * validness checks to the cluster. + */ + public static void verifyClearBrokerValidness( + ClusterInfo cluster, Predicate isDemoted, Predicate allowedTopics) { + var disallowedTopicsToClear = + cluster.topicPartitionReplicas().stream() + .filter(tpr -> isDemoted.test(tpr.brokerId())) + .filter(tpr -> !allowedTopics.test(tpr.topic())) + .collect(Collectors.toUnmodifiableSet()); + if (!disallowedTopicsToClear.isEmpty()) + throw new IllegalArgumentException( + "Attempts to clear some brokers, but some of them contain topics that forbidden from being changed due to \"" + + BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX + + "\": " + + disallowedTopicsToClear); + + var ongoingEventReplica = + cluster.replicas().stream() + .filter(r -> isDemoted.test(r.nodeInfo().id())) + .filter(r -> r.isAdding() || r.isRemoving() || r.isFuture()) + .map(Replica::topicPartitionReplica) + .collect(Collectors.toUnmodifiableSet()); + if (!ongoingEventReplica.isEmpty()) + throw new IllegalArgumentException( + "Attempts to clear broker with ongoing migration event (adding/removing/future replica): " + + ongoingEventReplica); + } + + /** + * Move all the replicas at the demoting broker to other allowed brokers. BE CAREFUL, The + * implementation made no assumption for MoveCost or ClusterCost of the returned ClusterInfo. + * Be aware of this limitation before using it as the starting point for a solution search. Some + * balancer implementation might have trouble finding answer when starting at a state where the + * MoveCost is already violated. + */ + public static ClusterInfo clearedCluster( + ClusterInfo initial, Predicate clearBrokers, Predicate allowedBrokers) { + final var allowed = + initial.nodes().stream() + .filter(node -> allowedBrokers.test(node.id())) + .filter(node -> Predicate.not(clearBrokers).test(node.id())) + .collect(Collectors.toUnmodifiableSet()); + final var nextBroker = Stream.generate(() -> allowed).flatMap(Collection::stream).iterator(); + final var nextBrokerFolder = + initial.brokerFolders().entrySet().stream() + .collect( + Collectors.toUnmodifiableMap( + Map.Entry::getKey, + x -> Stream.generate(x::getValue).flatMap(Collection::stream).iterator())); + + var trackingReplicaList = + initial.topicPartitions().stream() + .collect( + Collectors.toUnmodifiableMap( + tp -> tp, + tp -> + initial.replicas(tp).stream() + .map(Replica::nodeInfo) + .collect(Collectors.toSet()))); + return ClusterInfo.builder(initial) + .mapLog( + replica -> { + if (!clearBrokers.test(replica.nodeInfo().id())) return replica; + var currentReplicaList = trackingReplicaList.get(replica.topicPartition()); + var broker = + IntStream.range(0, allowed.size()) + .mapToObj(i -> nextBroker.next()) + .filter(b -> !currentReplicaList.contains(b)) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + "Unable to clear replica " + + replica.topicPartitionReplica() + + " for broker " + + replica.nodeInfo().id() + + ", the allowed destination brokers are " + + allowed.stream() + .map(NodeInfo::id) + .collect(Collectors.toUnmodifiableSet()) + + " but all of them already hosting a replica for this partition. " + + "There is no broker can adopt this replica.")); + var folder = nextBrokerFolder.get(broker.id()).next(); + + // update the tracking list. have to do this to avoid putting two replicas from the + // same tp to one broker. + currentReplicaList.remove(replica.nodeInfo()); + currentReplicaList.add(broker); + + return Replica.builder(replica).nodeInfo(broker).path(folder).build(); + }) + .build(); + } + + public enum BalancingModes implements EnumInfo { + BALANCING, + DEMOTED, + EXCLUDED; + + public static BalancingModes ofAlias(String alias) { + return EnumInfo.ignoreCaseEnum(BalancingModes.class, alias); + } + + @Override + public String alias() { + return name(); + } + + @Override + public String toString() { + return alias(); + } + } +} diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index 1278d85c56..779ff5a9f8 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.DoubleAccumulator; import java.util.concurrent.atomic.LongAdder; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.regex.Pattern; @@ -32,6 +33,7 @@ import org.astraea.common.balancer.AlgorithmConfig; import org.astraea.common.balancer.Balancer; import org.astraea.common.balancer.BalancerConfigs; +import org.astraea.common.balancer.BalancerUtils; import org.astraea.common.balancer.tweakers.ShuffleTweaker; import org.astraea.common.cost.ClusterCost; import org.astraea.common.metrics.MBeanRegister; @@ -140,26 +142,38 @@ public Optional offer(AlgorithmConfig config) { .regexString(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX) .map(Pattern::asMatchPredicate) .orElse((ignore) -> true); - final var allowedBrokers = - config - .balancerConfig() - .regexString(BalancerConfigs.BALANCER_ALLOWED_BROKERS_REGEX) - .map(Pattern::asMatchPredicate) - .>map( - predicate -> (brokerId) -> predicate.test(Integer.toString(brokerId))) - .orElse((ignore) -> true); + final var balancingMode = + BalancerUtils.balancingMode( + config.clusterInfo(), + config + .balancerConfig() + .string(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE) + .orElse("")); + final Predicate isBalancing = + id -> balancingMode.get(id) == BalancerUtils.BalancingModes.BALANCING; + final Predicate isDemoted = + id -> balancingMode.get(id) == BalancerUtils.BalancingModes.DEMOTED; + final var hasDemoted = + balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.DEMOTED); + BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted, allowedTopics); - final var currentClusterInfo = config.clusterInfo(); + final var currentClusterInfo = + BalancerUtils.clearedCluster(config.clusterInfo(), isDemoted, isBalancing); final var clusterBean = config.clusterBean(); final var allocationTweaker = ShuffleTweaker.builder() .numberOfShuffle(() -> ThreadLocalRandom.current().nextInt(minStep, maxStep)) .allowedTopics(allowedTopics) - .allowedBrokers(allowedBrokers) + .allowedBrokers(isBalancing) .build(); - final var clusterCostFunction = config.clusterCostFunction(); final var moveCostFunction = config.moveCostFunction(); - final var initialCost = clusterCostFunction.clusterCost(currentClusterInfo, clusterBean); + final Function evaluateCost = + (cluster) -> { + final var filteredCluster = + hasDemoted ? ClusterInfo.builder(cluster).removeNodes(isDemoted).build() : cluster; + return config.clusterCostFunction().clusterCost(filteredCluster, clusterBean); + }; + final var initialCost = evaluateCost.apply(currentClusterInfo); final var loop = new AtomicInteger(iteration); final var start = System.currentTimeMillis(); @@ -182,7 +196,7 @@ public Optional offer(AlgorithmConfig config) { config.clusterInfo(), initialCost, newAllocation, - clusterCostFunction.clusterCost(newAllocation, clusterBean))) + evaluateCost.apply(newAllocation))) .filter(plan -> plan.proposalClusterCost().value() < currentCost.value()) .findFirst(); var currentCost = initialCost; @@ -211,6 +225,25 @@ public Optional offer(AlgorithmConfig config) { currentCost = currentSolution.get().proposalClusterCost(); currentAllocation = currentSolution.get().proposal(); } - return currentSolution; + return currentSolution.or( + () -> { + // With demotion, the implementation detail start search from a demoted state. It is + // possible + // that the start state is already the ideal answer. In this case, it is directly + // returned. + if (hasDemoted + && initialCost.value() == 0.0 + && !moveCostFunction + .moveCost(config.clusterInfo(), currentClusterInfo, clusterBean) + .overflow()) { + return Optional.of( + new Plan( + config.clusterInfo(), + config.clusterCostFunction().clusterCost(config.clusterInfo(), clusterBean), + currentClusterInfo, + initialCost)); + } + return Optional.empty(); + }); } } diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java index 0b8cc0bafe..3f170b369f 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java @@ -21,13 +21,17 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Function; import java.util.function.Predicate; import java.util.regex.Pattern; import org.astraea.common.Utils; +import org.astraea.common.admin.ClusterInfo; import org.astraea.common.balancer.AlgorithmConfig; import org.astraea.common.balancer.Balancer; import org.astraea.common.balancer.BalancerConfigs; +import org.astraea.common.balancer.BalancerUtils; import org.astraea.common.balancer.tweakers.ShuffleTweaker; +import org.astraea.common.cost.ClusterCost; /** This algorithm proposes rebalance plan by tweaking the log allocation once. */ public class SingleStepBalancer implements Balancer { @@ -68,27 +72,39 @@ public Optional offer(AlgorithmConfig config) { .regexString(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX) .map(Pattern::asMatchPredicate) .orElse((ignore) -> true); - final var allowedBrokers = - config - .balancerConfig() - .regexString(BalancerConfigs.BALANCER_ALLOWED_BROKERS_REGEX) - .map(Pattern::asMatchPredicate) - .>map( - predicate -> (brokerId) -> predicate.test(Integer.toString(brokerId))) - .orElse((ignore) -> true); + final var balancingMode = + BalancerUtils.balancingMode( + config.clusterInfo(), + config + .balancerConfig() + .string(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE) + .orElse("")); + final Predicate isBalancing = + id -> balancingMode.get(id) == BalancerUtils.BalancingModes.BALANCING; + final Predicate isDemoted = + id -> balancingMode.get(id) == BalancerUtils.BalancingModes.DEMOTED; + final var hasDemoted = + balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.DEMOTED); + BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted, allowedTopics); - final var currentClusterInfo = config.clusterInfo(); + final var currentClusterInfo = + BalancerUtils.clearedCluster(config.clusterInfo(), isDemoted, isBalancing); final var clusterBean = config.clusterBean(); final var allocationTweaker = ShuffleTweaker.builder() .numberOfShuffle(() -> ThreadLocalRandom.current().nextInt(minStep, maxStep)) .allowedTopics(allowedTopics) - .allowedBrokers(allowedBrokers) + .allowedBrokers(isBalancing) .build(); - final var clusterCostFunction = config.clusterCostFunction(); final var moveCostFunction = config.moveCostFunction(); - final var currentCost = - config.clusterCostFunction().clusterCost(currentClusterInfo, clusterBean); + + final Function evaluateCost = + (cluster) -> { + final var filteredCluster = + hasDemoted ? ClusterInfo.builder(cluster).removeNodes(isDemoted).build() : cluster; + return config.clusterCostFunction().clusterCost(filteredCluster, clusterBean); + }; + final var currentCost = evaluateCost.apply(currentClusterInfo); var start = System.currentTimeMillis(); return allocationTweaker @@ -107,8 +123,28 @@ public Optional offer(AlgorithmConfig config) { config.clusterInfo(), currentCost, newAllocation, - clusterCostFunction.clusterCost(newAllocation, clusterBean))) + evaluateCost.apply(newAllocation))) .filter(plan -> plan.proposalClusterCost().value() < currentCost.value()) - .min(Comparator.comparing(plan -> plan.proposalClusterCost().value())); + .min(Comparator.comparing(plan -> plan.proposalClusterCost().value())) + .or( + () -> { + // With demotion, the implementation detail start search from a demoted state. It is + // possible + // that the start state is already the ideal answer. In this case, it is directly + // returned. + if (hasDemoted + && currentCost.value() == 0.0 + && !moveCostFunction + .moveCost(config.clusterInfo(), currentClusterInfo, clusterBean) + .overflow()) { + return Optional.of( + new Plan( + config.clusterInfo(), + config.clusterCostFunction().clusterCost(config.clusterInfo(), clusterBean), + currentClusterInfo, + currentCost)); + } + return Optional.empty(); + }); } } diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java index b7d2851632..0f2e5244d3 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java @@ -341,4 +341,23 @@ void testFakeBrokerInteraction(int id, String host, int port) { Assertions.assertEquals(node0, node1); Assertions.assertNotEquals(node0, node2); } + + @Test + void testRemoveNodes() { + var base = ClusterInfo.builder().addNode(Set.of(1, 2, 3, 4, 5, 6, 7, 8, 9)).build(); + Assertions.assertEquals( + Set.of(1, 2, 3), + ClusterInfo.builder(base) + .removeNodes(x -> Set.of(4, 5, 6, 7, 8, 9).contains(x)) + .build() + .nodes() + .stream() + .map(NodeInfo::id) + .collect(Collectors.toSet())); + Assertions.assertEquals( + Set.of(1, 3, 5, 7, 9), + ClusterInfo.builder(base).removeNodes(x -> x % 2 == 0).build().nodes().stream() + .map(NodeInfo::id) + .collect(Collectors.toSet())); + } } diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index 693c2be146..e3c2e66b35 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -17,17 +17,23 @@ package org.astraea.common.balancer; import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Replica; import org.astraea.common.cost.ClusterCost; import org.astraea.common.cost.HasClusterCost; +import org.astraea.common.cost.ReplicaLeaderCost; import org.astraea.common.metrics.ClusterBean; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -121,7 +127,7 @@ public void testBalancerAllowedTopicsRegex() { } @Test - public void testBalancerAllowedBrokersRegex() { + public void testBalancingMode() { final var balancer = Utils.construct(balancerClass, Configuration.EMPTY); final var cluster = cluster(10, 10, 10, (short) 5); @@ -134,7 +140,7 @@ public void testBalancerAllowedBrokersRegex() { .clusterCost(decreasingCost()) .timeout(Duration.ofSeconds(2)) .configs(customConfig.raw()) - .config(BalancerConfigs.BALANCER_ALLOWED_BROKERS_REGEX, "[0-9]*") + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "default:balancing") .build()); AssertionsHelper.assertSomeMovement(cluster, plan.orElseThrow().proposal(), testName); } @@ -148,7 +154,7 @@ public void testBalancerAllowedBrokersRegex() { .clusterCost(decreasingCost()) .timeout(Duration.ofSeconds(2)) .configs(customConfig.raw()) - .config(BalancerConfigs.BALANCER_ALLOWED_BROKERS_REGEX, "NoMatch") + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "default:excluded") .build()); // since nothing can be moved. It is ok to return no plan. if (plan.isPresent()) { @@ -160,8 +166,10 @@ public void testBalancerAllowedBrokersRegex() { { var testName = "[test some match]"; var allowedBrokers = IntStream.range(1, 6).boxed().collect(Collectors.toUnmodifiableSet()); - var rawRegex = - allowedBrokers.stream().map(Object::toString).collect(Collectors.joining("|", "(", ")")); + var config = + allowedBrokers.stream() + .map(i -> i + ":balancing") + .collect(Collectors.joining(",", "default:excluded,", "")); var plan = balancer.offer( AlgorithmConfig.builder() @@ -169,13 +177,345 @@ public void testBalancerAllowedBrokersRegex() { .clusterCost(decreasingCost()) .timeout(Duration.ofSeconds(2)) .configs(customConfig.raw()) - .config(BalancerConfigs.BALANCER_ALLOWED_BROKERS_REGEX, rawRegex) + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, config) .build()); AssertionsHelper.assertOnlyAllowedBrokerMovement( cluster, plan.orElseThrow().proposal(), allowedBrokers::contains, testName); } } + @Test + public void testBalancingModeDemoted() { + final var balancer = Utils.construct(balancerClass, Configuration.EMPTY); + final var cluster = cluster(10, 30, 10, (short) 5); + + { + var testName = "[test all clear]"; + Assertions.assertThrows( + Exception.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(cluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "default:demoted") + .build()), + testName); + } + + { + var testName = "[test some clear]"; + var plan = + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(cluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "0:demoted,1:demoted,2:demoted") + .build()); + Assertions.assertTrue(plan.isPresent(), testName); + var finalCluster = plan.get().proposal(); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.nodeInfo().id() == 0)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.nodeInfo().id() == 1)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.nodeInfo().id() == 2)); + Assertions.assertTrue( + finalCluster.replicas().stream().noneMatch(x -> x.nodeInfo().id() == 0)); + Assertions.assertTrue( + finalCluster.replicas().stream().noneMatch(x -> x.nodeInfo().id() == 1)); + Assertions.assertTrue( + finalCluster.replicas().stream().noneMatch(x -> x.nodeInfo().id() == 2)); + AssertionsHelper.assertBrokerEmpty( + finalCluster, (x) -> Set.of(0, 1, 2).contains(x), testName); + } + + { + var testName = "[test replication factor violation]"; + // 6 brokers, clear 3 brokers, remain 3 brokers, topic with replication factor 3 can fit this + // cluster. + var noViolatedCluster = cluster(6, 10, 10, (short) 3); + Assertions.assertDoesNotThrow( + () -> { + var solution = + balancer + .offer( + AlgorithmConfig.builder() + .clusterInfo(noViolatedCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "0:demoted,1:demoted,2:demoted") + .build()) + .orElseThrow() + .proposal(); + AssertionsHelper.assertBrokerEmpty( + solution, (x) -> Set.of(0, 1, 2).contains(x), testName); + }, + testName); + + // 5 brokers, clear 3 brokers, remain 2 brokers, topic with replication factor 3 CANNOT fit + // this cluster. + var violatedCluster = cluster(5, 10, 10, (short) 3); + Assertions.assertThrows( + Exception.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(violatedCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "0:demoted,1:demoted,2:demoted") + .build())); + } + + { + var testName = "[test if allowed topics is used, clear disallow topic will raise an error]"; + var base = + ClusterInfo.builder() + .addNode(Set.of(1, 2, 3)) + .addFolders( + Map.ofEntries( + Map.entry(1, Set.of("/folder")), + Map.entry(2, Set.of("/folder")), + Map.entry(3, Set.of("/folder")))) + .build(); + var node12 = Stream.of(1, 2).map(base::node).iterator(); + var node13 = Stream.of(1, 3).map(base::node).iterator(); + var node123 = Stream.of(1, 2, 3).map(base::node).iterator(); + var testCluster = + ClusterInfo.builder(base) + .addTopic("OK", 1, (short) 1, r -> Replica.builder(r).nodeInfo(base.node(1)).build()) + .addTopic( + "OK_SKIP", 2, (short) 1, r -> Replica.builder(r).nodeInfo(node12.next()).build()) + .addTopic( + "Replica", 1, (short) 2, r -> Replica.builder(r).nodeInfo(node13.next()).build()) + .addTopic( + "Partition", + 3, + (short) 1, + r -> Replica.builder(r).nodeInfo(node123.next()).build()) + .build(); + + Assertions.assertDoesNotThrow( + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(testCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // allow anything other than "OK" topic + .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!OK).*") + // clear broker 3 + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") + // this won't raise an error since that topic didn't locate at 3 + .build()), + testName); + Assertions.assertDoesNotThrow( + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(testCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // allow anything other than "OK" topic + .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!OK_SKIP).*") + // clear broker 3 + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") + // this won't raise an error since that topic didn't locate at 3 + .build()), + testName); + Assertions.assertThrows( + Exception.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(testCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // allow anything other than "Replica" topic + .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!Replica).*") + // clear broker 3 + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") + // this will raise an error since that topic has a replica at 3 + .build()), + testName); + Assertions.assertThrows( + Exception.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(testCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // allow anything other than "Replica" topic + .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!Partition).*") + // clear broker 3 + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") + // this will raise an error since that topic has a partition at 3 + .build()), + testName); + } + + { + var testName = "[test if allowed brokers is used, disallowed broker won't be altered]"; + var solution = + balancer + .offer( + AlgorithmConfig.builder() + .clusterInfo(cluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // clear broker 0 + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "0:demoted," + + + // allow broker 1,2,3,4,5,6 + "1:balancing,2:balancing,3:balancing,4:balancing,5:balancing,6:balancing,default:excluded") + // this will be ok since any replica at 0 can move to 1~6 without breaking + // replica factors + .build()) + .orElseThrow() + .proposal(); + var before = cluster.topicPartitionReplicas(); + var after = solution.topicPartitionReplicas(); + var changed = + after.stream() + .filter(Predicate.not(before::contains)) + .collect(Collectors.toUnmodifiableSet()); + Assertions.assertTrue(after.stream().noneMatch(r -> r.brokerId() == 0), testName); + Assertions.assertTrue( + changed.stream().allMatch(r -> Set.of(1, 2, 3, 4, 5, 6).contains(r.brokerId())), + testName); + } + + { + var testName = + "[test if allowed brokers is used, insufficient allowed broker to fit replica factor requirement will raise an error]"; + Assertions.assertThrows( + Exception.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(cluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // clear broker 0, allow broker 1 + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "0:demoted,1:balancing,default:excluded") + // this will raise an error if a partition has replicas at both 0 and 1. In + // this case, there is no allowed broker to adopt replica from 0, since the + // only allowed broker already has one replica on it. we cannot assign two + // replicas to one broker. + .build()), + testName); + } + + { + var testName = "[if replica on clear broker is adding/removing/future, raise an exception]"; + var adding = + ClusterInfo.builder(cluster) + .mapLog(r -> r.nodeInfo().id() != 0 ? r : Replica.builder(r).isAdding(true).build()) + .build(); + var removing = + ClusterInfo.builder(cluster) + .mapLog(r -> r.nodeInfo().id() != 0 ? r : Replica.builder(r).isRemoving(true).build()) + .build(); + var future = + ClusterInfo.builder(cluster) + .mapLog(r -> r.nodeInfo().id() != 0 ? r : Replica.builder(r).isFuture(true).build()) + .build(); + for (var cc : List.of(adding, removing, future)) { + Assertions.assertThrows( + Exception.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(cc) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(1)) + .configs(customConfig.raw()) + // clear broker 0 allow broker 1,2,3,4,5,6 + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "0:demoted," + + "1:balancing,2:balancing,3:balancing,4:balancing,5:balancing,6:balancing") + .build()), + testName); + } + for (var cc : List.of(adding, removing, future)) { + Assertions.assertDoesNotThrow( + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(cc) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(1)) + .configs(customConfig.raw()) + // clear broker 1 allow broker 0,2,3,4,5,6,7 + .config( + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, + "1:demoted," + + "0:balancing,2:balancing,3:balancing,4:balancing,5:balancing,6:balancing," + + "7:balancing,default:excluded") + // adding/removing/future at 0 not 1, unrelated so no error + .build()), + testName); + } + } + + { + // Some balancer implementations have such logic flaw: + // 1. The initial state[A] cannot be solution. + // 2. There are brokers that need to be demoted. + // 3. The load on those brokers been redistributed to other brokers. Creating the start + // state[B] for the solution search. + // 4. The start state[B] solution is actually the best solution. + // 5. Balancer think the start state[B] is the initial state[A]. And cannot be a solution(as + // mentioned in 1). + // 6. In fact, the start state[B] doesn't equal to the initial state[A]. Since there is a + // cleaning work performed at step 3. + // 7. Balancer cannot find any solution that is better than the start state(4) and therefore + // returns no solution. + var testName = + "[If the cluster after clear is the best solution, balancer should be able to return it]"; + var testCluster = + ClusterInfo.builder() + .addNode(Set.of(1, 2)) + .addFolders( + Map.ofEntries(Map.entry(1, Set.of("/folder")), Map.entry(2, Set.of("/folder")))) + .addTopic("topic", 100, (short) 1) + .build(); + Assertions.assertNotEquals( + Optional.empty(), + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(testCluster) + .clusterBean(ClusterBean.EMPTY) + .clusterCost(new ReplicaLeaderCost()) + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "1:demoted") + .timeout(Duration.ofSeconds(2)) + .build()), + testName); + } + } + private static ClusterInfo cluster(int nodes, int topics, int partitions, short replicas) { var builder = ClusterInfo.builder() @@ -258,5 +598,16 @@ static void assertOnlyAllowedBrokerMovement( }); }); } + + static void assertBrokerEmpty(ClusterInfo target, Predicate clearBroker, String name) { + var violated = + target + .replicaStream() + .filter(i -> clearBroker.test(i.nodeInfo().id())) + .collect(Collectors.toUnmodifiableSet()); + Assertions.assertTrue( + violated.isEmpty(), + name + ": the following replica should move to somewhere else " + violated); + } } } diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java new file mode 100644 index 0000000000..bf27b63426 --- /dev/null +++ b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java @@ -0,0 +1,208 @@ +/* + * 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.balancer; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Stream; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Replica; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +class BalancerUtilsTest { + + @Test + void testBalancingMode() { + var cluster = ClusterInfo.builder().addNode(Set.of(1, 2, 3, 4, 5)).build(); + + Assertions.assertThrows(Exception.class, () -> BalancerUtils.balancingMode(cluster, "bad")); + Assertions.assertThrows(Exception.class, () -> BalancerUtils.balancingMode(cluster, "bad:bad")); + Assertions.assertThrows( + Exception.class, () -> BalancerUtils.balancingMode(cluster, "bad:bad:bad")); + Assertions.assertThrows( + Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:balancing,bad:bad")); + Assertions.assertThrows( + Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:balancing,bad:bad:bad")); + Assertions.assertThrows( + Exception.class, + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,4:oops")); + Assertions.assertThrows( + Exception.class, + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,4:")); + Assertions.assertThrows( + Exception.class, + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,1:")); + Assertions.assertThrows( + Exception.class, + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,:")); + Assertions.assertThrows( + Exception.class, + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,::")); + Assertions.assertThrows(Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:")); + Assertions.assertThrows( + Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:balancing,1:balancing")); + + Assertions.assertDoesNotThrow( + () -> BalancerUtils.balancingMode(cluster, "reserved_usage:balancing"), + "Intentionally reserved this usage"); + + Assertions.assertEquals( + BalancerUtils.BalancingModes.BALANCING, + BalancerUtils.balancingMode(cluster, "").get(1), + "default"); + Assertions.assertEquals( + BalancerUtils.BalancingModes.DEMOTED, + BalancerUtils.balancingMode(cluster, "1:demoted").get(1), + "value"); + Assertions.assertEquals( + BalancerUtils.BalancingModes.DEMOTED, + BalancerUtils.balancingMode(cluster, "default:demoted").get(5), + "user defined default"); + Assertions.assertEquals( + BalancerUtils.BalancingModes.EXCLUDED, + BalancerUtils.balancingMode(cluster, "3:excluded,4:excluded").get(3)); + Assertions.assertEquals( + BalancerUtils.BalancingModes.EXCLUDED, + BalancerUtils.balancingMode(cluster, "3:excluded,4:excluded").get(4)); + Assertions.assertEquals( + BalancerUtils.BalancingModes.BALANCING, + BalancerUtils.balancingMode(cluster, "3:excluded,4:excluded,1:balancing").get(1)); + Assertions.assertEquals( + Set.of(1, 2, 3, 4, 5), BalancerUtils.balancingMode(cluster, "").keySet()); + } + + @Test + void testVerifyClearBrokerValidness() { + var base = + ClusterInfo.builder() + .addNode(Set.of(1, 2, 3)) + .addFolders( + Map.ofEntries( + Map.entry(1, Set.of("/folder")), + Map.entry(2, Set.of("/folder")), + Map.entry(3, Set.of("/folder")))) + .build(); + var iter = Stream.of(1, 2, 3).map(base::node).iterator(); + var cluster = + ClusterInfo.builder(base) + .addTopic("A", 1, (short) 1, r -> Replica.builder(r).nodeInfo(iter.next()).build()) + .addTopic("B", 1, (short) 1, r -> Replica.builder(r).nodeInfo(iter.next()).build()) + .addTopic("C", 1, (short) 1, r -> Replica.builder(r).nodeInfo(iter.next()).build()) + .build(); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 1, t -> !t.equals("A"))); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 2, t -> !t.equals("B"))); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 3, t -> !t.equals("C"))); + Assertions.assertDoesNotThrow( + () -> BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 1, t -> t.equals("A"))); + Assertions.assertDoesNotThrow( + () -> BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 2, t -> t.equals("B"))); + Assertions.assertDoesNotThrow( + () -> BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 3, t -> t.equals("C"))); + + var hasAdding = + ClusterInfo.builder(cluster).mapLog(r -> Replica.builder(r).isAdding(true).build()).build(); + var hasRemoving = + ClusterInfo.builder(cluster) + .mapLog(r -> Replica.builder(r).isRemoving(true).build()) + .build(); + var hasFuture = + ClusterInfo.builder(cluster).mapLog(r -> Replica.builder(r).isFuture(true).build()).build(); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> BalancerUtils.verifyClearBrokerValidness(hasAdding, x -> true, x -> true)); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> BalancerUtils.verifyClearBrokerValidness(hasRemoving, x -> true, x -> true)); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> BalancerUtils.verifyClearBrokerValidness(hasFuture, x -> true, x -> true)); + Assertions.assertDoesNotThrow( + () -> BalancerUtils.verifyClearBrokerValidness(hasAdding, x -> false, x -> true)); + Assertions.assertDoesNotThrow( + () -> BalancerUtils.verifyClearBrokerValidness(hasRemoving, x -> false, x -> true)); + Assertions.assertDoesNotThrow( + () -> BalancerUtils.verifyClearBrokerValidness(hasFuture, x -> false, x -> true)); + } + + @Test + void testClearedCluster() { + var cluster = + ClusterInfo.builder() + .addNode(Set.of(1, 2)) + .addFolders(Map.of(1, Set.of("/folder"))) + .addFolders(Map.of(2, Set.of("/folder"))) + .addTopic("topic", 100, (short) 2) + .addNode(Set.of(3, 4)) + .addFolders(Map.of(3, Set.of("/folder"))) + .addFolders(Map.of(4, Set.of("/folder"))) + .build(); + Assertions.assertThrows( + Exception.class, + () -> BalancerUtils.clearedCluster(cluster, id -> id == 1 || id == 2, id -> id == 3), + "Insufficient brokers to meet replica factor requirement"); + var clearedCluster = + Assertions.assertDoesNotThrow( + () -> + BalancerUtils.clearedCluster( + cluster, id -> id == 1 || id == 2, id -> id == 3 || id == 4)); + + Assertions.assertEquals( + List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 1).toList()); + Assertions.assertEquals( + List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 2).toList()); + Assertions.assertNotEquals( + List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 3).toList()); + Assertions.assertNotEquals( + List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 4).toList()); + + var sameCluster = + Assertions.assertDoesNotThrow( + () -> BalancerUtils.clearedCluster(cluster, id -> false, id -> true)); + Assertions.assertEquals( + Set.of(), + ClusterInfo.findNonFulfilledAllocation(cluster, sameCluster), + "Nothing to clear, nothing to change"); + + var aCluster = + Assertions.assertDoesNotThrow( + () -> BalancerUtils.clearedCluster(cluster, id -> id == 1, id -> id == 3)); + Assertions.assertEquals( + 0, aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 1).count(), "Demoted"); + Assertions.assertEquals( + 100, + aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 2).count(), + "Not allowed or cleared"); + Assertions.assertEquals( + 100, + aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 3).count(), + "Accept replicas broker demoted broker"); + Assertions.assertEquals( + 0, aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 4).count(), "Not allowed"); + } +} diff --git a/docs/web_server/web_api_balancer_chinese.md b/docs/web_server/web_api_balancer_chinese.md index 91b044d47a..309f924645 100644 --- a/docs/web_server/web_api_balancer_chinese.md +++ b/docs/web_server/web_api_balancer_chinese.md @@ -26,10 +26,10 @@ POST /balancer `balancerConfig` 是 balancer 實作開放給使用者設定的內部演算法行為參數,我們有針對常用情境的 balancer config 規範出一些固定的名稱, 參數是否支援要看 Balancer 實作本身。當指定的參數不被 balancer 實作支援時,該實作可能會丟出錯誤提示使用者。 -| config key | config value | -|--------------------------------|-------------------------------------------------------------------------------------------------------------------| -| balancer.allowed.topics.regex | 一個正則表達式,表達允許進行搬移操作的 topic 名稱,當沒有指定的時候,代表沒有任何限制,所有 topic 都可以做搬移 | -| balancer.allowed.brokers.regex | 一個正則表達式,表達允許進行搬移操作的 broker 編號,當沒有指定的時候,代表沒有任何限制,所有 broker 都可以做負載更動。當有指定時,只有那些 broker 編號有匹配此正則表達式的 broker 能進行負載的更動 | +| config key | config value | +|--------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| balancer.allowed.topics.regex | 一個正則表達式,表達允許進行搬移操作的 topic 名稱,當沒有指定的時候,代表沒有任何限制,所有 topic 都可以做搬移 | +| balancer.broker.balancing.mode | 這個參數指定每個 Broker 要採取的負載平衡設定,目前擁有的模式包含 `balancing`, `demoted` 和 `excluded`。`balancing` 代表特定節點要參予負載平衡的過程,該節點身上的負載可以被變動。`demoted` 代表特定節點身上的負載必須要全部移除,這個功能能夠協助使用者優雅地下線一個節點。`excluded` 代表特定節點不能夠參予負載平衡的過程,節點不能新增或移除負載。這個參數的格式是一系列的 key/value pair 的字串,每個 pair 之間透過逗號 "," 間隔,而 key/value 之間透過冒號 ":" 間隔,如 `(brokerId_A |"default"):(mode),(brokerId_B):(mode), ...`,其中 `key` 欄位代表這個是描述某 id 節點的設定,而對應的 `value` 欄位則是該節點要套用的負載平衡模式(`balancing`, `demoted` 或 `excluded`),另外 `key` 欄位可以填寫特殊字串 `default`,代表沒有被設定所提及的節點應該使用的負載平衡模式,比如填寫 `default:excluded` 可以使設定沒有提到的節點不參予負載平衡的過程,預設的 `default` 模式是 `balancing`,意即所有設定沒有提到的節點都將參予負載平衡的過程。 | costConfig: From 28ec2c6882c2fad0fffc7367d16e458dbd2e091c Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Tue, 23 May 2023 19:40:02 +0800 Subject: [PATCH 24/77] [METRICS] Migrate topic collector to balancer, partitioiner (#1765) --- .../java/org/astraea/app/web/WebService.java | 71 ++++++++++----- .../org/astraea/app/web/TopicHandlerTest.java | 4 +- .../org/astraea/app/web/WebServiceTest.java | 86 ++++++++++++++++++- .../partitioner/StrictCostPartitioner.java | 57 ++++++++---- .../StrictCostPartitionerTest.java | 79 +++++++++++++++++ 5 files changed, 257 insertions(+), 40 deletions(-) diff --git a/app/src/main/java/org/astraea/app/web/WebService.java b/app/src/main/java/org/astraea/app/web/WebService.java index c81f2a7b36..354bb95cf1 100644 --- a/app/src/main/java/org/astraea/app/web/WebService.java +++ b/app/src/main/java/org/astraea/app/web/WebService.java @@ -24,17 +24,20 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletionStage; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; import org.astraea.app.argument.DurationField; import org.astraea.app.argument.IntegerMapField; import org.astraea.app.argument.NonNegativeIntegerField; +import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.NodeInfo; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.MBeanClient; @@ -42,6 +45,10 @@ import org.astraea.common.metrics.collector.MetricStore; public class WebService implements AutoCloseable { + public static final String METRIC_STORE_KEY = "metric.store"; + public static final String METRIC_STORE_LOCAL = "local"; + public static final String METRIC_STORE_TOPIC = "topic"; + public static final String BOOTSTRAP_SERVERS_KEY = "bootstrap.servers"; private final HttpServer server; private final Admin admin; @@ -51,32 +58,48 @@ public WebService( Admin admin, int port, Function brokerIdToJmxPort, - Duration beanExpiration) { + Duration beanExpiration, + Configuration config) { this.admin = admin; - Supplier>> clientSupplier = + Supplier>> sensorsSupplier = () -> - admin - .brokers() - .thenApply( - brokers -> - brokers.stream() - .collect( - Collectors.toUnmodifiableMap( - NodeInfo::id, - b -> - JndiClient.of(b.host(), brokerIdToJmxPort.apply(b.id()))))); + sensors.metricSensors().stream() + .distinct() + .collect( + Collectors.toUnmodifiableMap(Function.identity(), ignored -> (id, ee) -> {})); + + List receivers = + switch (config.string(METRIC_STORE_KEY).orElse(METRIC_STORE_LOCAL)) { + case METRIC_STORE_LOCAL -> { + Function, Map> asBeanClientMap = + brokers -> + brokers.stream() + .collect( + Collectors.toUnmodifiableMap( + NodeInfo::id, + b -> JndiClient.of(b.host(), brokerIdToJmxPort.apply(b.id())))); + yield List.of( + MetricStore.Receiver.local(() -> admin.brokers().thenApply(asBeanClientMap))); + } + case METRIC_STORE_TOPIC -> List.of( + MetricStore.Receiver.topic(config.requireString(BOOTSTRAP_SERVERS_KEY)), + MetricStore.Receiver.local( + () -> CompletableFuture.completedStage(Map.of(-1, JndiClient.local())))); + default -> throw new IllegalArgumentException( + "unknown metric store type: " + + config.string(METRIC_STORE_KEY) + + ". use " + + METRIC_STORE_LOCAL + + " or " + + METRIC_STORE_TOPIC); + }; var metricStore = MetricStore.builder() .beanExpiration(beanExpiration) - .receivers(List.of(MetricStore.Receiver.local(clientSupplier))) - .sensorsSupplier( - () -> - sensors.metricSensors().stream() - .distinct() - .collect( - Collectors.toUnmodifiableMap( - Function.identity(), ignored -> (id, ee) -> {}))) + .receivers(receivers) + .sensorsSupplier(sensorsSupplier) .build(); + server = Utils.packException(() -> HttpServer.create(new InetSocketAddress(port), 0)); server.createContext("/topics", to(new TopicHandler(admin))); server.createContext("/groups", to(new GroupHandler(admin))); @@ -109,7 +132,11 @@ public static void main(String[] args) throws Exception { throw new IllegalArgumentException("you must define either --jmx.port or --jmx.ports"); try (var service = new WebService( - Admin.of(arg.configs()), arg.port, arg::jmxPortMapping, arg.beanExpiration)) { + Admin.of(arg.configs()), + arg.port, + arg::jmxPortMapping, + arg.beanExpiration, + new Configuration(arg.configs()))) { if (arg.ttl == null) { System.out.println("enter ctrl + c to terminate web service"); TimeUnit.MILLISECONDS.sleep(Long.MAX_VALUE); diff --git a/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java b/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java index 972394489e..ce8569ee66 100644 --- a/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java @@ -27,6 +27,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.admin.TopicPartition; @@ -69,7 +70,8 @@ void testWithWebService() { Admin.of(SERVICE.bootstrapServers()), 0, id -> SERVICE.jmxServiceURL().getPort(), - Duration.ofMillis(5))) { + Duration.ofMillis(5), + Configuration.EMPTY)) { Response response = HttpExecutor.builder() .build() diff --git a/app/src/test/java/org/astraea/app/web/WebServiceTest.java b/app/src/test/java/org/astraea/app/web/WebServiceTest.java index d40700a0b8..5ba689465a 100644 --- a/app/src/test/java/org/astraea/app/web/WebServiceTest.java +++ b/app/src/test/java/org/astraea/app/web/WebServiceTest.java @@ -17,13 +17,18 @@ package org.astraea.app.web; import java.time.Duration; +import java.util.Map; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; import org.astraea.app.argument.Argument; +import org.astraea.common.Configuration; import org.astraea.common.admin.Admin; +import org.astraea.common.metrics.collector.MetricStore; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.mockito.Mockito; +import org.mockito.stubbing.Answer; public class WebServiceTest { @@ -40,7 +45,9 @@ void testArgument() { @Timeout(10) @Test void testClose() { - var web = new WebService(Mockito.mock(Admin.class), 0, id -> -1, Duration.ofMillis(5)); + var web = + new WebService( + Mockito.mock(Admin.class), 0, id -> -1, Duration.ofMillis(5), Configuration.EMPTY); web.close(); } @@ -84,4 +91,81 @@ void testJmxPort() { Assertions.assertThrows( IllegalArgumentException.class, () -> noDefaultArgument.jmxPortMapping(4)); } + + @Test + void testMetricStoreConfiguration() { + try (var mockedReceiver = Mockito.mockStatic(MetricStore.Receiver.class)) { + var topicReceiverCount = new AtomicInteger(0); + var localReceiverCount = new AtomicInteger(0); + mockedReceiver + .when(() -> MetricStore.Receiver.topic(Mockito.any())) + .then( + (Answer) + invocation -> { + topicReceiverCount.incrementAndGet(); + return Mockito.mock(MetricStore.Receiver.class); + }); + mockedReceiver + .when(() -> MetricStore.Receiver.local(Mockito.any())) + .then( + (Answer) + invocation -> { + localReceiverCount.incrementAndGet(); + return Mockito.mock(MetricStore.Receiver.class); + }); + // Test default metric store configuration + try (var web = + new WebService( + Mockito.mock(Admin.class), 0, id -> -1, Duration.ofMillis(5), Configuration.EMPTY)) { + + Assertions.assertEquals(1, localReceiverCount.get()); + Assertions.assertEquals(0, topicReceiverCount.get()); + } + localReceiverCount.set(0); + topicReceiverCount.set(0); + // Test local metric store configuration + try (var web = + new WebService( + Mockito.mock(Admin.class), + 0, + id -> -1, + Duration.ofMillis(5), + new Configuration( + Map.of(WebService.METRIC_STORE_KEY, WebService.METRIC_STORE_LOCAL)))) { + + Assertions.assertEquals(1, localReceiverCount.get()); + Assertions.assertEquals(0, topicReceiverCount.get()); + } + localReceiverCount.set(0); + topicReceiverCount.set(0); + // Test topic metric store configuration + try (var web = + new WebService( + Mockito.mock(Admin.class), + 0, + id -> -1, + Duration.ofMillis(5), + new Configuration( + Map.of( + WebService.METRIC_STORE_KEY, + WebService.METRIC_STORE_TOPIC, + WebService.BOOTSTRAP_SERVERS_KEY, + "ignore")))) { + + // topic collector may create local receiver to receive local jmx metric + Assertions.assertEquals(1, topicReceiverCount.get()); + } + + // Test invalid metric store configuration + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + new WebService( + Mockito.mock(Admin.class), + 0, + id -> -1, + Duration.ofMillis(5), + new Configuration(Map.of(WebService.METRIC_STORE_KEY, "unknown")))); + } + } } diff --git a/common/src/main/java/org/astraea/common/partitioner/StrictCostPartitioner.java b/common/src/main/java/org/astraea/common/partitioner/StrictCostPartitioner.java index 8b9fa570db..e06a2983fe 100644 --- a/common/src/main/java/org/astraea/common/partitioner/StrictCostPartitioner.java +++ b/common/src/main/java/org/astraea/common/partitioner/StrictCostPartitioner.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.function.Function; import java.util.function.Supplier; @@ -38,6 +39,7 @@ import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.MBeanClient; import org.astraea.common.metrics.collector.MetricStore; +import org.astraea.common.producer.ProducerConfigs; /** * this partitioner scores the nodes by multiples cost functions. Each function evaluate the target @@ -55,6 +57,9 @@ * `org.astraea.cost.ThroughputCost=1,org.astraea.cost.broker.BrokerOutputCost=1`. */ public class StrictCostPartitioner extends Partitioner { + public static final String METRIC_STORE_KEY = "metric.store"; + public static final String METRIC_STORE_TOPIC = "topic"; + public static final String METRIC_STORE_LOCAL = "local"; static final int ROUND_ROBIN_LENGTH = 400; static final String JMX_PORT = "jmx.port"; static final String ROUND_ROBIN_LEASE_KEY = "round.robin.lease"; @@ -140,27 +145,47 @@ public void configure(Configuration config) { .string(ROUND_ROBIN_LEASE_KEY) .map(Utils::toDuration) .ifPresent(d -> this.roundRobinLease = d); - Supplier>> clientSupplier = - () -> - admin - .brokers() - .thenApply( - brokers -> { - var map = new HashMap(); - brokers.forEach( - b -> - map.put( - b.id(), JndiClient.of(b.host(), jmxPortGetter.apply(b.id())))); - // add local client to fetch consumer metrics - map.put(-1, JndiClient.local()); - return Collections.unmodifiableMap(map); - }); + List receivers = + switch (config.string(METRIC_STORE_KEY).orElse(METRIC_STORE_LOCAL)) { + case METRIC_STORE_TOPIC -> List.of( + MetricStore.Receiver.topic( + config.requireString(ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG)), + MetricStore.Receiver.local( + () -> CompletableFuture.completedStage(Map.of(-1, JndiClient.local())))); + case METRIC_STORE_LOCAL -> { + Supplier>> clientSupplier = + () -> + admin + .brokers() + .thenApply( + brokers -> { + var map = new HashMap(); + brokers.forEach( + b -> + map.put( + b.id(), + JndiClient.of(b.host(), jmxPortGetter.apply(b.id())))); + // add local client to fetch consumer metrics + map.put(-1, JndiClient.local()); + return Collections.unmodifiableMap(map); + }); + yield List.of(MetricStore.Receiver.local(clientSupplier)); + } + default -> throw new IllegalArgumentException( + "unknown metric store type: " + + config.string(METRIC_STORE_KEY) + + ". Use " + + METRIC_STORE_TOPIC + + " or " + + METRIC_STORE_LOCAL); + }; metricStore = MetricStore.builder() - .receivers(List.of(MetricStore.Receiver.local(clientSupplier))) + .receivers(receivers) .sensorsSupplier(() -> Map.of(this.costFunction.metricSensor(), (integer, e) -> {})) .build(); + this.roundRobinKeeper = RoundRobinKeeper.of(ROUND_ROBIN_LENGTH, roundRobinLease); } diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 4912bb8517..233be18563 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.Configuration; @@ -36,9 +37,12 @@ import org.astraea.common.cost.NodeThroughputCost; import org.astraea.common.cost.ReplicaLeaderCost; import org.astraea.common.metrics.ClusterBean; +import org.astraea.common.metrics.collector.MetricStore; +import org.astraea.common.producer.ProducerConfigs; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; +import org.mockito.stubbing.Answer; public class StrictCostPartitionerTest { @@ -291,4 +295,79 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { } } } + + /** Test if the partitioner use correct metric store */ + @Test + void testMetricStoreConfigure() { + try (var mockedReceiver = Mockito.mockStatic(MetricStore.Receiver.class)) { + var topicReceiverCount = new AtomicInteger(0); + var localReceiverCount = new AtomicInteger(0); + mockedReceiver + .when(() -> MetricStore.Receiver.topic(Mockito.any())) + .then( + (Answer) + invocation -> { + topicReceiverCount.incrementAndGet(); + return Mockito.mock(MetricStore.Receiver.class); + }); + mockedReceiver + .when(() -> MetricStore.Receiver.local(Mockito.any())) + .then( + (Answer) + invocation -> { + localReceiverCount.incrementAndGet(); + return Mockito.mock(MetricStore.Receiver.class); + }); + + try (var partitioner = new StrictCostPartitioner()) { + // Check default metric store + var config = + new Configuration(Map.of(ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092")); + partitioner.configure(config); + Assertions.assertNotEquals(0, localReceiverCount.get()); + Assertions.assertEquals(0, topicReceiverCount.get()); + + // Check topic metric store + localReceiverCount.set(0); + topicReceiverCount.set(0); + config = + new Configuration( + Map.of( + ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG, + "localhost:9092", + StrictCostPartitioner.METRIC_STORE_KEY, + StrictCostPartitioner.METRIC_STORE_TOPIC)); + partitioner.configure(config); + Assertions.assertNotEquals(0, topicReceiverCount.get()); + // topic collector may use local receiver to get local jmx metric + + // Check local metric store + topicReceiverCount.set(0); + localReceiverCount.set(0); + config = + new Configuration( + Map.of( + ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG, + "localhost:9092", + StrictCostPartitioner.METRIC_STORE_KEY, + StrictCostPartitioner.METRIC_STORE_LOCAL)); + partitioner.configure(config); + Assertions.assertNotEquals(0, localReceiverCount.get()); + Assertions.assertEquals(0, topicReceiverCount.get()); + + // Check unknown metric store + localReceiverCount.set(0); + topicReceiverCount.set(0); + var config2 = + new Configuration( + Map.of( + ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG, + "localhost:9092", + StrictCostPartitioner.METRIC_STORE_KEY, + "unknown")); + Assertions.assertThrows( + IllegalArgumentException.class, () -> partitioner.configure(config2)); + } + } + } } From ced840f9ded6e4167ccbbc86a20f79f7ec6be0c0 Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Wed, 24 May 2023 11:43:59 +0800 Subject: [PATCH 25/77] [COMMON] Replace NodeInfo with Broker (#1763) --- .../java/org/astraea/app/backup/Backup.java | 2 +- .../astraea/app/performance/Performance.java | 2 +- .../app/publisher/MetricPublisher.java | 16 +- .../org/astraea/app/web/BalancerHandler.java | 2 +- .../org/astraea/app/web/BrokerHandler.java | 10 +- .../astraea/app/web/ReassignmentHandler.java | 6 +- .../app/web/SkewedPartitionScenario.java | 4 +- .../org/astraea/app/web/ThrottleHandler.java | 16 +- .../org/astraea/app/web/TopicHandler.java | 2 +- .../java/org/astraea/app/web/WebService.java | 3 +- .../app/performance/PerformanceTest.java | 6 +- .../astraea/app/web/BalancerHandlerTest.java | 8 +- .../app/web/ReassignmentHandlerTest.java | 14 +- .../astraea/app/web/ThrottleHandlerTest.java | 10 +- .../java/org/astraea/common/ByteUtils.java | 19 +-- .../java/org/astraea/common/admin/Admin.java | 7 +- .../org/astraea/common/admin/AdminImpl.java | 51 +++--- .../java/org/astraea/common/admin/Broker.java | 149 +++++++----------- .../org/astraea/common/admin/ClusterInfo.java | 34 ++-- .../common/admin/ClusterInfoBuilder.java | 123 +++------------ .../astraea/common/admin/ConsumerGroup.java | 2 +- .../org/astraea/common/admin/NodeInfo.java | 89 ----------- .../common/admin/OptimizedClusterInfo.java | 19 +-- .../org/astraea/common/admin/Partition.java | 8 +- .../org/astraea/common/admin/Replica.java | 4 +- .../astraea/common/admin/ReplicaBuilder.java | 24 +-- .../common/balancer/BalancerConsoleImpl.java | 8 +- .../common/balancer/BalancerUtils.java | 20 +-- .../executor/StraightPlanExecutor.java | 3 +- .../balancer/tweakers/ShuffleTweaker.java | 10 +- .../common/cost/BrokerDiskSpaceCost.java | 6 +- .../org/astraea/common/cost/CostUtils.java | 6 +- .../astraea/common/cost/MigrationCost.java | 12 +- .../org/astraea/common/cost/NetworkCost.java | 10 +- .../common/cost/NetworkIngressCost.java | 2 +- .../astraea/common/cost/NodeMetricsCost.java | 6 +- .../astraea/common/cost/RecordSizeCost.java | 6 +- .../common/cost/ReplicaLeaderCost.java | 2 +- .../common/cost/ReplicaLeaderSizeCost.java | 2 +- .../common/cost/ReplicaNumberCost.java | 6 +- .../common/cost/utils/ClusterInfoSensor.java | 8 +- .../common/metrics/MetricSeriesBuilder.java | 16 +- .../common/partitioner/RoundRobinKeeper.java | 4 +- .../org/astraea/common/ByteUtilsTest.java | 46 ------ .../org/astraea/common/admin/AdminTest.java | 41 ++--- .../admin/AdminWithOfflineBrokerTest.java | 20 +-- .../common/admin/ClusterInfoBuilderTest.java | 60 +++---- .../astraea/common/admin/ClusterInfoTest.java | 8 +- .../admin/ClusterInfoWithOfflineNodeTest.java | 9 +- .../astraea/common/admin/NodeInfoTest.java | 38 ----- .../admin/SomePartitionOfflineTest.java | 4 +- .../balancer/BalancerConfigTestSuite.java | 36 ++--- .../common/balancer/BalancerConsoleTest.java | 2 +- .../astraea/common/balancer/BalancerTest.java | 6 +- .../common/balancer/BalancerUtilsTest.java | 22 +-- .../common/balancer/FakeClusterInfo.java | 78 ++------- .../executor/StraightPlanExecutorTest.java | 8 +- .../balancer/tweakers/ShuffleTweakerTest.java | 20 +-- .../common/cost/BrokerDiskSpaceCostTest.java | 73 +++++---- .../astraea/common/cost/CostUtilsTest.java | 41 +++-- .../common/cost/MigrationCostTest.java | 34 ++-- .../astraea/common/cost/NetworkCostTest.java | 4 +- .../common/cost/RecordSizeCostTest.java | 10 +- .../common/cost/ReplicaLeaderCostTest.java | 35 ++-- .../cost/ReplicaLeaderSizeCostTest.java | 12 +- .../cost/utils/ClusterInfoSensorTest.java | 4 +- .../common/partitioner/PartitionerTest.java | 2 +- .../StrictCostPartitionerPerfTest.java | 55 ++++--- .../StrictCostPartitionerTest.java | 14 +- .../main/java/org/astraea/gui/Context.java | 6 +- .../java/org/astraea/gui/tab/BrokerNode.java | 8 +- .../java/org/astraea/gui/tab/ClientNode.java | 2 +- .../java/org/astraea/gui/tab/SettingNode.java | 6 +- .../astraea/gui/tab/health/BalancerNode.java | 8 +- .../astraea/gui/tab/health/HealthNode.java | 10 +- .../astraea/gui/tab/topic/PartitionNode.java | 2 +- .../astraea/gui/tab/topic/ReplicaNode.java | 4 +- .../org/astraea/gui/tab/topic/TopicNode.java | 3 +- .../gui/tab/health/BalancerNodeTest.java | 13 +- .../gui/tab/topic/ReplicaNodeTest.java | 12 +- 80 files changed, 566 insertions(+), 947 deletions(-) delete mode 100644 common/src/main/java/org/astraea/common/admin/NodeInfo.java delete mode 100644 common/src/test/java/org/astraea/common/admin/NodeInfoTest.java diff --git a/app/src/main/java/org/astraea/app/backup/Backup.java b/app/src/main/java/org/astraea/app/backup/Backup.java index 4b92ee152c..055ea1c0b1 100644 --- a/app/src/main/java/org/astraea/app/backup/Backup.java +++ b/app/src/main/java/org/astraea/app/backup/Backup.java @@ -50,7 +50,7 @@ public void restoreDistribution(ClusterInfo clusterInfo, String bootstrapServers .sorted( Comparator.comparing( replica -> !replica.isLeader())) - .map(replica -> replica.nodeInfo().id()) + .map(replica -> replica.broker().id()) .toList()))))) .configs(topic.config().raw()) .run() diff --git a/app/src/main/java/org/astraea/app/performance/Performance.java b/app/src/main/java/org/astraea/app/performance/Performance.java index f3a8b3f6d1..875ccaf1b1 100644 --- a/app/src/main/java/org/astraea/app/performance/Performance.java +++ b/app/src/main/java/org/astraea/app/performance/Performance.java @@ -382,7 +382,7 @@ else if (specifiedByBroker) { .join() .replicaStream() .filter(Replica::isLeader) - .filter(replica -> specifyBrokers.contains(replica.nodeInfo().id())) + .filter(replica -> specifyBrokers.contains(replica.broker().id())) .map(replica -> TopicPartition.of(replica.topic(), replica.partition())) .distinct() .toList(); diff --git a/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java b/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java index d99c040995..e260bf42ac 100644 --- a/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java +++ b/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java @@ -25,7 +25,7 @@ import org.astraea.app.argument.StringMapField; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; -import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Broker; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.collector.MetricFetcher; @@ -50,17 +50,17 @@ static void execute(Arguments arguments) { .clientSupplier( () -> admin - .nodeInfos() + .brokers() .thenApply( - nodes -> - nodes.stream() + brokers -> + brokers.stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, - node -> + Broker::id, + broker -> JndiClient.of( - node.host(), - arguments.idToJmxPort().apply(node.id())))))) + broker.host(), + arguments.idToJmxPort().apply(broker.id())))))) .fetchBeanDelay(arguments.period) .fetchMetadataDelay(Duration.ofMinutes(5)) .threads(3) diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 84889b19a3..6a382ddba4 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -255,7 +255,7 @@ static class Placement { final Optional size; Placement(Replica replica, Optional size) { - this.brokerId = replica.nodeInfo().id(); + this.brokerId = replica.broker().id(); this.directory = replica.path(); this.size = size; } diff --git a/app/src/main/java/org/astraea/app/web/BrokerHandler.java b/app/src/main/java/org/astraea/app/web/BrokerHandler.java index 849fb4d333..a59ad5536a 100644 --- a/app/src/main/java/org/astraea/app/web/BrokerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BrokerHandler.java @@ -25,7 +25,7 @@ import java.util.concurrent.CompletionStage; import java.util.stream.Collectors; import org.astraea.common.admin.Admin; -import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.TopicPartition; class BrokerHandler implements Handler { @@ -44,8 +44,12 @@ CompletionStage> brokers(Optional target) { .orElseGet( () -> admin - .nodeInfos() - .thenApply(ns -> ns.stream().map(NodeInfo::id).collect(Collectors.toSet()))); + .brokers() + .thenApply( + ns -> + ns.stream() + .map(org.astraea.common.admin.Broker::id) + .collect(Collectors.toSet()))); } catch (NumberFormatException e) { return CompletableFuture.failedFuture( new NoSuchElementException("the broker id must be number")); diff --git a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java index f798c05362..2c2245e7e3 100644 --- a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java +++ b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java @@ -31,8 +31,8 @@ import java.util.stream.Stream; import org.astraea.common.FutureUtils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.admin.TopicPartitionReplica; @@ -110,7 +110,7 @@ public CompletionStage post(Channel channel) { availableBrokers.stream() .filter( b -> b.topicPartitions().contains(tp)) - .map(NodeInfo::id) + .map(Broker::id) .toList(); if (!ids.isEmpty()) return ids; return List.of( @@ -211,7 +211,7 @@ static class AddingReplica implements Response { AddingReplica(Replica addingReplica, long leaderSize) { this.topicName = addingReplica.topic(); this.partition = addingReplica.partition(); - this.broker = addingReplica.nodeInfo().id(); + this.broker = addingReplica.broker().id(); this.dataFolder = addingReplica.path(); this.size = addingReplica.size(); this.leaderSize = leaderSize; diff --git a/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java b/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java index 7fa6eb074c..963dbb29b8 100644 --- a/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java +++ b/app/src/main/java/org/astraea/app/web/SkewedPartitionScenario.java @@ -30,7 +30,7 @@ import org.apache.commons.math3.distribution.IntegerDistribution; import org.apache.commons.math3.util.Pair; import org.astraea.common.admin.Admin; -import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.TopicPartition; import org.astraea.common.admin.TopicPartitionReplica; @@ -62,7 +62,7 @@ public CompletionStage apply(Admin admin) { admin.waitPartitionLeaderSynced( Map.of(topicName, partitions), Duration.ofSeconds(4))) .thenCompose(ignored -> admin.brokers()) - .thenApply(brokers -> brokers.stream().map(NodeInfo::id).sorted().toList()) + .thenApply(brokers -> brokers.stream().map(Broker::id).sorted().toList()) .thenCompose( brokerIds -> { var distribution = diff --git a/app/src/main/java/org/astraea/app/web/ThrottleHandler.java b/app/src/main/java/org/astraea/app/web/ThrottleHandler.java index d47e14a856..67cc5843de 100644 --- a/app/src/main/java/org/astraea/app/web/ThrottleHandler.java +++ b/app/src/main/java/org/astraea/app/web/ThrottleHandler.java @@ -30,8 +30,8 @@ import org.astraea.common.EnumInfo; import org.astraea.common.FutureUtils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.BrokerConfigs; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.TopicConfigs; import org.astraea.common.admin.TopicPartitionReplica; import org.astraea.common.json.TypeRef; @@ -100,9 +100,8 @@ public CompletionStage post(Channel channel) { var topicToAppends = admin - .nodeInfos() - .thenApply( - nodeInfos -> nodeInfos.stream().map(NodeInfo::id).collect(Collectors.toSet())) + .brokers() + .thenApply(brokers -> brokers.stream().map(Broker::id).collect(Collectors.toSet())) .thenCompose(admin::topicPartitionReplicas) .thenApply( replicas -> @@ -197,9 +196,8 @@ public CompletionStage delete(Channel channel) { var topicToSubtracts = admin - .nodeInfos() - .thenApply( - nodeInfos -> nodeInfos.stream().map(NodeInfo::id).collect(Collectors.toSet())) + .brokers() + .thenApply(brokers -> brokers.stream().map(Broker::id).collect(Collectors.toSet())) .thenCompose(admin::topicPartitionReplicas) .thenApply( replicas -> { @@ -251,11 +249,11 @@ public CompletionStage delete(Channel channel) { var brokerToUnset = admin - .nodeInfos() + .brokers() .thenApply( ns -> ns.stream() - .map(NodeInfo::id) + .map(Broker::id) .filter( id -> !channel.queries().containsKey("broker") diff --git a/app/src/main/java/org/astraea/app/web/TopicHandler.java b/app/src/main/java/org/astraea/app/web/TopicHandler.java index 1e7beec0f2..28537281d1 100644 --- a/app/src/main/java/org/astraea/app/web/TopicHandler.java +++ b/app/src/main/java/org/astraea/app/web/TopicHandler.java @@ -316,7 +316,7 @@ private Replica() { Replica(org.astraea.common.admin.Replica replica) { this( - replica.nodeInfo().id(), + replica.broker().id(), replica.lag(), replica.size(), replica.isLeader(), diff --git a/app/src/main/java/org/astraea/app/web/WebService.java b/app/src/main/java/org/astraea/app/web/WebService.java index 354bb95cf1..e4e9eab6b4 100644 --- a/app/src/main/java/org/astraea/app/web/WebService.java +++ b/app/src/main/java/org/astraea/app/web/WebService.java @@ -38,7 +38,6 @@ import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.admin.Broker; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.MBeanClient; import org.astraea.common.metrics.collector.MetricSensor; @@ -76,7 +75,7 @@ public WebService( brokers.stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, + Broker::id, b -> JndiClient.of(b.host(), brokerIdToJmxPort.apply(b.id())))); yield List.of( MetricStore.Receiver.local(() -> admin.brokers().thenApply(asBeanClientMap))); diff --git a/app/src/test/java/org/astraea/app/performance/PerformanceTest.java b/app/src/test/java/org/astraea/app/performance/PerformanceTest.java index c1dad4673b..90f56293a5 100644 --- a/app/src/test/java/org/astraea/app/performance/PerformanceTest.java +++ b/app/src/test/java/org/astraea/app/performance/PerformanceTest.java @@ -193,7 +193,7 @@ void testPartitionSupplier() { .join() .replicaStream() .filter(Replica::isLeader) - .filter(r -> r.nodeInfo().id() == 1) + .filter(r -> r.broker().id() == 1) .map(Replica::topicPartition) .collect(Collectors.toUnmodifiableSet()); @@ -239,7 +239,7 @@ void testPartitionSupplier() { .join() .replicaStream() .filter(Replica::isLeader) - .filter(replica -> replica.nodeInfo().id() == 1) + .filter(replica -> replica.broker().id() == 1) .map(Replica::topicPartition) .collect(Collectors.toSet()); var selector2 = args.topicPartitionSelector(); @@ -273,7 +273,7 @@ void testPartitionSupplier() { .replicaStream() .findFirst() .get() - .nodeInfo() + .broker() .id(); var noPartitionBroker = (validBroker == 3) ? 1 : validBroker + 1; args = diff --git a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java index f6727111cb..53725255ed 100644 --- a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java @@ -58,8 +58,8 @@ import org.astraea.common.DataSize; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.balancer.AlgorithmConfig; @@ -976,7 +976,7 @@ void testChangeOrder() { (short) 10, r -> Replica.builder(r) - .nodeInfo(base.node(srcIter.next())) + .broker(base.node(srcIter.next())) .isPreferredLeader(srcPrefIter.next()) .path(srcDirIter.next()) .build()) @@ -992,7 +992,7 @@ void testChangeOrder() { (short) 10, r -> Replica.builder(r) - .nodeInfo(base.node(dstIter.next())) + .broker(base.node(dstIter.next())) .isPreferredLeader(dstPrefIter.next()) .path(dstDirIter.next()) .build()) @@ -1343,7 +1343,7 @@ private MetricStore metricStore(Admin admin, Set costFunctions) { brokers.stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, + Broker::id, b -> JndiClient.of(b.host(), brokerIdToJmxPort.apply(b.id()))))); var cf = Utils.costFunctions(costFunctions, HasClusterCost.class, Configuration.EMPTY); diff --git a/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java b/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java index 83577b5616..01570fdca4 100644 --- a/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java @@ -60,7 +60,7 @@ void testMigrateToAnotherBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id(); var nextBroker = SERVICE.dataFolders().keySet().stream().filter(i -> i != currentBroker).findAny().get(); @@ -88,7 +88,7 @@ void testMigrateToAnotherBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id()); } } @@ -111,7 +111,7 @@ void testMigrateToAnotherPath() { .findFirst() .get(); - var currentBroker = currentReplica.nodeInfo().id(); + var currentBroker = currentReplica.broker().id(); var currentPath = currentReplica.path(); var nextPath = SERVICE.dataFolders().get(currentBroker).stream() @@ -170,7 +170,7 @@ void testExcludeSpecificBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id(); var body = @@ -194,7 +194,7 @@ void testExcludeSpecificBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id()); Assertions.assertEquals( 0, @@ -221,7 +221,7 @@ void testExcludeSpecificBrokerTopic() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id(); var body = @@ -247,7 +247,7 @@ void testExcludeSpecificBrokerTopic() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id()); Assertions.assertNotEquals( 0, diff --git a/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java b/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java index 35ebc92214..b917b4dbb5 100644 --- a/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/ThrottleHandlerTest.java @@ -30,8 +30,8 @@ import org.astraea.common.DataRate; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.BrokerConfigs; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.TopicConfigs; import org.astraea.common.admin.TopicPartition; import org.astraea.common.json.JsonConverter; @@ -55,13 +55,13 @@ static void closeService() { public void cleanup() { try (var admin = Admin.of(SERVICE.bootstrapServers())) { admin - .nodeInfos() + .brokers() .thenApply( ns -> ns.stream() .collect( Collectors.toMap( - NodeInfo::id, + Broker::id, ignored -> Set.of( BrokerConfigs.LEADER_REPLICATION_THROTTLED_RATE_CONFIG, @@ -509,11 +509,11 @@ void testDelete() { Runnable setThrottle = () -> { admin - .nodeInfos() + .brokers() .thenApply( ns -> ns.stream() - .map(NodeInfo::id) + .map(Broker::id) .collect( Collectors.toMap( n -> n, diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index 29c304a041..405c01f3c6 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -27,9 +27,9 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Config; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.Topic; import org.astraea.common.admin.TopicPartition; @@ -186,12 +186,13 @@ public static byte[] toBytes(BeanObject value) { return beanBuilder.build().toByteArray(); } + // TODO: Due to the change of NodeInfo to Broker. This and the test should be updated. /** Serialize ClusterInfo by protocol buffer. */ public static byte[] toBytes(ClusterInfo value) { return ClusterInfoOuterClass.ClusterInfo.newBuilder() .setClusterId(value.clusterId()) .addAllNodeInfo( - value.nodes().stream() + value.brokers().stream() .map( nodeInfo -> ClusterInfoOuterClass.ClusterInfo.NodeInfo.newBuilder() @@ -223,9 +224,9 @@ public static byte[] toBytes(ClusterInfo value) { .setPartition(replica.partition()) .setNodeInfo( ClusterInfoOuterClass.ClusterInfo.NodeInfo.newBuilder() - .setId(replica.nodeInfo().id()) - .setHost(replica.nodeInfo().host()) - .setPort(replica.nodeInfo().port()) + .setId(replica.broker().id()) + .setHost(replica.broker().host()) + .setPort(replica.broker().port()) .build()) .setLag(replica.lag()) .setSize(replica.size()) @@ -327,6 +328,7 @@ public static BeanObject readBeanObject(byte[] bytes) throws SerializationExcept } } + // TODO: Due to the change of NodeInfo to Broker. This and the test should be updated. /** Deserialize to ClusterInfo with protocol buffer */ public static ClusterInfo readClusterInfo(byte[] bytes) { try { @@ -334,8 +336,7 @@ public static ClusterInfo readClusterInfo(byte[] bytes) { return ClusterInfo.of( outerClusterInfo.getClusterId(), outerClusterInfo.getNodeInfoList().stream() - .map( - nodeInfo -> NodeInfo.of(nodeInfo.getId(), nodeInfo.getHost(), nodeInfo.getPort())) + .map(nodeInfo -> Broker.of(nodeInfo.getId(), nodeInfo.getHost(), nodeInfo.getPort())) .collect(Collectors.toList()), outerClusterInfo.getTopicList().stream() .map( @@ -370,8 +371,8 @@ public Set topicPartitions() { Replica.builder() .topic(replica.getTopic()) .partition(replica.getPartition()) - .nodeInfo( - NodeInfo.of( + .broker( + Broker.of( replica.getNodeInfo().getId(), replica.getNodeInfo().getHost(), replica.getNodeInfo().getPort())) diff --git a/common/src/main/java/org/astraea/common/admin/Admin.java b/common/src/main/java/org/astraea/common/admin/Admin.java index 68a39c4b96..d6c7068ae2 100644 --- a/common/src/main/java/org/astraea/common/admin/Admin.java +++ b/common/src/main/java/org/astraea/common/admin/Admin.java @@ -164,11 +164,6 @@ default CompletionStage>>> lates CompletionStage> partitions(Set topics); - /** - * @return online node information - */ - CompletionStage> nodeInfos(); - /** * @return online broker information */ @@ -187,7 +182,7 @@ default CompletionStage>> brokerFolders() { brokers.stream() .collect( Collectors.toMap( - NodeInfo::id, + Broker::id, n -> n.dataFolders().stream() .map(Broker.DataFolder::path) diff --git a/common/src/main/java/org/astraea/common/admin/AdminImpl.java b/common/src/main/java/org/astraea/common/admin/AdminImpl.java index 1f51b1ac0a..2d0c01eba5 100644 --- a/common/src/main/java/org/astraea/common/admin/AdminImpl.java +++ b/common/src/main/java/org/astraea/common/admin/AdminImpl.java @@ -405,7 +405,11 @@ public CompletionStage> partitions(Set topics) { ts -> ts.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().isInternal()))), - (earliestOffsets, latestOffsets, maxTimestamps, tpInfos, topicAndInternal) -> + brokers() + .thenApply( + brokers -> + brokers.stream().collect(Collectors.toMap(Broker::id, broker -> broker))), + (earliestOffsets, latestOffsets, maxTimestamps, tpInfos, topicAndInternal, brokers) -> tpInfos.keySet().stream() .map( tp -> { @@ -416,11 +420,15 @@ public CompletionStage> partitions(Set topics) { var leader = tpInfo.leader() == null || tpInfo.leader().isEmpty() ? null - : NodeInfo.of(tpInfo.leader()); + : tpInfo.leader().id(); var replicas = - tpInfo.replicas().stream().map(NodeInfo::of).collect(Collectors.toList()); + tpInfo.replicas().stream() + .map(node -> brokers.getOrDefault(node.id(), Broker.of(node))) + .collect(Collectors.toList()); var isr = - tpInfo.isr().stream().map(NodeInfo::of).collect(Collectors.toList()); + tpInfo.isr().stream() + .map(node -> brokers.getOrDefault(node.id(), Broker.of(node))) + .collect(Collectors.toList()); return new Partition( tp.topic(), tp.partition(), @@ -436,13 +444,6 @@ public CompletionStage> partitions(Set topics) { .collect(Collectors.toList())); } - @Override - public CompletionStage> nodeInfos() { - return to(kafkaAdmin.describeCluster().nodes()) - .thenApply( - nodes -> nodes.stream().map(NodeInfo::of).collect(Collectors.toUnmodifiableList())); - } - @Override public CompletionStage> brokers() { return clusterIdAndBrokers().thenApply(Map.Entry::getValue); @@ -483,7 +484,7 @@ private CompletionStage>> clusterIdAndBrokers() { configs.get(String.valueOf(node.id())), logDirs.get(node.id()), topics.values())) - .sorted(Comparator.comparing(NodeInfo::id)) + .sorted(Comparator.comparing(Broker::id)) .collect(Collectors.toList()))); } @@ -522,7 +523,7 @@ public CompletionStage> consumerGroups(Set consumerG groupId, consumerGroupDescriptions.get(groupId).partitionAssignor(), consumerGroupDescriptions.get(groupId).state().name(), - NodeInfo.of(consumerGroupDescriptions.get(groupId).coordinator()), + consumerGroupDescriptions.get(groupId).coordinator().id(), consumerGroupMetadata.get(groupId).entrySet().stream() .collect( Collectors.toUnmodifiableMap( @@ -622,12 +623,7 @@ public CompletionStage clusterInfo(Set topics) { var topicMap = topicList.stream().collect(Collectors.toUnmodifiableMap(Topic::name, t -> t)); return ClusterInfo.of( - clusterIdAndBrokers.getKey(), - clusterIdAndBrokers.getValue().stream() - .map(x -> (NodeInfo) x) - .collect(Collectors.toUnmodifiableList()), - topicMap, - replicas); + clusterIdAndBrokers.getKey(), clusterIdAndBrokers.getValue(), topicMap, replicas); }); } @@ -642,7 +638,11 @@ private CompletionStage> replicas(Set topics) { // supported version: 2.4.0 // https://issues.apache.org/jira/browse/KAFKA-8345 .exceptionally(exceptionHandler(UnsupportedVersionException.class, Map.of())), - (logDirs, ts, reassignmentMap) -> + brokers() + .thenApply( + brokers -> + brokers.stream().collect(Collectors.toMap(Broker::id, broker -> broker))), + (logDirs, ts, reassignmentMap, brokers) -> ts.values().stream() .flatMap(topic -> topic.partitions().stream().map(p -> Map.entry(topic.name(), p))) .flatMap( @@ -687,7 +687,9 @@ private CompletionStage> replicas(Set topics) { .internal(internal) .isAdding(isAdding) .isRemoving(isRemoving) - .nodeInfo(NodeInfo.of(node)) + .broker( + brokers.getOrDefault( + node.id(), Broker.of(node))) .lag(pathAndReplica.getValue().offsetLag()) .size(pathAndReplica.getValue().size()) .isLeader( @@ -716,7 +718,7 @@ private CompletionStage> replicas(Set topics) { .sorted( Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) - .thenComparing(r -> r.nodeInfo().id())) + .thenComparing(r -> r.broker().id())) .collect(Collectors.toUnmodifiableList())); } @@ -1374,10 +1376,9 @@ public void close() { Integer, Map>>> logDirs() { - return nodeInfos() + return brokers() .thenApply( - nodeInfos -> - nodeInfos.stream().map(NodeInfo::id).collect(Collectors.toUnmodifiableSet())) + brokers -> brokers.stream().map(Broker::id).collect(Collectors.toUnmodifiableSet())) .thenCompose(ids -> to(kafkaAdmin.describeLogDirs(ids).allDescriptions())) .thenApply( ds -> diff --git a/common/src/main/java/org/astraea/common/admin/Broker.java b/common/src/main/java/org/astraea/common/admin/Broker.java index 69ecb71e97..9e99102178 100644 --- a/common/src/main/java/org/astraea/common/admin/Broker.java +++ b/common/src/main/java/org/astraea/common/admin/Broker.java @@ -23,9 +23,42 @@ import java.util.stream.Collectors; import org.apache.kafka.common.requests.DescribeLogDirsResponse; -public interface Broker extends NodeInfo { +/** + * @param id + * @param host + * @param port + * @param isController + * @param config config used by this node + * @param dataFolders the disk folder used to stored data by this node + * @param topicPartitions + * @param topicPartitionLeaders partition leaders hosted by this broker + */ +public record Broker( + int id, + String host, + int port, + boolean isController, + Config config, + List dataFolders, + Set topicPartitions, + Set topicPartitionLeaders) { + + /** + * @return true if the broker is offline. An offline node can't offer host or port information. + */ + public boolean offline() { + return host() == null || host().isEmpty() || port() < 0; + } + + public static Broker of(int id, String host, int port) { + return new Broker(id, host, port, false, Config.EMPTY, List.of(), Set.of(), Set.of()); + } + + public static Broker of(org.apache.kafka.common.Node node) { + return of(node.id(), node.host(), node.port()); + } - static Broker of( + public static Broker of( boolean isController, org.apache.kafka.common.Node nodeInfo, Map configs, @@ -60,26 +93,9 @@ static Broker of( tpAndSize -> !partitionsFromTopicDesc.contains(tpAndSize.getKey())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - return (DataFolder) - new DataFolder() { - - @Override - public String path() { - return path; - } - - @Override - public Map partitionSizes() { - return partitionSizes; - } - - @Override - public Map orphanPartitionSizes() { - return orphanPartitionSizes; - } - }; + return new DataFolder(path, partitionSizes, orphanPartitionSizes); }) - .collect(Collectors.toList()); + .toList(); var topicPartitionLeaders = topics.stream() .flatMap( @@ -88,83 +104,24 @@ public Map orphanPartitionSizes() { .filter(p -> p.leader() != null && p.leader().id() == nodeInfo.id()) .map(p -> TopicPartition.of(topic.name(), p.partition()))) .collect(Collectors.toUnmodifiableSet()); - return new Broker() { - @Override - public String host() { - return nodeInfo.host(); - } - - @Override - public int port() { - return nodeInfo.port(); - } - - @Override - public int id() { - return nodeInfo.id(); - } - - @Override - public boolean isController() { - return isController; - } - - @Override - public Config config() { - return config; - } - - @Override - public List dataFolders() { - return folders; - } - - @Override - public Set topicPartitions() { - return partitionsFromTopicDesc; - } - - @Override - public Set topicPartitionLeaders() { - return topicPartitionLeaders; - } - }; + return new Broker( + nodeInfo.id(), + nodeInfo.host(), + nodeInfo.port(), + isController, + config, + folders, + partitionsFromTopicDesc, + topicPartitionLeaders); } - boolean isController(); - - /** - * @return config used by this node - */ - Config config(); - - /** - * @return the disk folder used to stored data by this node - */ - List dataFolders(); - - Set topicPartitions(); - /** - * @return partition leaders hosted by this broker + * @param path the path on the local disk + * @param partitionSizes topic partition hosed by this node and size of files + * @param orphanPartitionSizes topic partition located by this node but not traced by cluster */ - Set topicPartitionLeaders(); - - interface DataFolder { - - /** - * @return the path on the local disk - */ - String path(); - - /** - * @return topic partition hosed by this node and size of files - */ - Map partitionSizes(); - - /** - * @return topic partition located by this node but not traced by cluster - */ - Map orphanPartitionSizes(); - } + public record DataFolder( + String path, + Map partitionSizes, + Map orphanPartitionSizes) {} } 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 9411ed170c..0be6ff795a 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java @@ -87,14 +87,14 @@ static boolean placementMatch( .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(r -> r.nodeInfo().id())) + .thenComparing(r -> r.broker().id())) .collect(Collectors.toUnmodifiableList()); final var targetIds = targetReplicas.stream() .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(r -> r.nodeInfo().id())) + .thenComparing(r -> r.broker().id())) .collect(Collectors.toUnmodifiableList()); return IntStream.range(0, sourceIds.size()) .allMatch( @@ -102,7 +102,7 @@ static boolean placementMatch( final var source = sourceIds.get(index); final var target = targetIds.get(index); return source.isPreferredLeader() == target.isPreferredLeader() - && source.nodeInfo().id() == target.nodeInfo().id() + && source.broker().id() == target.broker().id() && Objects.equals(source.path(), target.path()); }); } @@ -121,7 +121,7 @@ static String toString(ClusterInfo allocation) { .forEach( log -> stringBuilder.append( - String.format("(%s, %s) ", log.nodeInfo().id(), log.path()))); + String.format("(%s, %s) ", log.broker().id(), log.path()))); stringBuilder.append(System.lineSeparator()); }); @@ -147,7 +147,7 @@ static String toString(ClusterInfo allocation) { * @return cluster info */ static ClusterInfo of( - String clusterId, List nodes, Map topics, List replicas) { + String clusterId, List nodes, Map topics, List replicas) { return new OptimizedClusterInfo(clusterId, nodes, topics, replicas); } @@ -306,8 +306,8 @@ default Set topicPartitionReplicas() { * @return the node information. It throws NoSuchElementException if specify node id is not * associated to any node */ - default NodeInfo node(int id) { - return nodes().stream() + default Broker node(int id) { + return brokers().stream() .filter(n -> n.id() == id) .findAny() .orElseThrow(() -> new NoSuchElementException(id + " is nonexistent")); @@ -321,7 +321,7 @@ default Map> brokerFolders() { return brokers().stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, + Broker::id, node -> node.dataFolders().stream() .map(Broker.DataFolder::path) @@ -332,7 +332,7 @@ default Map> brokerFolders() { // implements following methods by smart index to speed up the queries default Stream replicaStream(int broker) { - return replicaStream().filter(r -> r.nodeInfo().id() == broker); + return replicaStream().filter(r -> r.broker().id() == broker); } default Stream replicaStream(String topic) { @@ -340,8 +340,7 @@ default Stream replicaStream(String topic) { } default Stream replicaStream(BrokerTopic brokerTopic) { - return replicaStream(brokerTopic.topic()) - .filter(r -> r.nodeInfo().id() == brokerTopic.broker()); + return replicaStream(brokerTopic.topic()).filter(r -> r.broker().id() == brokerTopic.broker()); } default Stream replicaStream(TopicPartition partition) { @@ -350,22 +349,15 @@ default Stream replicaStream(TopicPartition partition) { default Stream replicaStream(TopicPartitionReplica replica) { return replicaStream(replica.topicPartition()) - .filter(r -> r.nodeInfo().id() == replica.brokerId()); + .filter(r -> r.broker().id() == replica.brokerId()); } // ---------------------[abstract methods]---------------------// /** - * @return The known nodes + * @return The known brokers */ - List nodes(); - - default List brokers() { - return nodes().stream() - .filter(n -> n instanceof Broker) - .map(n -> (Broker) n) - .collect(Collectors.toUnmodifiableList()); - } + List brokers(); /** * @return replica stream to offer effective way to operate a bunch of replicas diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java index 1aa93e1c40..2661d46071 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java @@ -37,7 +37,7 @@ public class ClusterInfoBuilder { private final ClusterInfo sourceCluster; private final List< - BiFunction, List, Map.Entry, List>>> + BiFunction, List, Map.Entry, List>>> alterations; ClusterInfoBuilder(ClusterInfo source) { @@ -52,7 +52,7 @@ public class ClusterInfoBuilder { * @return this. */ public ClusterInfoBuilder applyNodes( - BiFunction, List, List> alteration) { + BiFunction, List, List> alteration) { this.alterations.add( (nodes, replicas) -> Map.entry(alteration.apply(nodes, replicas), replicas)); return this; @@ -65,7 +65,7 @@ public ClusterInfoBuilder applyNodes( * @return this. */ public ClusterInfoBuilder applyReplicas( - BiFunction, List, List> alteration) { + BiFunction, List, List> alteration) { this.alterations.add((nodes, replicas) -> Map.entry(nodes, alteration.apply(nodes, replicas))); return this; } @@ -121,14 +121,15 @@ public ClusterInfoBuilder addFolders(Map> folders) { .map( node -> { if (folders.containsKey(node.id())) - return FakeBroker.of( + return fakeBroker( node.id(), node.host(), node.port(), Stream.concat( - ((Broker) node).dataFolders().stream(), - folders.get(node.id()).stream().map(FakeDataFolder::of)) - .collect(Collectors.toUnmodifiableList())); + node.dataFolders().stream(), + folders.get(node.id()).stream() + .map(ClusterInfoBuilder::fakeDataFolder)) + .toList()); else return node; }) .collect(Collectors.toUnmodifiableList()); @@ -160,16 +161,13 @@ public ClusterInfoBuilder addTopic( String topicName, int partitionSize, short replicaFactor, Function mapper) { return applyReplicas( (nodes, replicas) -> { - if (nodes.stream().anyMatch(node -> !(node instanceof Broker))) - throw new IllegalStateException("All the nodes must include the folder info"); if (nodes.size() < replicaFactor) throw new IllegalArgumentException( "Insufficient node for this replica factor: " + nodes.size() + " < " + replicaFactor); - var nodeSelector = - Stream.generate(nodes::stream).flatMap(x -> x).map(x -> (Broker) x).iterator(); + var nodeSelector = Stream.generate(nodes::stream).flatMap(x -> x).iterator(); // simulate the actual Kafka logic of log placement var folderLogCounter = @@ -178,14 +176,13 @@ public ClusterInfoBuilder addTopic( Collectors.toUnmodifiableMap( node -> node, node -> - ((Broker) node) - .dataFolders().stream() - .collect( - Collectors.toMap( - Broker.DataFolder::path, x -> new AtomicInteger())))); + node.dataFolders().stream() + .collect( + Collectors.toMap( + Broker.DataFolder::path, x -> new AtomicInteger())))); replicas.forEach( replica -> - folderLogCounter.get(replica.nodeInfo()).get(replica.path()).incrementAndGet()); + folderLogCounter.get(replica.broker()).get(replica.path()).incrementAndGet()); folderLogCounter.forEach( (node, folders) -> { @@ -215,7 +212,7 @@ public ClusterInfoBuilder addTopic( return Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .nodeInfo(broker) + .broker(broker) .isAdding(false) .isRemoving(false) .lag(0) @@ -272,7 +269,7 @@ public ClusterInfoBuilder reassignReplica( r -> { if (r.topicPartitionReplica().equals(replica)) { matched.set(true); - return Replica.builder(r).nodeInfo(newNode).path(toDir).build(); + return Replica.builder(r).broker(newNode).path(toDir).build(); } else { return r; } @@ -322,7 +319,7 @@ public ClusterInfoBuilder setPreferredLeader(TopicPartitionReplica replica) { * ClusterInfo}. */ public ClusterInfo build() { - var nodes = sourceCluster.nodes(); + var nodes = sourceCluster.brokers(); var replicas = sourceCluster.replicas(); for (var alteration : alterations) { var e = alteration.apply(nodes, replicas); @@ -338,90 +335,14 @@ private static Broker fakeNode(int brokerId) { var port = new Random(brokerId).nextInt(65535) + 1; var folders = List.of(); - return FakeBroker.of(brokerId, host, port, folders); + return fakeBroker(brokerId, host, port, folders); } - interface FakeBroker extends Broker { - - static FakeBroker of(int id, String host, int port, List folders) { - var hashCode = Objects.hash(id, host, port); - return new FakeBroker() { - @Override - public List dataFolders() { - return folders; - } - - @Override - public String host() { - return host; - } - - @Override - public int port() { - return port; - } - - @Override - public int id() { - return id; - } - - @Override - public String toString() { - return "FakeNodeInfo{" + "host=" + host() + ", id=" + id() + ", port=" + port() + '}'; - } - - @Override - public int hashCode() { - return hashCode; - } - - @Override - public boolean equals(Object other) { - if (other instanceof NodeInfo) { - var node = (NodeInfo) other; - return id() == node.id() && port() == node.port() && host().equals(node.host()); - } - return false; - } - }; - } - - @Override - default boolean isController() { - throw new UnsupportedOperationException(); - } - - @Override - default Config config() { - throw new UnsupportedOperationException(); - } - - @Override - default Set topicPartitions() { - throw new UnsupportedOperationException(); - } - - @Override - default Set topicPartitionLeaders() { - throw new UnsupportedOperationException(); - } + static Broker fakeBroker(int Id, String host, int port, List dataFolders) { + return new Broker(Id, host, port, false, Config.EMPTY, dataFolders, Set.of(), Set.of()); } - interface FakeDataFolder extends Broker.DataFolder { - - static FakeDataFolder of(String path) { - return () -> path; - } - - @Override - default Map partitionSizes() { - throw new UnsupportedOperationException(); - } - - @Override - default Map orphanPartitionSizes() { - throw new UnsupportedOperationException(); - } + private static Broker.DataFolder fakeDataFolder(String path) { + return new Broker.DataFolder(path, Map.of(), Map.of()); } } diff --git a/common/src/main/java/org/astraea/common/admin/ConsumerGroup.java b/common/src/main/java/org/astraea/common/admin/ConsumerGroup.java index 29691e8337..ef8a696e83 100644 --- a/common/src/main/java/org/astraea/common/admin/ConsumerGroup.java +++ b/common/src/main/java/org/astraea/common/admin/ConsumerGroup.java @@ -23,6 +23,6 @@ public record ConsumerGroup( String groupId, String assignor, String state, - NodeInfo coordinator, + int coordinatorId, Map consumeProgress, Map> assignment) {} diff --git a/common/src/main/java/org/astraea/common/admin/NodeInfo.java b/common/src/main/java/org/astraea/common/admin/NodeInfo.java deleted file mode 100644 index fffd74cc35..0000000000 --- a/common/src/main/java/org/astraea/common/admin/NodeInfo.java +++ /dev/null @@ -1,89 +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.admin; - -import java.util.Objects; - -public interface NodeInfo { - - static NodeInfo of(org.apache.kafka.common.Node node) { - return of(node.id(), node.host(), node.port()); - } - - static NodeInfo of(int id, String host, int port) { - return new NodeInfo() { - @Override - public String toString() { - return "NodeInfo{" + "host=" + host + ", id=" + id + ", port=" + port + '}'; - } - - // NodeInfo is used to be key of Map commonly, so creating hash can reduce the memory pressure - private final int hashCode = Objects.hash(id, host, port); - - @Override - public String host() { - return host; - } - - @Override - public int id() { - return id; - } - - @Override - public int port() { - return port; - } - - @Override - public int hashCode() { - return hashCode; - } - - @Override - public boolean equals(Object other) { - if (other instanceof NodeInfo) { - var node = (NodeInfo) other; - return id() == node.id() && port() == node.port() && host().equals(node.host()); - } - return false; - } - }; - } - - /** - * @return The host name for this node - */ - String host(); - - /** - * @return The client (kafka data, jmx, etc.) port for this node - */ - int port(); - - /** - * @return id of broker node. it must be unique. - */ - int id(); - - /** - * @return true if the node is offline. An offline node can't offer host or port information. - */ - default boolean offline() { - return host() == null || host().isEmpty() || port() < 0; - } -} diff --git a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java index d9448b92a6..e034b9368e 100644 --- a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java @@ -28,7 +28,7 @@ /** It optimizes all queries by pre-allocated Map collection. */ class OptimizedClusterInfo implements ClusterInfo { private final String clusterId; - private final List nodeInfos; + private final List brokers; private final List all; private final Lazy> topics; @@ -44,12 +44,9 @@ class OptimizedClusterInfo implements ClusterInfo { private final Lazy>> byReplica; OptimizedClusterInfo( - String clusterId, - List nodeInfos, - Map topics, - List replicas) { + String clusterId, List brokers, Map topics, List replicas) { this.clusterId = clusterId; - this.nodeInfos = nodeInfos; + this.brokers = brokers; this.all = replicas; this.topics = Lazy.of( @@ -104,7 +101,7 @@ && internal() == objTopic.internal() all.stream() .collect( Collectors.groupingBy( - r -> BrokerTopic.of(r.nodeInfo().id(), r.topic()), + r -> BrokerTopic.of(r.broker().id(), r.topic()), Collectors.toUnmodifiableList()))); this.byBrokerTopicForLeader = Lazy.of( @@ -114,7 +111,7 @@ && internal() == objTopic.internal() .filter(Replica::isLeader) .collect( Collectors.groupingBy( - r -> BrokerTopic.of(r.nodeInfo().id(), r.topic()), + r -> BrokerTopic.of(r.broker().id(), r.topic()), Collectors.toUnmodifiableList()))); this.byBroker = @@ -123,7 +120,7 @@ && internal() == objTopic.internal() all.stream() .collect( Collectors.groupingBy( - r -> r.nodeInfo().id(), Collectors.toUnmodifiableList()))); + r -> r.broker().id(), Collectors.toUnmodifiableList()))); this.byTopic = Lazy.of( @@ -204,8 +201,8 @@ public Set topicNames() { } @Override - public List nodes() { - return nodeInfos; + public List brokers() { + return brokers; } @Override diff --git a/common/src/main/java/org/astraea/common/admin/Partition.java b/common/src/main/java/org/astraea/common/admin/Partition.java index 1bbc40ecc6..1addd8330b 100644 --- a/common/src/main/java/org/astraea/common/admin/Partition.java +++ b/common/src/main/java/org/astraea/common/admin/Partition.java @@ -24,7 +24,7 @@ * @param latestOffset existent latest offset * @param maxTimestamp max timestamp of existent records. If the kafka servers don't support to * fetch max timestamp, this method will return empty - * @param leader null if the node gets offline. otherwise, it returns node info. + * @param leaderId null if the node gets offline. otherwise, it returns node id. * @param internal true if this topic is internal (system) topic */ public record Partition( @@ -33,9 +33,9 @@ public record Partition( long earliestOffset, long latestOffset, Optional maxTimestamp, - Optional leader, - List replicas, - List isr, + Optional leaderId, + List replicas, + List isr, boolean internal) { public TopicPartition topicPartition() { diff --git a/common/src/main/java/org/astraea/common/admin/Replica.java b/common/src/main/java/org/astraea/common/admin/Replica.java index 4bab1648c2..84a82b3c13 100644 --- a/common/src/main/java/org/astraea/common/admin/Replica.java +++ b/common/src/main/java/org/astraea/common/admin/Replica.java @@ -32,7 +32,7 @@ static ReplicaBuilder builder(Replica replica) { * @return TopicPartitionReplica */ default TopicPartitionReplica topicPartitionReplica() { - return TopicPartitionReplica.of(topic(), partition(), nodeInfo().id()); + return TopicPartitionReplica.of(topic(), partition(), broker().id()); } /** @@ -57,7 +57,7 @@ default TopicPartition topicPartition() { /** * @return information of the node hosts this replica */ - NodeInfo nodeInfo(); + Broker broker(); /** * @return true if this replica is a leader replica diff --git a/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java b/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java index a499e79e1a..b7e1929643 100644 --- a/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java +++ b/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java @@ -22,7 +22,7 @@ public class ReplicaBuilder { private String topic; private int partition; - private NodeInfo nodeInfo; + private Broker broker; private long lag; private long size; @@ -40,7 +40,7 @@ public class ReplicaBuilder { ReplicaBuilder replica(Replica replica) { this.topic = replica.topic(); this.partition = replica.partition(); - this.nodeInfo = replica.nodeInfo(); + this.broker = replica.broker(); this.lag = replica.lag(); this.size = replica.size(); this.isLeader = replica.isLeader(); @@ -63,8 +63,8 @@ public ReplicaBuilder partition(int partition) { return this; } - public ReplicaBuilder nodeInfo(NodeInfo nodeInfo) { - this.nodeInfo = nodeInfo; + public ReplicaBuilder broker(Broker broker) { + this.broker = broker; return this; } @@ -166,7 +166,7 @@ public Replica build() { private static class ReplicaImpl implements Replica { private final String topic; private final int partition; - private final NodeInfo nodeInfo; + private final Broker broker; private final long lag; private final long size; @@ -185,7 +185,7 @@ private static class ReplicaImpl implements Replica { private ReplicaImpl(ReplicaBuilder builder) { this.topic = Objects.requireNonNull(builder.topic); this.partition = builder.partition; - this.nodeInfo = Objects.requireNonNull(builder.nodeInfo); + this.broker = Objects.requireNonNull(builder.broker); this.isAdding = builder.isAdding; this.isRemoving = builder.isRemoving; this.lag = builder.lag; @@ -240,8 +240,8 @@ public int partition() { } @Override - public NodeInfo nodeInfo() { - return nodeInfo; + public Broker broker() { + return broker; } @Override @@ -286,7 +286,9 @@ public boolean equals(Object o) { && isOffline == replica.isOffline && isPreferredLeader == replica.isPreferredLeader && topic.equals(replica.topic) - && nodeInfo.equals(replica.nodeInfo) + && broker.id() == replica.broker.id() + && broker.host().equals(replica.broker.host()) + && broker.port() == replica.broker.port() && Objects.equals(path, replica.path); } @@ -299,7 +301,7 @@ public String toString() { + ", partition=" + partition() + ", broker=" - + nodeInfo() + + broker() + ", path='" + path() + '\'' @@ -313,7 +315,7 @@ public int hashCode() { return Objects.hash( topic, partition, - nodeInfo, + broker, lag, size, internal, diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java index b13e679d40..aa9b5c2343 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java @@ -270,8 +270,8 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(x -> x.nodeInfo().id())) - .map(x -> Map.entry(x.nodeInfo().id(), x.path())) + .thenComparing(x -> x.broker().id())) + .map(x -> Map.entry(x.broker().id(), x.path())) .collect(Collectors.toUnmodifiableList()))); return admin .topicNames(false) @@ -292,8 +292,8 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(x -> x.nodeInfo().id())) - .map(x -> Map.entry(x.nodeInfo().id(), x.path())) + .thenComparing(x -> x.broker().id())) + .map(x -> Map.entry(x.broker().id(), x.path())) .collect(Collectors.toUnmodifiableList()))); var mismatchPartitions = before.entrySet().stream() diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java index d846905f0b..c352de416f 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java @@ -26,8 +26,8 @@ import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.EnumInfo; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; public final class BalancerUtils { @@ -57,7 +57,7 @@ public static Map balancingMode(ClusterInfo cluster, St (id) -> map.getOrDefault(id, map.getOrDefault("default", BalancingModes.BALANCING)); return cluster.brokers().stream() - .map(NodeInfo::id) + .map(Broker::id) .collect(Collectors.toUnmodifiableMap(Function.identity(), mode)); } @@ -82,7 +82,7 @@ public static void verifyClearBrokerValidness( var ongoingEventReplica = cluster.replicas().stream() - .filter(r -> isDemoted.test(r.nodeInfo().id())) + .filter(r -> isDemoted.test(r.broker().id())) .filter(r -> r.isAdding() || r.isRemoving() || r.isFuture()) .map(Replica::topicPartitionReplica) .collect(Collectors.toUnmodifiableSet()); @@ -102,7 +102,7 @@ public static void verifyClearBrokerValidness( public static ClusterInfo clearedCluster( ClusterInfo initial, Predicate clearBrokers, Predicate allowedBrokers) { final var allowed = - initial.nodes().stream() + initial.brokers().stream() .filter(node -> allowedBrokers.test(node.id())) .filter(node -> Predicate.not(clearBrokers).test(node.id())) .collect(Collectors.toUnmodifiableSet()); @@ -121,12 +121,12 @@ public static ClusterInfo clearedCluster( tp -> tp, tp -> initial.replicas(tp).stream() - .map(Replica::nodeInfo) + .map(Replica::broker) .collect(Collectors.toSet()))); return ClusterInfo.builder(initial) .mapLog( replica -> { - if (!clearBrokers.test(replica.nodeInfo().id())) return replica; + if (!clearBrokers.test(replica.broker().id())) return replica; var currentReplicaList = trackingReplicaList.get(replica.topicPartition()); var broker = IntStream.range(0, allowed.size()) @@ -139,10 +139,10 @@ public static ClusterInfo clearedCluster( "Unable to clear replica " + replica.topicPartitionReplica() + " for broker " - + replica.nodeInfo().id() + + replica.broker().id() + ", the allowed destination brokers are " + allowed.stream() - .map(NodeInfo::id) + .map(Broker::id) .collect(Collectors.toUnmodifiableSet()) + " but all of them already hosting a replica for this partition. " + "There is no broker can adopt this replica.")); @@ -150,10 +150,10 @@ public static ClusterInfo clearedCluster( // update the tracking list. have to do this to avoid putting two replicas from the // same tp to one broker. - currentReplicaList.remove(replica.nodeInfo()); + currentReplicaList.remove(replica.broker()); currentReplicaList.add(broker); - return Replica.builder(replica).nodeInfo(broker).path(folder).build(); + return Replica.builder(replica).broker(broker).path(folder).build(); }) .build(); } diff --git a/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java b/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java index d295bc66fc..d932266afd 100644 --- a/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java +++ b/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java @@ -79,8 +79,7 @@ public CompletionStage run(Admin admin, ClusterInfo logAllocation, Duratio .collect( Collectors.groupingBy( Replica::topicPartition, - Collectors.mapping( - r -> r.nodeInfo().id(), Collectors.toList())))) + Collectors.mapping(r -> r.broker().id(), Collectors.toList())))) .thenApply(ignored -> replicas)) // step 2: wait replicas get reassigned .thenCompose( diff --git a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java index b75bddfafd..5584294b6f 100644 --- a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java +++ b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java @@ -65,13 +65,13 @@ public static Builder builder() { public Stream generate(ClusterInfo baseAllocation) { // There is no broker - if (baseAllocation.nodes().isEmpty()) return Stream.of(); + if (baseAllocation.brokers().isEmpty()) return Stream.of(); // No replica to working on. if (baseAllocation.replicas().size() == 0) return Stream.of(); // Only one broker & one folder exists, unable to do any meaningful log migration - if (baseAllocation.nodes().size() == 1 + if (baseAllocation.brokers().size() == 1 && baseAllocation.brokerFolders().values().stream().findFirst().orElseThrow().size() == 1) return Stream.of(); @@ -80,7 +80,7 @@ public Stream generate(ClusterInfo baseAllocation) { .filter(tp -> this.allowedTopics.test(tp.topic())) .filter(tp -> eligiblePartition(baseAllocation.replicas(tp))) .flatMap(baseAllocation::replicaStream) - .filter(r -> this.allowedBrokers.test(r.nodeInfo().id())) + .filter(r -> this.allowedBrokers.test(r.broker().id())) .toList(); return Stream.generate( @@ -108,7 +108,7 @@ public Stream generate(ClusterInfo baseAllocation) { // leader pair follower, follower pair leader .filter(r -> r.isFollower() != sourceReplica.isFollower()) // this follower is located at allowed broker - .filter(r -> this.allowedBrokers.test(r.nodeInfo().id())) + .filter(r -> this.allowedBrokers.test(r.broker().id())) // not forbidden .filter(r -> !forbiddenReplica.contains(r.topicPartitionReplica())) .map(r -> Map.entry(r, ThreadLocalRandom.current().nextInt())) @@ -136,7 +136,7 @@ public Stream generate(ClusterInfo baseAllocation) { baseAllocation.brokers().stream() // the candidate should not be part of the replica list .filter( - b -> replicaList.stream().noneMatch(r -> r.nodeInfo().id() == b.id())) + b -> replicaList.stream().noneMatch(r -> r.broker().id() == b.id())) // should be an allowed broker .filter(b -> this.allowedBrokers.test(b.id())) .map(b -> Map.entry(b, ThreadLocalRandom.current().nextInt())) diff --git a/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java b/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java index c9925fa301..db06eb3101 100644 --- a/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java +++ b/common/src/main/java/org/astraea/common/cost/BrokerDiskSpaceCost.java @@ -22,8 +22,8 @@ import java.util.stream.Stream; import org.astraea.common.Configuration; import org.astraea.common.DataSize; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; @@ -73,8 +73,8 @@ private Map brokerMoveCostLimit(Configuration configuration) static boolean brokerDiskUsageSizeOverflow( ClusterInfo before, ClusterInfo after, Map brokerMoveCostLimit) { for (var id : - Stream.concat(before.nodes().stream(), after.nodes().stream()) - .map(NodeInfo::id) + Stream.concat(before.brokers().stream(), after.brokers().stream()) + .map(Broker::id) .parallel() .collect(Collectors.toSet())) { diff --git a/common/src/main/java/org/astraea/common/cost/CostUtils.java b/common/src/main/java/org/astraea/common/cost/CostUtils.java index b23985d657..32506b9782 100644 --- a/common/src/main/java/org/astraea/common/cost/CostUtils.java +++ b/common/src/main/java/org/astraea/common/cost/CostUtils.java @@ -19,8 +19,8 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; final class CostUtils { @@ -32,8 +32,8 @@ static boolean changedRecordSizeOverflow( var totalRemovedSize = 0L; var totalAddedSize = 0L; for (var id : - Stream.concat(before.nodes().stream(), after.nodes().stream()) - .map(NodeInfo::id) + Stream.concat(before.brokers().stream(), after.brokers().stream()) + .map(Broker::id) .parallel() .collect(Collectors.toSet())) { var removed = diff --git a/common/src/main/java/org/astraea/common/cost/MigrationCost.java b/common/src/main/java/org/astraea/common/cost/MigrationCost.java index f580d6ee71..ef673b40be 100644 --- a/common/src/main/java/org/astraea/common/cost/MigrationCost.java +++ b/common/src/main/java/org/astraea/common/cost/MigrationCost.java @@ -22,8 +22,8 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; public class MigrationCost { @@ -107,19 +107,19 @@ private static Map migratedChanged( }) .collect( Collectors.groupingBy( - r -> r.nodeInfo().id(), + r -> r.broker().id(), Collectors.mapping( Function.identity(), Collectors.summingLong(replicaFunction::apply)))); - return Stream.concat(dest.nodes().stream(), source.nodes().stream()) - .map(NodeInfo::id) + return Stream.concat(dest.brokers().stream(), source.brokers().stream()) + .map(Broker::id) .distinct() .parallel() .collect(Collectors.toMap(Function.identity(), n -> cost.getOrDefault(n, 0L))); } private static Map changedReplicaNumber(ClusterInfo before, ClusterInfo after) { - return Stream.concat(before.nodes().stream(), after.nodes().stream()) - .map(NodeInfo::id) + return Stream.concat(before.brokers().stream(), after.brokers().stream()) + .map(Broker::id) .distinct() .parallel() .collect( diff --git a/common/src/main/java/org/astraea/common/cost/NetworkCost.java b/common/src/main/java/org/astraea/common/cost/NetworkCost.java index bc12ae8e99..1619b954ee 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkCost.java @@ -28,9 +28,9 @@ import org.astraea.common.Configuration; import org.astraea.common.DataRate; import org.astraea.common.EnumInfo; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.BrokerTopic; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.cost.utils.ClusterInfoSensor; @@ -114,7 +114,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) clusterInfo.brokers().stream() .collect( Collectors.toMap( - NodeInfo::id, + Broker::id, broker -> clusterInfo .replicaStream(broker.id()) @@ -128,7 +128,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) clusterInfo.brokers().stream() .collect( Collectors.toMap( - NodeInfo::id, + Broker::id, broker -> clusterInfo .replicaStream(broker.id()) @@ -154,7 +154,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) }) .sum())); // add the brokers having no replicas into map - clusterInfo.nodes().stream() + clusterInfo.brokers().stream() .filter(node -> !brokerIngressRate.containsKey(node.id())) .forEach( node -> { @@ -210,7 +210,7 @@ private Map> mapLeaderAllocation(ClusterInfo clusterI .replicaStream() .filter(Replica::isOnline) .filter(Replica::isLeader) - .map(r -> Map.entry(BrokerTopic.of(r.nodeInfo().id(), r.topic()), r)) + .map(r -> Map.entry(BrokerTopic.of(r.broker().id(), r.topic()), r)) .collect( Collectors.groupingBy( Map.Entry::getKey, diff --git a/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java b/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java index 02d54cff92..aa4f99055f 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java @@ -60,7 +60,7 @@ public PartitionCost partitionCost(ClusterInfo clusterInfo, ClusterBean clusterB .filter(Replica::isOnline) .collect( Collectors.groupingBy( - replica -> replica.nodeInfo().id(), + replica -> replica.broker().id(), Collectors.toMap( Replica::topicPartition, r -> partitionTraffic.get(r.topicPartition())))); diff --git a/common/src/main/java/org/astraea/common/cost/NodeMetricsCost.java b/common/src/main/java/org/astraea/common/cost/NodeMetricsCost.java index 3fc5431e81..f6acad9bef 100644 --- a/common/src/main/java/org/astraea/common/cost/NodeMetricsCost.java +++ b/common/src/main/java/org/astraea/common/cost/NodeMetricsCost.java @@ -21,8 +21,8 @@ import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.metrics.ClusterBean; import org.astraea.common.metrics.client.HasNodeMetrics; import org.astraea.common.metrics.client.producer.ProducerMetrics; @@ -59,8 +59,8 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { .max() .ifPresent( max -> - clusterInfo.nodes().stream() - .map(NodeInfo::id) + clusterInfo.brokers().stream() + .map(Broker::id) .filter(id -> !result.containsKey(id)) .forEach(id -> result.put(id, max))); return () -> result; diff --git a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java index 6b1b4c6cab..6a1085f792 100644 --- a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java @@ -22,8 +22,8 @@ import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.DataSize; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; @@ -43,10 +43,10 @@ public RecordSizeCost(Configuration config) { @Override public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { var result = - clusterInfo.nodes().stream() + clusterInfo.brokers().stream() .collect( Collectors.toMap( - NodeInfo::id, + Broker::id, n -> clusterInfo.replicaStream(n.id()).mapToDouble(Replica::size).sum())); return () -> result; } diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java index 7d4f4346e6..0ea33d71d2 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java @@ -62,7 +62,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) } static Map leaderCount(ClusterInfo clusterInfo) { - return clusterInfo.nodes().stream() + return clusterInfo.brokers().stream() .map(nodeInfo -> Map.entry(nodeInfo.id(), clusterInfo.replicaLeaders(nodeInfo.id()).size())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index 6d9c72451f..e722dc141d 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -67,7 +67,7 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { clusterInfo.replicas().stream() .collect( Collectors.groupingBy( - r -> r.nodeInfo().id(), + r -> r.broker().id(), Collectors.mapping( r -> clusterInfo diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java index 7307c7e998..0f84bbe705 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java @@ -21,8 +21,8 @@ import java.util.Map; import java.util.stream.Collectors; import org.astraea.common.Configuration; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.metrics.ClusterBean; /** more replicas migrate -> higher cost */ @@ -59,12 +59,12 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) var replicaPerBroker = clusterInfo .replicaStream() - .collect(Collectors.groupingBy(r -> r.nodeInfo().id(), Collectors.counting())); + .collect(Collectors.groupingBy(r -> r.broker().id(), Collectors.counting())); var summary = replicaPerBroker.values().stream().mapToLong(x -> x).summaryStatistics(); var anyBrokerEmpty = clusterInfo.brokers().stream() - .map(NodeInfo::id) + .map(Broker::id) .anyMatch(alive -> !replicaPerBroker.containsKey(alive)); var max = summary.getMax(); var min = anyBrokerEmpty ? 0 : summary.getMin(); diff --git a/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java b/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java index ac2db0446e..df3f62b795 100644 --- a/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java +++ b/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java @@ -21,8 +21,8 @@ import java.util.Map; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; import org.astraea.common.metrics.HasBeanObject; @@ -58,8 +58,8 @@ public static ClusterInfo metricViewCluster(ClusterBean clusterBean) { var nodes = clusterBean.brokerIds().stream() .filter(id -> id != -1) - .map(id -> NodeInfo.of(id, "", -1)) - .collect(Collectors.toUnmodifiableMap(NodeInfo::id, x -> x)); + .map(id -> Broker.of(id, "", -1)) + .collect(Collectors.toUnmodifiableMap(Broker::id, x -> x)); var replicaMap = clusterBean.brokerIds().stream() .collect( @@ -108,7 +108,7 @@ public static ClusterInfo metricViewCluster(ClusterBean clusterBean) { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .nodeInfo(nodes.get(broker)) + .broker(nodes.get(broker)) .path("") .size(size.value()); var isLeader = m.value() != 0; diff --git a/common/src/main/java/org/astraea/common/metrics/MetricSeriesBuilder.java b/common/src/main/java/org/astraea/common/metrics/MetricSeriesBuilder.java index f4c00cdedc..206b7c40e7 100644 --- a/common/src/main/java/org/astraea/common/metrics/MetricSeriesBuilder.java +++ b/common/src/main/java/org/astraea/common/metrics/MetricSeriesBuilder.java @@ -27,8 +27,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.function.Bi3Function; @@ -109,7 +109,7 @@ public MetricSeriesBuilder seriesByBroker( start, (t) -> t.isBefore(end) || t.isEqual(end), (t) -> t.plus(interval)) .flatMap( time -> - cluster.nodes().stream() + cluster.brokers().stream() .map( node -> Map.entry(node.id(), seriesGenerator.apply(time, node.id())))) @@ -128,10 +128,10 @@ public MetricSeriesBuilder seriesByBrokerTopic( final var interval = sampleInterval; this.series.add( () -> - cluster.nodes().stream() + cluster.brokers().stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, + Broker::id, node -> Stream.iterate( start, @@ -159,10 +159,10 @@ public MetricSeriesBuilder seriesByBrokerPartition( final var interval = sampleInterval; this.series.add( () -> - cluster.nodes().stream() + cluster.brokers().stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, + Broker::id, node -> Stream.iterate( start, @@ -189,10 +189,10 @@ public MetricSeriesBuilder seriesByBrokerReplica( final var interval = sampleInterval; this.series.add( () -> - cluster.nodes().stream() + cluster.brokers().stream() .collect( Collectors.toUnmodifiableMap( - NodeInfo::id, + Broker::id, node -> Stream.iterate( start, diff --git a/common/src/main/java/org/astraea/common/partitioner/RoundRobinKeeper.java b/common/src/main/java/org/astraea/common/partitioner/RoundRobinKeeper.java index c0f0680a15..b93ebb3b3d 100644 --- a/common/src/main/java/org/astraea/common/partitioner/RoundRobinKeeper.java +++ b/common/src/main/java/org/astraea/common/partitioner/RoundRobinKeeper.java @@ -22,8 +22,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; public class RoundRobinKeeper { private final AtomicInteger next = new AtomicInteger(0); @@ -46,7 +46,7 @@ void tryToUpdate(ClusterInfo clusterInfo, Supplier> costToS == now) { var roundRobin = RoundRobin.smooth(costToScore.get()); var ids = - clusterInfo.nodes().stream().map(NodeInfo::id).collect(Collectors.toUnmodifiableSet()); + clusterInfo.brokers().stream().map(Broker::id).collect(Collectors.toUnmodifiableSet()); // TODO: make ROUND_ROBIN_LENGTH configurable ??? for (var index = 0; index < this.roundRobin.length; ++index) this.roundRobin[index] = roundRobin.next(ids).orElse(-1); diff --git a/common/src/test/java/org/astraea/common/ByteUtilsTest.java b/common/src/test/java/org/astraea/common/ByteUtilsTest.java index 7cdef95b3c..616229e122 100644 --- a/common/src/test/java/org/astraea/common/ByteUtilsTest.java +++ b/common/src/test/java/org/astraea/common/ByteUtilsTest.java @@ -17,11 +17,6 @@ package org.astraea.common; import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.util.Set; -import org.astraea.common.admin.Admin; -import org.astraea.common.admin.ClusterInfo; -import org.astraea.it.Service; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -73,45 +68,4 @@ void testBoolean2Bytes() { Assertions.assertArrayEquals(new byte[] {1}, ByteUtils.toBytes(true)); Assertions.assertArrayEquals(new byte[] {0}, ByteUtils.toBytes(false)); } - - @Test - void testReadAndToBytesClusterInfo() { - var topic = Utils.randomString(); - try (var service = Service.builder().numberOfBrokers(3).build()) { - try (var admin = Admin.of(service.bootstrapServers())) { - admin - .creator() - .topic(topic) - .numberOfPartitions(1) - .numberOfReplicas((short) 3) - .run() - .toCompletableFuture() - .join(); - Utils.sleep(Duration.ofSeconds(1)); - var clusterInfo = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join(); - - Assertions.assertDoesNotThrow(() -> ByteUtils.toBytes(clusterInfo)); - var bytes = ByteUtils.toBytes(clusterInfo); - Assertions.assertDoesNotThrow(() -> ByteUtils.readClusterInfo(bytes)); - var deserializedClusterInfo = ByteUtils.readClusterInfo(bytes); - - Assertions.assertEquals(clusterInfo.clusterId(), deserializedClusterInfo.clusterId()); - Assertions.assertTrue(clusterInfo.nodes().containsAll(deserializedClusterInfo.nodes())); - Assertions.assertEquals(clusterInfo.topics(), deserializedClusterInfo.topics()); - Assertions.assertEquals(clusterInfo.replicas(), deserializedClusterInfo.replicas()); - } - } - } - - @Test - void testReadAndToBytesEmptyClusterInfo() { - var clusterInfo = ClusterInfo.empty(); - var serializedInfo = ByteUtils.toBytes(clusterInfo); - var deserializedClusterInfo = ByteUtils.readClusterInfo(serializedInfo); - - Assertions.assertEquals(clusterInfo.clusterId(), deserializedClusterInfo.clusterId()); - Assertions.assertEquals(clusterInfo.nodes(), deserializedClusterInfo.nodes()); - Assertions.assertEquals(clusterInfo.topics(), deserializedClusterInfo.topics()); - Assertions.assertEquals(clusterInfo.replicas(), deserializedClusterInfo.replicas()); - } } diff --git a/common/src/test/java/org/astraea/common/admin/AdminTest.java b/common/src/test/java/org/astraea/common/admin/AdminTest.java index 42465de7a5..a0675e8184 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminTest.java @@ -24,6 +24,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.SortedSet; @@ -345,7 +346,7 @@ void testOrder() { .toCompletableFuture() .join()); - Assertions.assertInstanceOf(List.class, admin.nodeInfos().toCompletableFuture().join()); + Assertions.assertInstanceOf(List.class, admin.brokers().toCompletableFuture().join()); var brokers = admin.brokers().toCompletableFuture().join(); Assertions.assertEquals( @@ -358,7 +359,7 @@ void testOrder() { .sorted( Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) - .thenComparing(r -> r.nodeInfo().id())) + .thenComparing(r -> r.broker().id())) .collect(Collectors.toList()), replicas); } @@ -378,10 +379,10 @@ void testMoveLeaderBroker() { var ids = List.of( SERVICE.dataFolders().keySet().stream() - .filter(i -> i != partition.leader().get().id()) + .filter(i -> !Objects.equals(i, partition.leaderId().get())) .findFirst() .get(), - partition.leader().get().id()); + partition.leaderId().get()); admin.moveToBrokers(Map.of(TopicPartition.of(topic, 0), ids)).toCompletableFuture().join(); Utils.sleep(Duration.ofSeconds(2)); @@ -390,7 +391,7 @@ void testMoveLeaderBroker() { var newPartition = newPartitions.get(0); Assertions.assertEquals(ids.size(), newPartition.replicas().size()); Assertions.assertEquals(ids.size(), newPartition.isr().size()); - Assertions.assertNotEquals(ids.get(0), newPartition.leader().get().id()); + Assertions.assertNotEquals(ids.get(0), newPartition.leaderId().get()); admin .preferredLeaderElection( @@ -400,7 +401,7 @@ void testMoveLeaderBroker() { Utils.sleep(Duration.ofSeconds(2)); Assertions.assertEquals( ids.get(0), - admin.partitions(Set.of(topic)).toCompletableFuture().join().get(0).leader().get().id()); + admin.partitions(Set.of(topic)).toCompletableFuture().join().get(0).leaderId().get()); } } @@ -416,7 +417,7 @@ void testMoveToAnotherFolder() { var replica = replicas.get(0); var idAndFolder = SERVICE.dataFolders().entrySet().stream() - .filter(e -> e.getKey() == replica.nodeInfo().id()) + .filter(e -> e.getKey() == replica.broker().id()) .map(e -> Map.of(e.getKey(), e.getValue().iterator().next())) .findFirst() .get(); @@ -433,7 +434,7 @@ void testMoveToAnotherFolder() { Assertions.assertEquals(1, newReplicas.size()); var newReplica = newReplicas.get(0); - Assertions.assertEquals(idAndFolder.get(newReplica.nodeInfo().id()), newReplica.path()); + Assertions.assertEquals(idAndFolder.get(newReplica.broker().id()), newReplica.path()); } } @@ -589,7 +590,7 @@ void testDeclarePreferredFoldersWithNoCrossBrokerMovement() { r -> Replica.builder(r) .path( - source.brokerFolders().get(r.nodeInfo().id()).stream() + source.brokerFolders().get(r.broker().id()).stream() .filter(p -> !p.equals(r.path())) .findAny() .orElseThrow()) @@ -719,7 +720,7 @@ void testMigrateToOtherFolders() { .replicaStream() .iterator() .next() - .nodeInfo() + .broker() .id(); var paths = new ArrayList<>(SERVICE.dataFolders().get(id)); @@ -854,12 +855,12 @@ void testCreateTopicWithReplicasAssignment() { Assertions.assertTrue(config.raw().containsValue("lz4")); Assertions.assertEquals( List.of(0, 2), - partitions.get(0).replicas().stream().map(NodeInfo::id).collect(Collectors.toList())); + partitions.get(0).replicas().stream().map(Broker::id).collect(Collectors.toList())); Assertions.assertEquals( List.of(2, 1), - partitions.get(1).replicas().stream().map(NodeInfo::id).collect(Collectors.toList())); - Assertions.assertEquals(0, partitions.get(0).leader().get().id()); - Assertions.assertEquals(2, partitions.get(1).leader().get().id()); + partitions.get(1).replicas().stream().map(Broker::id).collect(Collectors.toList())); + Assertions.assertEquals(0, partitions.get(0).leaderId().get()); + Assertions.assertEquals(2, partitions.get(1).leaderId().get()); } } @@ -999,7 +1000,7 @@ void testConsumerGroups() { .forEach( c -> { Assertions.assertNotNull(c.groupId()); - Assertions.assertNotNull(c.coordinator()); + Assertions.assertNotNull(c.coordinatorId()); Assertions.assertNotNull(c.assignor()); Assertions.assertNotNull(c.state()); }); @@ -1047,7 +1048,7 @@ void testMigrateSinglePartition() { var partitionReplicas = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join().replicas(); return partitionReplicas.size() == 1 - && partitionReplicas.get(0).nodeInfo().id() == broker; + && partitionReplicas.get(0).broker().id() == broker; }); var currentBroker = @@ -1059,7 +1060,7 @@ void testMigrateSinglePartition() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .nodeInfo() + .broker() .id(); var allPath = admin.brokerFolders().toCompletableFuture().join(); var otherPath = @@ -1114,7 +1115,7 @@ void testIllegalMigrationArgument() { .findFirst() .get(); - var currentBroker = currentReplica.nodeInfo().id(); + var currentBroker = currentReplica.broker().id(); var notExistReplica = (currentBroker + 1) % SERVICE.dataFolders().keySet().size(); var nextDir = SERVICE.dataFolders().get(notExistReplica).iterator().next(); @@ -1144,7 +1145,7 @@ void testMigrateAllPartitions() { Utils.waitFor( () -> { var replicas = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join().replicas(); - return replicas.stream().allMatch(r -> r.nodeInfo().id() == broker); + return replicas.stream().allMatch(r -> r.broker().id() == broker); }); } } @@ -1388,7 +1389,7 @@ void testReplicasPreferredLeaderFlag() { Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()), Collectors.mapping( - replica -> replica.nodeInfo().id(), Collectors.toList()))); + replica -> replica.broker().id(), Collectors.toList()))); IntStream.range(0, partitionCount) .forEach(p -> admin.moveToBrokers(Map.of(TopicPartition.of(topic, p), List.of(0, 1, 2)))); diff --git a/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java b/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java index 0b9daa3af5..43a9ebfc3a 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java @@ -80,7 +80,7 @@ static void closeOneBroker() { var allPs = admin.partitions(Set.of(TOPIC_NAME)).toCompletableFuture().join(); NUMBER_OF_ONLINE_PARTITIONS = PARTITIONS - - (int) allPs.stream().filter(p -> p.leader().get().id() == CLOSED_BROKER_ID).count(); + - (int) allPs.stream().filter(p -> p.leaderId().get() == CLOSED_BROKER_ID).count(); Assertions.assertEquals(PARTITIONS, allPs.size()); Utils.sleep(Duration.ofSeconds(2)); } @@ -106,18 +106,6 @@ void testProducerStates() { } } - @Timeout(10) - @Test - void testNodeInfos() { - try (var admin = Admin.of(SERVICE.bootstrapServers())) { - var nodeInfos = admin.nodeInfos().toCompletableFuture().join(); - Assertions.assertEquals(2, nodeInfos.size()); - var offlineNodeInfos = - nodeInfos.stream().filter(NodeInfo::offline).collect(Collectors.toList()); - Assertions.assertEquals(0, offlineNodeInfos.size()); - } - } - @Timeout(10) @Test void testBrokers() { @@ -128,7 +116,7 @@ void testBrokers() { b -> b.dataFolders() .forEach(d -> Assertions.assertEquals(0, d.orphanPartitionSizes().size()))); - var offlineBrokers = brokers.stream().filter(NodeInfo::offline).collect(Collectors.toList()); + var offlineBrokers = brokers.stream().filter(Broker::offline).collect(Collectors.toList()); Assertions.assertEquals(0, offlineBrokers.size()); } } @@ -140,7 +128,7 @@ void testPartitions() { var partitions = admin.partitions(Set.of(TOPIC_NAME)).toCompletableFuture().join(); Assertions.assertEquals(PARTITIONS, partitions.size()); var offlinePartitions = - partitions.stream().filter(p -> p.leader().isEmpty()).collect(Collectors.toList()); + partitions.stream().filter(p -> p.leaderId().isEmpty()).collect(Collectors.toList()); offlinePartitions.forEach( p -> { Assertions.assertEquals(1, p.replicas().size()); @@ -165,7 +153,7 @@ void testReplicas() { var offlineReplicas = replicas.stream().filter(Replica::isOffline).collect(Collectors.toList()); Assertions.assertNotEquals(PARTITIONS, offlineReplicas.size()); - offlineReplicas.forEach(r -> Assertions.assertTrue(r.nodeInfo().offline())); + offlineReplicas.forEach(r -> Assertions.assertTrue(r.broker().offline())); offlineReplicas.forEach(r -> Assertions.assertNull(r.path())); offlineReplicas.forEach(r -> Assertions.assertEquals(-1, r.size())); offlineReplicas.forEach(r -> Assertions.assertEquals(-1, r.lag())); diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java index 0f2e5244d3..8bebbe37a4 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java @@ -31,14 +31,14 @@ class ClusterInfoBuilderTest { @Test void testBuild() { - var host1000 = NodeInfo.of(1000, "host1000", 9092); - var host2000 = NodeInfo.of(2000, "host2000", 9092); - var host3000 = NodeInfo.of(3000, "host3000", 9092); + var host1000 = Broker.of(1000, "host1000", 9092); + var host2000 = Broker.of(2000, "host2000", 9092); + var host3000 = Broker.of(3000, "host3000", 9092); var replica = Replica.builder() .topic("MyTopic") .partition(0) - .nodeInfo(host1000) + .broker(host1000) .size(1024) .isPreferredLeader(true) .isLeader(true) @@ -47,23 +47,23 @@ void testBuild() { ClusterInfo.of("fake", List.of(host1000, host2000, host3000), Map.of(), List.of(replica)); Assertions.assertEquals( - List.of(host1000, host2000, host3000), ClusterInfo.builder(cluster).build().nodes()); + List.of(host1000, host2000, host3000), ClusterInfo.builder(cluster).build().brokers()); Assertions.assertEquals(List.of(replica), ClusterInfo.builder(cluster).build().replicas()); - Assertions.assertEquals(List.of(), ClusterInfo.builder().build().nodes()); + Assertions.assertEquals(List.of(), ClusterInfo.builder().build().brokers()); Assertions.assertEquals(List.of(), ClusterInfo.builder().build().replicas()); } @Test void addNode() { - Assertions.assertEquals(List.of(), ClusterInfo.builder().addNode(Set.of()).build().nodes()); + Assertions.assertEquals(List.of(), ClusterInfo.builder().addNode(Set.of()).build().brokers()); Assertions.assertEquals( Set.of(1, 2, 3, 4, 5, 100), - ClusterInfo.builder().addNode(Set.of(1, 2, 3, 4, 5, 100)).build().nodes().stream() - .map(NodeInfo::id) + ClusterInfo.builder().addNode(Set.of(1, 2, 3, 4, 5, 100)).build().brokers().stream() + .map(Broker::id) .collect(Collectors.toSet())); Assertions.assertEquals( - ClusterInfo.builder().addNode(Set.of(1, 2, 3)).build().nodes(), - ClusterInfo.builder().addNode(Set.of(1, 2, 3)).build().nodes(), + ClusterInfo.builder().addNode(Set.of(1, 2, 3)).build().brokers(), + ClusterInfo.builder().addNode(Set.of(1, 2, 3)).build().brokers(), "The port number is generated by deterministic random"); } @@ -160,7 +160,7 @@ void addTopic() { path -> cluster .replicaStream() - .filter(r -> r.nodeInfo().id() == id) + .filter(r -> r.broker().id() == id) .filter(r -> r.path().equals(path)) .count())); var summary = folderLogs.values().stream().mapToLong(x -> x).summaryStatistics(); @@ -214,7 +214,7 @@ void reassignReplica() { (short) 2, (replica) -> Replica.builder(replica) - .nodeInfo(base.node(replica.isPreferredLeader() ? 1 : 2)) + .broker(base.node(replica.isPreferredLeader() ? 1 : 2)) .path(replica.isPreferredLeader() ? "/ssd1" : "/ssd2") .build()) .build(); @@ -236,22 +236,22 @@ void reassignReplica() { original .replicaStream() .filter(Replica::isPreferredLeader) - .allMatch(r -> r.nodeInfo().id() == 1 && r.path().equals("/ssd1"))); + .allMatch(r -> r.broker().id() == 1 && r.path().equals("/ssd1"))); Assertions.assertTrue( original .replicaStream() .filter(Predicate.not(Replica::isPreferredLeader)) - .allMatch(r -> r.nodeInfo().id() == 2 && r.path().equals("/ssd2"))); + .allMatch(r -> r.broker().id() == 2 && r.path().equals("/ssd2"))); Assertions.assertTrue( altered .replicaStream() .filter(Replica::isPreferredLeader) - .allMatch(r -> r.nodeInfo().id() == 3 && r.path().equals("/ssd3"))); + .allMatch(r -> r.broker().id() == 3 && r.path().equals("/ssd3"))); Assertions.assertTrue( altered .replicaStream() .filter(Predicate.not(Replica::isPreferredLeader)) - .allMatch(r -> r.nodeInfo().id() == 4 && r.path().equals("/ssd4"))); + .allMatch(r -> r.broker().id() == 4 && r.path().equals("/ssd4"))); Assertions.assertThrows( IllegalArgumentException.class, @@ -287,8 +287,8 @@ void setPreferredLeader() { (short) 4, (replica) -> Replica.builder(replica) - .isLeader(replica.nodeInfo().id() == 0) - .isPreferredLeader(replica.nodeInfo().id() == 0) + .isLeader(replica.broker().id() == 0) + .isPreferredLeader(replica.broker().id() == 0) .build()) .build(); var altered = @@ -303,19 +303,19 @@ void setPreferredLeader() { original .replicaStream() .filter((Replica::isPreferredLeader)) - .allMatch(r -> r.nodeInfo().id() == 0)); + .allMatch(r -> r.broker().id() == 0)); Assertions.assertTrue( - original.replicaStream().filter((Replica::isLeader)).allMatch(r -> r.nodeInfo().id() == 0)); + original.replicaStream().filter((Replica::isLeader)).allMatch(r -> r.broker().id() == 0)); Assertions.assertTrue( altered .replicaStream() .filter(Replica::isPreferredLeader) - .allMatch(r -> r.nodeInfo().id() == r.partition())); + .allMatch(r -> r.broker().id() == r.partition())); Assertions.assertTrue( altered .replicaStream() .filter(Replica::isLeader) - .allMatch(r -> r.nodeInfo().id() == r.partition())); + .allMatch(r -> r.broker().id() == r.partition())); Assertions.assertThrows( IllegalArgumentException.class, () -> @@ -332,9 +332,9 @@ void setPreferredLeader() { "300, host3, 3000", }) void testFakeBrokerInteraction(int id, String host, int port) { - var node0 = ClusterInfoBuilder.FakeBroker.of(id, host, port, List.of()); - var node1 = NodeInfo.of(id, host, port); - var node2 = NodeInfo.of(id + 1, host, port); + var node0 = ClusterInfoBuilder.fakeBroker(id, host, port, List.of()); + var node1 = Broker.of(id, host, port); + var node2 = Broker.of(id + 1, host, port); Assertions.assertEquals(node0.hashCode(), node1.hashCode()); Assertions.assertNotEquals(node0.hashCode(), node2.hashCode()); @@ -350,14 +350,14 @@ void testRemoveNodes() { ClusterInfo.builder(base) .removeNodes(x -> Set.of(4, 5, 6, 7, 8, 9).contains(x)) .build() - .nodes() + .brokers() .stream() - .map(NodeInfo::id) + .map(Broker::id) .collect(Collectors.toSet())); Assertions.assertEquals( Set.of(1, 3, 5, 7, 9), - ClusterInfo.builder(base).removeNodes(x -> x % 2 == 0).build().nodes().stream() - .map(NodeInfo::id) + ClusterInfo.builder(base).removeNodes(x -> x % 2 == 0).build().brokers().stream() + .map(Broker::id) .collect(Collectors.toSet())); } } 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 f965fd57ad..126b46754a 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java @@ -48,8 +48,8 @@ public static ClusterInfo of(List replicas) { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::nodeInfo) - .collect(Collectors.groupingBy(NodeInfo::id, Collectors.reducing((x, y) -> x))) + .map(Replica::broker) + .collect(Collectors.groupingBy(Broker::id, Collectors.reducing((x, y) -> x))) .values() .stream() .flatMap(Optional::stream) @@ -61,7 +61,7 @@ public static ClusterInfo of(List replicas) { @Test void testEmptyCluster() { var emptyCluster = ClusterInfo.empty(); - Assertions.assertEquals(0, emptyCluster.nodes().size()); + Assertions.assertEquals(0, emptyCluster.brokers().size()); Assertions.assertEquals(0, emptyCluster.replicaStream().count()); } @@ -100,7 +100,7 @@ void testReturnCollectionUnmodifiable() { Replica.builder() .topic("topic") .partition(0) - .nodeInfo(NodeInfo.of(0, "", -1)) + .broker(Broker.of(0, "", -1)) .path("f") .buildLeader(); Assertions.assertThrows(Exception.class, () -> cluster.replicas().add(replica)); diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java index 4eed309d31..e5b251fd95 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java @@ -77,18 +77,17 @@ void testClusterInfoWithOfflineNode() { "One of the rest replicas should take over the leadership"); Assertions.assertTrue( after.availableReplicas(topicName).stream() - .allMatch(x -> x.nodeInfo().id() != brokerToClose)); + .allMatch(x -> x.broker().id() != brokerToClose)); Assertions.assertTrue( - after.replicaLeaders(topicName).stream() - .allMatch(x -> x.nodeInfo().id() != brokerToClose)); + after.replicaLeaders(topicName).stream().allMatch(x -> x.broker().id() != brokerToClose)); Assertions.assertTrue( after.replicas(topicName).stream() .filter(Replica::isOffline) - .allMatch(x -> x.nodeInfo().id() == brokerToClose)); + .allMatch(x -> x.broker().id() == brokerToClose)); Assertions.assertTrue( after.replicas(topicName).stream() .filter(x -> !x.isOffline()) - .allMatch(x -> x.nodeInfo().id() != brokerToClose)); + .allMatch(x -> x.broker().id() != brokerToClose)); } } } diff --git a/common/src/test/java/org/astraea/common/admin/NodeInfoTest.java b/common/src/test/java/org/astraea/common/admin/NodeInfoTest.java deleted file mode 100644 index 5c3d24d9fe..0000000000 --- a/common/src/test/java/org/astraea/common/admin/NodeInfoTest.java +++ /dev/null @@ -1,38 +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.admin; - -import org.apache.kafka.common.Node; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class NodeInfoTest { - - static Node node() { - return new Node(10, "host", 100); - } - - @Test - void testAllGetters() { - var kafkaNode = node(); - var node = NodeInfo.of(kafkaNode); - - Assertions.assertEquals(kafkaNode.host(), node.host()); - Assertions.assertEquals(kafkaNode.id(), node.id()); - Assertions.assertEquals(kafkaNode.port(), node.port()); - } -} diff --git a/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java b/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java index 812d78c3cb..624379db1e 100644 --- a/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java +++ b/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java @@ -63,7 +63,7 @@ void somePartitionsOffline() { .toCompletableFuture() .join() .replicaStream() - .filter(replica -> replica.nodeInfo().id() == 0) + .filter(replica -> replica.broker().id() == 0) .collect( Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()))); @@ -78,7 +78,7 @@ void somePartitionsOffline() { .toCompletableFuture() .join() .replicaStream() - .filter(replica -> replica.nodeInfo().id() == 0) + .filter(replica -> replica.broker().id() == 0) .collect( Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()))); diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index e3c2e66b35..22c0e0a608 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -220,15 +220,12 @@ public void testBalancingModeDemoted() { .build()); Assertions.assertTrue(plan.isPresent(), testName); var finalCluster = plan.get().proposal(); - Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.nodeInfo().id() == 0)); - Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.nodeInfo().id() == 1)); - Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.nodeInfo().id() == 2)); - Assertions.assertTrue( - finalCluster.replicas().stream().noneMatch(x -> x.nodeInfo().id() == 0)); - Assertions.assertTrue( - finalCluster.replicas().stream().noneMatch(x -> x.nodeInfo().id() == 1)); - Assertions.assertTrue( - finalCluster.replicas().stream().noneMatch(x -> x.nodeInfo().id() == 2)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.broker().id() == 0)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.broker().id() == 1)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.broker().id() == 2)); + Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.broker().id() == 0)); + Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.broker().id() == 1)); + Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.broker().id() == 2)); AssertionsHelper.assertBrokerEmpty( finalCluster, (x) -> Set.of(0, 1, 2).contains(x), testName); } @@ -293,16 +290,13 @@ public void testBalancingModeDemoted() { var node123 = Stream.of(1, 2, 3).map(base::node).iterator(); var testCluster = ClusterInfo.builder(base) - .addTopic("OK", 1, (short) 1, r -> Replica.builder(r).nodeInfo(base.node(1)).build()) + .addTopic("OK", 1, (short) 1, r -> Replica.builder(r).broker(base.node(1)).build()) .addTopic( - "OK_SKIP", 2, (short) 1, r -> Replica.builder(r).nodeInfo(node12.next()).build()) + "OK_SKIP", 2, (short) 1, r -> Replica.builder(r).broker(node12.next()).build()) .addTopic( - "Replica", 1, (short) 2, r -> Replica.builder(r).nodeInfo(node13.next()).build()) + "Replica", 1, (short) 2, r -> Replica.builder(r).broker(node13.next()).build()) .addTopic( - "Partition", - 3, - (short) 1, - r -> Replica.builder(r).nodeInfo(node123.next()).build()) + "Partition", 3, (short) 1, r -> Replica.builder(r).broker(node123.next()).build()) .build(); Assertions.assertDoesNotThrow( @@ -431,15 +425,15 @@ public void testBalancingModeDemoted() { var testName = "[if replica on clear broker is adding/removing/future, raise an exception]"; var adding = ClusterInfo.builder(cluster) - .mapLog(r -> r.nodeInfo().id() != 0 ? r : Replica.builder(r).isAdding(true).build()) + .mapLog(r -> r.broker().id() != 0 ? r : Replica.builder(r).isAdding(true).build()) .build(); var removing = ClusterInfo.builder(cluster) - .mapLog(r -> r.nodeInfo().id() != 0 ? r : Replica.builder(r).isRemoving(true).build()) + .mapLog(r -> r.broker().id() != 0 ? r : Replica.builder(r).isRemoving(true).build()) .build(); var future = ClusterInfo.builder(cluster) - .mapLog(r -> r.nodeInfo().id() != 0 ? r : Replica.builder(r).isFuture(true).build()) + .mapLog(r -> r.broker().id() != 0 ? r : Replica.builder(r).isFuture(true).build()) .build(); for (var cc : List.of(adding, removing, future)) { Assertions.assertThrows( @@ -579,7 +573,7 @@ static void assertOnlyAllowedBrokerMovement( source .replicaStream() // for those replicas that are not allowed to move - .filter(r -> !allowedBroker.test(r.nodeInfo().id())) + .filter(r -> !allowedBroker.test(r.broker().id())) // they should exist as-is in the target allocation .forEach( fixedReplica -> { @@ -603,7 +597,7 @@ static void assertBrokerEmpty(ClusterInfo target, Predicate clearBroker var violated = target .replicaStream() - .filter(i -> clearBroker.test(i.nodeInfo().id())) + .filter(i -> clearBroker.test(i.broker().id())) .collect(Collectors.toUnmodifiableSet()); Assertions.assertTrue( violated.isEmpty(), diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java index d66e1f29a4..78b25a664e 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java @@ -286,7 +286,7 @@ void testCheckNoOngoingMigration() { CompletableFuture.completedFuture( ClusterInfo.of( cluster.clusterId(), - cluster.nodes(), + cluster.brokers(), cluster.topics(), cluster .replicaStream() diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerTest.java index 5fcb74dc10..9b57cfeb75 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerTest.java @@ -31,8 +31,8 @@ import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.balancer.algorithms.GreedyBalancer; @@ -73,10 +73,10 @@ void testLeaderCountRebalance(Class theClass) { .clusterInfo(admin.topicNames(false).toCompletableFuture().join()) .toCompletableFuture() .join(); - return clusterInfo.nodes().stream() + return clusterInfo.brokers().stream() .collect( Collectors.toMap( - NodeInfo::id, + Broker::id, n -> clusterInfo .replicaStream(n.id()) diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java index bf27b63426..341968b28b 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java @@ -101,9 +101,9 @@ void testVerifyClearBrokerValidness() { var iter = Stream.of(1, 2, 3).map(base::node).iterator(); var cluster = ClusterInfo.builder(base) - .addTopic("A", 1, (short) 1, r -> Replica.builder(r).nodeInfo(iter.next()).build()) - .addTopic("B", 1, (short) 1, r -> Replica.builder(r).nodeInfo(iter.next()).build()) - .addTopic("C", 1, (short) 1, r -> Replica.builder(r).nodeInfo(iter.next()).build()) + .addTopic("A", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) + .addTopic("B", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) + .addTopic("C", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) .build(); Assertions.assertThrows( @@ -173,13 +173,13 @@ void testClearedCluster() { cluster, id -> id == 1 || id == 2, id -> id == 3 || id == 4)); Assertions.assertEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 1).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 1).toList()); Assertions.assertEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 2).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 2).toList()); Assertions.assertNotEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 3).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 3).toList()); Assertions.assertNotEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.nodeInfo().id() == 4).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 4).toList()); var sameCluster = Assertions.assertDoesNotThrow( @@ -193,16 +193,16 @@ void testClearedCluster() { Assertions.assertDoesNotThrow( () -> BalancerUtils.clearedCluster(cluster, id -> id == 1, id -> id == 3)); Assertions.assertEquals( - 0, aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 1).count(), "Demoted"); + 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 1).count(), "Demoted"); Assertions.assertEquals( 100, - aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 2).count(), + aCluster.replicas().stream().filter(r -> r.broker().id() == 2).count(), "Not allowed or cleared"); Assertions.assertEquals( 100, - aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 3).count(), + aCluster.replicas().stream().filter(r -> r.broker().id() == 3).count(), "Accept replicas broker demoted broker"); Assertions.assertEquals( - 0, aCluster.replicas().stream().filter(r -> r.nodeInfo().id() == 4).count(), "Not allowed"); + 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 4).count(), "Not allowed"); } } diff --git a/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java b/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java index 3857c4d145..8331e68f5d 100644 --- a/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java +++ b/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java @@ -26,7 +26,6 @@ import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Config; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; @@ -76,70 +75,21 @@ public static ClusterInfo of( .collect(Collectors.toUnmodifiableSet()); final var nodes = IntStream.range(0, nodeCount) - .mapToObj(nodeId -> NodeInfo.of(nodeId, "host" + nodeId, 9092)) + .mapToObj(nodeId -> Broker.of(nodeId, "host" + nodeId, 9092)) .map( node -> - new Broker() { - @Override - public boolean isController() { - throw new UnsupportedOperationException(); - } - - @Override - public Config config() { - throw new UnsupportedOperationException(); - } - - @Override - public List dataFolders() { - return dataDirectories.stream() - .map( - x -> - new DataFolder() { - @Override - public String path() { - return x; - } - - @Override - public Map partitionSizes() { - throw new UnsupportedOperationException(); - } - - @Override - public Map orphanPartitionSizes() { - throw new UnsupportedOperationException(); - } - }) - .collect(Collectors.toUnmodifiableList()); - } - - @Override - public Set topicPartitions() { - throw new UnsupportedOperationException(); - } - - @Override - public Set topicPartitionLeaders() { - throw new UnsupportedOperationException(); - } - - @Override - public String host() { - return node.host(); - } - - @Override - public int port() { - return node.port(); - } - - @Override - public int id() { - return node.id(); - } - }) - .collect(Collectors.toUnmodifiableList()); + new Broker( + node.id(), + node.host(), + node.port(), + false, + new Config(Map.of()), + dataDirectories.stream() + .map(path -> new Broker.DataFolder(path, Map.of(), Map.of())) + .toList(), + Set.of(), + Set.of())) + .toList(); final var dataDirectoryList = List.copyOf(dataDirectories); final var topics = topicNameGenerator.apply(topicCount); final var replicas = @@ -156,7 +106,7 @@ public int id() { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .nodeInfo(nodes.get(r)) + .broker(nodes.get(r)) .lag(0) .size(-1) .isLeader(r == 0) diff --git a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java index 511eac0e22..59afe2d50b 100644 --- a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java +++ b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java @@ -27,9 +27,9 @@ import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoTest; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.it.Service; @@ -78,7 +78,7 @@ void testAsyncRun() { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .nodeInfo(NodeInfo.of(broker0, "", -1)) + .broker(Broker.of(broker0, "", -1)) .lag(0) .size(0) .isLeader(true) @@ -91,7 +91,7 @@ void testAsyncRun() { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .nodeInfo(NodeInfo.of(broker1, "", -1)) + .broker(Broker.of(broker1, "", -1)) .lag(0) .size(0) .isLeader(false) @@ -163,7 +163,7 @@ void testDisableDataDirMigration() { replica -> Replica.builder(replica) .path( - source.brokerFolders().get(replica.nodeInfo().id()).stream() + source.brokerFolders().get(replica.broker().id()).stream() .filter(p -> !replica.path().equals(p)) .findAny() .orElseThrow()) diff --git a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java index baef67f81a..3ce16cb6cc 100644 --- a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java @@ -25,9 +25,9 @@ import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; import org.astraea.common.Utils; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoTest; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.balancer.FakeClusterInfo; @@ -160,14 +160,14 @@ void testEligiblePartition() { 0, Set.of("/a", "/b", "c"), 1, Set.of("/a", "/b", "c"), 2, Set.of("/a", "/b", "c")); - var nodeA = NodeInfo.of(0, "", -1); - var nodeB = NodeInfo.of(1, "", -1); - var nodeC = NodeInfo.of(2, "", -1); + var nodeA = Broker.of(0, "", -1); + var nodeB = Broker.of(1, "", -1); + var nodeC = Broker.of(2, "", -1); var base = Replica.builder() .topic("topic") .partition(0) - .nodeInfo(nodeA) + .broker(nodeA) .lag(0) .size(0) .isLeader(false) @@ -185,8 +185,8 @@ void testEligiblePartition() { .isLeader(true) .isPreferredLeader(true) .build(), - Replica.builder(base).topic("normal-topic").nodeInfo(nodeB).build(), - Replica.builder(base).topic("normal-topic").nodeInfo(nodeC).build(), + Replica.builder(base).topic("normal-topic").broker(nodeB).build(), + Replica.builder(base).topic("normal-topic").broker(nodeC).build(), Replica.builder(base) .topic("offline-single") .isPreferredLeader(true) @@ -195,10 +195,10 @@ void testEligiblePartition() { Replica.builder(base) .topic("no-leader") .isPreferredLeader(true) - .nodeInfo(nodeA) + .broker(nodeA) .build(), - Replica.builder(base).topic("no-leader").nodeInfo(nodeB).build(), - Replica.builder(base).topic("no-leader").nodeInfo(nodeC).build())); + Replica.builder(base).topic("no-leader").broker(nodeB).build(), + Replica.builder(base).topic("no-leader").broker(nodeC).build())); shuffleTweaker .generate(allocation) .limit(30) diff --git a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java index f7f7dc9b82..c91f089582 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java @@ -28,7 +28,6 @@ import org.astraea.common.DataSize; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; import org.junit.jupiter.api.Assertions; @@ -54,42 +53,42 @@ void testMoveCosts() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build()); @@ -98,42 +97,42 @@ void testMoveCosts() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(dataSize.bytes()) .path("/path1") .build(), Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(dataSize.bytes()) .path("/path1") .build(), Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(dataSize.bytes()) .path("/path0") .build()); @@ -235,15 +234,15 @@ public static ClusterInfo of(List replicas) { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::nodeInfo) + .map(Replica::broker) .distinct() .map( - nodeInfo -> + broker -> Broker.of( false, - new Node(nodeInfo.id(), "", nodeInfo.port()), + new Node(broker.id(), "", broker.port()), Map.of(), - dataPath.get(nodeInfo.id()), + dataPath.get(broker.id()), List.of())) .collect(Collectors.toList()), Map.of(), @@ -282,7 +281,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(100) .isLeader(true) .path("/path0") @@ -290,7 +289,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(99) .isLeader(false) .path("/path0") @@ -298,7 +297,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(500) .isLeader(true) .path("/path0") @@ -306,7 +305,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(499) .isLeader(false) .path("/path0") @@ -314,7 +313,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(1000) .isLeader(true) .path("/path0") @@ -322,7 +321,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(1000) .isLeader(false) .path("/path0") @@ -330,15 +329,15 @@ private static ClusterInfo beforeClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::nodeInfo) + .map(Replica::broker) .distinct() .map( - nodeInfo -> + broker -> Broker.of( false, - new Node(nodeInfo.id(), "", nodeInfo.port()), + new Node(broker.id(), "", broker.port()), Map.of(), - dataPath.get(nodeInfo.id()), + dataPath.get(broker.id()), List.of())) .collect(Collectors.toList()), Map.of(), @@ -363,7 +362,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(100) .isLeader(true) .path("/path1") @@ -371,7 +370,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(99) .isLeader(false) .path("/path0") @@ -379,7 +378,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(500) .isLeader(true) .path("/path0") @@ -387,7 +386,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(500) .isLeader(false) .path("/path1") @@ -395,7 +394,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(1000) .isLeader(true) .path("/path1") @@ -403,7 +402,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(1000) .isLeader(false) .path("/path0") @@ -411,15 +410,15 @@ private static ClusterInfo afterClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::nodeInfo) + .map(Replica::broker) .distinct() .map( - nodeInfo -> + broker -> Broker.of( false, - new Node(nodeInfo.id(), "", nodeInfo.port()), + new Node(broker.id(), "", broker.port()), Map.of(), - dataPath.get(nodeInfo.id()), + dataPath.get(broker.id()), List.of())) .collect(Collectors.toList()), Map.of(), diff --git a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java index e086242e57..db1817e725 100644 --- a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java +++ b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -90,7 +89,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(100) .isLeader(true) .path("/path0") @@ -98,7 +97,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(99) .isLeader(false) .path("/path0") @@ -106,7 +105,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(500) .isLeader(true) .path("/path0") @@ -114,7 +113,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(499) .isLeader(false) .path("/path0") @@ -122,7 +121,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(1000) .isLeader(true) .path("/path0") @@ -130,7 +129,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(1000) .isLeader(false) .path("/path0") @@ -138,15 +137,15 @@ private static ClusterInfo beforeClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::nodeInfo) + .map(Replica::broker) .distinct() .map( - nodeInfo -> + broker -> Broker.of( false, - new Node(nodeInfo.id(), "", nodeInfo.port()), + new Node(broker.id(), "", broker.port()), Map.of(), - dataPath.get(nodeInfo.id()), + dataPath.get(broker.id()), List.of())) .collect(Collectors.toList()), Map.of(), @@ -171,7 +170,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(100) .isLeader(true) .path("/path1") @@ -179,7 +178,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(99) .isLeader(false) .path("/path0") @@ -187,7 +186,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(500) .isLeader(true) .path("/path0") @@ -195,7 +194,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .size(500) .isLeader(false) .path("/path1") @@ -203,7 +202,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .size(1000) .isLeader(true) .path("/path1") @@ -211,7 +210,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .size(1000) .isLeader(false) .path("/path0") @@ -219,15 +218,15 @@ private static ClusterInfo afterClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::nodeInfo) + .map(Replica::broker) .distinct() .map( - nodeInfo -> + broker -> Broker.of( false, - new Node(nodeInfo.id(), "", nodeInfo.port()), + new Node(broker.id(), "", broker.port()), Map.of(), - dataPath.get(nodeInfo.id()), + dataPath.get(broker.id()), List.of())) .collect(Collectors.toList()), Map.of(), diff --git a/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java b/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java index 89ddd813bf..c7ccc33aa0 100644 --- a/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java @@ -21,8 +21,8 @@ import static org.astraea.common.cost.MigrationCost.replicaNumChanged; import java.util.List; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfoTest; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -36,7 +36,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -49,7 +49,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -62,7 +62,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -75,7 +75,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -90,7 +90,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -103,7 +103,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -116,7 +116,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -129,7 +129,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -176,7 +176,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -189,7 +189,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -202,7 +202,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -215,7 +215,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -230,7 +230,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -243,7 +243,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .nodeInfo(NodeInfo.of(1, "broker0", 1111)) + .broker(Broker.of(1, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) @@ -256,7 +256,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(0, "broker0", 1111)) + .broker(Broker.of(0, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(true) @@ -269,7 +269,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .nodeInfo(NodeInfo.of(2, "broker0", 1111)) + .broker(Broker.of(2, "broker0", 1111)) .lag(-1) .size(-1) .isLeader(false) diff --git a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java index e85a4b0d76..80544becc8 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java @@ -184,7 +184,7 @@ void testReplicationAware() { (short) 2, (r) -> Replica.builder(r) - .nodeInfo(base.node(iter1.next())) + .broker(base.node(iter1.next())) .isPreferredLeader(iter2.next()) .isLeader(iter3.next()) .size(1) @@ -195,7 +195,7 @@ void testReplicationAware() { (short) 2, (r) -> Replica.builder(r) - .nodeInfo(base.node(iter4.next())) + .broker(base.node(iter4.next())) .isPreferredLeader(iter5.next()) .isLeader(iter6.next()) .size(1) diff --git a/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java b/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java index 75698f3ad6..6d099b062d 100644 --- a/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java @@ -18,8 +18,8 @@ import java.util.List; import java.util.Map; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.metrics.ClusterBean; @@ -34,27 +34,27 @@ public class RecordSizeCostTest { private final ClusterInfo clusterInfo = ClusterInfo.of( "fake", - List.of(NodeInfo.of(0, "aa", 22), NodeInfo.of(1, "aa", 22), NodeInfo.of(2, "aa", 22)), + List.of(Broker.of(0, "aa", 22), Broker.of(1, "aa", 22), Broker.of(2, "aa", 22)), Map.of(), List.of( Replica.builder() .topic("topic") .partition(0) - .nodeInfo(NodeInfo.of(1, "aa", 22)) + .broker(Broker.of(1, "aa", 22)) .size(100) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(0) - .nodeInfo(NodeInfo.of(2, "aa", 22)) + .broker(Broker.of(2, "aa", 22)) .size(99) .path("/tmp/aa") .buildInSyncFollower(), Replica.builder() .topic("topic") .partition(1) - .nodeInfo(NodeInfo.of(1, "aa", 22)) + .broker(Broker.of(1, "aa", 22)) .size(11) .path("/tmp/aa") .buildLeader())); diff --git a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java index 47884a01be..caa18ea1e7 100644 --- a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java @@ -21,8 +21,8 @@ import java.util.Set; import java.util.stream.Collectors; import org.astraea.common.Configuration; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; import org.junit.jupiter.api.Assertions; @@ -42,28 +42,16 @@ void testLeaderCount() { var sourceCluster = ClusterInfo.builder(baseCluster) .addTopic( - "topic1", - 3, - (short) 1, - r -> Replica.builder(r).nodeInfo(baseCluster.node(1)).build()) + "topic1", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(1)).build()) .addTopic( - "topic2", - 3, - (short) 1, - r -> Replica.builder(r).nodeInfo(baseCluster.node(2)).build()) + "topic2", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(2)).build()) .build(); var overFlowTargetCluster = ClusterInfo.builder(baseCluster) .addTopic( - "topic1", - 3, - (short) 1, - r -> Replica.builder(r).nodeInfo(baseCluster.node(2)).build()) + "topic1", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(2)).build()) .addTopic( - "topic2", - 3, - (short) 1, - r -> Replica.builder(r).nodeInfo(baseCluster.node(1)).build()) + "topic2", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(1)).build()) .build(); var overFlowMoveCost = @@ -88,30 +76,31 @@ void testNoMetrics() { .topic("topic") .partition(0) .isLeader(true) - .nodeInfo(NodeInfo.of(10, "broker0", 1111)) + .broker(Broker.of(10, "broker0", 1111)) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(1) .isLeader(true) - .nodeInfo(NodeInfo.of(10, "broker0", 1111)) + .partition(0) + .broker(Broker.of(10, "broker0", 1111)) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(0) - .nodeInfo(NodeInfo.of(11, "broker1", 1111)) .isLeader(true) + .broker(Broker.of(11, "broker1", 1111)) .path("/tmp/aa") .buildLeader()); var clusterInfo = ClusterInfo.of( "fake", List.of( - NodeInfo.of(10, "host1", 8080), - NodeInfo.of(11, "host1", 8080), - NodeInfo.of(12, "host1", 8080)), + Broker.of(10, "host1", 8080), + Broker.of(11, "host1", 8080), + Broker.of(12, "host1", 8080)), Map.of(), replicas); var brokerCost = ReplicaLeaderCost.leaderCount(clusterInfo); diff --git a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java index 8826579dd1..d78ae5ad12 100644 --- a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java @@ -18,8 +18,8 @@ import java.util.List; import java.util.Map; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.metrics.ClusterBean; @@ -74,33 +74,33 @@ private ClusterInfo clusterInfo() { .topic("t") .partition(10) .isLeader(true) - .nodeInfo(NodeInfo.of(0, "", -1)) + .broker(Broker.of(0, "", -1)) .size(777) .build(), Replica.builder() .topic("t") .partition(11) .isLeader(true) - .nodeInfo(NodeInfo.of(1, "", -1)) + .broker(Broker.of(1, "", -1)) .size(700) .build(), Replica.builder() .topic("t") .partition(12) .isLeader(true) - .nodeInfo(NodeInfo.of(2, "", -1)) + .broker(Broker.of(2, "", -1)) .size(500) .build(), Replica.builder() .topic("t") .partition(12) .isLeader(false) - .nodeInfo(NodeInfo.of(0, "", -1)) + .broker(Broker.of(0, "", -1)) .size(499) .build()); return ClusterInfo.of( "fake", - List.of(NodeInfo.of(0, "", -1), NodeInfo.of(1, "", -1), NodeInfo.of(2, "", -1)), + List.of(Broker.of(0, "", -1), Broker.of(1, "", -1), Broker.of(2, "", -1)), Map.of(), replicas); } diff --git a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java index 639d8872d1..94fbcc9b4a 100644 --- a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java +++ b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java @@ -101,7 +101,7 @@ void testClusterInfoSensor() { TopicPartition.of(topic, 1), TopicPartition.of(topic, 2)))); // compare broker id - Assertions.assertTrue(info.replicaStream().allMatch(r -> r.nodeInfo().id() == aBroker.id())); + Assertions.assertTrue(info.replicaStream().allMatch(r -> r.broker().id() == aBroker.id())); // compare replica size var realCluster = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join(); Assertions.assertTrue( @@ -241,7 +241,7 @@ void testClusterId() { var info = ClusterInfoSensor.metricViewCluster(cb); - Assertions.assertEquals(1, info.nodes().size()); + Assertions.assertEquals(1, info.brokers().size()); Assertions.assertEquals(id, info.clusterId()); } diff --git a/common/src/test/java/org/astraea/common/partitioner/PartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/PartitionerTest.java index af5fa82063..a877df3912 100644 --- a/common/src/test/java/org/astraea/common/partitioner/PartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/PartitionerTest.java @@ -79,7 +79,7 @@ public int partition(String topic, byte[] key, byte[] value, ClusterInfo cluster Map.of(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, SERVICE.bootstrapServers())); Assertions.assertNotNull(partitioner.admin); Utils.sleep(Duration.ofSeconds(3)); - Assertions.assertNotEquals(0, partitioner.clusterInfo.nodes().size()); + Assertions.assertNotEquals(0, partitioner.clusterInfo.brokers().size()); } } diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java index 1fbaad6424..cef2691d5b 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java @@ -29,33 +29,34 @@ import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; -import org.astraea.common.metrics.MBeanRegister; +import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.ClusterBean; +import org.astraea.common.metrics.client.HasNodeMetrics; +import org.astraea.common.metrics.collector.MetricStore; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; public class StrictCostPartitionerPerfTest { - private static AtomicLong createMetric(int brokerId) { - var latency = new AtomicLong(100); - MBeanRegister.local() - .domainName("kafka.producer") - .property("type", "producer-node-metrics") - .property("node-id", String.valueOf(brokerId)) - .property("client-id", "xxxx") - .attribute("request-latency-avg", Double.class, () -> (double) latency.get()) - .register(); - return latency; + private static BeanObject getBeanObject(int brokerId, double latency) { + return new BeanObject( + "kafka.producer", + Map.of( + "type", "producer-node-metrics", + "node-id", String.valueOf(brokerId), + "client-id", "xxxx"), + Map.of("request-latency-avg", latency)); } @Test void test() { - var node0 = NodeInfo.of(0, "node0", 2222); - var node1 = NodeInfo.of(1, "node1", 2222); - var node2 = NodeInfo.of(2, "node2", 2222); + var node0 = Broker.of(0, "node0", 2222); + var node1 = Broker.of(1, "node1", 2222); + var node2 = Broker.of(2, "node2", 2222); var clusterInfo = ClusterInfo.of( "fake", @@ -65,19 +66,19 @@ void test() { Replica.builder() .topic("topic") .partition(0) - .nodeInfo(node0) + .broker(node0) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(1) - .nodeInfo(node1) + .broker(node1) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(2) - .nodeInfo(node2) + .broker(node2) .path("/tmp/aa") .buildLeader())); var admin = Mockito.mock(Admin.class); @@ -86,15 +87,27 @@ void test() { Mockito.when(admin.clusterInfo(Mockito.anySet())) .thenReturn(CompletableFuture.completedStage(clusterInfo)); - var node0Latency = createMetric(0); - var node1Latency = createMetric(1); - var node2Latency = createMetric(2); + var node0Latency = new AtomicLong(100); + var node1Latency = new AtomicLong(100); + var node2Latency = new AtomicLong(100); + + var metricStore = Mockito.mock(MetricStore.class); + Mockito.when(metricStore.clusterBean()) + .thenReturn( + ClusterBean.of( + Map.of( + -1, + List.of( + (HasNodeMetrics) () -> getBeanObject(0, node0Latency.get()), + () -> getBeanObject(1, node1Latency.get()), + () -> getBeanObject(2, node2Latency.get()))))); var key = "key".getBytes(StandardCharsets.UTF_8); var value = "value".getBytes(StandardCharsets.UTF_8); try (var partitioner = new StrictCostPartitioner()) { partitioner.admin = admin; partitioner.configure(new Configuration(Map.of("round.robin.lease", "2s"))); + partitioner.metricStore = metricStore; Supplier>> resultSupplier = () -> { diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 233be18563..8809be5932 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -26,9 +26,9 @@ import java.util.stream.IntStream; import org.astraea.common.Configuration; import org.astraea.common.Utils; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoTest; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.cost.BrokerCost; import org.astraea.common.cost.BrokerInputCost; @@ -110,13 +110,13 @@ void testNoAvailableBrokers() { @Test void testSingleBroker() { - var nodeInfo = NodeInfo.of(10, "host", 11111); + var nodeInfo = Broker.of(10, "host", 11111); var replicaInfo = Replica.builder() .topic("topic") .partition(10) .path("/tmp/aa") - .nodeInfo(nodeInfo) + .broker(nodeInfo) .buildLeader(); try (var partitioner = new StrictCostPartitioner()) { partitioner.configure(Configuration.EMPTY); @@ -142,14 +142,14 @@ void testCostFunctionWithoutSensor() { .topic("topic") .partition(0) .path("/tmp/aa") - .nodeInfo(NodeInfo.of(10, "host", 11111)) + .broker(Broker.of(10, "host", 11111)) .buildLeader(); var replicaInfo1 = Replica.builder() .topic("topic") .partition(0) .path("/tmp/aa") - .nodeInfo(NodeInfo.of(12, "host2", 11111)) + .broker(Broker.of(12, "host2", 11111)) .buildLeader(); try (var partitioner = new StrictCostPartitioner()) { partitioner.configure( @@ -196,14 +196,14 @@ void testReturnedPartition() { .topic("topic") .partition(partitionId) .path("/tmp/aa") - .nodeInfo(NodeInfo.of(brokerId, "host", 11111)) + .broker(Broker.of(brokerId, "host", 11111)) .buildLeader(); var replicaInfo1 = Replica.builder() .topic("topic") .partition(1) .path("/tmp/aa") - .nodeInfo(NodeInfo.of(1111, "host2", 11111)) + .broker(Broker.of(1111, "host2", 11111)) .buildLeader(); Assertions.assertEquals( partitionId, diff --git a/gui/src/main/java/org/astraea/gui/Context.java b/gui/src/main/java/org/astraea/gui/Context.java index ecf305b7a3..c10baec19e 100644 --- a/gui/src/main/java/org/astraea/gui/Context.java +++ b/gui/src/main/java/org/astraea/gui/Context.java @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicReference; import javafx.stage.Stage; import org.astraea.common.admin.Admin; -import org.astraea.common.admin.NodeInfo; +import org.astraea.common.admin.Broker; import org.astraea.common.connector.ConnectorClient; import org.astraea.common.metrics.JndiClient; @@ -73,9 +73,9 @@ public void workerJmxPort(int workerJmxPort) { } @SuppressWarnings("resource") - public Map addBrokerClients(List nodeInfos) { + public Map addBrokerClients(List brokers) { if (brokerClients == null) return Map.of(); - nodeInfos.forEach( + brokers.forEach( n -> brokerClients.clients.computeIfAbsent( n.id(), ignored -> JndiClient.of(n.host(), brokerClients.jmxPort))); diff --git a/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java b/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java index d66ac07ae0..e9655be4aa 100644 --- a/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java @@ -38,7 +38,6 @@ import org.astraea.common.MapUtils; import org.astraea.common.admin.Broker; import org.astraea.common.admin.BrokerConfigs; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.TopicPartition; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.broker.ControllerMetrics; @@ -224,7 +223,7 @@ static Node metricsNode(Context context) { (argument, logger) -> context .admin() - .nodeInfos() + .brokers() .thenApply( nodes -> context.addBrokerClients(nodes).entrySet().stream() @@ -378,12 +377,11 @@ private static Node configNode(Context context) { var unset = brokers.stream() .collect( - Collectors.toMap( - NodeInfo::id, b -> input.emptyValueKeys())); + Collectors.toMap(Broker::id, b -> input.emptyValueKeys())); var set = brokers.stream() .collect( - Collectors.toMap(NodeInfo::id, b -> input.nonEmptyTexts())); + Collectors.toMap(Broker::id, b -> input.nonEmptyTexts())); if (unset.isEmpty() && set.isEmpty()) { logger.log("nothing to alter"); return CompletableFuture.completedStage(null); diff --git a/gui/src/main/java/org/astraea/gui/tab/ClientNode.java b/gui/src/main/java/org/astraea/gui/tab/ClientNode.java index f3378f5111..abad4cd470 100644 --- a/gui/src/main/java/org/astraea/gui/tab/ClientNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/ClientNode.java @@ -184,7 +184,7 @@ static List> consumerResult( result.put("group", cg.groupId()); result.put("assignor", cg.assignor()); result.put("state", cg.state()); - result.put("coordinator", cg.coordinator().id()); + result.put("coordinator", cg.coordinatorId()); result.put("topic", tp.topic()); result.put("partition", tp.partition()); Optional.ofNullable(cg.consumeProgress().get(tp)) diff --git a/gui/src/main/java/org/astraea/gui/tab/SettingNode.java b/gui/src/main/java/org/astraea/gui/tab/SettingNode.java index 2b03786ddd..1e0f40b006 100644 --- a/gui/src/main/java/org/astraea/gui/tab/SettingNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/SettingNode.java @@ -149,16 +149,16 @@ public static Node of(Context context) { .url(Utils.packException(() -> new URL("http://" + url))) .build()); return FutureUtils.combine( - newAdmin.nodeInfos(), + newAdmin.brokers(), client .map(ConnectorClient::activeWorkers) .orElse(CompletableFuture.completedFuture(List.of())), - (nodeInfos, workers) -> { + (brokers, workers) -> { context.replace(newAdmin); client.ifPresent(context::replace); prop.brokerJmxPort.ifPresent(context::brokerJmxPort); prop.workerJmxPort.ifPresent(context::workerJmxPort); - context.addBrokerClients(nodeInfos); + context.addBrokerClients(brokers); context.addWorkerClients( workers.stream() .map(WorkerStatus::hostname) diff --git a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java index 2561971e49..b2ccf6415d 100644 --- a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java @@ -123,20 +123,20 @@ static List> assignmentResult(Balancer.Plan plan) { .filter(Replica::isLeader) .findFirst() .ifPresent( - r -> result.put(PREVIOUS_LEADER_KEY, r.nodeInfo().id() + ":" + r.path())); + r -> result.put(PREVIOUS_LEADER_KEY, r.broker().id() + ":" + r.path())); newAssignments.stream() .filter(Replica::isLeader) .findFirst() - .ifPresent(r -> result.put(NEW_LEADER_KEY, r.nodeInfo().id() + ":" + r.path())); + .ifPresent(r -> result.put(NEW_LEADER_KEY, r.broker().id() + ":" + r.path())); var previousFollowers = previousAssignments.stream() .filter(r -> !r.isLeader()) - .map(r -> r.nodeInfo().id() + ":" + r.path()) + .map(r -> r.broker().id() + ":" + r.path()) .collect(Collectors.joining(",")); var newFollowers = newAssignments.stream() .filter(r -> !r.isLeader()) - .map(r -> r.nodeInfo().id() + ":" + r.path()) + .map(r -> r.broker().id() + ":" + r.path()) .collect(Collectors.joining(",")); if (!previousFollowers.isBlank()) result.put(PREVIOUS_FOLLOWER_KEY, previousFollowers); diff --git a/gui/src/main/java/org/astraea/gui/tab/health/HealthNode.java b/gui/src/main/java/org/astraea/gui/tab/health/HealthNode.java index 168161d10c..93fac415f2 100644 --- a/gui/src/main/java/org/astraea/gui/tab/health/HealthNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/health/HealthNode.java @@ -109,15 +109,13 @@ static CompletionStage>> unavailablePartitions(Admin ad .filter( p -> p.isr().size() < minInSync.getOrDefault(p.topic(), 1) - || p.leader().isEmpty()) + || p.leaderId().isEmpty()) .map( p -> { var r = new LinkedHashMap(); r.put("topic", p.topic()); r.put("partition", p.partition()); - r.put( - "leader", - p.leader().map(n -> String.valueOf(n.id())).orElse("null")); + r.put("leader", p.leaderId().map(String::valueOf).orElse("null")); r.put( "in-sync replicas", p.isr().stream() @@ -126,10 +124,10 @@ static CompletionStage>> unavailablePartitions(Admin ad r.put( TopicConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, minInSync.getOrDefault(p.topic(), 1)); - r.put("readable", p.leader().isPresent()); + r.put("readable", p.leaderId().isPresent()); r.put( "writable", - p.leader().isPresent() + p.leaderId().isPresent() && p.isr().size() >= minInSync.getOrDefault(p.topic(), 1)); return (Map) r; }) diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/PartitionNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/PartitionNode.java index b1d6e43416..49df25c46d 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/PartitionNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/PartitionNode.java @@ -59,7 +59,7 @@ private static List> basicResult(List ps) { result.put(TOPIC_NAME_KEY, p.topic()); result.put(PARTITION_KEY, p.partition()); result.put("internal", p.internal()); - p.leader().ifPresent(l -> result.put("leader", l.id())); + p.leaderId().ifPresent(l -> result.put("leader", l)); result.put( "replicas", p.replicas().stream() diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java index 3ee8168fd9..8d0e1905a5 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java @@ -64,7 +64,7 @@ static List> allResult(ClusterInfo clusterInfo) { result.put(TOPIC_NAME_KEY, replica.topic()); result.put(PARTITION_KEY, replica.partition()); result.put("internal", replica.internal()); - result.put("broker", replica.nodeInfo().id()); + result.put("broker", replica.broker().id()); if (replica.path() != null) result.put(PATH_KEY, replica.path()); result.put("isLeader", replica.isLeader()); result.put("isPreferredLeader", replica.isPreferredLeader()); @@ -195,7 +195,7 @@ static List> allResult(ClusterInfo clusterInfo) { .allMatch( entry -> rs.replicas(entry.getKey()).stream() - .map(r -> r.nodeInfo().id()) + .map(r -> r.broker().id()) .collect(Collectors.toSet()) .containsAll(entry.getValue())), Duration.ofSeconds(10), diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java index bf3ec4fa2d..7fcdf3e730 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java @@ -38,7 +38,6 @@ import org.astraea.common.MapUtils; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ConsumerGroup; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Partition; import org.astraea.common.admin.ProducerState; import org.astraea.common.admin.TopicConfigs; @@ -413,7 +412,7 @@ private static List> basicResult( "number of producer id", topicProducers.getOrDefault(topic, Set.of()).size()); ps.stream() .flatMap(p -> p.replicas().stream()) - .collect(Collectors.groupingBy(NodeInfo::id)) + .collect(Collectors.groupingBy(Broker::id)) .entrySet() .stream() .sorted(Map.Entry.comparingByKey()) diff --git a/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java b/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java index ebfe02274f..90ca3ca23b 100644 --- a/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java +++ b/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java @@ -25,8 +25,8 @@ import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.common.balancer.Balancer; import org.astraea.common.cost.ReplicaLeaderSizeCost; @@ -104,8 +104,7 @@ void testGenerator() { void testResult() { var topic = Utils.randomString(); var leaderSize = 100; - var allNodes = - List.of(NodeInfo.of(0, "aa", 0), NodeInfo.of(1, "aa", 0), NodeInfo.of(3, "aa", 0)); + var allNodes = List.of(Broker.of(0, "aa", 0), Broker.of(1, "aa", 0), Broker.of(3, "aa", 0)); var beforeReplicas = List.of( Replica.builder() @@ -113,7 +112,7 @@ void testResult() { .isPreferredLeader(false) .topic(topic) .partition(0) - .nodeInfo(allNodes.get(0)) + .broker(allNodes.get(0)) .size(leaderSize) .path("/tmp/aaa") .build(), @@ -122,7 +121,7 @@ void testResult() { .isPreferredLeader(true) .topic(topic) .partition(0) - .nodeInfo(allNodes.get(1)) + .broker(allNodes.get(1)) .size(leaderSize) .path("/tmp/bbb") .build()); @@ -133,7 +132,7 @@ void testResult() { .isPreferredLeader(false) .topic(topic) .partition(0) - .nodeInfo(allNodes.get(2)) + .broker(allNodes.get(2)) .size(leaderSize) .path("/tmp/ddd") .build(), @@ -142,7 +141,7 @@ void testResult() { .isPreferredLeader(true) .topic(topic) .partition(0) - .nodeInfo(allNodes.get(1)) + .broker(allNodes.get(1)) .size(leaderSize) .path("/tmp/bbb") .build()); diff --git a/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java b/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java index cea7020372..0530ff0b77 100644 --- a/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java +++ b/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java @@ -25,8 +25,8 @@ import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; -import org.astraea.common.admin.NodeInfo; import org.astraea.common.admin.Replica; import org.astraea.gui.Context; import org.astraea.gui.pane.Argument; @@ -124,7 +124,7 @@ void testTableAction() { .join() .replicas() .get(0) - .nodeInfo() + .broker() .id()); Assertions.assertEquals( path, @@ -143,14 +143,14 @@ void testResult() { var topic = Utils.randomString(); var partition = 0; var leaderSize = 100; - var nodes = List.of(NodeInfo.of(0, "aa", 0), NodeInfo.of(1, "aa", 0), NodeInfo.of(2, "aa", 0)); + var nodes = List.of(Broker.of(0, "aa", 0), Broker.of(1, "aa", 0), Broker.of(2, "aa", 0)); var replicas = List.of( Replica.builder() .isLeader(true) .topic(topic) .partition(partition) - .nodeInfo(nodes.get(0)) + .broker(nodes.get(0)) .size(leaderSize) .path("/tmp/aaa") .build(), @@ -158,14 +158,14 @@ void testResult() { .isLeader(false) .topic(topic) .partition(partition) - .nodeInfo(nodes.get(1)) + .broker(nodes.get(1)) .size(20) .build(), Replica.builder() .isLeader(false) .topic(topic) .partition(partition) - .nodeInfo(nodes.get(2)) + .broker(nodes.get(2)) .size(30) .build()); var results = ReplicaNode.allResult(ClusterInfo.of("fake", nodes, Map.of(), replicas)); From 6e6bafbf3466598b16d677ee87326e91fe0973dc Mon Sep 17 00:00:00 2001 From: Haser Date: Fri, 26 May 2023 11:23:52 +0800 Subject: [PATCH 26/77] [CONNECTOR] rewrite PerfSource by java17 toList #1776 --- .../java/org/astraea/connector/perf/PerfSource.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java index c3063b239f..919330be7c 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java @@ -166,7 +166,7 @@ protected List takeConfiguration(int maxTasks) { copy.put(SourceConnector.TOPICS_KEY, t); return new Configuration(copy); }) - .collect(Collectors.toUnmodifiableList()); + .toList(); return Utils.chunk(topics, maxTasks).stream() .map( tps -> { @@ -174,7 +174,7 @@ protected List takeConfiguration(int maxTasks) { copy.put(SourceConnector.TOPICS_KEY, String.join(",", tps)); return new Configuration(copy); }) - .collect(Collectors.toUnmodifiableList()); + .toList(); } @Override @@ -243,13 +243,11 @@ protected void init(Configuration configuration, MetadataStorage storage) { RecordGenerator.builder() .batchSize(batchSize) .keyTableSeed(keyTableSeed) - .keyRange( - LongStream.rangeClosed(0, 10000).boxed().collect(Collectors.toUnmodifiableList())) + .keyRange(LongStream.rangeClosed(0, 10000).boxed().toList()) .keyDistribution(keyDistribution.create(10000, configuration)) .keySizeDistribution(keySizeDistribution.create((int) KeySize.bytes(), configuration)) .valueTableSeed(valueTableSeed) - .valueRange( - LongStream.rangeClosed(0, 10000).boxed().collect(Collectors.toUnmodifiableList())) + .valueRange(LongStream.rangeClosed(0, 10000).boxed().toList()) .valueDistribution(valueDistribution.create(10000, configuration)) .valueSizeDistribution( valueSizeDistribution.create((int) valueSize.bytes(), configuration)) From 4c9034959115dbf94cd8b97d840b968d69828225 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 26 May 2023 21:30:50 +0800 Subject: [PATCH 27/77] [COMMON] rewrite admin.Topic by java 17 record (#1775) --- .../java/org/astraea/common/ByteUtils.java | 28 ++------- .../common/admin/OptimizedClusterInfo.java | 48 +++----------- .../java/org/astraea/common/admin/Topic.java | 62 ++++++------------- 3 files changed, 34 insertions(+), 104 deletions(-) diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index 405c01f3c6..9f6f01dc92 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -341,29 +341,11 @@ public static ClusterInfo readClusterInfo(byte[] bytes) { outerClusterInfo.getTopicList().stream() .map( protoTopic -> - new Topic() { - @Override - public String name() { - return protoTopic.getName(); - } - - @Override - public Config config() { - return new Config(protoTopic.getConfigMap()); - } - - @Override - public boolean internal() { - return protoTopic.getInternal(); - } - - @Override - public Set topicPartitions() { - return protoTopic.getPartitionList().stream() - .map(tp -> TopicPartition.of(protoTopic.getName(), tp)) - .collect(Collectors.toSet()); - } - }) + new Topic( + protoTopic.getName(), + new Config(protoTopic.getConfigMap()), + protoTopic.getInternal(), + Set.copyOf(protoTopic.getPartitionList()))) .collect(Collectors.toMap(Topic::name, Function.identity())), outerClusterInfo.getReplicaList().stream() .map( diff --git a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java index e034b9368e..e463d2bc45 100644 --- a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -56,45 +55,18 @@ class OptimizedClusterInfo implements ClusterInfo { .distinct() .map( topic -> - new Topic() { - @Override - public String name() { - return topic; - } - - @Override - public Config config() { - return Optional.ofNullable(topics.get(name())) + new Topic( + topic, + Optional.ofNullable(topics.get(topic)) .map(Topic::config) - .orElse(Config.EMPTY); - } - - @Override - public boolean internal() { - return Optional.ofNullable(topics.get(name())) + .orElse(Config.EMPTY), + Optional.ofNullable(topics.get(topic)) .map(Topic::internal) - .orElse(false); - } - - @Override - public Set topicPartitions() { - return OptimizedClusterInfo.this.replicas(topic).stream() - .map(Replica::topicPartition) - .collect(Collectors.toUnmodifiableSet()); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null || getClass() != obj.getClass()) return false; - var objTopic = (Topic) obj; - return Objects.equals(name(), objTopic.name()) - && config().raw().equals(objTopic.config().raw()) - && internal() == objTopic.internal() - && topicPartitions().equals(objTopic.topicPartitions()); - } - }) - .collect(Collectors.toUnmodifiableMap(t -> t.name(), t -> t))); + .orElse(false), + OptimizedClusterInfo.this.replicas(topic).stream() + .map(r -> r.topicPartition().partition()) + .collect(Collectors.toUnmodifiableSet()))) + .collect(Collectors.toUnmodifiableMap(Topic::name, t -> t))); this.byBrokerTopic = Lazy.of( () -> diff --git a/common/src/main/java/org/astraea/common/admin/Topic.java b/common/src/main/java/org/astraea/common/admin/Topic.java index f7e7b8ff19..9db850eb3d 100644 --- a/common/src/main/java/org/astraea/common/admin/Topic.java +++ b/common/src/main/java/org/astraea/common/admin/Topic.java @@ -19,56 +19,32 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.kafka.common.TopicPartitionInfo; -public interface Topic { +/** + * @param name topic name + * @param config topic configs. it includes both default configs and override configs + * @param internal true if this topic is internal (system) topic + * @param partitionIds partition id related to this topic + */ +public record Topic(String name, Config config, boolean internal, Set partitionIds) { static Topic of( String name, org.apache.kafka.clients.admin.TopicDescription topicDescription, Map kafkaConfig) { - - var config = new Config(kafkaConfig); - var topicPartitions = + return new Topic( + name, + new Config(kafkaConfig), + topicDescription.isInternal(), topicDescription.partitions().stream() - .map(p -> TopicPartition.of(name, p.partition())) - .collect(Collectors.toUnmodifiableSet()); - return new Topic() { - @Override - public String name() { - return name; - } - - @Override - public Config config() { - return config; - } - - @Override - public boolean internal() { - return topicDescription.isInternal(); - } - - @Override - public Set topicPartitions() { - return topicPartitions; - } - }; + .map(TopicPartitionInfo::partition) + .collect(Collectors.toUnmodifiableSet())); } - /** - * @return topic name - */ - String name(); - - /** - * @return config used by this topic - */ - Config config(); - - /** - * @return true if this topic is internal (system) topic - */ - boolean internal(); - - Set topicPartitions(); + public Set topicPartitions() { + return partitionIds.stream() + .map(id -> new TopicPartition(name, id)) + .collect(Collectors.toUnmodifiableSet()); + } } From 5283ea3adeddba83e44eea9f152bf6d0fe9ae197 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Fri, 26 May 2023 22:50:43 +0800 Subject: [PATCH 28/77] [BALANCER] Reject unsupported balancer configuration (#1768) --- .../common/balancer/BalancerConfigs.java | 1 - .../common/balancer/BalancerUtils.java | 16 +++++ .../balancer/algorithms/GreedyBalancer.java | 6 ++ .../algorithms/SingleStepBalancer.java | 6 ++ .../balancer/BalancerConfigTestSuite.java | 24 +++++++ .../common/balancer/BalancerConfigsTest.java | 68 +++++++++++++++++++ .../common/balancer/BalancerUtilsTest.java | 21 ++++++ 7 files changed, 141 insertions(+), 1 deletion(-) create mode 100644 common/src/test/java/org/astraea/common/balancer/BalancerConfigsTest.java diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java index da74c14425..63ed30d823 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java @@ -23,7 +23,6 @@ * raise an exception. */ public final class BalancerConfigs { - // TODO: Add tests for the above requirement described in javadoc. private BalancerConfigs() {} diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java index c352de416f..38a5cb5be1 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java @@ -19,12 +19,14 @@ import java.util.Arrays; import java.util.Collection; import java.util.Map; +import java.util.Set; import java.util.function.Function; import java.util.function.Predicate; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import org.astraea.common.Configuration; import org.astraea.common.EnumInfo; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; @@ -158,6 +160,20 @@ public static ClusterInfo clearedCluster( .build(); } + public static void balancerConfigCheck(Configuration configs, Set supportedConfig) { + var unsupportedBalancerConfigs = + configs.raw().keySet().stream() + .filter(key -> key.startsWith("balancer.")) + .filter(Predicate.not(supportedConfig::contains)) + .collect(Collectors.toSet()); + if (!unsupportedBalancerConfigs.isEmpty()) + throw new IllegalArgumentException( + "Unsupported balancer configs: " + + unsupportedBalancerConfigs + + ", this implementation support " + + supportedConfig); + } + public enum BalancingModes implements EnumInfo { BALANCING, DEMOTED, diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index 779ff5a9f8..19a3c0dec4 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -113,6 +113,12 @@ public class GreedyBalancer implements Balancer { @Override public Optional offer(AlgorithmConfig config) { + BalancerUtils.balancerConfigCheck( + config.balancerConfig(), + Set.of( + BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE)); + final var minStep = config .balancerConfig() diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java index 3f170b369f..c8b2939221 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java @@ -45,6 +45,12 @@ public class SingleStepBalancer implements Balancer { @Override public Optional offer(AlgorithmConfig config) { + BalancerUtils.balancerConfigCheck( + config.balancerConfig(), + Set.of( + BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE)); + final var minStep = config .balancerConfig() diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index 22c0e0a608..2495e54722 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -49,6 +49,30 @@ public BalancerConfigTestSuite(Class balancerClass, Configur this.customConfig = custom; } + @Test + public void testUnsupportedConfigException() { + final var balancer = Utils.construct(balancerClass, Configuration.EMPTY); + final var cluster = cluster(20, 10, 10, (short) 5); + final var testName = + """ + Balancer implementation should raise an exception \ + when seeing an unsupported config with 'balancer.' prefix. + """; + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(cluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + .config("balancer.no.such.configuration", "oops") + .build()), + testName); + } + @Test public void testBalancerAllowedTopicsRegex() { final var balancer = Utils.construct(balancerClass, Configuration.EMPTY); diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigsTest.java new file mode 100644 index 0000000000..fb92ae83f4 --- /dev/null +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigsTest.java @@ -0,0 +1,68 @@ +/* + * 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.balancer; + +import java.util.Arrays; +import java.util.List; +import org.astraea.common.Utils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opentest4j.AssertionFailedError; + +class BalancerConfigsTest { + + @ParameterizedTest + @ValueSource(classes = BalancerConfigs.class) + void testBalancerConfigPrefix(Class aClass) { + record Field(String name, T value) {} + + var badFields = + Arrays.stream(aClass.getDeclaredFields()) + .map(x -> new Field<>(x.getName(), Utils.packException(() -> x.get(null)))) + .filter(x -> x.value instanceof String configKey && !configKey.startsWith("balancer.")) + .toList(); + + Assertions.assertEquals( + List.of(), + badFields, + "The class '%s' contains bad config key pattern, it should start with 'balancer.'" + .formatted(aClass)); + } + + @Test + void testPrefixCheckCorrect() { + @SuppressWarnings("unused") + interface CorrectClass { + String correct0 = "balancer.good.config.A"; + String correct1 = "balancer.good.config.B"; + String correct2 = "balancer.good.config.C"; + } + + @SuppressWarnings("unused") + interface IncorrectClass { + String correct0 = "balancer.good.config.A"; + String incorrect1 = "not.balancer.good.config.B"; + String incorrect2 = "not.balancer.good.config.C"; + } + + Assertions.assertDoesNotThrow(() -> testBalancerConfigPrefix(CorrectClass.class)); + Assertions.assertThrows( + AssertionFailedError.class, () -> testBalancerConfigPrefix(IncorrectClass.class)); + } +} diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java index 341968b28b..5680dbb57a 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Stream; +import org.astraea.common.Configuration; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; import org.junit.jupiter.api.Assertions; @@ -205,4 +206,24 @@ void testClearedCluster() { Assertions.assertEquals( 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 4).count(), "Not allowed"); } + + @Test + void testBalancerConfigCheck() { + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + BalancerUtils.balancerConfigCheck( + new Configuration(Map.ofEntries(Map.entry("balancer.unsupported.config", ""))), + Set.of("balancer.supported.config"))); + Assertions.assertDoesNotThrow( + () -> + BalancerUtils.balancerConfigCheck( + new Configuration(Map.ofEntries(Map.entry("balancer.supported.config", ""))), + Set.of("balancer.supported.config"))); + Assertions.assertDoesNotThrow( + () -> + BalancerUtils.balancerConfigCheck( + new Configuration(Map.ofEntries(Map.entry("not.balancer.config", ""))), + Set.of("balancer.supported.config"))); + } } From cf700870f37d7dcc4a46c275dc9d5bb411f74f5f Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Fri, 26 May 2023 23:11:59 +0800 Subject: [PATCH 29/77] [COMMON] Rewrite file under connector by java 17 record (#1779) --- .../common/connector/ConnectorReq.java | 22 ++--- .../common/connector/ConnectorStatus.java | 74 +++++------------ .../astraea/common/connector/PluginInfo.java | 26 ++---- .../astraea/common/connector/TaskStatus.java | 69 ++++------------ .../common/connector/WorkerStatus.java | 80 +++++-------------- 5 files changed, 66 insertions(+), 205 deletions(-) diff --git a/common/src/main/java/org/astraea/common/connector/ConnectorReq.java b/common/src/main/java/org/astraea/common/connector/ConnectorReq.java index ea0ebc9422..90ab2644b7 100644 --- a/common/src/main/java/org/astraea/common/connector/ConnectorReq.java +++ b/common/src/main/java/org/astraea/common/connector/ConnectorReq.java @@ -18,20 +18,8 @@ import java.util.Map; -public class ConnectorReq { - private final String name; - private final Map config; - - public ConnectorReq(String name, Map config) { - this.name = name; - this.config = config; - } - - public String name() { - return name; - } - - public Map config() { - return config; - } -} +/** + * @param name + * @param config + */ +public record ConnectorReq(String name, Map config) {} diff --git a/common/src/main/java/org/astraea/common/connector/ConnectorStatus.java b/common/src/main/java/org/astraea/common/connector/ConnectorStatus.java index f187fd2872..b42272989c 100644 --- a/common/src/main/java/org/astraea/common/connector/ConnectorStatus.java +++ b/common/src/main/java/org/astraea/common/connector/ConnectorStatus.java @@ -20,59 +20,21 @@ import java.util.Map; import java.util.Optional; -/** this is not a kind of json response from kafka. */ -public class ConnectorStatus { - - private final String name; - - private final String state; - - private final String workerId; - - // The type is always null before kafka 2.0.2 - // see https://issues.apache.org/jira/browse/KAFKA-7253 - private final Optional type; - - private final Map configs; - - private final List tasks; - - ConnectorStatus( - String name, - String state, - String workerId, - Optional type, - Map configs, - List tasks) { - this.name = name; - this.state = state; - this.workerId = workerId; - this.type = type; - this.configs = Map.copyOf(configs); - this.tasks = List.copyOf(tasks); - } - - public String name() { - return name; - } - - public String state() { - return state; - } - - public String workerId() { - return workerId; - } - - public Optional type() { - return type; - } - - public Map configs() { - return configs; - } - - public List tasks() { - return tasks; - } -} +/** + * this is not a kind of json response from kafka. + * + * @param name + * @param state + * @param workerId + * @param type The type is always null before kafka 2.0.2 see KAFKA-7253 + * @param configs + * @param tasks + */ +public record ConnectorStatus( + String name, + String state, + String workerId, + Optional type, + Map configs, + List tasks) {} diff --git a/common/src/main/java/org/astraea/common/connector/PluginInfo.java b/common/src/main/java/org/astraea/common/connector/PluginInfo.java index 7a81fb1ce2..16efa149c2 100644 --- a/common/src/main/java/org/astraea/common/connector/PluginInfo.java +++ b/common/src/main/java/org/astraea/common/connector/PluginInfo.java @@ -18,22 +18,10 @@ import java.util.List; -/** this is not a kind of json response from kafka. We compose it with definition. */ -public class PluginInfo { - private final String className; - - private final List definitions; - - public PluginInfo(String className, List definitions) { - this.className = className; - this.definitions = definitions; - } - - public String className() { - return className; - } - - public List definitions() { - return definitions; - } -} +/** + * this is not a kind of json response from kafka. We compose it with definition. + * + * @param className + * @param definitions + */ +public record PluginInfo(String className, List definitions) {} diff --git a/common/src/main/java/org/astraea/common/connector/TaskStatus.java b/common/src/main/java/org/astraea/common/connector/TaskStatus.java index 9ba59b1126..61d9be612e 100644 --- a/common/src/main/java/org/astraea/common/connector/TaskStatus.java +++ b/common/src/main/java/org/astraea/common/connector/TaskStatus.java @@ -19,55 +19,20 @@ import java.util.Map; import java.util.Optional; -/** this is not a kind of json response from kafka. */ -public class TaskStatus { - - private final String connectorName; - private final int id; - private final String state; - - private final String workerId; - - private final Map configs; - - private final Optional error; - - TaskStatus( - String connectorName, - int id, - String state, - String workerId, - Map configs, - Optional error) { - this.connectorName = connectorName; - this.id = id; - this.state = state; - this.workerId = workerId; - this.configs = Map.copyOf(configs); - this.error = error; - } - - public String connectorName() { - return connectorName; - } - - public int id() { - return id; - } - - public String state() { - return state; - } - - public String workerId() { - return workerId; - } - - public Map configs() { - return configs; - } - - public Optional error() { - return error; - } -} +/** + * this is not a kind of json response from kafka. + * + * @param connectorName + * @param id + * @param state + * @param workerId + * @param configs + * @param error + */ +public record TaskStatus( + String connectorName, + int id, + String state, + String workerId, + Map configs, + Optional error) {} diff --git a/common/src/main/java/org/astraea/common/connector/WorkerStatus.java b/common/src/main/java/org/astraea/common/connector/WorkerStatus.java index d810695563..081ae3b459 100644 --- a/common/src/main/java/org/astraea/common/connector/WorkerStatus.java +++ b/common/src/main/java/org/astraea/common/connector/WorkerStatus.java @@ -16,64 +16,22 @@ */ package org.astraea.common.connector; -/** this is not a kind of json response from kafka. We compose it with worker hostname and port. */ -public class WorkerStatus { - - private final String hostname; - - private final int port; - - private final String version; - private final String commit; - - private final String kafkaClusterId; - - private final long numberOfConnectors; - - private final long numberOfTasks; - - WorkerStatus( - String hostname, - int port, - String version, - String commit, - String kafkaClusterId, - long numberOfConnectors, - long numberOfTasks) { - this.hostname = hostname; - this.port = port; - this.version = version; - this.commit = commit; - this.kafkaClusterId = kafkaClusterId; - this.numberOfConnectors = numberOfConnectors; - this.numberOfTasks = numberOfTasks; - } - - public String hostname() { - return hostname; - } - - public int port() { - return port; - } - - public String version() { - return version; - } - - public String commit() { - return commit; - } - - public String kafkaClusterId() { - return kafkaClusterId; - } - - public long numberOfConnectors() { - return numberOfConnectors; - } - - public long numberOfTasks() { - return numberOfTasks; - } -} +/** + * this is not a kind of json response from kafka. We compose it with worker hostname and port. + * + * @param hostname + * @param port + * @param version + * @param commit + * @param kafkaClusterId + * @param numberOfConnectors + * @param numberOfTasks + */ +public record WorkerStatus( + String hostname, + int port, + String version, + String commit, + String kafkaClusterId, + long numberOfConnectors, + long numberOfTasks) {} From e3372dbf5ba3f9da413986d5bb4e0ba80cc9f552 Mon Sep 17 00:00:00 2001 From: Haser Date: Fri, 26 May 2023 23:59:23 +0800 Subject: [PATCH 30/77] [COMMON] Rewrite common.main by java17 toList and instanceof (#1777) --- .../java/org/astraea/common/ByteUtils.java | 24 +++++++++---------- .../main/java/org/astraea/common/Header.java | 3 +-- .../main/java/org/astraea/common/Utils.java | 4 ++-- 3 files changed, 15 insertions(+), 16 deletions(-) diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index 9f6f01dc92..cf0301f3a4 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -380,18 +380,18 @@ public static ClusterInfo readClusterInfo(byte[] bytes) { * and "char" in Protocol Buffers. Use "int" and "String" instead. */ private static PrimitiveOuterClass.Primitive primitive(Object v) throws SerializationException { - if (v instanceof Integer) - return PrimitiveOuterClass.Primitive.newBuilder().setInt((int) v).build(); - else if (v instanceof Long) - return PrimitiveOuterClass.Primitive.newBuilder().setLong((long) v).build(); - else if (v instanceof Float) - return PrimitiveOuterClass.Primitive.newBuilder().setFloat((float) v).build(); - else if (v instanceof Double) - return PrimitiveOuterClass.Primitive.newBuilder().setDouble((double) v).build(); - else if (v instanceof Boolean) - return PrimitiveOuterClass.Primitive.newBuilder().setBoolean((boolean) v).build(); - else if (v instanceof String) - return PrimitiveOuterClass.Primitive.newBuilder().setStr(v.toString()).build(); + if (v instanceof Integer value) + return PrimitiveOuterClass.Primitive.newBuilder().setInt(value).build(); + else if (v instanceof Long value) + return PrimitiveOuterClass.Primitive.newBuilder().setLong(value).build(); + else if (v instanceof Float value) + return PrimitiveOuterClass.Primitive.newBuilder().setFloat(value).build(); + else if (v instanceof Double value) + return PrimitiveOuterClass.Primitive.newBuilder().setDouble(value).build(); + else if (v instanceof Boolean value) + return PrimitiveOuterClass.Primitive.newBuilder().setBoolean(value).build(); + else if (v instanceof String value) + return PrimitiveOuterClass.Primitive.newBuilder().setStr(value).build(); else throw new SerializationException( "Type " diff --git a/common/src/main/java/org/astraea/common/Header.java b/common/src/main/java/org/astraea/common/Header.java index 0beea8ef7d..4637408afe 100644 --- a/common/src/main/java/org/astraea/common/Header.java +++ b/common/src/main/java/org/astraea/common/Header.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Spliterators; -import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.kafka.common.header.Headers; @@ -29,7 +28,7 @@ public static List
of(Headers headers) { if (!iter.hasNext()) return List.of(); return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iter, 0), false) .map(h -> new Header(h.key(), h.value())) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static Header of(String key, byte[] value) { diff --git a/common/src/main/java/org/astraea/common/Utils.java b/common/src/main/java/org/astraea/common/Utils.java index 59d420b2b4..002fc3affd 100644 --- a/common/src/main/java/org/astraea/common/Utils.java +++ b/common/src/main/java/org/astraea/common/Utils.java @@ -148,8 +148,8 @@ public static R packException(Getter getter) { } public static void close(Object obj) { - if (obj instanceof AutoCloseable) { - packException(() -> ((AutoCloseable) obj).close()); + if (obj instanceof AutoCloseable autoCloseableObj) { + packException(autoCloseableObj::close); } } From fb58f057e49788741ffb4ae5f4ae0ccda25c52f5 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sat, 27 May 2023 10:11:38 +0800 Subject: [PATCH 31/77] [COMMON] rewrite admin.Replica by java 17 record (#1782) --- .../org/astraea/common/admin/AdminImpl.java | 2 +- .../common/admin/ClusterInfoBuilder.java | 2 +- .../org/astraea/common/admin/Replica.java | 295 +++++++++++----- .../astraea/common/admin/ReplicaBuilder.java | 332 ------------------ .../org/astraea/common/admin/AdminTest.java | 2 +- .../astraea/gui/tab/topic/ReplicaNode.java | 2 +- 6 files changed, 218 insertions(+), 417 deletions(-) delete mode 100644 common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java diff --git a/common/src/main/java/org/astraea/common/admin/AdminImpl.java b/common/src/main/java/org/astraea/common/admin/AdminImpl.java index 2d0c01eba5..38b13322ff 100644 --- a/common/src/main/java/org/astraea/common/admin/AdminImpl.java +++ b/common/src/main/java/org/astraea/common/admin/AdminImpl.java @@ -684,7 +684,7 @@ private CompletionStage> replicas(Set topics) { Replica.builder() .topic(topicName) .partition(partitionId) - .internal(internal) + .isInternal(internal) .isAdding(isAdding) .isRemoving(isRemoving) .broker( diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java index 2661d46071..c259f1c9d3 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java @@ -216,7 +216,7 @@ public ClusterInfoBuilder addTopic( .isAdding(false) .isRemoving(false) .lag(0) - .internal(false) + .isInternal(false) .isLeader(index == 0) .isSync(true) .isFuture(false) diff --git a/common/src/main/java/org/astraea/common/admin/Replica.java b/common/src/main/java/org/astraea/common/admin/Replica.java index 84a82b3c13..9f95af2ad2 100644 --- a/common/src/main/java/org/astraea/common/admin/Replica.java +++ b/common/src/main/java/org/astraea/common/admin/Replica.java @@ -16,13 +16,49 @@ */ package org.astraea.common.admin; -public interface Replica { +import java.util.Objects; - static ReplicaBuilder builder() { - return new ReplicaBuilder(); +/** + * @param topic topic name + * @param partition partition id + * @param broker information of the node hosts this replica + * @param isLeader true if this replica is a leader replica + * @param isSync true if this replica is synced + * @param isOffline true if this replica is offline + * @param isAdding true if this replica is adding and syncing data + * @param isRemoving true if this replica will be deleted in the future. + * @param isFuture true if this log is created by AlterReplicaLogDirsRequest and will replace the + * current log of the replica at some time in the future. + * @param isPreferredLeader true if the replica is the preferred leader + * @param lag (LEO - high watermark) if it is the current log, * (LEO) if it is the future log, (-1) + * if the host of replica is offline + * @param size The size of all log segments in this replica in bytes. It returns -1 if the host of + * replica is offline + * @param path that indicates the data folder path which stored this replica on a specific Kafka + * node. It returns null if the host of replica is offline + * @param isInternal true if this replica belongs to internal topic + */ +public record Replica( + String topic, + int partition, + Broker broker, + boolean isLeader, + boolean isSync, + boolean isOffline, + boolean isAdding, + boolean isRemoving, + boolean isFuture, + boolean isPreferredLeader, + long lag, + long size, + String path, + boolean isInternal) { + + public static Builder builder() { + return new Builder(); } - static ReplicaBuilder builder(Replica replica) { + public static Builder builder(Replica replica) { return Replica.builder().replica(replica); } @@ -31,7 +67,7 @@ static ReplicaBuilder builder(Replica replica) { * * @return TopicPartitionReplica */ - default TopicPartitionReplica topicPartitionReplica() { + public TopicPartitionReplica topicPartitionReplica() { return TopicPartitionReplica.of(topic(), partition(), broker().id()); } @@ -40,105 +76,202 @@ default TopicPartitionReplica topicPartitionReplica() { * * @return TopicPartition */ - default TopicPartition topicPartition() { + public TopicPartition topicPartition() { return TopicPartition.of(topic(), partition()); } - /** - * @return topic name - */ - String topic(); - - /** - * @return partition id - */ - int partition(); - - /** - * @return information of the node hosts this replica - */ - Broker broker(); - - /** - * @return true if this replica is a leader replica - */ - boolean isLeader(); - /** * @return true if this replica is a follower replica */ - default boolean isFollower() { + public boolean isFollower() { return !isLeader(); } - /** - * @return true if this replica is synced - */ - boolean isSync(); - - /** - * @return true if this replica is offline - */ - boolean isOffline(); - /** * @return true if this replica is online */ - default boolean isOnline() { + public boolean isOnline() { return !isOffline(); } - /** - * @return true if this replica is adding and syncing data - */ - boolean isAdding(); - - /** - * @return true if this replica will be deleted in the future. - */ - boolean isRemoving(); - - /** - * Whether this replica has been created by a AlterReplicaLogDirsRequest but not yet replaced the - * current replica on the broker. - * - * @return true if this log is created by AlterReplicaLogDirsRequest and will replace the current - * log of the replica at some time in the future. - */ - boolean isFuture(); - /** * @return true if this is current log of replica. */ - default boolean isCurrent() { + public boolean isCurrent() { return !isFuture(); } - /** - * @return true if the replica is the preferred leader - */ - boolean isPreferredLeader(); + public static class Builder { - /** - * @return (LEO - high watermark) if it is the current log, * (LEO) if it is the future log, * - * (-1) if the host of replica is offline - */ - long lag(); + private String topic; + private int partition; + private Broker broker; + private long lag; + private long size; - /** - * @return The size of all log segments in this replica in bytes. It returns -1 if the host of - * replica is offline - */ - long size(); + private boolean isAdding; - /** - * @return that indicates the data folder path which stored this replica on a specific Kafka node. - * It returns null if the host of replica is offline - */ - String path(); + private boolean isRemoving; + private boolean isInternal; + private boolean isLeader; + private boolean isSync; + private boolean isFuture; + private boolean isOffline; + private boolean isPreferredLeader; + private String path; - /** - * @return true if this replica belongs to internal topic - */ - boolean internal(); + private Builder() {} + + public Builder replica(Replica replica) { + this.topic = replica.topic(); + this.partition = replica.partition(); + this.broker = replica.broker(); + this.lag = replica.lag(); + this.size = replica.size(); + this.isAdding = replica.isAdding; + this.isRemoving = replica.isRemoving; + this.isInternal = replica.isInternal; + this.isLeader = replica.isLeader(); + this.isSync = replica.isSync(); + this.isFuture = replica.isFuture(); + this.isOffline = replica.isOffline(); + this.isPreferredLeader = replica.isPreferredLeader(); + this.path = replica.path(); + return this; + } + + public Builder topic(String topic) { + this.topic = topic; + return this; + } + + public Builder partition(int partition) { + this.partition = partition; + return this; + } + + public Builder broker(Broker broker) { + this.broker = broker; + return this; + } + + public Builder lag(long lag) { + this.lag = lag; + return this; + } + + public Builder size(long size) { + this.size = size; + return this; + } + + public Builder isAdding(boolean isAdding) { + this.isAdding = isAdding; + return this; + } + + public Builder isRemoving(boolean isRemoving) { + this.isRemoving = isRemoving; + return this; + } + + public Builder isInternal(boolean isInternal) { + this.isInternal = isInternal; + return this; + } + + public Builder isLeader(boolean leader) { + this.isLeader = leader; + return this; + } + + public Builder isSync(boolean isSync) { + this.isSync = isSync; + return this; + } + + public Builder isFuture(boolean isFuture) { + this.isFuture = isFuture; + return this; + } + + public Builder isOffline(boolean offline) { + this.isOffline = offline; + return this; + } + + public Builder isPreferredLeader(boolean isPreferredLeader) { + this.isPreferredLeader = isPreferredLeader; + return this; + } + + public Builder path(String path) { + this.path = path; + return this; + } + + /** + * a helper used to set all flags for a replica leader. + * + * @return a replica leader + */ + public Replica buildLeader() { + return new Replica( + Objects.requireNonNull(topic), + partition, + Objects.requireNonNull(broker), + true, + true, + false, + false, + false, + false, + true, + 0, + size, + Objects.requireNonNull(path), + isInternal); + } + + /** + * a helper used to set all flags for a in-sync replica follower. + * + * @return a replica leader + */ + public Replica buildInSyncFollower() { + return new Replica( + Objects.requireNonNull(topic), + partition, + Objects.requireNonNull(broker), + false, + true, + false, + false, + false, + false, + false, + 0, + size, + Objects.requireNonNull(path), + isInternal); + } + + public Replica build() { + return new Replica( + Objects.requireNonNull(topic), + partition, + Objects.requireNonNull(broker), + isLeader, + isSync, + isOffline, + isAdding, + isRemoving, + isFuture, + isPreferredLeader, + lag, + size, + path, + isInternal); + } + } } diff --git a/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java b/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java deleted file mode 100644 index b7e1929643..0000000000 --- a/common/src/main/java/org/astraea/common/admin/ReplicaBuilder.java +++ /dev/null @@ -1,332 +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.admin; - -import java.util.Objects; - -public class ReplicaBuilder { - - private String topic; - private int partition; - private Broker broker; - private long lag; - private long size; - - private boolean isAdding; - - private boolean isRemoving; - private boolean internal; - private boolean isLeader; - private boolean isSync; - private boolean isFuture; - private boolean isOffline; - private boolean isPreferredLeader; - private String path; - - ReplicaBuilder replica(Replica replica) { - this.topic = replica.topic(); - this.partition = replica.partition(); - this.broker = replica.broker(); - this.lag = replica.lag(); - this.size = replica.size(); - this.isLeader = replica.isLeader(); - this.isSync = replica.isSync(); - this.isFuture = replica.isFuture(); - this.isOffline = replica.isOffline(); - this.isPreferredLeader = replica.isPreferredLeader(); - this.path = replica.path(); - - return this; - } - - public ReplicaBuilder topic(String topic) { - this.topic = topic; - return this; - } - - public ReplicaBuilder partition(int partition) { - this.partition = partition; - return this; - } - - public ReplicaBuilder broker(Broker broker) { - this.broker = broker; - return this; - } - - public ReplicaBuilder lag(long lag) { - this.lag = lag; - return this; - } - - public ReplicaBuilder size(long size) { - this.size = size; - return this; - } - - public ReplicaBuilder isAdding(boolean isAdding) { - this.isAdding = isAdding; - return this; - } - - public ReplicaBuilder isRemoving(boolean isRemoving) { - this.isRemoving = isRemoving; - return this; - } - - public ReplicaBuilder internal(boolean internal) { - this.internal = internal; - return this; - } - - public ReplicaBuilder isLeader(boolean leader) { - this.isLeader = leader; - return this; - } - - public ReplicaBuilder isSync(boolean isSync) { - this.isSync = isSync; - return this; - } - - public ReplicaBuilder isFuture(boolean isFuture) { - this.isFuture = isFuture; - return this; - } - - public ReplicaBuilder isOffline(boolean offline) { - this.isOffline = offline; - return this; - } - - public ReplicaBuilder isPreferredLeader(boolean isPreferredLeader) { - this.isPreferredLeader = isPreferredLeader; - return this; - } - - public ReplicaBuilder path(String path) { - this.path = path; - return this; - } - - /** - * a helper used to set all flags for a replica leader. - * - * @return a replica leader - */ - public Replica buildLeader() { - Objects.requireNonNull(path); - return new ReplicaImpl( - this.isLeader(true) - .isPreferredLeader(true) - .isSync(true) - .isFuture(false) - .isOffline(false) - .isRemoving(false) - .isAdding(false) - .lag(0)); - } - - /** - * a helper used to set all flags for a in-sync replica follower. - * - * @return a replica leader - */ - public Replica buildInSyncFollower() { - Objects.requireNonNull(path); - return new ReplicaImpl( - this.isLeader(false) - .isPreferredLeader(false) - .isSync(true) - .isFuture(false) - .isOffline(false) - .isRemoving(false) - .isAdding(false) - .lag(0)); - } - - public Replica build() { - return new ReplicaImpl(this); - } - - private static class ReplicaImpl implements Replica { - private final String topic; - private final int partition; - private final Broker broker; - private final long lag; - private final long size; - - private final boolean internal; - private final boolean isLeader; - - private final boolean isAdding; - - private final boolean isRemoving; - private final boolean isSync; - private final boolean isFuture; - private final boolean isOffline; - private final boolean isPreferredLeader; - private final String path; - - private ReplicaImpl(ReplicaBuilder builder) { - this.topic = Objects.requireNonNull(builder.topic); - this.partition = builder.partition; - this.broker = Objects.requireNonNull(builder.broker); - this.isAdding = builder.isAdding; - this.isRemoving = builder.isRemoving; - this.lag = builder.lag; - this.size = builder.size; - this.internal = builder.internal; - this.isLeader = builder.isLeader; - this.isSync = builder.isSync; - this.isFuture = builder.isFuture; - this.isOffline = builder.isOffline; - this.isPreferredLeader = builder.isPreferredLeader; - this.path = builder.path; - } - - @Override - public boolean isFuture() { - return isFuture; - } - - @Override - public boolean isPreferredLeader() { - return isPreferredLeader; - } - - @Override - public long lag() { - return lag; - } - - @Override - public long size() { - return size; - } - - @Override - public String path() { - return path; - } - - @Override - public boolean internal() { - return internal; - } - - @Override - public String topic() { - return topic; - } - - @Override - public int partition() { - return partition; - } - - @Override - public Broker broker() { - return broker; - } - - @Override - public boolean isLeader() { - return isLeader; - } - - @Override - public boolean isSync() { - return isSync; - } - - @Override - public boolean isOffline() { - return isOffline; - } - - @Override - public boolean isAdding() { - return isAdding; - } - - @Override - public boolean isRemoving() { - return isRemoving; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ReplicaImpl replica = (ReplicaImpl) o; - return partition == replica.partition - && lag == replica.lag - && size == replica.size - && internal == replica.internal - && isLeader == replica.isLeader - && isAdding == replica.isAdding - && isRemoving == replica.isRemoving - && isSync == replica.isSync - && isFuture == replica.isFuture - && isOffline == replica.isOffline - && isPreferredLeader == replica.isPreferredLeader - && topic.equals(replica.topic) - && broker.id() == replica.broker.id() - && broker.host().equals(replica.broker.host()) - && broker.port() == replica.broker.port() - && Objects.equals(path, replica.path); - } - - @Override - public String toString() { - return "Replica{" - + "topic='" - + topic() - + '\'' - + ", partition=" - + partition() - + ", broker=" - + broker() - + ", path='" - + path() - + '\'' - + ", " - + (isOffline() ? "offline" : isLeader() ? "leader" : "follower") - + '}'; - } - - @Override - public int hashCode() { - return Objects.hash( - topic, - partition, - broker, - lag, - size, - internal, - isLeader, - isAdding, - isRemoving, - isSync, - isFuture, - isOffline, - isPreferredLeader, - path); - } - } -} diff --git a/common/src/test/java/org/astraea/common/admin/AdminTest.java b/common/src/test/java/org/astraea/common/admin/AdminTest.java index a0675e8184..efab4420c6 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminTest.java @@ -201,7 +201,7 @@ void testClusterInfo() { var replicas = admin.clusterInfo(Set.of(t.name())).toCompletableFuture().join().replicas(); Assertions.assertNotEquals(0, replicas.size()); - replicas.forEach(r -> Assertions.assertEquals(t.internal(), r.internal())); + replicas.forEach(r -> Assertions.assertEquals(t.internal(), r.isInternal())); replicas.forEach(r -> Assertions.assertFalse(r.isAdding())); replicas.forEach(r -> Assertions.assertFalse(r.isRemoving())); }); diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java index 8d0e1905a5..ff0026948b 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java @@ -63,7 +63,7 @@ static List> allResult(ClusterInfo clusterInfo) { var result = new LinkedHashMap(); result.put(TOPIC_NAME_KEY, replica.topic()); result.put(PARTITION_KEY, replica.partition()); - result.put("internal", replica.internal()); + result.put("internal", replica.isInternal()); result.put("broker", replica.broker().id()); if (replica.path() != null) result.put(PATH_KEY, replica.path()); result.put("isLeader", replica.isLeader()); From 26687d235a3a05e707749cb4591e0c8d83d57ddc Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Sat, 27 May 2023 10:14:56 +0800 Subject: [PATCH 32/77] [STATISTICS] Add sum statistic (#1766) --- .../org/astraea/common/metrics/stats/Avg.java | 42 ------------------- .../org/astraea/common/metrics/stats/Sum.java | 37 ++++++++++++++++ .../astraea/common/metrics/stats/AvgTest.java | 23 ---------- .../astraea/common/metrics/stats/SumTest.java | 30 +++++++++++++ 4 files changed, 67 insertions(+), 65 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/metrics/stats/Sum.java create mode 100644 common/src/test/java/org/astraea/common/metrics/stats/SumTest.java diff --git a/common/src/main/java/org/astraea/common/metrics/stats/Avg.java b/common/src/main/java/org/astraea/common/metrics/stats/Avg.java index adb5a00a38..36e662308d 100644 --- a/common/src/main/java/org/astraea/common/metrics/stats/Avg.java +++ b/common/src/main/java/org/astraea/common/metrics/stats/Avg.java @@ -17,7 +17,6 @@ package org.astraea.common.metrics.stats; import java.time.Duration; -import java.util.concurrent.ConcurrentLinkedDeque; public class Avg { public static Stat of() { @@ -75,45 +74,4 @@ public synchronized Double measure() { } }; } - - /** Compute the average of value recorded in the given time period. */ - public static Stat byTime(Duration period) { - if (period.toMillis() <= 0) { - throw new IllegalArgumentException( - "Stat, Average by time, needs period longer than 1 millisecond."); - } - return new Stat<>() { - private final ConcurrentLinkedDeque> past = - new ConcurrentLinkedDeque<>(); - - @Override - public void record(Double value) { - past.add(new ValueAndTime<>(value, System.currentTimeMillis())); - popOutdated(); - } - - @Override - public Double measure() { - popOutdated(); - return past.stream().mapToDouble(e -> e.value).average().orElse(Double.NaN); - } - - private void popOutdated() { - var outdated = System.currentTimeMillis() - period.toMillis(); - while (!past.isEmpty() && past.peekFirst().timestamp < outdated) { - past.poll(); - } - } - }; - } - - private static class ValueAndTime { - public final V value; - public final long timestamp; - - public ValueAndTime(V value, long timestamp) { - this.value = value; - this.timestamp = timestamp; - } - } } diff --git a/common/src/main/java/org/astraea/common/metrics/stats/Sum.java b/common/src/main/java/org/astraea/common/metrics/stats/Sum.java new file mode 100644 index 0000000000..c1798dce1c --- /dev/null +++ b/common/src/main/java/org/astraea/common/metrics/stats/Sum.java @@ -0,0 +1,37 @@ +/* + * 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.metrics.stats; + +import java.util.concurrent.atomic.LongAdder; + +public interface Sum extends Stat { + static Sum ofLong() { + return new Sum<>() { + private final LongAdder sum = new LongAdder(); + + @Override + public void record(Long value) { + sum.add(value); + } + + @Override + public Long measure() { + return sum.sum(); + } + }; + } +} diff --git a/common/src/test/java/org/astraea/common/metrics/stats/AvgTest.java b/common/src/test/java/org/astraea/common/metrics/stats/AvgTest.java index ba9ef4c486..47cfd0b955 100644 --- a/common/src/test/java/org/astraea/common/metrics/stats/AvgTest.java +++ b/common/src/test/java/org/astraea/common/metrics/stats/AvgTest.java @@ -51,27 +51,4 @@ void testExpWeightByTime() throws InterruptedException { Assertions.assertEquals(10 * 0.5 * 0.5 + 50 * 0.5, rateByTime.measure()); } - - @Test - void testByTime() throws InterruptedException { - var byTime = Avg.byTime(Duration.ofMillis(150)); - - byTime.record(10.0); - Assertions.assertEquals(10.0, byTime.measure()); - byTime.record(14.0); - Assertions.assertEquals(12.0, byTime.measure()); - - Thread.sleep(20); - - byTime.record(18.0); - Assertions.assertEquals(14.0, byTime.measure()); - - Thread.sleep(140); - - Assertions.assertEquals(18.0, byTime.measure()); - - Thread.sleep(20); - - Assertions.assertEquals(Double.NaN, byTime.measure()); - } } diff --git a/common/src/test/java/org/astraea/common/metrics/stats/SumTest.java b/common/src/test/java/org/astraea/common/metrics/stats/SumTest.java new file mode 100644 index 0000000000..27f262e007 --- /dev/null +++ b/common/src/test/java/org/astraea/common/metrics/stats/SumTest.java @@ -0,0 +1,30 @@ +/* + * 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.metrics.stats; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class SumTest { + @Test + void testLongOf() { + var stat = Sum.ofLong(); + stat.record(1L); + stat.record(2L); + Assertions.assertEquals(3L, stat.measure()); + } +} From 6b4e546d6040599737d5e68942900aba5656e52c Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Sat, 27 May 2023 12:24:05 +0800 Subject: [PATCH 33/77] [BALANCER] Expose internal topics to Balancer (#1767) --- .../org/astraea/app/web/BalancerHandler.java | 2 +- .../common/balancer/BalancerConfigs.java | 13 +- .../common/balancer/BalancerConsoleImpl.java | 6 +- .../common/balancer/BalancerUtils.java | 21 +--- .../balancer/algorithms/GreedyBalancer.java | 14 ++- .../algorithms/SingleStepBalancer.java | 14 ++- .../balancer/tweakers/ShuffleTweaker.java | 20 +-- .../balancer/BalancerConfigTestSuite.java | 118 +++++++----------- .../common/balancer/BalancerUtilsTest.java | 31 +---- .../balancer/tweakers/ShuffleTweakerTest.java | 74 +++++++++++ docs/web_server/web_api_balancer_chinese.md | 11 +- 11 files changed, 182 insertions(+), 142 deletions(-) diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 6a382ddba4..6cfbc6718b 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -75,7 +75,7 @@ public CompletionStage post(Channel channel) { var balancerPostRequest = channel.request(TypeRef.of(BalancerPostRequest.class)); var request = admin - .topicNames(false) + .topicNames(true) .thenCompose(admin::clusterInfo) .thenApply( currentClusterInfo -> diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java index 63ed30d823..a93e84f27c 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java @@ -27,9 +27,8 @@ public final class BalancerConfigs { private BalancerConfigs() {} /** - * A regular expression indicates which topics are eligible for rebalancing. When specified, - * topics that don't match this expression cannot be altered and must stay at their original - * position. + * A regular expression indicates which topics are eligible for balancing. When specified, topics + * that don't match this expression cannot be altered and must stay at their original position. */ public static final String BALANCER_ALLOWED_TOPICS_REGEX = "balancer.allowed.topics.regex"; @@ -65,9 +64,11 @@ private BalancerConfigs() {} *

Flag Interaction:

* *
    - *
  1. When this flag is used in conjunction with {@link - * BalancerConfigs#BALANCER_ALLOWED_TOPICS_REGEX}, if a demoted broker contains partition - * from those forbidden topics, an exception should be raised. + *
  2. All partitions on the demoting brokers will be compelled to participate in the balancing + * process, regardless of the explicit prohibition specified by the {@link + * BalancerConfigs#BALANCER_ALLOWED_TOPICS_REGEX} configuration. This exception solely + * applies to partitions located at a demoting broker, while disallowed partitions on + * balancing brokers will remain excluded from the balancing decision. *
* *

Limitation:

diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java index aa9b5c2343..5170d02d2c 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java @@ -125,7 +125,7 @@ public CompletionStage generate() { var clusterInfo = this.checkNoOngoingMigration ? BalancerConsoleImpl.this.checkNoOngoingMigration() - : admin.topicNames(false).thenCompose(admin::clusterInfo); + : admin.topicNames(true).thenCompose(admin::clusterInfo); return planGenerations.compute( taskId, (id, previousTask) -> { @@ -232,7 +232,7 @@ private void checkNotClosed() { private CompletionStage checkNoOngoingMigration() { return admin - .topicNames(false) + .topicNames(true) .thenCompose(admin::clusterInfo) .thenApply( cluster -> { @@ -274,7 +274,7 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .map(x -> Map.entry(x.broker().id(), x.path())) .collect(Collectors.toUnmodifiableList()))); return admin - .topicNames(false) + .topicNames(true) .thenCompose(admin::clusterInfo) .thenAccept( currentCluster -> { diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java index 38a5cb5be1..1f8ad18156 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java @@ -63,25 +63,8 @@ public static Map balancingMode(ClusterInfo cluster, St .collect(Collectors.toUnmodifiableMap(Function.identity(), mode)); } - /** - * Verify there is no logic conflict between {@link BalancerConfigs#BALANCER_ALLOWED_TOPICS_REGEX} - * and {@link BalancerConfigs#BALANCER_BROKER_BALANCING_MODE}. It also performs other common - * validness checks to the cluster. - */ - public static void verifyClearBrokerValidness( - ClusterInfo cluster, Predicate isDemoted, Predicate allowedTopics) { - var disallowedTopicsToClear = - cluster.topicPartitionReplicas().stream() - .filter(tpr -> isDemoted.test(tpr.brokerId())) - .filter(tpr -> !allowedTopics.test(tpr.topic())) - .collect(Collectors.toUnmodifiableSet()); - if (!disallowedTopicsToClear.isEmpty()) - throw new IllegalArgumentException( - "Attempts to clear some brokers, but some of them contain topics that forbidden from being changed due to \"" - + BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX - + "\": " - + disallowedTopicsToClear); - + /** Performs common validness checks to the cluster. */ + public static void verifyClearBrokerValidness(ClusterInfo cluster, Predicate isDemoted) { var ongoingEventReplica = cluster.replicas().stream() .filter(r -> isDemoted.test(r.broker().id())) diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index 19a3c0dec4..26b628f95a 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -28,6 +28,7 @@ import java.util.function.Predicate; import java.util.function.Supplier; import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.balancer.AlgorithmConfig; @@ -161,15 +162,24 @@ public Optional offer(AlgorithmConfig config) { id -> balancingMode.get(id) == BalancerUtils.BalancingModes.DEMOTED; final var hasDemoted = balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.DEMOTED); - BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted, allowedTopics); + BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted); final var currentClusterInfo = BalancerUtils.clearedCluster(config.clusterInfo(), isDemoted, isBalancing); final var clusterBean = config.clusterBean(); + final var fixedReplicas = + config + .clusterInfo() + .replicaStream() + // if a topic is not allowed to move, it should be fixed. + // if a topic is not allowed to move, but originally it located on a demoting broker, it + // is ok to move. + .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isDemoted.test(tpr.broker().id())) + .collect(Collectors.toUnmodifiableSet()); final var allocationTweaker = ShuffleTweaker.builder() .numberOfShuffle(() -> ThreadLocalRandom.current().nextInt(minStep, maxStep)) - .allowedTopics(allowedTopics) + .allowedReplicas(r -> !fixedReplicas.contains(r)) .allowedBrokers(isBalancing) .build(); final var moveCostFunction = config.moveCostFunction(); diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java index c8b2939221..03b8cbfc29 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java @@ -24,6 +24,7 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.balancer.AlgorithmConfig; @@ -91,15 +92,24 @@ public Optional offer(AlgorithmConfig config) { id -> balancingMode.get(id) == BalancerUtils.BalancingModes.DEMOTED; final var hasDemoted = balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.DEMOTED); - BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted, allowedTopics); + BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted); final var currentClusterInfo = BalancerUtils.clearedCluster(config.clusterInfo(), isDemoted, isBalancing); final var clusterBean = config.clusterBean(); + final var fixedReplicas = + config + .clusterInfo() + .replicaStream() + // if a topic is not allowed to move, it should be fixed. + // if a topic is not allowed to move, but originally it located on a demoting broker, it + // is ok to move. + .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isDemoted.test(tpr.broker().id())) + .collect(Collectors.toUnmodifiableSet()); final var allocationTweaker = ShuffleTweaker.builder() .numberOfShuffle(() -> ThreadLocalRandom.current().nextInt(minStep, maxStep)) - .allowedTopics(allowedTopics) + .allowedReplicas(r -> !fixedReplicas.contains(r)) .allowedBrokers(isBalancing) .build(); final var moveCostFunction = config.moveCostFunction(); diff --git a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java index 5584294b6f..07db764350 100644 --- a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java +++ b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java @@ -47,15 +47,15 @@ public class ShuffleTweaker { private final Supplier numberOfShuffle; - private final Predicate allowedTopics; + private final Predicate allowedReplicas; private final Predicate allowedBrokers; public ShuffleTweaker( Supplier numberOfShuffle, - Predicate allowedTopics, + Predicate allowedReplicas, Predicate allowedBrokers) { this.numberOfShuffle = numberOfShuffle; - this.allowedTopics = allowedTopics; + this.allowedReplicas = allowedReplicas; this.allowedBrokers = allowedBrokers; } @@ -77,10 +77,10 @@ public Stream generate(ClusterInfo baseAllocation) { final var legalReplicas = baseAllocation.topicPartitions().stream() - .filter(tp -> this.allowedTopics.test(tp.topic())) .filter(tp -> eligiblePartition(baseAllocation.replicas(tp))) .flatMap(baseAllocation::replicaStream) .filter(r -> this.allowedBrokers.test(r.broker().id())) + .filter(this.allowedReplicas) .toList(); return Stream.generate( @@ -107,8 +107,10 @@ public Stream generate(ClusterInfo baseAllocation) { .replicaStream(sourceReplica.topicPartition()) // leader pair follower, follower pair leader .filter(r -> r.isFollower() != sourceReplica.isFollower()) - // this follower is located at allowed broker + // this leader/follower is located at allowed broker .filter(r -> this.allowedBrokers.test(r.broker().id())) + // this leader/follower is allowed to tweak + .filter(this.allowedReplicas) // not forbidden .filter(r -> !forbiddenReplica.contains(r.topicPartitionReplica())) .map(r -> Map.entry(r, ThreadLocalRandom.current().nextInt())) @@ -224,7 +226,7 @@ public String toString() { public static class Builder { private Supplier numberOfShuffle = () -> ThreadLocalRandom.current().nextInt(1, 5); - private Predicate allowedTopics = (name) -> true; + private Predicate allowedReplicas = (replica) -> true; private Predicate allowedBrokers = (name) -> true; private Builder() {} @@ -234,8 +236,8 @@ public Builder numberOfShuffle(Supplier numberOfShuffle) { return this; } - public Builder allowedTopics(Predicate allowedTopics) { - this.allowedTopics = allowedTopics; + public Builder allowedReplicas(Predicate allowedReplicas) { + this.allowedReplicas = allowedReplicas; return this; } @@ -245,7 +247,7 @@ public Builder allowedBrokers(Predicate allowedBrokers) { } public ShuffleTweaker build() { - return new ShuffleTweaker(numberOfShuffle, allowedTopics, allowedBrokers); + return new ShuffleTweaker(numberOfShuffle, allowedReplicas, allowedBrokers); } } } diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index 2495e54722..047162a648 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -26,7 +26,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.Stream; import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; @@ -299,7 +298,8 @@ public void testBalancingModeDemoted() { } { - var testName = "[test if allowed topics is used, clear disallow topic will raise an error]"; + var testName = + "[test if allowed topics is used, disallowed partitions on demoted broker will be force to move]"; var base = ClusterInfo.builder() .addNode(Set.of(1, 2, 3)) @@ -309,82 +309,52 @@ public void testBalancingModeDemoted() { Map.entry(2, Set.of("/folder")), Map.entry(3, Set.of("/folder")))) .build(); - var node12 = Stream.of(1, 2).map(base::node).iterator(); - var node13 = Stream.of(1, 3).map(base::node).iterator(); - var node123 = Stream.of(1, 2, 3).map(base::node).iterator(); + var node3 = base.node(3); var testCluster = ClusterInfo.builder(base) - .addTopic("OK", 1, (short) 1, r -> Replica.builder(r).broker(base.node(1)).build()) - .addTopic( - "OK_SKIP", 2, (short) 1, r -> Replica.builder(r).broker(node12.next()).build()) - .addTopic( - "Replica", 1, (short) 2, r -> Replica.builder(r).broker(node13.next()).build()) - .addTopic( - "Partition", 3, (short) 1, r -> Replica.builder(r).broker(node123.next()).build()) + .addTopic("topic", 3, (short) 1) + .addTopic("ok0", 10, (short) 1, r -> Replica.builder(r).broker(node3).build()) + .addTopic("ok1", 10, (short) 1, r -> Replica.builder(r).broker(node3).build()) + .addTopic("ok2", 10, (short) 1, r -> Replica.builder(r).broker(node3).build()) .build(); - Assertions.assertDoesNotThrow( - () -> - balancer.offer( - AlgorithmConfig.builder() - .clusterInfo(testCluster) - .clusterCost(decreasingCost()) - .timeout(Duration.ofSeconds(2)) - .configs(customConfig.raw()) - // allow anything other than "OK" topic - .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!OK).*") - // clear broker 3 - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") - // this won't raise an error since that topic didn't locate at 3 - .build()), - testName); - Assertions.assertDoesNotThrow( - () -> - balancer.offer( - AlgorithmConfig.builder() - .clusterInfo(testCluster) - .clusterCost(decreasingCost()) - .timeout(Duration.ofSeconds(2)) - .configs(customConfig.raw()) - // allow anything other than "OK" topic - .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!OK_SKIP).*") - // clear broker 3 - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") - // this won't raise an error since that topic didn't locate at 3 - .build()), - testName); - Assertions.assertThrows( - Exception.class, - () -> - balancer.offer( - AlgorithmConfig.builder() - .clusterInfo(testCluster) - .clusterCost(decreasingCost()) - .timeout(Duration.ofSeconds(2)) - .configs(customConfig.raw()) - // allow anything other than "Replica" topic - .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!Replica).*") - // clear broker 3 - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") - // this will raise an error since that topic has a replica at 3 - .build()), - testName); - Assertions.assertThrows( - Exception.class, - () -> - balancer.offer( - AlgorithmConfig.builder() - .clusterInfo(testCluster) - .clusterCost(decreasingCost()) - .timeout(Duration.ofSeconds(2)) - .configs(customConfig.raw()) - // allow anything other than "Replica" topic - .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!Partition).*") - // clear broker 3 - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") - // this will raise an error since that topic has a partition at 3 - .build()), - testName); + var result = + Assertions.assertDoesNotThrow( + () -> + balancer.offer( + AlgorithmConfig.builder() + .clusterInfo(testCluster) + .clusterCost(decreasingCost()) + .timeout(Duration.ofSeconds(2)) + .configs(customConfig.raw()) + // allow anything other than this topic + .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!topic).*") + // clear broker 3 + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") + // partition at broker 3 will be forced to move + .build()), + testName); + + Assertions.assertTrue(result.isPresent()); + Assertions.assertNotEquals( + List.of(), + testCluster.replicas().stream().filter(x -> x.broker().id() == 3).toList(), + "Originally, some replica located at broker 3"); + Assertions.assertEquals( + List.of(), + result.get().proposal().replicas().stream().filter(x -> x.broker().id() == 3).toList(), + "Returned allocation has no replica located at broker 3"); + var toStay = + testCluster.replicas().stream() + .filter(x -> x.topic().equals("topic")) + .filter(x -> x.broker().id() != 3) + .collect(Collectors.toSet()); + Assertions.assertTrue( + result.get().proposal().replicas().stream() + .filter(x -> x.topic().equals("topic")) + .collect(Collectors.toSet()) + .containsAll(toStay), + "Disallowed partition stay still except those at broker 3"); } { diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java index 5680dbb57a..b18804d433 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java @@ -107,25 +107,6 @@ void testVerifyClearBrokerValidness() { .addTopic("C", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) .build(); - Assertions.assertThrows( - IllegalArgumentException.class, - () -> - BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 1, t -> !t.equals("A"))); - Assertions.assertThrows( - IllegalArgumentException.class, - () -> - BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 2, t -> !t.equals("B"))); - Assertions.assertThrows( - IllegalArgumentException.class, - () -> - BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 3, t -> !t.equals("C"))); - Assertions.assertDoesNotThrow( - () -> BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 1, t -> t.equals("A"))); - Assertions.assertDoesNotThrow( - () -> BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 2, t -> t.equals("B"))); - Assertions.assertDoesNotThrow( - () -> BalancerUtils.verifyClearBrokerValidness(cluster, id -> id == 3, t -> t.equals("C"))); - var hasAdding = ClusterInfo.builder(cluster).mapLog(r -> Replica.builder(r).isAdding(true).build()).build(); var hasRemoving = @@ -136,19 +117,19 @@ void testVerifyClearBrokerValidness() { ClusterInfo.builder(cluster).mapLog(r -> Replica.builder(r).isFuture(true).build()).build(); Assertions.assertThrows( IllegalArgumentException.class, - () -> BalancerUtils.verifyClearBrokerValidness(hasAdding, x -> true, x -> true)); + () -> BalancerUtils.verifyClearBrokerValidness(hasAdding, x -> true)); Assertions.assertThrows( IllegalArgumentException.class, - () -> BalancerUtils.verifyClearBrokerValidness(hasRemoving, x -> true, x -> true)); + () -> BalancerUtils.verifyClearBrokerValidness(hasRemoving, x -> true)); Assertions.assertThrows( IllegalArgumentException.class, - () -> BalancerUtils.verifyClearBrokerValidness(hasFuture, x -> true, x -> true)); + () -> BalancerUtils.verifyClearBrokerValidness(hasFuture, x -> true)); Assertions.assertDoesNotThrow( - () -> BalancerUtils.verifyClearBrokerValidness(hasAdding, x -> false, x -> true)); + () -> BalancerUtils.verifyClearBrokerValidness(hasAdding, x -> false)); Assertions.assertDoesNotThrow( - () -> BalancerUtils.verifyClearBrokerValidness(hasRemoving, x -> false, x -> true)); + () -> BalancerUtils.verifyClearBrokerValidness(hasRemoving, x -> false)); Assertions.assertDoesNotThrow( - () -> BalancerUtils.verifyClearBrokerValidness(hasFuture, x -> false, x -> true)); + () -> BalancerUtils.verifyClearBrokerValidness(hasFuture, x -> false)); } @Test diff --git a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java index 3ce16cb6cc..a80143fa18 100644 --- a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java @@ -212,4 +212,78 @@ void testEligiblePartition() { "only normal-topic get altered. Actual: " + notFulfilled); }); } + + @Test + void testAllowedBrokers() { + var notAllowed = ThreadLocalRandom.current().nextInt(1, 4); + var tweaker = ShuffleTweaker.builder().allowedBrokers(b -> b != notAllowed).build(); + + var testCluster = + ClusterInfo.builder() + .addNode(Set.of(1, 2, 3)) + .addFolders(Map.of(1, Set.of("/folder"))) + .addFolders(Map.of(2, Set.of("/folder"))) + .addFolders(Map.of(3, Set.of("/folder"))) + .addTopic("A", 6, (short) 2) + .addTopic("B", 6, (short) 2) + .addTopic("C", 6, (short) 2) + .build(); + for (int i = 0; i < 100; i++) { + var result = tweaker.generate(testCluster).findFirst().orElseThrow(); + var expected = + testCluster + .replicaStream() + .map(Replica::topicPartitionReplica) + .filter(x -> x.brokerId() == notAllowed) + .collect(Collectors.toSet()); + var actual = + result + .replicaStream() + .map(Replica::topicPartitionReplica) + .filter(x -> x.brokerId() == notAllowed) + .collect(Collectors.toSet()); + Assertions.assertEquals(expected, actual, "Replica at not allowed broker should stay"); + Assertions.assertNotEquals( + Set.of(), + ClusterInfo.findNonFulfilledAllocation(testCluster, result), + "Some tweak has been performed"); + } + } + + @Test + void testAllowedReplicas() { + var tweaker = ShuffleTweaker.builder().allowedReplicas(Replica::isFollower).build(); + + var testCluster = + ClusterInfo.builder() + .addNode(Set.of(1, 2, 3)) + .addFolders(Map.of(1, Set.of("/folder"))) + .addFolders(Map.of(2, Set.of("/folder"))) + .addFolders(Map.of(3, Set.of("/folder"))) + .addTopic("A", 6, (short) 2) + .addTopic("B", 6, (short) 2) + .addTopic("C", 6, (short) 2) + .build(); + + for (int i = 0; i < 100; i++) { + var result = tweaker.generate(testCluster).findFirst().orElseThrow(); + var expected = + testCluster + .replicaStream() + .filter(Replica::isLeader) + .map(Replica::topicPartitionReplica) + .collect(Collectors.toSet()); + var actual = + result + .replicaStream() + .filter(Replica::isLeader) + .map(Replica::topicPartitionReplica) + .collect(Collectors.toSet()); + Assertions.assertEquals(expected, actual, "Only follower can be tweaked"); + Assertions.assertNotEquals( + Set.of(), + ClusterInfo.findNonFulfilledAllocation(testCluster, result), + "Some tweak has been performed"); + } + } } diff --git a/docs/web_server/web_api_balancer_chinese.md b/docs/web_server/web_api_balancer_chinese.md index 309f924645..41e5e8a366 100644 --- a/docs/web_server/web_api_balancer_chinese.md +++ b/docs/web_server/web_api_balancer_chinese.md @@ -28,9 +28,18 @@ POST /balancer | config key | config value | |--------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| balancer.allowed.topics.regex | 一個正則表達式,表達允許進行搬移操作的 topic 名稱,當沒有指定的時候,代表沒有任何限制,所有 topic 都可以做搬移 | +| balancer.allowed.topics.regex | 一個正則表達式,表達允許進行搬移操作的 topic 名稱,當沒有指定的時候,代表沒有任何限制,所有 topic 都可以做搬移,包含 internal topics,表格下面有附上一些常見的設定範例,可以當做參考。 | | balancer.broker.balancing.mode | 這個參數指定每個 Broker 要採取的負載平衡設定,目前擁有的模式包含 `balancing`, `demoted` 和 `excluded`。`balancing` 代表特定節點要參予負載平衡的過程,該節點身上的負載可以被變動。`demoted` 代表特定節點身上的負載必須要全部移除,這個功能能夠協助使用者優雅地下線一個節點。`excluded` 代表特定節點不能夠參予負載平衡的過程,節點不能新增或移除負載。這個參數的格式是一系列的 key/value pair 的字串,每個 pair 之間透過逗號 "," 間隔,而 key/value 之間透過冒號 ":" 間隔,如 `(brokerId_A |"default"):(mode),(brokerId_B):(mode), ...`,其中 `key` 欄位代表這個是描述某 id 節點的設定,而對應的 `value` 欄位則是該節點要套用的負載平衡模式(`balancing`, `demoted` 或 `excluded`),另外 `key` 欄位可以填寫特殊字串 `default`,代表沒有被設定所提及的節點應該使用的負載平衡模式,比如填寫 `default:excluded` 可以使設定沒有提到的節點不參予負載平衡的過程,預設的 `default` 模式是 `balancing`,意即所有設定沒有提到的節點都將參予負載平衡的過程。 | +一些可以參考的 `balancer.allowed.topics.regex` 設定: + +| `balancer.allowed.topics.regex` 設定範例 | 效果 | +|-----------------------------------------------|-----------------------------------------------------------------------------------------| +| `^(?!__).*$` | 所有非 `__` 開頭的 topic 都可以搬移,如 `__consumer_offsets` 和 `__transaction_state` 之外的 topic 都可以搬移 | +| `^(?!__).*(? Date: Sat, 27 May 2023 21:20:48 +0800 Subject: [PATCH 34/77] [SERIALIZATION] Serialize Borker info in Clusterinfo (#1721) --- .../java/org/astraea/common/ByteUtils.java | 218 +++++++++++------- .../common/generated/ClusterInfo.proto | 37 --- .../common/generated/admin/Broker.proto | 22 ++ .../common/generated/admin/ClusterInfo.proto | 14 ++ .../common/generated/admin/Replica.proto | 22 ++ .../common/generated/admin/Topic.proto | 10 + .../generated/admin/TopicPartition.proto | 8 + .../org/astraea/common/ByteUtilsTest.java | 45 ++++ 8 files changed, 258 insertions(+), 118 deletions(-) delete mode 100644 common/src/main/proto/org/astraea/common/generated/ClusterInfo.proto create mode 100644 common/src/main/proto/org/astraea/common/generated/admin/Broker.proto create mode 100644 common/src/main/proto/org/astraea/common/generated/admin/ClusterInfo.proto create mode 100644 common/src/main/proto/org/astraea/common/generated/admin/Replica.proto create mode 100644 common/src/main/proto/org/astraea/common/generated/admin/Topic.proto create mode 100644 common/src/main/proto/org/astraea/common/generated/admin/TopicPartition.proto diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index cf0301f3a4..ad30ffa53d 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -34,8 +34,12 @@ import org.astraea.common.admin.Topic; import org.astraea.common.admin.TopicPartition; import org.astraea.common.generated.BeanObjectOuterClass; -import org.astraea.common.generated.ClusterInfoOuterClass; import org.astraea.common.generated.PrimitiveOuterClass; +import org.astraea.common.generated.admin.BrokerOuterClass; +import org.astraea.common.generated.admin.ClusterInfoOuterClass; +import org.astraea.common.generated.admin.ReplicaOuterClass; +import org.astraea.common.generated.admin.TopicOuterClass; +import org.astraea.common.generated.admin.TopicPartitionOuterClass; import org.astraea.common.metrics.BeanObject; public final class ByteUtils { @@ -186,58 +190,13 @@ public static byte[] toBytes(BeanObject value) { return beanBuilder.build().toByteArray(); } - // TODO: Due to the change of NodeInfo to Broker. This and the test should be updated. /** Serialize ClusterInfo by protocol buffer. */ public static byte[] toBytes(ClusterInfo value) { return ClusterInfoOuterClass.ClusterInfo.newBuilder() .setClusterId(value.clusterId()) - .addAllNodeInfo( - value.brokers().stream() - .map( - nodeInfo -> - ClusterInfoOuterClass.ClusterInfo.NodeInfo.newBuilder() - .setId(nodeInfo.id()) - .setHost(nodeInfo.host()) - .setPort(nodeInfo.port()) - .build()) - .collect(Collectors.toList())) - .addAllTopic( - value.topics().values().stream() - .map( - topicClass -> - ClusterInfoOuterClass.ClusterInfo.Topic.newBuilder() - .setName(topicClass.name()) - .putAllConfig(topicClass.config().raw()) - .setInternal(topicClass.internal()) - .addAllPartition( - topicClass.topicPartitions().stream() - .map(TopicPartition::partition) - .collect(Collectors.toList())) - .build()) - .collect(Collectors.toList())) - .addAllReplica( - value.replicas().stream() - .map( - replica -> - ClusterInfoOuterClass.ClusterInfo.Replica.newBuilder() - .setTopic(replica.topic()) - .setPartition(replica.partition()) - .setNodeInfo( - ClusterInfoOuterClass.ClusterInfo.NodeInfo.newBuilder() - .setId(replica.broker().id()) - .setHost(replica.broker().host()) - .setPort(replica.broker().port()) - .build()) - .setLag(replica.lag()) - .setSize(replica.size()) - .setIsLeader(replica.isLeader()) - .setIsSync(replica.isSync()) - .setIsFuture(replica.isFuture()) - .setIsOffline(replica.isOffline()) - .setIsPreferredLeader(replica.isPreferredLeader()) - .setPath(replica.path()) - .build()) - .collect(Collectors.toList())) + .addAllBroker(value.brokers().stream().map(ByteUtils::toOuterClass).toList()) + .addAllTopic(value.topics().values().stream().map(ByteUtils::toOuterClass).toList()) + .addAllReplica(value.replicas().stream().map(ByteUtils::toOuterClass).toList()) .build() .toByteArray(); } @@ -328,51 +287,148 @@ public static BeanObject readBeanObject(byte[] bytes) throws SerializationExcept } } - // TODO: Due to the change of NodeInfo to Broker. This and the test should be updated. /** Deserialize to ClusterInfo with protocol buffer */ public static ClusterInfo readClusterInfo(byte[] bytes) { try { var outerClusterInfo = ClusterInfoOuterClass.ClusterInfo.parseFrom(bytes); return ClusterInfo.of( outerClusterInfo.getClusterId(), - outerClusterInfo.getNodeInfoList().stream() - .map(nodeInfo -> Broker.of(nodeInfo.getId(), nodeInfo.getHost(), nodeInfo.getPort())) - .collect(Collectors.toList()), + outerClusterInfo.getBrokerList().stream().map(ByteUtils::toBroker).toList(), outerClusterInfo.getTopicList().stream() - .map( - protoTopic -> - new Topic( - protoTopic.getName(), - new Config(protoTopic.getConfigMap()), - protoTopic.getInternal(), - Set.copyOf(protoTopic.getPartitionList()))) + .map(ByteUtils::toTopic) .collect(Collectors.toMap(Topic::name, Function.identity())), - outerClusterInfo.getReplicaList().stream() - .map( - replica -> - Replica.builder() - .topic(replica.getTopic()) - .partition(replica.getPartition()) - .broker( - Broker.of( - replica.getNodeInfo().getId(), - replica.getNodeInfo().getHost(), - replica.getNodeInfo().getPort())) - .lag(replica.getLag()) - .size(replica.getSize()) - .isLeader(replica.getIsLeader()) - .isSync(replica.getIsSync()) - .isFuture(replica.getIsFuture()) - .isOffline(replica.getIsOffline()) - .isPreferredLeader(replica.getIsPreferredLeader()) - .path(replica.getPath()) - .build()) - .collect(Collectors.toList())); + outerClusterInfo.getReplicaList().stream().map(ByteUtils::toReplica).toList()); } catch (InvalidProtocolBufferException ex) { throw new SerializationException(ex); } } + // ---------------------------Serialize To ProtoBuf Outer Class------------------------------- // + + private static BrokerOuterClass.Broker.DataFolder toOuterClass(Broker.DataFolder dataFolder) { + return BrokerOuterClass.Broker.DataFolder.newBuilder() + .setPath(dataFolder.path()) + .putAllPartitionSizes( + dataFolder.partitionSizes().entrySet().stream() + .collect(Collectors.toMap(entry -> entry.getKey().toString(), Map.Entry::getValue))) + .putAllOrphanPartitionSizes( + dataFolder.orphanPartitionSizes().entrySet().stream() + .collect(Collectors.toMap(entry -> entry.getKey().toString(), Map.Entry::getValue))) + .build(); + } + + private static TopicPartitionOuterClass.TopicPartition toOuterClass( + TopicPartition topicPartition) { + return TopicPartitionOuterClass.TopicPartition.newBuilder() + .setPartition(topicPartition.partition()) + .setTopic(topicPartition.topic()) + .build(); + } + + private static BrokerOuterClass.Broker toOuterClass(Broker broker) { + return BrokerOuterClass.Broker.newBuilder() + .setId(broker.id()) + .setHost(broker.host()) + .setPort(broker.port()) + .setIsController(broker.isController()) + .putAllConfig(broker.config().raw()) + .addAllDataFolder(broker.dataFolders().stream().map(ByteUtils::toOuterClass).toList()) + .addAllTopicPartitions( + broker.topicPartitions().stream().map(ByteUtils::toOuterClass).toList()) + .addAllTopicPartitionLeaders( + broker.topicPartitionLeaders().stream().map(ByteUtils::toOuterClass).toList()) + .build(); + } + + private static TopicOuterClass.Topic toOuterClass(Topic topic) { + return TopicOuterClass.Topic.newBuilder() + .setName(topic.name()) + .putAllConfig(topic.config().raw()) + .setInternal(topic.internal()) + .addAllPartitionIds(topic.partitionIds()) + .build(); + } + + private static ReplicaOuterClass.Replica toOuterClass(Replica replica) { + return ReplicaOuterClass.Replica.newBuilder() + .setTopic(replica.topic()) + .setPartition(replica.partition()) + .setBroker(toOuterClass(replica.broker())) + .setLag(replica.lag()) + .setSize(replica.size()) + .setIsInternal(replica.isInternal()) + .setIsLeader(replica.isLeader()) + .setIsAdding(replica.isAdding()) + .setIsRemoving(replica.isRemoving()) + .setIsSync(replica.isSync()) + .setIsFuture(replica.isFuture()) + .setIsOffline(replica.isOffline()) + .setIsPreferredLeader(replica.isPreferredLeader()) + .setPath(replica.path()) + .build(); + } + + // -------------------------Deserialize From ProtoBuf Outer Class----------------------------- // + + private static Broker.DataFolder toDataFolder(BrokerOuterClass.Broker.DataFolder dataFolder) { + return new Broker.DataFolder( + dataFolder.getPath(), + dataFolder.getPartitionSizesMap().entrySet().stream() + .collect( + Collectors.toMap(entry -> TopicPartition.of(entry.getKey()), Map.Entry::getValue)), + dataFolder.getOrphanPartitionSizesMap().entrySet().stream() + .collect( + Collectors.toMap(entry -> TopicPartition.of(entry.getKey()), Map.Entry::getValue))); + } + + private static TopicPartition toTopicPartition( + TopicPartitionOuterClass.TopicPartition topicPartition) { + return new TopicPartition(topicPartition.getTopic(), topicPartition.getPartition()); + } + + private static Broker toBroker(BrokerOuterClass.Broker broker) { + return new Broker( + broker.getId(), + broker.getHost(), + broker.getPort(), + broker.getIsController(), + new Config(broker.getConfigMap()), + broker.getDataFolderList().stream().map(ByteUtils::toDataFolder).toList(), + broker.getTopicPartitionsList().stream() + .map(ByteUtils::toTopicPartition) + .collect(Collectors.toSet()), + broker.getTopicPartitionLeadersList().stream() + .map(ByteUtils::toTopicPartition) + .collect(Collectors.toSet())); + } + + private static Topic toTopic(TopicOuterClass.Topic topic) { + return new Topic( + topic.getName(), + new Config(topic.getConfigMap()), + topic.getInternal(), + Set.copyOf(topic.getPartitionIdsList())); + } + + private static Replica toReplica(ReplicaOuterClass.Replica replica) { + return Replica.builder() + .topic(replica.getTopic()) + .partition(replica.getPartition()) + .broker(toBroker(replica.getBroker())) + .lag(replica.getLag()) + .size(replica.getSize()) + .isInternal(replica.getIsInternal()) + .isLeader(replica.getIsLeader()) + .isAdding(replica.getIsAdding()) + .isRemoving(replica.getIsRemoving()) + .isSync(replica.getIsSync()) + .isFuture(replica.getIsFuture()) + .isOffline(replica.getIsOffline()) + .isPreferredLeader(replica.getIsPreferredLeader()) + .path(replica.getPath()) + .build(); + } + // --------------------------------ProtoBuf Primitive----------------------------------------- // /** diff --git a/common/src/main/proto/org/astraea/common/generated/ClusterInfo.proto b/common/src/main/proto/org/astraea/common/generated/ClusterInfo.proto deleted file mode 100644 index a9a6db12f4..0000000000 --- a/common/src/main/proto/org/astraea/common/generated/ClusterInfo.proto +++ /dev/null @@ -1,37 +0,0 @@ -syntax = "proto3"; - -package org.astraea.common.generated; - -message ClusterInfo{ - string clusterId = 1; - - repeated NodeInfo nodeInfo = 2; - message NodeInfo { - int32 id = 1; - string host = 2; - int32 port = 3; - } - - repeated Topic topic = 3; - message Topic { - string name = 1; - map config = 2; - bool internal = 3; - repeated int32 partition = 4; - } - - repeated Replica replica = 4; - message Replica{ - string topic = 1; - int32 partition = 2; - NodeInfo nodeInfo = 3; - int64 lag = 4; - int64 size = 5; - bool isLeader = 6; - bool isSync = 7; - bool isFuture = 8; - bool isOffline = 9; - bool isPreferredLeader = 10; - string path = 11; - } -} \ No newline at end of file diff --git a/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto b/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto new file mode 100644 index 0000000000..cf463e193c --- /dev/null +++ b/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto @@ -0,0 +1,22 @@ +syntax = "proto3"; + +package org.astraea.common.generated.admin; + +import "org/astraea/common/generated/admin/TopicPartition.proto"; + +message Broker { + int32 id = 1; + string host = 2; + int32 port = 3; + bool isController = 4; + map config = 5; + repeated DataFolder dataFolder = 6; + repeated TopicPartition topicPartitions = 7; + repeated TopicPartition topicPartitionLeaders = 8; + + message DataFolder { + string path = 1; + map partitionSizes = 2; + map orphanPartitionSizes = 3; + } +} \ No newline at end of file diff --git a/common/src/main/proto/org/astraea/common/generated/admin/ClusterInfo.proto b/common/src/main/proto/org/astraea/common/generated/admin/ClusterInfo.proto new file mode 100644 index 0000000000..260f65ec64 --- /dev/null +++ b/common/src/main/proto/org/astraea/common/generated/admin/ClusterInfo.proto @@ -0,0 +1,14 @@ +syntax = "proto3"; + +package org.astraea.common.generated.admin; + +import "org/astraea/common/generated/admin/Broker.proto"; +import "org/astraea/common/generated/admin/Topic.proto"; +import "org/astraea/common/generated/admin/Replica.proto"; + +message ClusterInfo { + string clusterId = 1; + repeated Broker broker = 2; + repeated Topic topic = 3; + repeated Replica replica = 4; +} \ No newline at end of file diff --git a/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto b/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto new file mode 100644 index 0000000000..598ade270e --- /dev/null +++ b/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto @@ -0,0 +1,22 @@ +syntax = "proto3"; + +package org.astraea.common.generated.admin; + +import "org/astraea/common/generated/admin/Broker.proto"; + +message Replica { + string topic = 1; + int32 partition = 2; + Broker broker = 3; + bool isLeader = 7; + bool isSync = 10; + bool isOffline = 12; + bool isAdding = 8; + bool isRemoving = 9; + bool isFuture = 11; + bool isPreferredLeader = 13; + int64 lag = 4; + int64 size = 5; + string path = 14; + bool isInternal = 6; +} \ No newline at end of file diff --git a/common/src/main/proto/org/astraea/common/generated/admin/Topic.proto b/common/src/main/proto/org/astraea/common/generated/admin/Topic.proto new file mode 100644 index 0000000000..cb46f4b9d7 --- /dev/null +++ b/common/src/main/proto/org/astraea/common/generated/admin/Topic.proto @@ -0,0 +1,10 @@ +syntax = "proto3"; + +package org.astraea.common.generated.admin; + +message Topic { + string name = 1; + map config = 2; + bool internal = 3; + repeated int32 partitionIds = 4; +} \ No newline at end of file diff --git a/common/src/main/proto/org/astraea/common/generated/admin/TopicPartition.proto b/common/src/main/proto/org/astraea/common/generated/admin/TopicPartition.proto new file mode 100644 index 0000000000..44f446c88d --- /dev/null +++ b/common/src/main/proto/org/astraea/common/generated/admin/TopicPartition.proto @@ -0,0 +1,8 @@ +syntax = "proto3"; + +package org.astraea.common.generated.admin; + +message TopicPartition { + int32 partition = 1; + string topic = 2; +} \ No newline at end of file diff --git a/common/src/test/java/org/astraea/common/ByteUtilsTest.java b/common/src/test/java/org/astraea/common/ByteUtilsTest.java index 616229e122..03e7a0cc76 100644 --- a/common/src/test/java/org/astraea/common/ByteUtilsTest.java +++ b/common/src/test/java/org/astraea/common/ByteUtilsTest.java @@ -17,6 +17,11 @@ package org.astraea.common; import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Set; +import org.astraea.common.admin.Admin; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.it.Service; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -68,4 +73,44 @@ void testBoolean2Bytes() { Assertions.assertArrayEquals(new byte[] {1}, ByteUtils.toBytes(true)); Assertions.assertArrayEquals(new byte[] {0}, ByteUtils.toBytes(false)); } + + @Test + void testReadAndToBytesClusterInfo() { + var topic = Utils.randomString(); + try (var service = Service.builder().numberOfBrokers(3).build()) { + try (var admin = Admin.of(service.bootstrapServers())) { + admin + .creator() + .topic(topic) + .numberOfPartitions(1) + .numberOfReplicas((short) 3) + .run() + .toCompletableFuture() + .join(); + Utils.sleep(Duration.ofSeconds(1)); + var clusterInfo = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join(); + + Assertions.assertDoesNotThrow(() -> ByteUtils.toBytes(clusterInfo)); + var bytes = ByteUtils.toBytes(clusterInfo); + Assertions.assertDoesNotThrow(() -> ByteUtils.readClusterInfo(bytes)); + var deserializedClusterInfo = ByteUtils.readClusterInfo(bytes); + Assertions.assertEquals(clusterInfo.clusterId(), deserializedClusterInfo.clusterId()); + Assertions.assertEquals(clusterInfo.brokers(), deserializedClusterInfo.brokers()); + Assertions.assertEquals(clusterInfo.topics(), deserializedClusterInfo.topics()); + Assertions.assertEquals(clusterInfo.replicas(), deserializedClusterInfo.replicas()); + } + } + } + + @Test + void testReadAndToBytesEmptyClusterInfo() { + var clusterInfo = ClusterInfo.empty(); + var serializedInfo = ByteUtils.toBytes(clusterInfo); + var deserializedClusterInfo = ByteUtils.readClusterInfo(serializedInfo); + + Assertions.assertEquals(clusterInfo.clusterId(), deserializedClusterInfo.clusterId()); + Assertions.assertEquals(clusterInfo.brokers(), deserializedClusterInfo.brokers()); + Assertions.assertEquals(clusterInfo.topics(), deserializedClusterInfo.topics()); + Assertions.assertEquals(clusterInfo.replicas(), deserializedClusterInfo.replicas()); + } } From 124a096884aec5732e0f0330f69becaae5da9d4e Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Sat, 27 May 2023 22:27:58 +0800 Subject: [PATCH 35/77] [BALANCER] Fix `ShuffleTweaker` logic error that cause multiple replicas at one broker (#1770) --- .../balancer/tweakers/ShuffleTweaker.java | 86 ++++++++++++------- .../balancer/tweakers/ShuffleTweakerTest.java | 54 ++++++++++++ 2 files changed, 107 insertions(+), 33 deletions(-) diff --git a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java index 07db764350..d797512145 100644 --- a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java +++ b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java @@ -16,9 +16,10 @@ */ package org.astraea.common.balancer.tweakers; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -29,7 +30,7 @@ import org.astraea.common.Utils; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; -import org.astraea.common.admin.TopicPartitionReplica; +import org.astraea.common.admin.TopicPartition; /** * The {@link ShuffleTweaker} proposes a new log placement based on the current log placement, but @@ -87,45 +88,51 @@ public Stream generate(ClusterInfo baseAllocation) { () -> { final var shuffleCount = numberOfShuffle.get(); final var replicaOrder = Utils.shuffledPermutation(legalReplicas).iterator(); - final var forbiddenReplica = new HashSet(); + final var allocation = new HashMap>(); - final var finalCluster = ClusterInfo.builder(baseAllocation); for (int shuffled = 0; replicaOrder.hasNext() && shuffled < shuffleCount; ) { final var sourceReplica = replicaOrder.next(); - // the leadership change operation will not only affect source target but also the - // target replica. To prevent mutating one replica twice in the tweaking loop. We have - // to mandatory exclude the target replica since it has been touched in this tweak. - // Tweaking a replica twice is meaningless and incompatible with the design of - // ClusterInfoBuilder. - if (forbiddenReplica.contains(sourceReplica.topicPartitionReplica())) continue; - Supplier leadershipChange = () -> { - var targetReplica = - baseAllocation - .replicaStream(sourceReplica.topicPartition()) + var replicaList = + allocation.computeIfAbsent( + sourceReplica.topicPartition(), + (tp) -> new ArrayList<>(baseAllocation.replicas(tp))); + if (!replicaList.contains(sourceReplica)) return false; + var maybeTargetReplica = + replicaList.stream() // leader pair follower, follower pair leader .filter(r -> r.isFollower() != sourceReplica.isFollower()) // this leader/follower is located at allowed broker .filter(r -> this.allowedBrokers.test(r.broker().id())) // this leader/follower is allowed to tweak .filter(this.allowedReplicas) - // not forbidden - .filter(r -> !forbiddenReplica.contains(r.topicPartitionReplica())) .map(r -> Map.entry(r, ThreadLocalRandom.current().nextInt())) .min(Map.Entry.comparingByValue()) .map(Map.Entry::getKey); // allowed broker filter might cause no legal exchange target - if (targetReplica.isPresent()) { - var theFollower = - sourceReplica.isFollower() ? sourceReplica : targetReplica.orElseThrow(); - finalCluster.setPreferredLeader(theFollower.topicPartitionReplica()); - - forbiddenReplica.add(sourceReplica.topicPartitionReplica()); - forbiddenReplica.add(targetReplica.orElseThrow().topicPartitionReplica()); - + if (maybeTargetReplica.isPresent()) { + var targetReplica = maybeTargetReplica.get(); + var newLeader = sourceReplica.isFollower() ? sourceReplica : targetReplica; + for (int i = 0; i < replicaList.size(); i++) { + if (replicaList.get(i).equals(newLeader)) + replicaList.set( + i, + Replica.builder(replicaList.get(i)) + .isLeader(true) + .isPreferredLeader(true) + .build()); + else if (replicaList.get(i).isLeader() + || replicaList.get(i).isPreferredLeader()) + replicaList.set( + i, + Replica.builder(replicaList.get(i)) + .isLeader(false) + .isPreferredLeader(false) + .build()); + } return true; } else { return false; @@ -133,7 +140,11 @@ public Stream generate(ClusterInfo baseAllocation) { }; Supplier replicaListChange = () -> { - var replicaList = baseAllocation.replicas(sourceReplica.topicPartition()); + var replicaList = + allocation.computeIfAbsent( + sourceReplica.topicPartition(), + (tp) -> new ArrayList<>(baseAllocation.replicas(tp))); + if (!replicaList.contains(sourceReplica)) return false; var targetBroker = baseAllocation.brokers().stream() // the candidate should not be part of the replica list @@ -146,13 +157,15 @@ public Stream generate(ClusterInfo baseAllocation) { .map(Map.Entry::getKey); if (targetBroker.isPresent()) { - finalCluster.reassignReplica( - sourceReplica.topicPartitionReplica(), - targetBroker.orElseThrow().id(), - randomElement( - baseAllocation.brokerFolders().get(targetBroker.orElseThrow().id()))); - - forbiddenReplica.add(sourceReplica.topicPartitionReplica()); + var targetReplica = + Replica.builder(sourceReplica) + .broker(targetBroker.get()) + .path( + randomElement( + baseAllocation.brokerFolders().get(targetBroker.get().id()))) + .build(); + replicaList.remove(sourceReplica); + replicaList.add(targetReplica); return true; } else { return false; @@ -175,7 +188,14 @@ public Stream generate(ClusterInfo baseAllocation) { shuffled += isFinished ? 1 : 0; } - return finalCluster.build(); + return ClusterInfo.of( + baseAllocation.clusterId(), + baseAllocation.brokers(), + baseAllocation.topics(), + baseAllocation.topicPartitions().stream() + .map(tp -> allocation.getOrDefault(tp, baseAllocation.replicas(tp))) + .flatMap(Collection::stream) + .toList()); }); } diff --git a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java index a80143fa18..b2b0e6ec64 100644 --- a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java @@ -213,6 +213,60 @@ void testEligiblePartition() { }); } + @Test + void testReplicaFactorSafety() { + var testCluster = + ClusterInfo.builder() + .addNode(Set.of(1, 2, 3, 4, 5, 6)) + .addFolders(Map.of(1, Set.of("/folder1", "/folder2", "/folder3"))) + .addFolders(Map.of(2, Set.of("/folder1", "/folder2", "/folder3"))) + .addFolders(Map.of(3, Set.of("/folder1", "/folder2", "/folder3"))) + .addFolders(Map.of(4, Set.of("/folder1", "/folder2", "/folder3"))) + .addFolders(Map.of(5, Set.of("/folder1", "/folder2", "/folder3"))) + .addFolders(Map.of(6, Set.of("/folder1", "/folder2", "/folder3"))) + .addTopic("topic2", 100, (short) 2) + .addTopic("topic3", 100, (short) 3) + .addTopic("topic4", 100, (short) 4) + .build(); + + var tweaker = ShuffleTweaker.builder().numberOfShuffle(() -> 50).build(); + + tweaker + .generate(testCluster) + .map(x -> tweaker.generate(x).findFirst().orElseThrow()) + .map(x -> tweaker.generate(x).findFirst().orElseThrow()) + .map(x -> tweaker.generate(x).findFirst().orElseThrow()) + .map(x -> tweaker.generate(x).findFirst().orElseThrow()) + .limit(1000) + .forEach( + cluster -> { + for (var partition : cluster.topicPartitions()) { + var replicaSet = + cluster.replicas(partition).stream() + .map(Replica::topicPartitionReplica) + .collect(Collectors.toUnmodifiableSet()); + switch (partition.topic()) { + case "topic2" -> Assertions.assertEquals( + 2, replicaSet.size(), replicaSet.toString()); + case "topic3" -> Assertions.assertEquals( + 3, replicaSet.size(), replicaSet.toString()); + case "topic4" -> Assertions.assertEquals( + 4, replicaSet.size(), replicaSet.toString()); + } + + var replicas = cluster.replicas(partition).stream().toList(); + Assertions.assertEquals( + 1, + replicas.stream().filter(Replica::isLeader).count(), + "One leader only: " + replicas); + Assertions.assertEquals( + 1, + replicas.stream().filter(Replica::isPreferredLeader).count(), + "One preferred leader only: " + replicas); + } + }); + } + @Test void testAllowedBrokers() { var notAllowed = ThreadLocalRandom.current().nextInt(1, 4); From 5b565f972f1b2969b4a7fc0ecc1ca7d98e7d817c Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sat, 27 May 2023 23:14:36 +0800 Subject: [PATCH 36/77] [COMMON] rewrite switch by java 17 switch (#1787) --- .../astraea/app/performance/MonkeyThread.java | 16 +++----- .../astraea/app/performance/ReportFormat.java | 9 +++-- .../org/astraea/app/web/BalancerHandler.java | 38 ++++++++----------- .../java/org/astraea/app/web/Channel.java | 22 ++++------- .../java/org/astraea/app/web/Handler.java | 19 ++++------ .../astraea/app/web/BalancerHandlerTest.java | 18 +++------ .../java/org/astraea/common/ByteUtils.java | 26 +++++-------- .../main/java/org/astraea/common/Utils.java | 27 +++++-------- .../common/backup/RecordReaderBuilder.java | 9 ++--- .../common/backup/RecordWriterBuilder.java | 10 ++--- .../org/astraea/common/cost/NetworkCost.java | 20 ++++------ .../common/cost/NeutralIntegratedCost.java | 29 ++++++-------- .../org/astraea/fs/ftp/FtpFileSystem.java | 10 ++--- gui/src/main/java/org/astraea/gui/Query.java | 17 ++++----- .../java/org/astraea/gui/tab/BrokerNode.java | 26 ++++++------- 15 files changed, 115 insertions(+), 181 deletions(-) diff --git a/app/src/main/java/org/astraea/app/performance/MonkeyThread.java b/app/src/main/java/org/astraea/app/performance/MonkeyThread.java index 5a7d573615..e4677f5d72 100644 --- a/app/src/main/java/org/astraea/app/performance/MonkeyThread.java +++ b/app/src/main/java/org/astraea/app/performance/MonkeyThread.java @@ -41,16 +41,12 @@ static List play(List consumerThreads, Performance System.out.println("create chaos monkey"); return param.monkeys.entrySet().stream() .map( - entry -> { - switch ((entry.getKey())) { - case "kill": - return killMonkey(consumerThreads, entry.getValue()); - case "add": - return addMonkey(consumerThreads, entry.getValue(), param); - default: - return unsubscribeMonkey(consumerThreads, entry.getValue()); - } - }) + entry -> + switch ((entry.getKey())) { + case "kill" -> killMonkey(consumerThreads, entry.getValue()); + case "add" -> addMonkey(consumerThreads, entry.getValue(), param); + default -> unsubscribeMonkey(consumerThreads, entry.getValue()); + }) .toList(); } diff --git a/app/src/main/java/org/astraea/app/performance/ReportFormat.java b/app/src/main/java/org/astraea/app/performance/ReportFormat.java index 667ef0f000..d57ca80a2a 100644 --- a/app/src/main/java/org/astraea/app/performance/ReportFormat.java +++ b/app/src/main/java/org/astraea/app/performance/ReportFormat.java @@ -80,7 +80,7 @@ public static Runnable createFileWriter( + reportFormat); var writer = new BufferedWriter(Utils.packException(() -> new FileWriter(filePath.toFile()))); switch (reportFormat) { - case CSV: + case CSV -> { initCSVFormat(writer, latencyAndIO()); return () -> { try { @@ -92,7 +92,8 @@ public static Runnable createFileWriter( Utils.close(writer); } }; - case JSON: + } + case JSON -> { Utils.packException(() -> writer.write("{")); return () -> { try { @@ -104,8 +105,8 @@ public static Runnable createFileWriter( Utils.close(writer); } }; - default: - throw new IllegalArgumentException("Invalid format."); + } + default -> throw new IllegalArgumentException("Invalid format."); } } diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 6cfbc6718b..09447f5d53 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -147,28 +147,22 @@ private PlanExecutionProgress progress(String taskId) { var exception = (Function) (phase) -> { - switch (phase) { - case Searching: - case Searched: - case Executing: - case Executed: - // No error message during the search & execution - return null; - case SearchFailed: - return planGenerations - .get(taskId) - .handle((plan, err) -> err != null ? err.toString() : null) - .toCompletableFuture() - .getNow(null); - case ExecutionFailed: - return planExecutions - .get(taskId) - .handle((ignore, err) -> err != null ? err.toString() : null) - .toCompletableFuture() - .getNow(null); - default: - throw new IllegalStateException("Unknown state: " + phase); - } + return switch (phase) { + case Searching, Searched, Executing, Executed -> + // No error message during the search & execution + null; + case SearchFailed -> planGenerations + .get(taskId) + .handle((plan, err) -> err != null ? err.toString() : null) + .toCompletableFuture() + .getNow(null); + case ExecutionFailed -> planExecutions + .get(taskId) + .handle((ignore, err) -> err != null ? err.toString() : null) + .toCompletableFuture() + .getNow(null); + default -> throw new IllegalStateException("Unknown state: " + phase); + }; }; var changes = (Function>) diff --git a/app/src/main/java/org/astraea/app/web/Channel.java b/app/src/main/java/org/astraea/app/web/Channel.java index c0b071fd40..1f09a9620c 100644 --- a/app/src/main/java/org/astraea/app/web/Channel.java +++ b/app/src/main/java/org/astraea/app/web/Channel.java @@ -181,20 +181,14 @@ static Channel of(HttpExchange exchange) { }; Function parseType = - name -> { - switch (name.toUpperCase(Locale.ROOT)) { - case "GET": - return Type.GET; - case "POST": - return Type.POST; - case "DELETE": - return Type.DELETE; - case "PUT": - return Type.PUT; - default: - return Type.UNKNOWN; - } - }; + name -> + switch (name.toUpperCase(Locale.ROOT)) { + case "GET" -> Type.GET; + case "POST" -> Type.POST; + case "DELETE" -> Type.DELETE; + case "PUT" -> Type.PUT; + default -> Type.UNKNOWN; + }; // TODO: there is a temporary needed for reading the network stream twice // remove this hack in future diff --git a/app/src/main/java/org/astraea/app/web/Handler.java b/app/src/main/java/org/astraea/app/web/Handler.java index 7449d1e2a9..c275e08e2f 100644 --- a/app/src/main/java/org/astraea/app/web/Handler.java +++ b/app/src/main/java/org/astraea/app/web/Handler.java @@ -34,18 +34,13 @@ static Set compare(Set all, Optional target) { default CompletionStage process(Channel channel) { var start = System.currentTimeMillis(); try { - switch (channel.type()) { - case GET: - return get(channel); - case POST: - return post(channel); - case DELETE: - return delete(channel); - case PUT: - return put(channel); - default: - return CompletableFuture.completedFuture(Response.NOT_FOUND); - } + return switch (channel.type()) { + case GET -> get(channel); + case POST -> post(channel); + case DELETE -> delete(channel); + case PUT -> put(channel); + default -> CompletableFuture.completedFuture(Response.NOT_FOUND); + }; } catch (Exception e) { e.printStackTrace(); return CompletableFuture.completedFuture(Response.of(e)); diff --git a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java index 53725255ed..8fadee55a2 100644 --- a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java @@ -321,18 +321,12 @@ void testMoveCost(String leaderLimit, String sizeLimit) { report.migrationCosts.forEach( migrationCost -> { switch (migrationCost.name) { - case TO_SYNC_BYTES: - case TO_FETCH_BYTES: - Assertions.assertTrue( - migrationCost.brokerCosts.values().stream().mapToLong(Long::intValue).sum() - <= DataSize.of(sizeLimit).bytes()); - break; - case REPLICA_LEADERS_TO_ADDED: - case REPLICA_LEADERS_TO_REMOVE: - Assertions.assertTrue( - migrationCost.brokerCosts.values().stream().mapToLong(Long::intValue).sum() - <= Long.parseLong(leaderLimit)); - break; + case TO_SYNC_BYTES, TO_FETCH_BYTES -> Assertions.assertTrue( + migrationCost.brokerCosts.values().stream().mapToLong(Long::intValue).sum() + <= DataSize.of(sizeLimit).bytes()); + case REPLICA_LEADERS_TO_ADDED, REPLICA_LEADERS_TO_REMOVE -> Assertions.assertTrue( + migrationCost.brokerCosts.values().stream().mapToLong(Long::intValue).sum() + <= Long.parseLong(leaderLimit)); } }); } diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index ad30ffa53d..68a233d217 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -458,23 +458,15 @@ else if (v instanceof String value) /** Retrieve field from "one of" field. */ private static Object toObject(PrimitiveOuterClass.Primitive v) { var oneOfCase = v.getValueCase(); - switch (oneOfCase) { - case INT: - return v.getInt(); - case LONG: - return v.getLong(); - case FLOAT: - return v.getFloat(); - case DOUBLE: - return v.getDouble(); - case BOOLEAN: - return v.getBoolean(); - case STR: - return v.getStr(); - case VALUE_NOT_SET: - default: - throw new IllegalArgumentException("The value is not set."); - } + return switch (oneOfCase) { + case INT -> v.getInt(); + case LONG -> v.getLong(); + case FLOAT -> v.getFloat(); + case DOUBLE -> v.getDouble(); + case BOOLEAN -> v.getBoolean(); + case STR -> v.getStr(); + default -> throw new IllegalArgumentException("The value is not set."); + }; } // ------------------------------------ByteBuffer--------------------------------------------- // diff --git a/common/src/main/java/org/astraea/common/Utils.java b/common/src/main/java/org/astraea/common/Utils.java index 002fc3affd..ea4eddf6d2 100644 --- a/common/src/main/java/org/astraea/common/Utils.java +++ b/common/src/main/java/org/astraea/common/Utils.java @@ -97,24 +97,15 @@ public static Duration toDuration(String input) { if (matcher.find()) { long value = Long.parseLong(matcher.group("value")); String unit = matcher.group("unit"); - switch (unit) { - case "days": - case "day": - return Duration.ofDays(value); - case "h": - return Duration.ofHours(value); - case "m": - return Duration.ofMinutes(value); - case "ms": - return Duration.ofMillis(value); - case "us": - return Duration.ofNanos(value * 1000); - case "ns": - return Duration.ofNanos(value); - case "s": - default: - return Duration.ofSeconds(value); - } + return switch (unit) { + case "days", "day" -> Duration.ofDays(value); + case "h" -> Duration.ofHours(value); + case "m" -> Duration.ofMinutes(value); + case "ms" -> Duration.ofMillis(value); + case "us" -> Duration.ofNanos(value * 1000); + case "ns" -> Duration.ofNanos(value); + default -> Duration.ofSeconds(value); + }; } else { throw new IllegalArgumentException("value \"" + input + "\" doesn't match any time format"); } diff --git a/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java index 53924d3281..d6f154bafd 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java @@ -104,11 +104,8 @@ public RecordReaderBuilder buffered(int size) { public RecordReader build() { var version = ByteUtils.readShort(fs); - switch (version) { - case 0: - return V0.apply(fs); - default: - throw new IllegalArgumentException("unsupported version: " + version); - } + if (version == 0) return V0.apply(fs); + + throw new IllegalArgumentException("unsupported version: " + version); } } diff --git a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java index 6732a2db10..52ca446357 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java @@ -135,13 +135,11 @@ public RecordWriterBuilder buffered(int size) { public RecordWriter build() { return Utils.packException( () -> { - switch (version) { - case 0: - fs.write(ByteUtils.toBytes(version)); - return V0.apply(fs); - default: - throw new IllegalArgumentException("unsupported version: " + version); + if (version == 0) { + fs.write(ByteUtils.toBytes(version)); + return V0.apply(fs); } + throw new IllegalArgumentException("unsupported version: " + version); }); } } diff --git a/common/src/main/java/org/astraea/common/cost/NetworkCost.java b/common/src/main/java/org/astraea/common/cost/NetworkCost.java index 1619b954ee..c1b61a576f 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkCost.java @@ -235,19 +235,15 @@ Map estimateRate( .filter(bean -> bean.type().equals(metric)) .max(Comparator.comparingLong(HasBeanObject::createdTimestamp)) .map( - hasRate -> { - switch (estimationMethod) { - case BROKER_TOPIC_ONE_MINUTE_RATE: - return hasRate.oneMinuteRate(); - case BROKER_TOPIC_FIVE_MINUTE_RATE: - return hasRate.fiveMinuteRate(); - case BROKER_TOPIC_FIFTEEN_MINUTE_RATE: - return hasRate.fifteenMinuteRate(); - default: - throw new IllegalStateException( + hasRate -> + switch (estimationMethod) { + case BROKER_TOPIC_ONE_MINUTE_RATE -> hasRate.oneMinuteRate(); + case BROKER_TOPIC_FIVE_MINUTE_RATE -> hasRate.fiveMinuteRate(); + case BROKER_TOPIC_FIFTEEN_MINUTE_RATE -> hasRate + .fifteenMinuteRate(); + default -> throw new IllegalStateException( "Unknown estimation method: " + estimationMethod); - } - }) + }) // no load metric for this partition, treat as zero load .orElse(0.0); if (Double.isNaN(totalShare) || totalShare < 0) diff --git a/common/src/main/java/org/astraea/common/cost/NeutralIntegratedCost.java b/common/src/main/java/org/astraea/common/cost/NeutralIntegratedCost.java index 45117865e3..289c01e8fc 100644 --- a/common/src/main/java/org/astraea/common/cost/NeutralIntegratedCost.java +++ b/common/src/main/java/org/astraea/common/cost/NeutralIntegratedCost.java @@ -135,28 +135,21 @@ static Map weight( .collect( Collectors.toMap( m -> m.metricName, - m -> { - switch (m) { - case inputThroughput: - return brokerMetrics.values().stream() + m -> + switch (m) { + case inputThroughput -> brokerMetrics.values().stream() .map(metrics -> metrics.inputScore) - .collect(Collectors.toUnmodifiableList()); - case outputThroughput: - return brokerMetrics.values().stream() + .toList(); + case outputThroughput -> brokerMetrics.values().stream() .map(metrics -> metrics.outputScore) - .collect(Collectors.toUnmodifiableList()); - case memory: - return brokerMetrics.values().stream() + .toList(); + case memory -> brokerMetrics.values().stream() .map(metrics -> metrics.memoryScore) - .collect(Collectors.toUnmodifiableList()); - case cpu: - return brokerMetrics.values().stream() + .toList(); + case cpu -> brokerMetrics.values().stream() .map(metrics -> metrics.cpuScore) - .collect(Collectors.toUnmodifiableList()); - default: - return List.of(); - } - })); + .toList(); + })); return weightProvider.weight(values); } diff --git a/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java b/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java index 0e03a580d7..e27e4bec67 100644 --- a/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java +++ b/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java @@ -127,18 +127,16 @@ public synchronized void delete(String path) { if (path.equals("/")) throw new IllegalArgumentException("Can't delete whole root folder"); switch (type(path)) { - case NONEXISTENT: - return; - case FILE: - client.deleteFile(path); - return; - case FOLDER: + case NONEXISTENT -> {} + case FILE -> client.deleteFile(path); + case FOLDER -> { for (var f : client.listFiles(path)) { var sub = FileSystem.path(path, f.getName()); if (f.isDirectory()) delete(sub); else client.deleteFile(sub); } client.removeDirectory(path); + } } }); } diff --git a/gui/src/main/java/org/astraea/gui/Query.java b/gui/src/main/java/org/astraea/gui/Query.java index 4df524c1a4..d10a7b3041 100644 --- a/gui/src/main/java/org/astraea/gui/Query.java +++ b/gui/src/main/java/org/astraea/gui/Query.java @@ -85,16 +85,13 @@ public boolean required(Map item) { for (var index = 0; index != queries.size(); ++index) { if (match == null) match = queries.get(index).required(item); else { - switch (ops.get(index - 1)) { - case "&&": - match = match && queries.get(index).required(item); - break; - case "||": - match = match || queries.get(index).required(item); - break; - default: - throw new IllegalArgumentException("unsupported op: " + ops.get(index - 1)); - } + match = + switch (ops.get(index - 1)) { + case "&&" -> match && queries.get(index).required(item); + case "||" -> match || queries.get(index).required(item); + default -> throw new IllegalArgumentException( + "unsupported op: " + ops.get(index - 1)); + }; } } return match == null || match; diff --git a/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java b/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java index e9655be4aa..67dc8930fb 100644 --- a/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java @@ -180,20 +180,18 @@ private enum MetricType { .collect( Collectors.toMap( ServerMetrics.BrokerTopic.Meter::metricsName, - m -> { - switch (m.type()) { - case BYTES_IN_PER_SEC: - case BYTES_OUT_PER_SEC: - case BYTES_REJECTED_PER_SEC: - case REASSIGNMENT_BYTES_OUT_PER_SEC: - case REASSIGNMENT_BYTES_IN_PER_SEC: - case REPLICATION_BYTES_IN_PER_SEC: - case REPLICATION_BYTES_OUT_PER_SEC: - return DataSize.Byte.of((long) m.fiveMinuteRate()); - default: - return m.fiveMinuteRate(); - } - }))); + m -> + switch (m.type()) { + case BYTES_IN_PER_SEC, + BYTES_OUT_PER_SEC, + BYTES_REJECTED_PER_SEC, + REASSIGNMENT_BYTES_OUT_PER_SEC, + REASSIGNMENT_BYTES_IN_PER_SEC, + REPLICATION_BYTES_IN_PER_SEC, + REPLICATION_BYTES_OUT_PER_SEC -> DataSize.Byte.of( + (long) m.fiveMinuteRate()); + default -> m.fiveMinuteRate(); + }))); private final Function> fetcher; private final String display; From 713e0e1461d6da35749ecabe4a3aad0f6b10bcf5 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 29 May 2023 00:25:29 +0800 Subject: [PATCH 37/77] [COMMON] avoid calling URL#hashCode (#1783) --- .../java/org/astraea/common/connector/Builder.java | 6 +++--- .../common/connector/ConnectorClientBuilderTest.java | 3 ++- .../astraea/common/connector/ConnectorClientTest.java | 11 +++++------ 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/common/src/main/java/org/astraea/common/connector/Builder.java b/common/src/main/java/org/astraea/common/connector/Builder.java index c04d129d90..eb218c4ef2 100644 --- a/common/src/main/java/org/astraea/common/connector/Builder.java +++ b/common/src/main/java/org/astraea/common/connector/Builder.java @@ -19,7 +19,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.net.URISyntaxException; import java.net.URL; -import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -45,8 +45,8 @@ public class Builder { private List urls = List.of(); private HttpExecutor builderHttpExecutor; - public Builder urls(Set urls) { - this.urls = new ArrayList<>(Objects.requireNonNull(urls)); + public Builder urls(Collection urls) { + this.urls = List.copyOf(urls); return this; } diff --git a/common/src/test/java/org/astraea/common/connector/ConnectorClientBuilderTest.java b/common/src/test/java/org/astraea/common/connector/ConnectorClientBuilderTest.java index d2472a359a..0adfd1591f 100644 --- a/common/src/test/java/org/astraea/common/connector/ConnectorClientBuilderTest.java +++ b/common/src/test/java/org/astraea/common/connector/ConnectorClientBuilderTest.java @@ -18,6 +18,7 @@ import java.net.MalformedURLException; import java.net.URL; +import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -37,7 +38,7 @@ void testUrlShouldSet() throws MalformedURLException { ConnectorClient.builder().url(new URL("https://github.com/skiptests/astraea/")).build(); ConnectorClient.builder() - .urls(Set.of(new URL("https://github.com/skiptests/astraea/"))) + .urls(List.of(new URL("https://github.com/skiptests/astraea/"))) .build(); } diff --git a/common/src/test/java/org/astraea/common/connector/ConnectorClientTest.java b/common/src/test/java/org/astraea/common/connector/ConnectorClientTest.java index 4e8126a990..8dfda2ea80 100644 --- a/common/src/test/java/org/astraea/common/connector/ConnectorClientTest.java +++ b/common/src/test/java/org/astraea/common/connector/ConnectorClientTest.java @@ -31,6 +31,7 @@ import java.time.Duration; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletionException; @@ -107,7 +108,7 @@ void testConnectors() { @Test void testCreateConnector() { var connectorName = Utils.randomString(10); - var connectorClient = ConnectorClient.builder().urls(Set.copyOf(SERVICE.workerUrls())).build(); + var connectorClient = ConnectorClient.builder().urls(List.copyOf(SERVICE.workerUrls())).build(); var exampleConnector = new HashMap<>(getExampleConnector()); exampleConnector.put("tasks.max", "3"); @@ -213,7 +214,7 @@ void testPlugin() { @Test void testUrls() { var connectorName = Utils.randomString(10); - var connectorClient = ConnectorClient.builder().urls(Set.copyOf(SERVICE.workerUrls())).build(); + var connectorClient = ConnectorClient.builder().urls(List.copyOf(SERVICE.workerUrls())).build(); connectorClient .createConnector(connectorName, getExampleConnector()) .toCompletableFuture() @@ -239,15 +240,13 @@ void testUrls() { @Test void testUrlsRoundRobin() { var servers = - IntStream.range(0, 3) - .mapToObj(x -> new Server(200, "[\"connector" + x + "\"]")) - .collect(Collectors.toList()); + IntStream.range(0, 3).mapToObj(x -> new Server(200, "[\"connector" + x + "\"]")).toList(); try { var urls = servers.stream() .map(x -> "http://" + Utils.hostname() + ":" + x.port()) .map(x -> Utils.packException(() -> new URL(x))) - .collect(Collectors.toSet()); + .toList(); var connectorClient = ConnectorClient.builder().urls(urls).build(); From eaac0b4b1015dc0b88215711f0399d4bac591322 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 29 May 2023 08:55:02 +0800 Subject: [PATCH 38/77] [COMMON] tweak Admin (#1790) --- .../org/astraea/common/admin/AdminImpl.java | 30 ++++--- .../java/org/astraea/common/admin/Broker.java | 10 +-- .../common/cost/BrokerDiskSpaceCostTest.java | 79 ++++++++++--------- .../astraea/common/cost/CostUtilsTest.java | 52 ++++++------ 4 files changed, 87 insertions(+), 84 deletions(-) diff --git a/common/src/main/java/org/astraea/common/admin/AdminImpl.java b/common/src/main/java/org/astraea/common/admin/AdminImpl.java index 38b13322ff..5af795dfa9 100644 --- a/common/src/main/java/org/astraea/common/admin/AdminImpl.java +++ b/common/src/main/java/org/astraea/common/admin/AdminImpl.java @@ -129,12 +129,12 @@ public CompletionStage> topics(Set topics) { topics.stream() .map(topic -> new ConfigResource(ConfigResource.Type.TOPIC, topic)) .collect(Collectors.toList())), - to(kafkaAdmin.describeTopics(topics).all()), + to(kafkaAdmin.describeTopics(topics).allTopicNames()), (configs, desc) -> configs.entrySet().stream() .map(entry -> Topic.of(entry.getKey(), desc.get(entry.getKey()), entry.getValue())) .sorted(Comparator.comparing(Topic::name)) - .collect(Collectors.toUnmodifiableList())); + .toList()); } @Override @@ -225,7 +225,7 @@ public CompletionStage deleteGroups(Set consumerGroups) { @Override public CompletionStage> topicPartitions(Set topics) { if (topics.isEmpty()) return CompletableFuture.completedFuture(Set.of()); - return to(kafkaAdmin.describeTopics(topics).all()) + return to(kafkaAdmin.describeTopics(topics).allTopicNames()) .thenApply( r -> r.entrySet().stream() @@ -240,7 +240,7 @@ public CompletionStage> topicPartitions(Set topics) public CompletionStage> topicPartitionReplicas(Set brokers) { if (brokers.isEmpty()) return CompletableFuture.completedFuture(Set.of()); return topicNames(true) - .thenCompose(topics -> to(kafkaAdmin.describeTopics(topics).all())) + .thenCompose(topics -> to(kafkaAdmin.describeTopics(topics).allTopicNames())) .thenApply( r -> r.entrySet().stream() @@ -266,7 +266,7 @@ public CompletionStage> topicPartitionReplicas(Set> updatableTopicPartitions(Set topics) { if (topics.isEmpty()) return CompletableFuture.completedFuture(Set.of()); - return to(kafkaAdmin.describeTopics(topics).all()) + return to(kafkaAdmin.describeTopics(topics).allTopicNames()) .thenApply( ts -> ts.entrySet().stream() @@ -380,7 +380,7 @@ public CompletionStage> maxTimestamps( public CompletionStage> partitions(Set topics) { if (topics.isEmpty()) return CompletableFuture.completedFuture(List.of()); var updatableTopicPartitions = updatableTopicPartitions(topics); - var topicDesc = to(kafkaAdmin.describeTopics(topics).all()); + var topicDesc = to(kafkaAdmin.describeTopics(topics).allTopicNames()); return FutureUtils.combine( updatableTopicPartitions.thenCompose(this::earliestOffsets), updatableTopicPartitions.thenCompose(this::latestOffsets), @@ -455,13 +455,13 @@ private CompletionStage>> clusterIdAndBrokers() { return FutureUtils.combine( to(cluster.clusterId()), to(cluster.controller()), - topicNames(true).thenCompose(names -> to(kafkaAdmin.describeTopics(names).all())), + topicNames(true).thenCompose(names -> to(kafkaAdmin.describeTopics(names).allTopicNames())), nodeFuture.thenCompose( nodes -> to( kafkaAdmin .describeLogDirs(nodes.stream().map(Node::id).collect(Collectors.toList())) - .all())), + .allDescriptions())), nodeFuture.thenCompose( nodes -> doGetConfigs( @@ -512,7 +512,7 @@ public CompletionStage> consumerGroups(Set consumerG .partitionsToOffsetAndMetadata() .thenApply(of -> Map.entry(id, of))) .map(f -> to(f).toCompletableFuture()) - .collect(Collectors.toUnmodifiableList())) + .toList()) .thenApply( s -> s.stream().collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))), (consumerGroupDescriptions, consumerGroupMetadata) -> @@ -553,7 +553,7 @@ public CompletionStage> producerStates(Set p return to(kafkaAdmin .describeTopics( partitions.stream().map(TopicPartition::topic).collect(Collectors.toSet())) - .all()) + .allTopicNames()) .thenApply( ts -> partitions.stream() @@ -573,9 +573,7 @@ public CompletionStage> producerStates(Set p availablePartitions -> to(kafkaAdmin .describeProducers( - availablePartitions.stream() - .map(TopicPartition::to) - .collect(Collectors.toUnmodifiableList())) + availablePartitions.stream().map(TopicPartition::to).toList()) .all()) // supported version: 2.8.0 // https://issues.apache.org/jira/browse/KAFKA-12238 @@ -610,7 +608,7 @@ public CompletionStage> transactions(Set transactionId ts.entrySet().stream() .map(e -> Transaction.of(e.getKey(), e.getValue())) .sorted(Comparator.comparing(Transaction::transactionId)) - .collect(Collectors.toUnmodifiableList())); + .toList()); } @Override @@ -719,7 +717,7 @@ private CompletionStage> replicas(Set topics) { Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) .thenComparing(r -> r.broker().id())) - .collect(Collectors.toUnmodifiableList())); + .toList()); } @Override @@ -1237,7 +1235,7 @@ private CompletionStage doSubtractConfigs( e.getKey().name(), Map.of()) .getOrDefault(entry.getKey(), "") .split(",")) - .collect(Collectors.toList()); + .toList(); // disable to subtract from * if (values.contains("*")) throw new IllegalArgumentException( diff --git a/common/src/main/java/org/astraea/common/admin/Broker.java b/common/src/main/java/org/astraea/common/admin/Broker.java index 9e99102178..3d2b042159 100644 --- a/common/src/main/java/org/astraea/common/admin/Broker.java +++ b/common/src/main/java/org/astraea/common/admin/Broker.java @@ -21,7 +21,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.kafka.common.requests.DescribeLogDirsResponse; +import org.apache.kafka.clients.admin.LogDirDescription; /** * @param id @@ -58,11 +58,11 @@ public static Broker of(org.apache.kafka.common.Node node) { return of(node.id(), node.host(), node.port()); } - public static Broker of( + static Broker of( boolean isController, org.apache.kafka.common.Node nodeInfo, Map configs, - Map dirs, + Map dirs, Collection topics) { var config = new Config(configs); var partitionsFromTopicDesc = @@ -79,10 +79,10 @@ public static Broker of( entry -> { var path = entry.getKey(); var allPartitionAndSize = - entry.getValue().replicaInfos.entrySet().stream() + entry.getValue().replicaInfos().entrySet().stream() .collect( Collectors.toUnmodifiableMap( - e -> TopicPartition.from(e.getKey()), e -> e.getValue().size)); + e -> TopicPartition.from(e.getKey()), e -> e.getValue().size())); var partitionSizes = allPartitionAndSize.entrySet().stream() .filter(tpAndSize -> partitionsFromTopicDesc.contains(tpAndSize.getKey())) diff --git a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java index c91f089582..76515ddf58 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java @@ -21,13 +21,13 @@ import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.astraea.common.Configuration; import org.astraea.common.DataSize; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Config; import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; import org.junit.jupiter.api.Assertions; @@ -222,15 +222,13 @@ public static ClusterInfo of(List replicas) { var dataPath = Map.of( 0, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 1, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 2, - Map.of( - "/path0", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), - "/path1", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + List.of( + new Broker.DataFolder("/path0", Map.of(), Map.of()), + new Broker.DataFolder("/path1", Map.of(), Map.of()))); return ClusterInfo.of( "fake", replicas.stream() @@ -238,12 +236,15 @@ public static ClusterInfo of(List replicas) { .distinct() .map( broker -> - Broker.of( + new Broker( + broker.id(), + "", + broker.port(), false, - new Node(broker.id(), "", broker.port()), - Map.of(), + Config.EMPTY, dataPath.get(broker.id()), - List.of())) + Set.of(), + Set.of())) .collect(Collectors.toList()), Map.of(), replicas); @@ -264,18 +265,15 @@ public static ClusterInfo of(List replicas) { p2 1000 */ private static ClusterInfo beforeClusterInfo() { + var dataPath = Map.of( - 0, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), - 1, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 0, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), + 1, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 2, - Map.of( - "/path0", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), - "/path1", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + List.of( + new Broker.DataFolder("/path0", Map.of(), Map.of()), + new Broker.DataFolder("/path1", Map.of(), Map.of()))); var replicas = List.of( Replica.builder() @@ -333,30 +331,30 @@ private static ClusterInfo beforeClusterInfo() { .distinct() .map( broker -> - Broker.of( + new Broker( + broker.id(), + "", + broker.port(), false, - new Node(broker.id(), "", broker.port()), - Map.of(), + Config.EMPTY, dataPath.get(broker.id()), - List.of())) + Set.of(), + Set.of())) .collect(Collectors.toList()), Map.of(), replicas); } private static ClusterInfo afterClusterInfo() { + var dataPath = Map.of( - 0, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), - 1, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + 0, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), + 1, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 2, - Map.of( - "/path0", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), - "/path1", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + List.of( + new Broker.DataFolder("/path0", Map.of(), Map.of()), + new Broker.DataFolder("/path1", Map.of(), Map.of()))); var replicas = List.of( Replica.builder() @@ -414,12 +412,15 @@ private static ClusterInfo afterClusterInfo() { .distinct() .map( broker -> - Broker.of( + new Broker( + broker.id(), + "", + broker.port(), false, - new Node(broker.id(), "", broker.port()), - Map.of(), + Config.EMPTY, dataPath.get(broker.id()), - List.of())) + Set.of(), + Set.of())) .collect(Collectors.toList()), Map.of(), replicas); diff --git a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java index db1817e725..6cac4f497d 100644 --- a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java +++ b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java @@ -22,11 +22,11 @@ import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.requests.DescribeLogDirsResponse; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Config; import org.astraea.common.admin.Replica; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -72,18 +72,17 @@ void testChangedRecordSizeOverflow() { p2 1000 */ private static ClusterInfo beforeClusterInfo() { + var dataPath = Map.of( 0, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 1, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 2, - Map.of( - "/path0", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), - "/path1", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + List.of( + new Broker.DataFolder("/path0", Map.of(), Map.of()), + new Broker.DataFolder("/path1", Map.of(), Map.of()))); var replicas = List.of( Replica.builder() @@ -141,30 +140,32 @@ private static ClusterInfo beforeClusterInfo() { .distinct() .map( broker -> - Broker.of( + new Broker( + broker.id(), + "", + broker.port(), false, - new Node(broker.id(), "", broker.port()), - Map.of(), + Config.EMPTY, dataPath.get(broker.id()), - List.of())) + Set.of(), + Set.of())) .collect(Collectors.toList()), Map.of(), replicas); } private static ClusterInfo afterClusterInfo() { + var dataPath = Map.of( 0, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 1, - Map.of("/path0", new DescribeLogDirsResponse.LogDirInfo(null, Map.of())), + List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), 2, - Map.of( - "/path0", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()), - "/path1", - new DescribeLogDirsResponse.LogDirInfo(null, Map.of()))); + List.of( + new Broker.DataFolder("/path0", Map.of(), Map.of()), + new Broker.DataFolder("/path1", Map.of(), Map.of()))); var replicas = List.of( Replica.builder() @@ -222,12 +223,15 @@ private static ClusterInfo afterClusterInfo() { .distinct() .map( broker -> - Broker.of( + new Broker( + broker.id(), + "", + broker.port(), false, - new Node(broker.id(), "", broker.port()), - Map.of(), + Config.EMPTY, dataPath.get(broker.id()), - List.of())) + Set.of(), + Set.of())) .collect(Collectors.toList()), Map.of(), replicas); From 212e5d496cdf997ed2ea04a67e74f09896f4052f Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Mon, 29 May 2023 15:15:36 +0800 Subject: [PATCH 39/77] [TEST] Fix flaky BackupTest#testRestoreDistribution() (#1785) --- .../org/astraea/app/backup/BackupTest.java | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/app/src/test/java/org/astraea/app/backup/BackupTest.java b/app/src/test/java/org/astraea/app/backup/BackupTest.java index ea7df4906a..9f6291ea09 100644 --- a/app/src/test/java/org/astraea/app/backup/BackupTest.java +++ b/app/src/test/java/org/astraea/app/backup/BackupTest.java @@ -18,8 +18,10 @@ import java.time.Duration; import java.util.Set; +import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; +import org.astraea.common.admin.Replica; import org.astraea.it.Service; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; @@ -41,16 +43,16 @@ void testRestoreDistribution() { admin .creator() .topic(topic1) - .numberOfPartitions(3) - .numberOfReplicas((short) 3) + .numberOfPartitions(2) + .numberOfReplicas((short) 2) .run() .toCompletableFuture() .join(); admin .creator() .topic(topic2) - .numberOfPartitions(3) - .numberOfReplicas((short) 3) + .numberOfPartitions(2) + .numberOfReplicas((short) 2) .run() .toCompletableFuture() .join(); @@ -67,7 +69,19 @@ void testRestoreDistribution() { admin.clusterInfo(Set.of(topic1, topic2)).toCompletableFuture().join(); Utils.sleep(Duration.ofSeconds(2)); - Assertions.assertEquals(clusterInfo.replicas(), restoredClusterInfo.replicas()); + // Comparing with partial information between ClusterInfos. We do this because in KRaft world, + // Kafka chooses a random broker node to report as the controller, resulting in different + // Replica.broker.isController values. + Assertions.assertEquals(clusterInfo.topics(), restoredClusterInfo.topics()); + Assertions.assertEquals( + clusterInfo.topicPartitionReplicas(), restoredClusterInfo.topicPartitionReplicas()); + Assertions.assertEquals( + clusterInfo.replicaLeaders().stream() + .map(Replica::topicPartitionReplica) + .collect(Collectors.toSet()), + restoredClusterInfo.replicaLeaders().stream() + .map(Replica::topicPartitionReplica) + .collect(Collectors.toSet())); } } } From b1c95667a8b0fa0ff8bd24900f3d67715420d3b3 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Mon, 29 May 2023 23:32:51 +0800 Subject: [PATCH 40/77] [BALANCER] Rename balancing mode (#1786) --- .../common/balancer/BalancerConfigs.java | 26 ++++++++-------- .../common/balancer/BalancerUtils.java | 10 +++--- .../balancer/algorithms/GreedyBalancer.java | 22 ++++++------- .../algorithms/SingleStepBalancer.java | 22 ++++++------- .../balancer/BalancerConfigTestSuite.java | 31 +++++++++---------- .../common/balancer/BalancerUtilsTest.java | 22 ++++++------- docs/web_server/web_api_balancer_chinese.md | 8 ++--- 7 files changed, 69 insertions(+), 72 deletions(-) diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java index a93e84f27c..44c6ceb165 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConfigs.java @@ -48,14 +48,14 @@ private BalancerConfigs() {} *
    *
  • balancing: The broker will participate in the load balancing process. The * replica assignment for this broker is eligible for changes. - *
  • demoted: The broker should become empty after the rebalance. This mode - * allows the user to clear all the loadings for certain brokers, enabling a graceful - * removal of those brokers. Note to the balancer implementation: A broker in this mode - * assumes it will be out of service after the balancing is finished. Therefore, when - * evaluating the cluster cost, the brokers to demote should be excluded. However, these - * brokers will be included in the move cost evaluation. Since these brokers are still part - * of the cluster right now, and move cost focusing on the cost associated during the - * ongoing balancing process itself. + *
  • clear: The broker should become empty after the rebalance. This mode allows + * the user to clear all the loadings for certain brokers, enabling a graceful removal of + * those brokers. Note to the balancer implementation: A broker in this mode assumes it will + * be out of service after the balancing is finished. Therefore, when evaluating the cluster + * cost, the brokers to clear should be excluded. However, these brokers will be included in + * the move cost evaluation. Since these brokers are still part of the cluster right now, + * and move cost focusing on the cost associated during the ongoing balancing process + * itself. *
  • excluded: The broker will not participate in the load balancing process. The * replica assignment for this broker is not eligible for changes. It will neither accept * replicas from other brokers nor reassign replicas to other brokers. @@ -64,23 +64,23 @@ private BalancerConfigs() {} *

    Flag Interaction:

    * *
      - *
    1. All partitions on the demoting brokers will be compelled to participate in the balancing + *
    2. All partitions on the clearing brokers will be compelled to participate in the balancing * process, regardless of the explicit prohibition specified by the {@link * BalancerConfigs#BALANCER_ALLOWED_TOPICS_REGEX} configuration. This exception solely - * applies to partitions located at a demoting broker, while disallowed partitions on + * applies to partitions located at a clearing broker, while disallowed partitions on * balancing brokers will remain excluded from the balancing decision. *
    * *

    Limitation:

    * *
      - *
    1. Demoting a broker may be infeasible if there are not enough brokers to fit the required + *
    2. Clearing a broker may be infeasible if there are not enough brokers to fit the required * replica factor for a specific partition. This situation is more likely to occur if there * are many excluded brokers that reject accepting new replicas. If such a case * is detected, an exception should be raised. *
    3. Any broker with ongoing replica-move-in, replica-move-out, or inter-folder movement - * cannot be the demoting target. An exception will be raised if any of the demoting brokers - * have such ongoing events. * + * cannot be the clearing target. An exception will be raised if any of the clearing brokers + * have such ongoing events. *
    */ public static final String BALANCER_BROKER_BALANCING_MODE = "balancer.broker.balancing.mode"; diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java index 1f8ad18156..a673085b40 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java @@ -49,7 +49,7 @@ public static Map balancingMode(ClusterInfo cluster, St s -> switch (s[1]) { case "balancing" -> BalancingModes.BALANCING; - case "demoted" -> BalancingModes.DEMOTED; + case "clear" -> BalancingModes.CLEAR; case "excluded" -> BalancingModes.EXCLUDED; default -> throw new IllegalArgumentException( "Unsupported balancing mode: " + s[1]); @@ -64,10 +64,10 @@ public static Map balancingMode(ClusterInfo cluster, St } /** Performs common validness checks to the cluster. */ - public static void verifyClearBrokerValidness(ClusterInfo cluster, Predicate isDemoted) { + public static void verifyClearBrokerValidness(ClusterInfo cluster, Predicate isClear) { var ongoingEventReplica = cluster.replicas().stream() - .filter(r -> isDemoted.test(r.broker().id())) + .filter(r -> isClear.test(r.broker().id())) .filter(r -> r.isAdding() || r.isRemoving() || r.isFuture()) .map(Replica::topicPartitionReplica) .collect(Collectors.toUnmodifiableSet()); @@ -78,7 +78,7 @@ public static void verifyClearBrokerValidness(ClusterInfo cluster, PredicateBE CAREFUL, The + * Move all the replicas at the clearing broker to other allowed brokers. BE CAREFUL, The * implementation made no assumption for MoveCost or ClusterCost of the returned ClusterInfo. * Be aware of this limitation before using it as the starting point for a solution search. Some * balancer implementation might have trouble finding answer when starting at a state where the @@ -159,7 +159,7 @@ public static void balancerConfigCheck(Configuration configs, Set suppor public enum BalancingModes implements EnumInfo { BALANCING, - DEMOTED, + CLEAR, EXCLUDED; public static BalancingModes ofAlias(String alias) { diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index 26b628f95a..b185c89c6b 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -158,23 +158,23 @@ public Optional offer(AlgorithmConfig config) { .orElse("")); final Predicate isBalancing = id -> balancingMode.get(id) == BalancerUtils.BalancingModes.BALANCING; - final Predicate isDemoted = - id -> balancingMode.get(id) == BalancerUtils.BalancingModes.DEMOTED; - final var hasDemoted = - balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.DEMOTED); - BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted); + final Predicate isClearing = + id -> balancingMode.get(id) == BalancerUtils.BalancingModes.CLEAR; + final var clearing = + balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.CLEAR); + BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isClearing); final var currentClusterInfo = - BalancerUtils.clearedCluster(config.clusterInfo(), isDemoted, isBalancing); + BalancerUtils.clearedCluster(config.clusterInfo(), isClearing, isBalancing); final var clusterBean = config.clusterBean(); final var fixedReplicas = config .clusterInfo() .replicaStream() // if a topic is not allowed to move, it should be fixed. - // if a topic is not allowed to move, but originally it located on a demoting broker, it + // if a topic is not allowed to move, but originally it located on a clearing broker, it // is ok to move. - .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isDemoted.test(tpr.broker().id())) + .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isClearing.test(tpr.broker().id())) .collect(Collectors.toUnmodifiableSet()); final var allocationTweaker = ShuffleTweaker.builder() @@ -186,7 +186,7 @@ public Optional offer(AlgorithmConfig config) { final Function evaluateCost = (cluster) -> { final var filteredCluster = - hasDemoted ? ClusterInfo.builder(cluster).removeNodes(isDemoted).build() : cluster; + clearing ? ClusterInfo.builder(cluster).removeNodes(isClearing).build() : cluster; return config.clusterCostFunction().clusterCost(filteredCluster, clusterBean); }; final var initialCost = evaluateCost.apply(currentClusterInfo); @@ -243,11 +243,11 @@ public Optional offer(AlgorithmConfig config) { } return currentSolution.or( () -> { - // With demotion, the implementation detail start search from a demoted state. It is + // With clearing, the implementation detail start search from a cleared state. It is // possible // that the start state is already the ideal answer. In this case, it is directly // returned. - if (hasDemoted + if (clearing && initialCost.value() == 0.0 && !moveCostFunction .moveCost(config.clusterInfo(), currentClusterInfo, clusterBean) diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java index 03b8cbfc29..850f6f4875 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java @@ -88,23 +88,23 @@ public Optional offer(AlgorithmConfig config) { .orElse("")); final Predicate isBalancing = id -> balancingMode.get(id) == BalancerUtils.BalancingModes.BALANCING; - final Predicate isDemoted = - id -> balancingMode.get(id) == BalancerUtils.BalancingModes.DEMOTED; - final var hasDemoted = - balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.DEMOTED); - BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isDemoted); + final Predicate isClearing = + id -> balancingMode.get(id) == BalancerUtils.BalancingModes.CLEAR; + final var clearing = + balancingMode.values().stream().anyMatch(i -> i == BalancerUtils.BalancingModes.CLEAR); + BalancerUtils.verifyClearBrokerValidness(config.clusterInfo(), isClearing); final var currentClusterInfo = - BalancerUtils.clearedCluster(config.clusterInfo(), isDemoted, isBalancing); + BalancerUtils.clearedCluster(config.clusterInfo(), isClearing, isBalancing); final var clusterBean = config.clusterBean(); final var fixedReplicas = config .clusterInfo() .replicaStream() // if a topic is not allowed to move, it should be fixed. - // if a topic is not allowed to move, but originally it located on a demoting broker, it + // if a topic is not allowed to move, but originally it located on a clearing broker, it // is ok to move. - .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isDemoted.test(tpr.broker().id())) + .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isClearing.test(tpr.broker().id())) .collect(Collectors.toUnmodifiableSet()); final var allocationTweaker = ShuffleTweaker.builder() @@ -117,7 +117,7 @@ public Optional offer(AlgorithmConfig config) { final Function evaluateCost = (cluster) -> { final var filteredCluster = - hasDemoted ? ClusterInfo.builder(cluster).removeNodes(isDemoted).build() : cluster; + clearing ? ClusterInfo.builder(cluster).removeNodes(isClearing).build() : cluster; return config.clusterCostFunction().clusterCost(filteredCluster, clusterBean); }; final var currentCost = evaluateCost.apply(currentClusterInfo); @@ -144,11 +144,11 @@ public Optional offer(AlgorithmConfig config) { .min(Comparator.comparing(plan -> plan.proposalClusterCost().value())) .or( () -> { - // With demotion, the implementation detail start search from a demoted state. It is + // With clearing, the implementation detail start search from a cleared state. It is // possible // that the start state is already the ideal answer. In this case, it is directly // returned. - if (hasDemoted + if (clearing && currentCost.value() == 0.0 && !moveCostFunction .moveCost(config.clusterInfo(), currentClusterInfo, clusterBean) diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index 047162a648..603fbe38af 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -208,7 +208,7 @@ public void testBalancingMode() { } @Test - public void testBalancingModeDemoted() { + public void testBalancingModeClear() { final var balancer = Utils.construct(balancerClass, Configuration.EMPTY); final var cluster = cluster(10, 30, 10, (short) 5); @@ -223,7 +223,7 @@ public void testBalancingModeDemoted() { .clusterCost(decreasingCost()) .timeout(Duration.ofSeconds(2)) .configs(customConfig.raw()) - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "default:demoted") + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "default:clear") .build()), testName); } @@ -237,9 +237,7 @@ public void testBalancingModeDemoted() { .clusterCost(decreasingCost()) .timeout(Duration.ofSeconds(2)) .configs(customConfig.raw()) - .config( - BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "0:demoted,1:demoted,2:demoted") + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "0:clear,1:clear,2:clear") .build()); Assertions.assertTrue(plan.isPresent(), testName); var finalCluster = plan.get().proposal(); @@ -270,7 +268,7 @@ public void testBalancingModeDemoted() { .configs(customConfig.raw()) .config( BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "0:demoted,1:demoted,2:demoted") + "0:clear,1:clear,2:clear") .build()) .orElseThrow() .proposal(); @@ -292,14 +290,13 @@ public void testBalancingModeDemoted() { .timeout(Duration.ofSeconds(2)) .configs(customConfig.raw()) .config( - BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "0:demoted,1:demoted,2:demoted") + BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "0:clear,1:clear,2:clear") .build())); } { var testName = - "[test if allowed topics is used, disallowed partitions on demoted broker will be force to move]"; + "[test if allowed topics is used, disallowed partitions on cleared broker will be force to move]"; var base = ClusterInfo.builder() .addNode(Set.of(1, 2, 3)) @@ -330,7 +327,7 @@ public void testBalancingModeDemoted() { // allow anything other than this topic .config(BalancerConfigs.BALANCER_ALLOWED_TOPICS_REGEX, "(?!topic).*") // clear broker 3 - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:demoted") + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "3:clear") // partition at broker 3 will be forced to move .build()), testName); @@ -370,7 +367,7 @@ public void testBalancingModeDemoted() { // clear broker 0 .config( BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "0:demoted," + "0:clear," + // allow broker 1,2,3,4,5,6 "1:balancing,2:balancing,3:balancing,4:balancing,5:balancing,6:balancing,default:excluded") @@ -406,7 +403,7 @@ public void testBalancingModeDemoted() { // clear broker 0, allow broker 1 .config( BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "0:demoted,1:balancing,default:excluded") + "0:clear,1:balancing,default:excluded") // this will raise an error if a partition has replicas at both 0 and 1. In // this case, there is no allowed broker to adopt replica from 0, since the // only allowed broker already has one replica on it. we cannot assign two @@ -442,7 +439,7 @@ public void testBalancingModeDemoted() { // clear broker 0 allow broker 1,2,3,4,5,6 .config( BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "0:demoted," + "0:clear," + "1:balancing,2:balancing,3:balancing,4:balancing,5:balancing,6:balancing") .build()), testName); @@ -459,7 +456,7 @@ public void testBalancingModeDemoted() { // clear broker 1 allow broker 0,2,3,4,5,6,7 .config( BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, - "1:demoted," + "1:clear," + "0:balancing,2:balancing,3:balancing,4:balancing,5:balancing,6:balancing," + "7:balancing,default:excluded") // adding/removing/future at 0 not 1, unrelated so no error @@ -471,14 +468,14 @@ public void testBalancingModeDemoted() { { // Some balancer implementations have such logic flaw: // 1. The initial state[A] cannot be solution. - // 2. There are brokers that need to be demoted. + // 2. There are brokers that need to be cleared. // 3. The load on those brokers been redistributed to other brokers. Creating the start // state[B] for the solution search. // 4. The start state[B] solution is actually the best solution. // 5. Balancer think the start state[B] is the initial state[A]. And cannot be a solution(as // mentioned in 1). // 6. In fact, the start state[B] doesn't equal to the initial state[A]. Since there is a - // cleaning work performed at step 3. + // clearing work performed at step 3. // 7. Balancer cannot find any solution that is better than the start state(4) and therefore // returns no solution. var testName = @@ -497,7 +494,7 @@ public void testBalancingModeDemoted() { .clusterInfo(testCluster) .clusterBean(ClusterBean.EMPTY) .clusterCost(new ReplicaLeaderCost()) - .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "1:demoted") + .config(BalancerConfigs.BALANCER_BROKER_BALANCING_MODE, "1:clear") .timeout(Duration.ofSeconds(2)) .build()), testName); diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java index b18804d433..c47b76b8a2 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java @@ -42,19 +42,19 @@ void testBalancingMode() { Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:balancing,bad:bad:bad")); Assertions.assertThrows( Exception.class, - () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,4:oops")); + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:clear,3:excluded,4:oops")); Assertions.assertThrows( Exception.class, - () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,4:")); + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:clear,3:excluded,4:")); Assertions.assertThrows( Exception.class, - () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,1:")); + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:clear,3:excluded,1:")); Assertions.assertThrows( Exception.class, - () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,:")); + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:clear,3:excluded,:")); Assertions.assertThrows( Exception.class, - () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:demoted,3:excluded,::")); + () -> BalancerUtils.balancingMode(cluster, "1:balancing,2:clear,3:excluded,::")); Assertions.assertThrows(Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:")); Assertions.assertThrows( Exception.class, () -> BalancerUtils.balancingMode(cluster, "1:balancing,1:balancing")); @@ -68,12 +68,12 @@ void testBalancingMode() { BalancerUtils.balancingMode(cluster, "").get(1), "default"); Assertions.assertEquals( - BalancerUtils.BalancingModes.DEMOTED, - BalancerUtils.balancingMode(cluster, "1:demoted").get(1), + BalancerUtils.BalancingModes.CLEAR, + BalancerUtils.balancingMode(cluster, "1:clear").get(1), "value"); Assertions.assertEquals( - BalancerUtils.BalancingModes.DEMOTED, - BalancerUtils.balancingMode(cluster, "default:demoted").get(5), + BalancerUtils.BalancingModes.CLEAR, + BalancerUtils.balancingMode(cluster, "default:clear").get(5), "user defined default"); Assertions.assertEquals( BalancerUtils.BalancingModes.EXCLUDED, @@ -175,7 +175,7 @@ void testClearedCluster() { Assertions.assertDoesNotThrow( () -> BalancerUtils.clearedCluster(cluster, id -> id == 1, id -> id == 3)); Assertions.assertEquals( - 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 1).count(), "Demoted"); + 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 1).count(), "Clear"); Assertions.assertEquals( 100, aCluster.replicas().stream().filter(r -> r.broker().id() == 2).count(), @@ -183,7 +183,7 @@ void testClearedCluster() { Assertions.assertEquals( 100, aCluster.replicas().stream().filter(r -> r.broker().id() == 3).count(), - "Accept replicas broker demoted broker"); + "Accept replicas from cleared broker"); Assertions.assertEquals( 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 4).count(), "Not allowed"); } diff --git a/docs/web_server/web_api_balancer_chinese.md b/docs/web_server/web_api_balancer_chinese.md index 41e5e8a366..432da98212 100644 --- a/docs/web_server/web_api_balancer_chinese.md +++ b/docs/web_server/web_api_balancer_chinese.md @@ -26,10 +26,10 @@ POST /balancer `balancerConfig` 是 balancer 實作開放給使用者設定的內部演算法行為參數,我們有針對常用情境的 balancer config 規範出一些固定的名稱, 參數是否支援要看 Balancer 實作本身。當指定的參數不被 balancer 實作支援時,該實作可能會丟出錯誤提示使用者。 -| config key | config value | -|--------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| balancer.allowed.topics.regex | 一個正則表達式,表達允許進行搬移操作的 topic 名稱,當沒有指定的時候,代表沒有任何限制,所有 topic 都可以做搬移,包含 internal topics,表格下面有附上一些常見的設定範例,可以當做參考。 | -| balancer.broker.balancing.mode | 這個參數指定每個 Broker 要採取的負載平衡設定,目前擁有的模式包含 `balancing`, `demoted` 和 `excluded`。`balancing` 代表特定節點要參予負載平衡的過程,該節點身上的負載可以被變動。`demoted` 代表特定節點身上的負載必須要全部移除,這個功能能夠協助使用者優雅地下線一個節點。`excluded` 代表特定節點不能夠參予負載平衡的過程,節點不能新增或移除負載。這個參數的格式是一系列的 key/value pair 的字串,每個 pair 之間透過逗號 "," 間隔,而 key/value 之間透過冒號 ":" 間隔,如 `(brokerId_A |"default"):(mode),(brokerId_B):(mode), ...`,其中 `key` 欄位代表這個是描述某 id 節點的設定,而對應的 `value` 欄位則是該節點要套用的負載平衡模式(`balancing`, `demoted` 或 `excluded`),另外 `key` 欄位可以填寫特殊字串 `default`,代表沒有被設定所提及的節點應該使用的負載平衡模式,比如填寫 `default:excluded` 可以使設定沒有提到的節點不參予負載平衡的過程,預設的 `default` 模式是 `balancing`,意即所有設定沒有提到的節點都將參予負載平衡的過程。 | +| config key | config value | +|--------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| balancer.allowed.topics.regex | 一個正則表達式,表達允許進行搬移操作的 topic 名稱,當沒有指定的時候,代表沒有任何限制,所有 topic 都可以做搬移,包含 internal topics,表格下面有附上一些常見的設定範例,可以當做參考。 | +| balancer.broker.balancing.mode | 這個參數指定每個 Broker 要採取的負載平衡設定,目前擁有的模式包含 `balancing`, `clear` 和 `excluded`。`balancing` 代表特定節點要參予負載平衡的過程,該節點身上的負載可以被變動。`clear` 代表特定節點身上的負載必須要全部移除,這個功能能夠協助使用者優雅地下線一個節點。`excluded` 代表特定節點不能夠參予負載平衡的過程,節點不能新增或移除負載。這個參數的格式是一系列的 key/value pair 的字串,每個 pair 之間透過逗號 "," 間隔,而 key/value 之間透過冒號 ":" 間隔,如 `(brokerId_A |"default"):(mode),(brokerId_B):(mode), ...`,其中 `key` 欄位代表這個是描述某 id 節點的設定,而對應的 `value` 欄位則是該節點要套用的負載平衡模式(`balancing`, `clear` 或 `excluded`),另外 `key` 欄位可以填寫特殊字串 `default`,代表沒有被設定所提及的節點應該使用的負載平衡模式,比如填寫 `default:excluded` 可以使設定沒有提到的節點不參予負載平衡的過程,預設的 `default` 模式是 `balancing`,意即所有設定沒有提到的節點都將參予負載平衡的過程。 | 一些可以參考的 `balancer.allowed.topics.regex` 設定: From c45312948da0bd5e059ed76da535cbb5f7926045 Mon Sep 17 00:00:00 2001 From: Xiang-Jun Sun Date: Tue, 30 May 2023 00:17:10 +0800 Subject: [PATCH 41/77] [COST] add ClusterBean to MigrationCost and revise Balancer#Plan --- .../org/astraea/app/web/BalancerHandler.java | 3 +- .../org/astraea/common/balancer/Balancer.java | 34 +++++-------------- .../balancer/algorithms/GreedyBalancer.java | 2 ++ .../algorithms/SingleStepBalancer.java | 2 ++ .../astraea/common/cost/MigrationCost.java | 4 ++- .../common/balancer/BalancerConsoleTest.java | 7 +++- .../astraea/gui/tab/health/BalancerNode.java | 4 ++- .../gui/tab/health/BalancerNodeTest.java | 2 ++ 8 files changed, 29 insertions(+), 29 deletions(-) diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 09447f5d53..9ee87413b7 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -186,7 +186,8 @@ private PlanExecutionProgress progress(String taskId) { solution -> new PlanReport( changes.apply(solution), - MigrationCost.migrationCosts(contextCluster, solution.proposal()))) + MigrationCost.migrationCosts( + contextCluster, solution.proposal(), solution.clusterBean()))) .orElse(null); var phase = balancerConsole.taskPhase(taskId).orElseThrow(); return new PlanExecutionProgress( diff --git a/common/src/main/java/org/astraea/common/balancer/Balancer.java b/common/src/main/java/org/astraea/common/balancer/Balancer.java index 5854f1da36..3118c76dad 100644 --- a/common/src/main/java/org/astraea/common/balancer/Balancer.java +++ b/common/src/main/java/org/astraea/common/balancer/Balancer.java @@ -24,6 +24,7 @@ import org.astraea.common.balancer.algorithms.GreedyBalancer; import org.astraea.common.balancer.algorithms.SingleStepBalancer; import org.astraea.common.cost.ClusterCost; +import org.astraea.common.metrics.ClusterBean; public interface Balancer { @@ -32,41 +33,24 @@ public interface Balancer { */ Optional offer(AlgorithmConfig config); - class Plan { - private final ClusterInfo initialClusterInfo; - private final ClusterCost initialClusterCost; - - private final ClusterInfo proposal; - private final ClusterCost proposalClusterCost; - - public Plan( - ClusterInfo initialClusterInfo, - ClusterCost initialClusterCost, - ClusterInfo proposal, - ClusterCost proposalClusterCost) { - this.initialClusterInfo = initialClusterInfo; - this.initialClusterCost = initialClusterCost; - this.proposal = proposal; - this.proposalClusterCost = proposalClusterCost; - } - - public ClusterInfo initialClusterInfo() { - return initialClusterInfo; - } + record Plan( + ClusterBean clusterBean, + ClusterInfo initialClusterInfo, + ClusterCost initialClusterCost, + ClusterInfo proposal, + ClusterCost proposalClusterCost) { /** * The {@link ClusterCost} score of the original {@link ClusterInfo} when this plan is start * generating. */ + @Override public ClusterCost initialClusterCost() { return initialClusterCost; } - public ClusterInfo proposal() { - return proposal; - } - /** The {@link ClusterCost} score of the proposed new allocation. */ + @Override public ClusterCost proposalClusterCost() { return proposalClusterCost; } diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index b185c89c6b..f9987385f5 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -209,6 +209,7 @@ public Optional offer(AlgorithmConfig config) { .map( newAllocation -> new Plan( + config.clusterBean(), config.clusterInfo(), initialCost, newAllocation, @@ -254,6 +255,7 @@ public Optional offer(AlgorithmConfig config) { .overflow()) { return Optional.of( new Plan( + config.clusterBean(), config.clusterInfo(), config.clusterCostFunction().clusterCost(config.clusterInfo(), clusterBean), currentClusterInfo, diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java index 850f6f4875..f92259c226 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java @@ -136,6 +136,7 @@ public Optional offer(AlgorithmConfig config) { .map( newAllocation -> new Plan( + config.clusterBean(), config.clusterInfo(), currentCost, newAllocation, @@ -155,6 +156,7 @@ public Optional offer(AlgorithmConfig config) { .overflow()) { return Optional.of( new Plan( + config.clusterBean(), config.clusterInfo(), config.clusterCostFunction().clusterCost(config.clusterInfo(), clusterBean), currentClusterInfo, diff --git a/common/src/main/java/org/astraea/common/cost/MigrationCost.java b/common/src/main/java/org/astraea/common/cost/MigrationCost.java index ef673b40be..c88c1d2488 100644 --- a/common/src/main/java/org/astraea/common/cost/MigrationCost.java +++ b/common/src/main/java/org/astraea/common/cost/MigrationCost.java @@ -25,6 +25,7 @@ import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; +import org.astraea.common.metrics.ClusterBean; public class MigrationCost { @@ -37,7 +38,8 @@ public class MigrationCost { public static final String REPLICA_LEADERS_TO_REMOVE = "leader number to remove"; public static final String CHANGED_REPLICAS = "changed replicas"; - public static List migrationCosts(ClusterInfo before, ClusterInfo after) { + public static List migrationCosts( + ClusterInfo before, ClusterInfo after, ClusterBean clusterBean) { var migrateInBytes = recordSizeToSync(before, after); var migrateOutBytes = recordSizeToFetch(before, after); var migrateReplicaNum = replicaNumChanged(before, after); diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java index 78b25a664e..67216abb30 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java @@ -400,7 +400,12 @@ public Optional offer(AlgorithmConfig config) { costFunction, Duration.ofMillis(sampleTimeMs - (System.currentTimeMillis() - startMs))); return Optional.of( - new Plan(config.clusterInfo(), () -> 0, config.clusterInfo(), () -> 0)); + new Plan( + config.clusterBean(), + config.clusterInfo(), + () -> 0, + config.clusterInfo(), + () -> 0)); } }; diff --git a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java index b2ccf6415d..563172af6e 100644 --- a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java @@ -105,7 +105,9 @@ static List> costResult(Balancer.Plan plan) { }) .put(migrationCost.name, count))); - process.accept(MigrationCost.migrationCosts(plan.initialClusterInfo(), plan.proposal())); + process.accept( + MigrationCost.migrationCosts( + plan.initialClusterInfo(), plan.proposal(), plan.clusterBean())); return List.copyOf(map.values()); } diff --git a/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java b/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java index 90ca3ca23b..b9de226662 100644 --- a/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java +++ b/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java @@ -30,6 +30,7 @@ import org.astraea.common.admin.Replica; import org.astraea.common.balancer.Balancer; import org.astraea.common.cost.ReplicaLeaderSizeCost; +import org.astraea.common.metrics.ClusterBean; import org.astraea.gui.Context; import org.astraea.gui.pane.Argument; import org.astraea.it.Service; @@ -150,6 +151,7 @@ void testResult() { var results = BalancerNode.assignmentResult( new Balancer.Plan( + ClusterBean.EMPTY, beforeClusterInfo, () -> 1.0D, ClusterInfo.of("fake", allNodes, Map.of(), afterReplicas), From eb0c07d1b14597763aa9d8b28eb85a43256056d2 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 30 May 2023 00:45:35 +0800 Subject: [PATCH 42/77] [COMMON] don't reference to Broker from Replica (#1791) --- .../java/org/astraea/app/backup/Backup.java | 2 +- .../astraea/app/performance/Performance.java | 2 +- .../org/astraea/app/web/BalancerHandler.java | 2 +- .../astraea/app/web/ReassignmentHandler.java | 2 +- .../org/astraea/app/web/TopicHandler.java | 2 +- .../app/performance/PerformanceTest.java | 7 +- .../astraea/app/web/BalancerHandlerTest.java | 4 +- .../app/web/ReassignmentHandlerTest.java | 20 ++--- .../java/org/astraea/common/ByteUtils.java | 4 +- .../org/astraea/common/admin/AdminImpl.java | 9 ++- .../org/astraea/common/admin/ClusterInfo.java | 19 +++-- .../common/admin/ClusterInfoBuilder.java | 25 +++--- .../common/admin/OptimizedClusterInfo.java | 7 +- .../org/astraea/common/admin/Replica.java | 18 ++--- .../common/balancer/BalancerConsoleImpl.java | 8 +- .../common/balancer/BalancerUtils.java | 16 ++-- .../balancer/algorithms/GreedyBalancer.java | 2 +- .../algorithms/SingleStepBalancer.java | 2 +- .../executor/StraightPlanExecutor.java | 2 +- .../balancer/tweakers/ShuffleTweaker.java | 9 +-- .../astraea/common/cost/MigrationCost.java | 2 +- .../org/astraea/common/cost/NetworkCost.java | 2 +- .../common/cost/NetworkIngressCost.java | 2 +- .../common/cost/ReplicaLeaderSizeCost.java | 2 +- .../common/cost/ReplicaNumberCost.java | 2 +- .../common/cost/utils/ClusterInfoSensor.java | 2 +- .../common/generated/admin/Replica.proto | 2 +- .../org/astraea/common/admin/AdminTest.java | 23 +++--- .../admin/AdminWithOfflineBrokerTest.java | 1 - .../common/admin/ClusterInfoBuilderTest.java | 26 +++---- .../astraea/common/admin/ClusterInfoTest.java | 22 +++--- .../admin/ClusterInfoWithOfflineNodeTest.java | 9 +-- .../admin/SomePartitionOfflineTest.java | 4 +- .../balancer/BalancerConfigTestSuite.java | 34 ++++---- .../common/balancer/BalancerUtilsTest.java | 22 +++--- .../common/balancer/FakeClusterInfo.java | 4 +- .../executor/StraightPlanExecutorTest.java | 7 +- .../balancer/tweakers/ShuffleTweakerTest.java | 12 +-- .../common/cost/BrokerDiskSpaceCostTest.java | 78 +++++++++---------- .../astraea/common/cost/CostUtilsTest.java | 44 +++++------ .../common/cost/MigrationCostTest.java | 33 ++++---- .../astraea/common/cost/NetworkCostTest.java | 4 +- .../common/cost/RecordSizeCostTest.java | 6 +- .../common/cost/ReplicaLeaderCostTest.java | 26 +++++-- .../cost/ReplicaLeaderSizeCostTest.java | 26 +------ .../cost/utils/ClusterInfoSensorTest.java | 2 +- .../StrictCostPartitionerPerfTest.java | 6 +- .../StrictCostPartitionerTest.java | 20 ++--- .../astraea/gui/tab/health/BalancerNode.java | 9 +-- .../astraea/gui/tab/topic/ReplicaNode.java | 4 +- .../gui/tab/health/BalancerNodeTest.java | 8 +- .../gui/tab/topic/ReplicaNodeTest.java | 9 +-- 52 files changed, 291 insertions(+), 324 deletions(-) diff --git a/app/src/main/java/org/astraea/app/backup/Backup.java b/app/src/main/java/org/astraea/app/backup/Backup.java index 055ea1c0b1..71225e369c 100644 --- a/app/src/main/java/org/astraea/app/backup/Backup.java +++ b/app/src/main/java/org/astraea/app/backup/Backup.java @@ -50,7 +50,7 @@ public void restoreDistribution(ClusterInfo clusterInfo, String bootstrapServers .sorted( Comparator.comparing( replica -> !replica.isLeader())) - .map(replica -> replica.broker().id()) + .map(replica -> replica.brokerId()) .toList()))))) .configs(topic.config().raw()) .run() diff --git a/app/src/main/java/org/astraea/app/performance/Performance.java b/app/src/main/java/org/astraea/app/performance/Performance.java index 875ccaf1b1..8776b38c0e 100644 --- a/app/src/main/java/org/astraea/app/performance/Performance.java +++ b/app/src/main/java/org/astraea/app/performance/Performance.java @@ -382,7 +382,7 @@ else if (specifiedByBroker) { .join() .replicaStream() .filter(Replica::isLeader) - .filter(replica -> specifyBrokers.contains(replica.broker().id())) + .filter(replica -> specifyBrokers.contains(replica.brokerId())) .map(replica -> TopicPartition.of(replica.topic(), replica.partition())) .distinct() .toList(); diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 9ee87413b7..45cf1626fc 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -250,7 +250,7 @@ static class Placement { final Optional size; Placement(Replica replica, Optional size) { - this.brokerId = replica.broker().id(); + this.brokerId = replica.brokerId(); this.directory = replica.path(); this.size = size; } diff --git a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java index 2c2245e7e3..cfd84680e6 100644 --- a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java +++ b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java @@ -211,7 +211,7 @@ static class AddingReplica implements Response { AddingReplica(Replica addingReplica, long leaderSize) { this.topicName = addingReplica.topic(); this.partition = addingReplica.partition(); - this.broker = addingReplica.broker().id(); + this.broker = addingReplica.brokerId(); this.dataFolder = addingReplica.path(); this.size = addingReplica.size(); this.leaderSize = leaderSize; diff --git a/app/src/main/java/org/astraea/app/web/TopicHandler.java b/app/src/main/java/org/astraea/app/web/TopicHandler.java index 28537281d1..e88295d6ff 100644 --- a/app/src/main/java/org/astraea/app/web/TopicHandler.java +++ b/app/src/main/java/org/astraea/app/web/TopicHandler.java @@ -316,7 +316,7 @@ private Replica() { Replica(org.astraea.common.admin.Replica replica) { this( - replica.broker().id(), + replica.brokerId(), replica.lag(), replica.size(), replica.isLeader(), diff --git a/app/src/test/java/org/astraea/app/performance/PerformanceTest.java b/app/src/test/java/org/astraea/app/performance/PerformanceTest.java index 90f56293a5..2150b5c979 100644 --- a/app/src/test/java/org/astraea/app/performance/PerformanceTest.java +++ b/app/src/test/java/org/astraea/app/performance/PerformanceTest.java @@ -193,7 +193,7 @@ void testPartitionSupplier() { .join() .replicaStream() .filter(Replica::isLeader) - .filter(r -> r.broker().id() == 1) + .filter(r -> r.brokerId() == 1) .map(Replica::topicPartition) .collect(Collectors.toUnmodifiableSet()); @@ -239,7 +239,7 @@ void testPartitionSupplier() { .join() .replicaStream() .filter(Replica::isLeader) - .filter(replica -> replica.broker().id() == 1) + .filter(replica -> replica.brokerId() == 1) .map(Replica::topicPartition) .collect(Collectors.toSet()); var selector2 = args.topicPartitionSelector(); @@ -273,8 +273,7 @@ void testPartitionSupplier() { .replicaStream() .findFirst() .get() - .broker() - .id(); + .brokerId(); var noPartitionBroker = (validBroker == 3) ? 1 : validBroker + 1; args = Argument.parse( diff --git a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java index 8fadee55a2..03bdd829a8 100644 --- a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java @@ -970,7 +970,7 @@ void testChangeOrder() { (short) 10, r -> Replica.builder(r) - .broker(base.node(srcIter.next())) + .brokerId(base.node(srcIter.next()).id()) .isPreferredLeader(srcPrefIter.next()) .path(srcDirIter.next()) .build()) @@ -986,7 +986,7 @@ void testChangeOrder() { (short) 10, r -> Replica.builder(r) - .broker(base.node(dstIter.next())) + .brokerId(base.node(dstIter.next()).id()) .isPreferredLeader(dstPrefIter.next()) .path(dstDirIter.next()) .build()) diff --git a/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java b/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java index 01570fdca4..fd763ca73a 100644 --- a/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/ReassignmentHandlerTest.java @@ -60,8 +60,7 @@ void testMigrateToAnotherBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id(); + .brokerId(); var nextBroker = SERVICE.dataFolders().keySet().stream().filter(i -> i != currentBroker).findAny().get(); @@ -88,8 +87,7 @@ void testMigrateToAnotherBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id()); + .brokerId()); } } @@ -111,7 +109,7 @@ void testMigrateToAnotherPath() { .findFirst() .get(); - var currentBroker = currentReplica.broker().id(); + var currentBroker = currentReplica.brokerId(); var currentPath = currentReplica.path(); var nextPath = SERVICE.dataFolders().get(currentBroker).stream() @@ -170,8 +168,7 @@ void testExcludeSpecificBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id(); + .brokerId(); var body = String.format("{\"excludeNodes\": [{\"%s\": \"%s\"}]}", EXCLUDE_KEY, currentBroker); @@ -194,8 +191,7 @@ void testExcludeSpecificBroker() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id()); + .brokerId()); Assertions.assertEquals( 0, admin.topicPartitionReplicas(Set.of(currentBroker)).toCompletableFuture().join().size()); @@ -221,8 +217,7 @@ void testExcludeSpecificBrokerTopic() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id(); + .brokerId(); var body = String.format( @@ -247,8 +242,7 @@ void testExcludeSpecificBrokerTopic() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id()); + .brokerId()); Assertions.assertNotEquals( 0, admin.topicPartitionReplicas(Set.of(currentBroker)).toCompletableFuture().join().size()); diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index 68a233d217..a97e93f14e 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -353,7 +353,7 @@ private static ReplicaOuterClass.Replica toOuterClass(Replica replica) { return ReplicaOuterClass.Replica.newBuilder() .setTopic(replica.topic()) .setPartition(replica.partition()) - .setBroker(toOuterClass(replica.broker())) + .setBrokerId(replica.brokerId()) .setLag(replica.lag()) .setSize(replica.size()) .setIsInternal(replica.isInternal()) @@ -414,7 +414,7 @@ private static Replica toReplica(ReplicaOuterClass.Replica replica) { return Replica.builder() .topic(replica.getTopic()) .partition(replica.getPartition()) - .broker(toBroker(replica.getBroker())) + .brokerId(replica.getBrokerId()) .lag(replica.getLag()) .size(replica.getSize()) .isInternal(replica.getIsInternal()) diff --git a/common/src/main/java/org/astraea/common/admin/AdminImpl.java b/common/src/main/java/org/astraea/common/admin/AdminImpl.java index 5af795dfa9..a29d57f56a 100644 --- a/common/src/main/java/org/astraea/common/admin/AdminImpl.java +++ b/common/src/main/java/org/astraea/common/admin/AdminImpl.java @@ -685,9 +685,10 @@ private CompletionStage> replicas(Set topics) { .isInternal(internal) .isAdding(isAdding) .isRemoving(isRemoving) - .broker( - brokers.getOrDefault( - node.id(), Broker.of(node))) + .brokerId( + brokers + .getOrDefault(node.id(), Broker.of(node)) + .id()) .lag(pathAndReplica.getValue().offsetLag()) .size(pathAndReplica.getValue().size()) .isLeader( @@ -716,7 +717,7 @@ private CompletionStage> replicas(Set topics) { .sorted( Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) - .thenComparing(r -> r.broker().id())) + .thenComparing(r -> r.brokerId())) .toList()); } 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 0be6ff795a..33c7231608 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java @@ -87,22 +87,22 @@ static boolean placementMatch( .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(r -> r.broker().id())) - .collect(Collectors.toUnmodifiableList()); + .thenComparing(Replica::brokerId)) + .toList(); final var targetIds = targetReplicas.stream() .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(r -> r.broker().id())) - .collect(Collectors.toUnmodifiableList()); + .thenComparing(Replica::brokerId)) + .toList(); return IntStream.range(0, sourceIds.size()) .allMatch( index -> { final var source = sourceIds.get(index); final var target = targetIds.get(index); return source.isPreferredLeader() == target.isPreferredLeader() - && source.broker().id() == target.broker().id() + && source.brokerId() == target.brokerId() && Objects.equals(source.path(), target.path()); }); } @@ -121,7 +121,7 @@ static String toString(ClusterInfo allocation) { .forEach( log -> stringBuilder.append( - String.format("(%s, %s) ", log.broker().id(), log.path()))); + String.format("(%s, %s) ", log.brokerId(), log.path()))); stringBuilder.append(System.lineSeparator()); }); @@ -332,7 +332,7 @@ default Map> brokerFolders() { // implements following methods by smart index to speed up the queries default Stream replicaStream(int broker) { - return replicaStream().filter(r -> r.broker().id() == broker); + return replicaStream().filter(r -> r.brokerId() == broker); } default Stream replicaStream(String topic) { @@ -340,7 +340,7 @@ default Stream replicaStream(String topic) { } default Stream replicaStream(BrokerTopic brokerTopic) { - return replicaStream(brokerTopic.topic()).filter(r -> r.broker().id() == brokerTopic.broker()); + return replicaStream(brokerTopic.topic()).filter(r -> r.brokerId() == brokerTopic.broker()); } default Stream replicaStream(TopicPartition partition) { @@ -348,8 +348,7 @@ default Stream replicaStream(TopicPartition partition) { } default Stream replicaStream(TopicPartitionReplica replica) { - return replicaStream(replica.topicPartition()) - .filter(r -> r.broker().id() == replica.brokerId()); + return replicaStream(replica.topicPartition()).filter(r -> r.brokerId() == replica.brokerId()); } // ---------------------[abstract methods]---------------------// diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java index c259f1c9d3..f3d01f5a1c 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java @@ -89,7 +89,7 @@ public ClusterInfoBuilder addNode(Set brokerIds) { + " but another broker with this id already existed"); }); return Stream.concat(nodes.stream(), brokerIds.stream().map(ClusterInfoBuilder::fakeNode)) - .collect(Collectors.toUnmodifiableList()); + .toList(); }); } @@ -132,7 +132,7 @@ public ClusterInfoBuilder addFolders(Map> folders) { .toList()); else return node; }) - .collect(Collectors.toUnmodifiableList()); + .toList(); }); } @@ -174,7 +174,7 @@ public ClusterInfoBuilder addTopic( nodes.stream() .collect( Collectors.toUnmodifiableMap( - node -> node, + Broker::id, node -> node.dataFolders().stream() .collect( @@ -182,7 +182,7 @@ public ClusterInfoBuilder addTopic( Broker.DataFolder::path, x -> new AtomicInteger())))); replicas.forEach( replica -> - folderLogCounter.get(replica.broker()).get(replica.path()).incrementAndGet()); + folderLogCounter.get(replica.brokerId()).get(replica.path()).incrementAndGet()); folderLogCounter.forEach( (node, folders) -> { @@ -200,7 +200,7 @@ public ClusterInfoBuilder addTopic( index -> { final Broker broker = nodeSelector.next(); final String path = - folderLogCounter.get(broker).entrySet().stream() + folderLogCounter.get(broker.id()).entrySet().stream() .min(Comparator.comparing(x -> x.getValue().get())) .map( entry -> { @@ -212,7 +212,7 @@ public ClusterInfoBuilder addTopic( return Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .broker(broker) + .brokerId(broker.id()) .isAdding(false) .isRemoving(false) .lag(0) @@ -227,8 +227,7 @@ public ClusterInfoBuilder addTopic( })) .map(mapper); - return Stream.concat(replicas.stream(), newTopic) - .collect(Collectors.toUnmodifiableList()); + return Stream.concat(replicas.stream(), newTopic).toList(); }); } @@ -239,9 +238,7 @@ public ClusterInfoBuilder addTopic( * @return this. */ public ClusterInfoBuilder mapLog(Function mapper) { - return applyReplicas( - (nodes, replicas) -> - replicas.stream().map(mapper).collect(Collectors.toUnmodifiableList())); + return applyReplicas((nodes, replicas) -> replicas.stream().map(mapper).toList()); } /** @@ -269,12 +266,12 @@ public ClusterInfoBuilder reassignReplica( r -> { if (r.topicPartitionReplica().equals(replica)) { matched.set(true); - return Replica.builder(r).broker(newNode).path(toDir).build(); + return Replica.builder(r).brokerId(newNode.id()).path(toDir).build(); } else { return r; } }) - .collect(Collectors.toUnmodifiableList()); + .toList(); if (!matched.get()) throw new IllegalArgumentException("No such replica: " + replica); return collect; }); @@ -307,7 +304,7 @@ public ClusterInfoBuilder setPreferredLeader(TopicPartitionReplica replica) { return r; } }) - .collect(Collectors.toUnmodifiableList()); + .toList(); if (!matched.get()) throw new IllegalArgumentException("No such replica: " + replica); return collect; diff --git a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java index e463d2bc45..cf874fc9d0 100644 --- a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java @@ -73,7 +73,7 @@ class OptimizedClusterInfo implements ClusterInfo { all.stream() .collect( Collectors.groupingBy( - r -> BrokerTopic.of(r.broker().id(), r.topic()), + r -> BrokerTopic.of(r.brokerId(), r.topic()), Collectors.toUnmodifiableList()))); this.byBrokerTopicForLeader = Lazy.of( @@ -83,7 +83,7 @@ class OptimizedClusterInfo implements ClusterInfo { .filter(Replica::isLeader) .collect( Collectors.groupingBy( - r -> BrokerTopic.of(r.broker().id(), r.topic()), + r -> BrokerTopic.of(r.brokerId(), r.topic()), Collectors.toUnmodifiableList()))); this.byBroker = @@ -91,8 +91,7 @@ class OptimizedClusterInfo implements ClusterInfo { () -> all.stream() .collect( - Collectors.groupingBy( - r -> r.broker().id(), Collectors.toUnmodifiableList()))); + Collectors.groupingBy(r -> r.brokerId(), Collectors.toUnmodifiableList()))); this.byTopic = Lazy.of( diff --git a/common/src/main/java/org/astraea/common/admin/Replica.java b/common/src/main/java/org/astraea/common/admin/Replica.java index 9f95af2ad2..11c656df19 100644 --- a/common/src/main/java/org/astraea/common/admin/Replica.java +++ b/common/src/main/java/org/astraea/common/admin/Replica.java @@ -41,7 +41,7 @@ public record Replica( String topic, int partition, - Broker broker, + int brokerId, boolean isLeader, boolean isSync, boolean isOffline, @@ -68,7 +68,7 @@ public static Builder builder(Replica replica) { * @return TopicPartitionReplica */ public TopicPartitionReplica topicPartitionReplica() { - return TopicPartitionReplica.of(topic(), partition(), broker().id()); + return TopicPartitionReplica.of(topic(), partition(), brokerId()); } /** @@ -105,7 +105,7 @@ public static class Builder { private String topic; private int partition; - private Broker broker; + private int brokerId; private long lag; private long size; @@ -125,7 +125,7 @@ private Builder() {} public Builder replica(Replica replica) { this.topic = replica.topic(); this.partition = replica.partition(); - this.broker = replica.broker(); + this.brokerId = replica.brokerId(); this.lag = replica.lag(); this.size = replica.size(); this.isAdding = replica.isAdding; @@ -150,8 +150,8 @@ public Builder partition(int partition) { return this; } - public Builder broker(Broker broker) { - this.broker = broker; + public Builder brokerId(int brokerId) { + this.brokerId = brokerId; return this; } @@ -219,7 +219,7 @@ public Replica buildLeader() { return new Replica( Objects.requireNonNull(topic), partition, - Objects.requireNonNull(broker), + brokerId, true, true, false, @@ -242,7 +242,7 @@ public Replica buildInSyncFollower() { return new Replica( Objects.requireNonNull(topic), partition, - Objects.requireNonNull(broker), + brokerId, false, true, false, @@ -260,7 +260,7 @@ public Replica build() { return new Replica( Objects.requireNonNull(topic), partition, - Objects.requireNonNull(broker), + brokerId, isLeader, isSync, isOffline, diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java index 5170d02d2c..fe4db19073 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java @@ -270,8 +270,8 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(x -> x.broker().id())) - .map(x -> Map.entry(x.broker().id(), x.path())) + .thenComparing(x -> x.brokerId())) + .map(x -> Map.entry(x.brokerId(), x.path())) .collect(Collectors.toUnmodifiableList()))); return admin .topicNames(true) @@ -292,8 +292,8 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(x -> x.broker().id())) - .map(x -> Map.entry(x.broker().id(), x.path())) + .thenComparing(x -> x.brokerId())) + .map(x -> Map.entry(x.brokerId(), x.path())) .collect(Collectors.toUnmodifiableList()))); var mismatchPartitions = before.entrySet().stream() diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java index a673085b40..ee30ace4bf 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerUtils.java @@ -67,7 +67,7 @@ public static Map balancingMode(ClusterInfo cluster, St public static void verifyClearBrokerValidness(ClusterInfo cluster, Predicate isClear) { var ongoingEventReplica = cluster.replicas().stream() - .filter(r -> isClear.test(r.broker().id())) + .filter(r -> isClear.test(r.brokerId())) .filter(r -> r.isAdding() || r.isRemoving() || r.isFuture()) .map(Replica::topicPartitionReplica) .collect(Collectors.toUnmodifiableSet()); @@ -106,17 +106,17 @@ public static ClusterInfo clearedCluster( tp -> tp, tp -> initial.replicas(tp).stream() - .map(Replica::broker) + .map(Replica::brokerId) .collect(Collectors.toSet()))); return ClusterInfo.builder(initial) .mapLog( replica -> { - if (!clearBrokers.test(replica.broker().id())) return replica; + if (!clearBrokers.test(replica.brokerId())) return replica; var currentReplicaList = trackingReplicaList.get(replica.topicPartition()); var broker = IntStream.range(0, allowed.size()) .mapToObj(i -> nextBroker.next()) - .filter(b -> !currentReplicaList.contains(b)) + .filter(b -> !currentReplicaList.contains(b.id())) .findFirst() .orElseThrow( () -> @@ -124,7 +124,7 @@ public static ClusterInfo clearedCluster( "Unable to clear replica " + replica.topicPartitionReplica() + " for broker " - + replica.broker().id() + + replica.brokerId() + ", the allowed destination brokers are " + allowed.stream() .map(Broker::id) @@ -135,10 +135,10 @@ public static ClusterInfo clearedCluster( // update the tracking list. have to do this to avoid putting two replicas from the // same tp to one broker. - currentReplicaList.remove(replica.broker()); - currentReplicaList.add(broker); + currentReplicaList.remove(replica.brokerId()); + currentReplicaList.add(broker.id()); - return Replica.builder(replica).broker(broker).path(folder).build(); + return Replica.builder(replica).brokerId(broker.id()).path(folder).build(); }) .build(); } diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index f9987385f5..3516b36a25 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -174,7 +174,7 @@ public Optional offer(AlgorithmConfig config) { // if a topic is not allowed to move, it should be fixed. // if a topic is not allowed to move, but originally it located on a clearing broker, it // is ok to move. - .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isClearing.test(tpr.broker().id())) + .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isClearing.test(tpr.brokerId())) .collect(Collectors.toUnmodifiableSet()); final var allocationTweaker = ShuffleTweaker.builder() diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java index f92259c226..c44cd9ded7 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/SingleStepBalancer.java @@ -104,7 +104,7 @@ public Optional offer(AlgorithmConfig config) { // if a topic is not allowed to move, it should be fixed. // if a topic is not allowed to move, but originally it located on a clearing broker, it // is ok to move. - .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isClearing.test(tpr.broker().id())) + .filter(tpr -> !allowedTopics.test(tpr.topic()) && !isClearing.test(tpr.brokerId())) .collect(Collectors.toUnmodifiableSet()); final var allocationTweaker = ShuffleTweaker.builder() diff --git a/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java b/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java index d932266afd..796fb89b14 100644 --- a/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java +++ b/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java @@ -79,7 +79,7 @@ public CompletionStage run(Admin admin, ClusterInfo logAllocation, Duratio .collect( Collectors.groupingBy( Replica::topicPartition, - Collectors.mapping(r -> r.broker().id(), Collectors.toList())))) + Collectors.mapping(r -> r.brokerId(), Collectors.toList())))) .thenApply(ignored -> replicas)) // step 2: wait replicas get reassigned .thenCompose( diff --git a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java index d797512145..bb919112e4 100644 --- a/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java +++ b/common/src/main/java/org/astraea/common/balancer/tweakers/ShuffleTweaker.java @@ -80,7 +80,7 @@ public Stream generate(ClusterInfo baseAllocation) { baseAllocation.topicPartitions().stream() .filter(tp -> eligiblePartition(baseAllocation.replicas(tp))) .flatMap(baseAllocation::replicaStream) - .filter(r -> this.allowedBrokers.test(r.broker().id())) + .filter(r -> this.allowedBrokers.test(r.brokerId())) .filter(this.allowedReplicas) .toList(); @@ -105,7 +105,7 @@ public Stream generate(ClusterInfo baseAllocation) { // leader pair follower, follower pair leader .filter(r -> r.isFollower() != sourceReplica.isFollower()) // this leader/follower is located at allowed broker - .filter(r -> this.allowedBrokers.test(r.broker().id())) + .filter(r -> this.allowedBrokers.test(r.brokerId())) // this leader/follower is allowed to tweak .filter(this.allowedReplicas) .map(r -> Map.entry(r, ThreadLocalRandom.current().nextInt())) @@ -148,8 +148,7 @@ else if (replicaList.get(i).isLeader() var targetBroker = baseAllocation.brokers().stream() // the candidate should not be part of the replica list - .filter( - b -> replicaList.stream().noneMatch(r -> r.broker().id() == b.id())) + .filter(b -> replicaList.stream().noneMatch(r -> r.brokerId() == b.id())) // should be an allowed broker .filter(b -> this.allowedBrokers.test(b.id())) .map(b -> Map.entry(b, ThreadLocalRandom.current().nextInt())) @@ -159,7 +158,7 @@ else if (replicaList.get(i).isLeader() if (targetBroker.isPresent()) { var targetReplica = Replica.builder(sourceReplica) - .broker(targetBroker.get()) + .brokerId(targetBroker.get().id()) .path( randomElement( baseAllocation.brokerFolders().get(targetBroker.get().id()))) diff --git a/common/src/main/java/org/astraea/common/cost/MigrationCost.java b/common/src/main/java/org/astraea/common/cost/MigrationCost.java index c88c1d2488..9bb6b92274 100644 --- a/common/src/main/java/org/astraea/common/cost/MigrationCost.java +++ b/common/src/main/java/org/astraea/common/cost/MigrationCost.java @@ -109,7 +109,7 @@ private static Map migratedChanged( }) .collect( Collectors.groupingBy( - r -> r.broker().id(), + r -> r.brokerId(), Collectors.mapping( Function.identity(), Collectors.summingLong(replicaFunction::apply)))); return Stream.concat(dest.brokers().stream(), source.brokers().stream()) diff --git a/common/src/main/java/org/astraea/common/cost/NetworkCost.java b/common/src/main/java/org/astraea/common/cost/NetworkCost.java index c1b61a576f..28d150eccc 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkCost.java @@ -210,7 +210,7 @@ private Map> mapLeaderAllocation(ClusterInfo clusterI .replicaStream() .filter(Replica::isOnline) .filter(Replica::isLeader) - .map(r -> Map.entry(BrokerTopic.of(r.broker().id(), r.topic()), r)) + .map(r -> Map.entry(BrokerTopic.of(r.brokerId(), r.topic()), r)) .collect( Collectors.groupingBy( Map.Entry::getKey, diff --git a/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java b/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java index aa4f99055f..d633a70774 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java @@ -60,7 +60,7 @@ public PartitionCost partitionCost(ClusterInfo clusterInfo, ClusterBean clusterB .filter(Replica::isOnline) .collect( Collectors.groupingBy( - replica -> replica.broker().id(), + replica -> replica.brokerId(), Collectors.toMap( Replica::topicPartition, r -> partitionTraffic.get(r.topicPartition())))); diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index e722dc141d..f2586401e8 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -67,7 +67,7 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { clusterInfo.replicas().stream() .collect( Collectors.groupingBy( - r -> r.broker().id(), + r -> r.brokerId(), Collectors.mapping( r -> clusterInfo diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java index 0f84bbe705..720c46f6f0 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java @@ -59,7 +59,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) var replicaPerBroker = clusterInfo .replicaStream() - .collect(Collectors.groupingBy(r -> r.broker().id(), Collectors.counting())); + .collect(Collectors.groupingBy(r -> r.brokerId(), Collectors.counting())); var summary = replicaPerBroker.values().stream().mapToLong(x -> x).summaryStatistics(); var anyBrokerEmpty = diff --git a/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java b/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java index df3f62b795..74e82da525 100644 --- a/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java +++ b/common/src/main/java/org/astraea/common/cost/utils/ClusterInfoSensor.java @@ -108,7 +108,7 @@ public static ClusterInfo metricViewCluster(ClusterBean clusterBean) { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .broker(nodes.get(broker)) + .brokerId(nodes.get(broker).id()) .path("") .size(size.value()); var isLeader = m.value() != 0; diff --git a/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto b/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto index 598ade270e..333ae33956 100644 --- a/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto +++ b/common/src/main/proto/org/astraea/common/generated/admin/Replica.proto @@ -7,7 +7,7 @@ import "org/astraea/common/generated/admin/Broker.proto"; message Replica { string topic = 1; int32 partition = 2; - Broker broker = 3; + int32 brokerId = 3; bool isLeader = 7; bool isSync = 10; bool isOffline = 12; diff --git a/common/src/test/java/org/astraea/common/admin/AdminTest.java b/common/src/test/java/org/astraea/common/admin/AdminTest.java index efab4420c6..eeaccaa638 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminTest.java @@ -359,7 +359,7 @@ void testOrder() { .sorted( Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) - .thenComparing(r -> r.broker().id())) + .thenComparing(r -> r.brokerId())) .collect(Collectors.toList()), replicas); } @@ -417,7 +417,7 @@ void testMoveToAnotherFolder() { var replica = replicas.get(0); var idAndFolder = SERVICE.dataFolders().entrySet().stream() - .filter(e -> e.getKey() == replica.broker().id()) + .filter(e -> e.getKey() == replica.brokerId()) .map(e -> Map.of(e.getKey(), e.getValue().iterator().next())) .findFirst() .get(); @@ -434,7 +434,7 @@ void testMoveToAnotherFolder() { Assertions.assertEquals(1, newReplicas.size()); var newReplica = newReplicas.get(0); - Assertions.assertEquals(idAndFolder.get(newReplica.broker().id()), newReplica.path()); + Assertions.assertEquals(idAndFolder.get(newReplica.brokerId()), newReplica.path()); } } @@ -590,7 +590,7 @@ void testDeclarePreferredFoldersWithNoCrossBrokerMovement() { r -> Replica.builder(r) .path( - source.brokerFolders().get(r.broker().id()).stream() + source.brokerFolders().get(r.brokerId()).stream() .filter(p -> !p.equals(r.path())) .findAny() .orElseThrow()) @@ -720,8 +720,7 @@ void testMigrateToOtherFolders() { .replicaStream() .iterator() .next() - .broker() - .id(); + .brokerId(); var paths = new ArrayList<>(SERVICE.dataFolders().get(id)); for (var path : paths) { @@ -1047,8 +1046,7 @@ void testMigrateSinglePartition() { () -> { var partitionReplicas = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join().replicas(); - return partitionReplicas.size() == 1 - && partitionReplicas.get(0).broker().id() == broker; + return partitionReplicas.size() == 1 && partitionReplicas.get(0).brokerId() == broker; }); var currentBroker = @@ -1060,8 +1058,7 @@ void testMigrateSinglePartition() { .filter(replica -> replica.partition() == 0) .findFirst() .get() - .broker() - .id(); + .brokerId(); var allPath = admin.brokerFolders().toCompletableFuture().join(); var otherPath = allPath.get(currentBroker).stream() @@ -1115,7 +1112,7 @@ void testIllegalMigrationArgument() { .findFirst() .get(); - var currentBroker = currentReplica.broker().id(); + var currentBroker = currentReplica.brokerId(); var notExistReplica = (currentBroker + 1) % SERVICE.dataFolders().keySet().size(); var nextDir = SERVICE.dataFolders().get(notExistReplica).iterator().next(); @@ -1145,7 +1142,7 @@ void testMigrateAllPartitions() { Utils.waitFor( () -> { var replicas = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join().replicas(); - return replicas.stream().allMatch(r -> r.broker().id() == broker); + return replicas.stream().allMatch(r -> r.brokerId() == broker); }); } } @@ -1389,7 +1386,7 @@ void testReplicasPreferredLeaderFlag() { Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()), Collectors.mapping( - replica -> replica.broker().id(), Collectors.toList()))); + replica -> replica.brokerId(), Collectors.toList()))); IntStream.range(0, partitionCount) .forEach(p -> admin.moveToBrokers(Map.of(TopicPartition.of(topic, p), List.of(0, 1, 2)))); diff --git a/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java b/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java index 43a9ebfc3a..2595b0e8d1 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java @@ -153,7 +153,6 @@ void testReplicas() { var offlineReplicas = replicas.stream().filter(Replica::isOffline).collect(Collectors.toList()); Assertions.assertNotEquals(PARTITIONS, offlineReplicas.size()); - offlineReplicas.forEach(r -> Assertions.assertTrue(r.broker().offline())); offlineReplicas.forEach(r -> Assertions.assertNull(r.path())); offlineReplicas.forEach(r -> Assertions.assertEquals(-1, r.size())); offlineReplicas.forEach(r -> Assertions.assertEquals(-1, r.lag())); diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java index 8bebbe37a4..85861e188b 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java @@ -38,7 +38,7 @@ void testBuild() { Replica.builder() .topic("MyTopic") .partition(0) - .broker(host1000) + .brokerId(host1000.id()) .size(1024) .isPreferredLeader(true) .isLeader(true) @@ -160,7 +160,7 @@ void addTopic() { path -> cluster .replicaStream() - .filter(r -> r.broker().id() == id) + .filter(r -> r.brokerId() == id) .filter(r -> r.path().equals(path)) .count())); var summary = folderLogs.values().stream().mapToLong(x -> x).summaryStatistics(); @@ -214,7 +214,7 @@ void reassignReplica() { (short) 2, (replica) -> Replica.builder(replica) - .broker(base.node(replica.isPreferredLeader() ? 1 : 2)) + .brokerId(base.node(replica.isPreferredLeader() ? 1 : 2).id()) .path(replica.isPreferredLeader() ? "/ssd1" : "/ssd2") .build()) .build(); @@ -236,22 +236,22 @@ void reassignReplica() { original .replicaStream() .filter(Replica::isPreferredLeader) - .allMatch(r -> r.broker().id() == 1 && r.path().equals("/ssd1"))); + .allMatch(r -> r.brokerId() == 1 && r.path().equals("/ssd1"))); Assertions.assertTrue( original .replicaStream() .filter(Predicate.not(Replica::isPreferredLeader)) - .allMatch(r -> r.broker().id() == 2 && r.path().equals("/ssd2"))); + .allMatch(r -> r.brokerId() == 2 && r.path().equals("/ssd2"))); Assertions.assertTrue( altered .replicaStream() .filter(Replica::isPreferredLeader) - .allMatch(r -> r.broker().id() == 3 && r.path().equals("/ssd3"))); + .allMatch(r -> r.brokerId() == 3 && r.path().equals("/ssd3"))); Assertions.assertTrue( altered .replicaStream() .filter(Predicate.not(Replica::isPreferredLeader)) - .allMatch(r -> r.broker().id() == 4 && r.path().equals("/ssd4"))); + .allMatch(r -> r.brokerId() == 4 && r.path().equals("/ssd4"))); Assertions.assertThrows( IllegalArgumentException.class, @@ -287,8 +287,8 @@ void setPreferredLeader() { (short) 4, (replica) -> Replica.builder(replica) - .isLeader(replica.broker().id() == 0) - .isPreferredLeader(replica.broker().id() == 0) + .isLeader(replica.brokerId() == 0) + .isPreferredLeader(replica.brokerId() == 0) .build()) .build(); var altered = @@ -303,19 +303,19 @@ void setPreferredLeader() { original .replicaStream() .filter((Replica::isPreferredLeader)) - .allMatch(r -> r.broker().id() == 0)); + .allMatch(r -> r.brokerId() == 0)); Assertions.assertTrue( - original.replicaStream().filter((Replica::isLeader)).allMatch(r -> r.broker().id() == 0)); + original.replicaStream().filter((Replica::isLeader)).allMatch(r -> r.brokerId() == 0)); Assertions.assertTrue( altered .replicaStream() .filter(Replica::isPreferredLeader) - .allMatch(r -> r.broker().id() == r.partition())); + .allMatch(r -> r.brokerId() == r.partition())); Assertions.assertTrue( altered .replicaStream() .filter(Replica::isLeader) - .allMatch(r -> r.broker().id() == r.partition())); + .allMatch(r -> r.brokerId() == r.partition())); Assertions.assertThrows( IllegalArgumentException.class, () -> 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 126b46754a..716ab3b4fc 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java @@ -48,12 +48,22 @@ public static ClusterInfo of(List replicas) { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::broker) + .map( + r -> + new Broker( + r.brokerId(), + "hpost", + 22222, + false, + Config.EMPTY, + List.of(), + Set.of(), + Set.of())) .collect(Collectors.groupingBy(Broker::id, Collectors.reducing((x, y) -> x))) .values() .stream() .flatMap(Optional::stream) - .collect(Collectors.toUnmodifiableList()), + .toList(), Map.of(), replicas); } @@ -96,13 +106,7 @@ void testTopics() { @Test void testReturnCollectionUnmodifiable() { var cluster = ClusterInfo.empty(); - var replica = - Replica.builder() - .topic("topic") - .partition(0) - .broker(Broker.of(0, "", -1)) - .path("f") - .buildLeader(); + var replica = Replica.builder().topic("topic").partition(0).brokerId(0).path("f").buildLeader(); Assertions.assertThrows(Exception.class, () -> cluster.replicas().add(replica)); Assertions.assertThrows(Exception.class, () -> cluster.replicas("t").add(replica)); Assertions.assertThrows( diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java index e5b251fd95..203cfc911a 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoWithOfflineNodeTest.java @@ -76,18 +76,17 @@ void testClusterInfoWithOfflineNode() { after.replicaLeaders(topicName).size(), "One of the rest replicas should take over the leadership"); Assertions.assertTrue( - after.availableReplicas(topicName).stream() - .allMatch(x -> x.broker().id() != brokerToClose)); + after.availableReplicas(topicName).stream().allMatch(x -> x.brokerId() != brokerToClose)); Assertions.assertTrue( - after.replicaLeaders(topicName).stream().allMatch(x -> x.broker().id() != brokerToClose)); + after.replicaLeaders(topicName).stream().allMatch(x -> x.brokerId() != brokerToClose)); Assertions.assertTrue( after.replicas(topicName).stream() .filter(Replica::isOffline) - .allMatch(x -> x.broker().id() == brokerToClose)); + .allMatch(x -> x.brokerId() == brokerToClose)); Assertions.assertTrue( after.replicas(topicName).stream() .filter(x -> !x.isOffline()) - .allMatch(x -> x.broker().id() != brokerToClose)); + .allMatch(x -> x.brokerId() != brokerToClose)); } } } diff --git a/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java b/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java index 624379db1e..c83d63dbcf 100644 --- a/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java +++ b/common/src/test/java/org/astraea/common/admin/SomePartitionOfflineTest.java @@ -63,7 +63,7 @@ void somePartitionsOffline() { .toCompletableFuture() .join() .replicaStream() - .filter(replica -> replica.broker().id() == 0) + .filter(replica -> replica.brokerId() == 0) .collect( Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()))); @@ -78,7 +78,7 @@ void somePartitionsOffline() { .toCompletableFuture() .join() .replicaStream() - .filter(replica -> replica.broker().id() == 0) + .filter(replica -> replica.brokerId() == 0) .collect( Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()))); diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index 603fbe38af..83407490fa 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -241,12 +241,12 @@ public void testBalancingModeClear() { .build()); Assertions.assertTrue(plan.isPresent(), testName); var finalCluster = plan.get().proposal(); - Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.broker().id() == 0)); - Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.broker().id() == 1)); - Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.broker().id() == 2)); - Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.broker().id() == 0)); - Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.broker().id() == 1)); - Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.broker().id() == 2)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.brokerId() == 0)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.brokerId() == 1)); + Assertions.assertTrue(cluster.replicas().stream().anyMatch(x -> x.brokerId() == 2)); + Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.brokerId() == 0)); + Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.brokerId() == 1)); + Assertions.assertTrue(finalCluster.replicas().stream().noneMatch(x -> x.brokerId() == 2)); AssertionsHelper.assertBrokerEmpty( finalCluster, (x) -> Set.of(0, 1, 2).contains(x), testName); } @@ -310,9 +310,9 @@ public void testBalancingModeClear() { var testCluster = ClusterInfo.builder(base) .addTopic("topic", 3, (short) 1) - .addTopic("ok0", 10, (short) 1, r -> Replica.builder(r).broker(node3).build()) - .addTopic("ok1", 10, (short) 1, r -> Replica.builder(r).broker(node3).build()) - .addTopic("ok2", 10, (short) 1, r -> Replica.builder(r).broker(node3).build()) + .addTopic("ok0", 10, (short) 1, r -> Replica.builder(r).brokerId(node3.id()).build()) + .addTopic("ok1", 10, (short) 1, r -> Replica.builder(r).brokerId(node3.id()).build()) + .addTopic("ok2", 10, (short) 1, r -> Replica.builder(r).brokerId(node3.id()).build()) .build(); var result = @@ -335,16 +335,16 @@ public void testBalancingModeClear() { Assertions.assertTrue(result.isPresent()); Assertions.assertNotEquals( List.of(), - testCluster.replicas().stream().filter(x -> x.broker().id() == 3).toList(), + testCluster.replicas().stream().filter(x -> x.brokerId() == 3).toList(), "Originally, some replica located at broker 3"); Assertions.assertEquals( List.of(), - result.get().proposal().replicas().stream().filter(x -> x.broker().id() == 3).toList(), + result.get().proposal().replicas().stream().filter(x -> x.brokerId() == 3).toList(), "Returned allocation has no replica located at broker 3"); var toStay = testCluster.replicas().stream() .filter(x -> x.topic().equals("topic")) - .filter(x -> x.broker().id() != 3) + .filter(x -> x.brokerId() != 3) .collect(Collectors.toSet()); Assertions.assertTrue( result.get().proposal().replicas().stream() @@ -416,15 +416,15 @@ public void testBalancingModeClear() { var testName = "[if replica on clear broker is adding/removing/future, raise an exception]"; var adding = ClusterInfo.builder(cluster) - .mapLog(r -> r.broker().id() != 0 ? r : Replica.builder(r).isAdding(true).build()) + .mapLog(r -> r.brokerId() != 0 ? r : Replica.builder(r).isAdding(true).build()) .build(); var removing = ClusterInfo.builder(cluster) - .mapLog(r -> r.broker().id() != 0 ? r : Replica.builder(r).isRemoving(true).build()) + .mapLog(r -> r.brokerId() != 0 ? r : Replica.builder(r).isRemoving(true).build()) .build(); var future = ClusterInfo.builder(cluster) - .mapLog(r -> r.broker().id() != 0 ? r : Replica.builder(r).isFuture(true).build()) + .mapLog(r -> r.brokerId() != 0 ? r : Replica.builder(r).isFuture(true).build()) .build(); for (var cc : List.of(adding, removing, future)) { Assertions.assertThrows( @@ -564,7 +564,7 @@ static void assertOnlyAllowedBrokerMovement( source .replicaStream() // for those replicas that are not allowed to move - .filter(r -> !allowedBroker.test(r.broker().id())) + .filter(r -> !allowedBroker.test(r.brokerId())) // they should exist as-is in the target allocation .forEach( fixedReplica -> { @@ -588,7 +588,7 @@ static void assertBrokerEmpty(ClusterInfo target, Predicate clearBroker var violated = target .replicaStream() - .filter(i -> clearBroker.test(i.broker().id())) + .filter(i -> clearBroker.test(i.brokerId())) .collect(Collectors.toUnmodifiableSet()); Assertions.assertTrue( violated.isEmpty(), diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java index c47b76b8a2..4ddb5805d9 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerUtilsTest.java @@ -102,9 +102,9 @@ void testVerifyClearBrokerValidness() { var iter = Stream.of(1, 2, 3).map(base::node).iterator(); var cluster = ClusterInfo.builder(base) - .addTopic("A", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) - .addTopic("B", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) - .addTopic("C", 1, (short) 1, r -> Replica.builder(r).broker(iter.next()).build()) + .addTopic("A", 1, (short) 1, r -> Replica.builder(r).brokerId(iter.next().id()).build()) + .addTopic("B", 1, (short) 1, r -> Replica.builder(r).brokerId(iter.next().id()).build()) + .addTopic("C", 1, (short) 1, r -> Replica.builder(r).brokerId(iter.next().id()).build()) .build(); var hasAdding = @@ -155,13 +155,13 @@ void testClearedCluster() { cluster, id -> id == 1 || id == 2, id -> id == 3 || id == 4)); Assertions.assertEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 1).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.brokerId() == 1).toList()); Assertions.assertEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 2).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.brokerId() == 2).toList()); Assertions.assertNotEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 3).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.brokerId() == 3).toList()); Assertions.assertNotEquals( - List.of(), clearedCluster.replicas().stream().filter(x -> x.broker().id() == 4).toList()); + List.of(), clearedCluster.replicas().stream().filter(x -> x.brokerId() == 4).toList()); var sameCluster = Assertions.assertDoesNotThrow( @@ -175,17 +175,17 @@ void testClearedCluster() { Assertions.assertDoesNotThrow( () -> BalancerUtils.clearedCluster(cluster, id -> id == 1, id -> id == 3)); Assertions.assertEquals( - 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 1).count(), "Clear"); + 0, aCluster.replicas().stream().filter(r -> r.brokerId() == 1).count(), "Clear"); Assertions.assertEquals( 100, - aCluster.replicas().stream().filter(r -> r.broker().id() == 2).count(), + aCluster.replicas().stream().filter(r -> r.brokerId() == 2).count(), "Not allowed or cleared"); Assertions.assertEquals( 100, - aCluster.replicas().stream().filter(r -> r.broker().id() == 3).count(), + aCluster.replicas().stream().filter(r -> r.brokerId() == 3).count(), "Accept replicas from cleared broker"); Assertions.assertEquals( - 0, aCluster.replicas().stream().filter(r -> r.broker().id() == 4).count(), "Not allowed"); + 0, aCluster.replicas().stream().filter(r -> r.brokerId() == 4).count(), "Not allowed"); } @Test diff --git a/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java b/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java index 8331e68f5d..460888447c 100644 --- a/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java +++ b/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java @@ -106,7 +106,7 @@ public static ClusterInfo of( Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .broker(nodes.get(r)) + .brokerId(nodes.get(r).id()) .lag(0) .size(-1) .isLeader(r == 0) @@ -118,7 +118,7 @@ public static ClusterInfo of( dataDirectoryList.get( tp.partition() % dataDirectories.size())) .build())) - .collect(Collectors.toUnmodifiableList()); + .toList(); return ClusterInfo.of("fake", List.copyOf(nodes), Map.of(), replicas); } diff --git a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java index 59afe2d50b..e0687ab5c0 100644 --- a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java +++ b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java @@ -27,7 +27,6 @@ import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; -import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.ClusterInfoTest; import org.astraea.common.admin.Replica; @@ -78,7 +77,7 @@ void testAsyncRun() { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .broker(Broker.of(broker0, "", -1)) + .brokerId(broker0) .lag(0) .size(0) .isLeader(true) @@ -91,7 +90,7 @@ void testAsyncRun() { Replica.builder() .topic(tp.topic()) .partition(tp.partition()) - .broker(Broker.of(broker1, "", -1)) + .brokerId(broker1) .lag(0) .size(0) .isLeader(false) @@ -163,7 +162,7 @@ void testDisableDataDirMigration() { replica -> Replica.builder(replica) .path( - source.brokerFolders().get(replica.broker().id()).stream() + source.brokerFolders().get(replica.brokerId()).stream() .filter(p -> !replica.path().equals(p)) .findAny() .orElseThrow()) diff --git a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java index b2b0e6ec64..cf3b43e84a 100644 --- a/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/tweakers/ShuffleTweakerTest.java @@ -167,7 +167,7 @@ void testEligiblePartition() { Replica.builder() .topic("topic") .partition(0) - .broker(nodeA) + .brokerId(nodeA.id()) .lag(0) .size(0) .isLeader(false) @@ -185,8 +185,8 @@ void testEligiblePartition() { .isLeader(true) .isPreferredLeader(true) .build(), - Replica.builder(base).topic("normal-topic").broker(nodeB).build(), - Replica.builder(base).topic("normal-topic").broker(nodeC).build(), + Replica.builder(base).topic("normal-topic").brokerId(nodeB.id()).build(), + Replica.builder(base).topic("normal-topic").brokerId(nodeC.id()).build(), Replica.builder(base) .topic("offline-single") .isPreferredLeader(true) @@ -195,10 +195,10 @@ void testEligiblePartition() { Replica.builder(base) .topic("no-leader") .isPreferredLeader(true) - .broker(nodeA) + .brokerId(nodeA.id()) .build(), - Replica.builder(base).topic("no-leader").broker(nodeB).build(), - Replica.builder(base).topic("no-leader").broker(nodeC).build())); + Replica.builder(base).topic("no-leader").brokerId(nodeB.id()).build(), + Replica.builder(base).topic("no-leader").brokerId(nodeC.id()).build())); shuffleTweaker .generate(allocation) .limit(30) diff --git a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java index 76515ddf58..773f24f955 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java @@ -53,42 +53,42 @@ void testMoveCosts() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(dataSize.bytes()) .path("/path0") .build()); @@ -97,42 +97,42 @@ void testMoveCosts() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(dataSize.bytes()) .path("/path1") .build(), Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(dataSize.bytes()) .path("/path1") .build(), Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(dataSize.bytes()) .path("/path0") .build(), Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(dataSize.bytes()) .path("/path0") .build()); @@ -232,17 +232,17 @@ public static ClusterInfo of(List replicas) { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::broker) + .map(Replica::brokerId) .distinct() .map( - broker -> + brokerId -> new Broker( - broker.id(), + brokerId, "", - broker.port(), + 2222, false, Config.EMPTY, - dataPath.get(broker.id()), + dataPath.get(brokerId), Set.of(), Set.of())) .collect(Collectors.toList()), @@ -279,7 +279,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(100) .isLeader(true) .path("/path0") @@ -287,7 +287,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(99) .isLeader(false) .path("/path0") @@ -295,7 +295,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(500) .isLeader(true) .path("/path0") @@ -303,7 +303,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(499) .isLeader(false) .path("/path0") @@ -311,7 +311,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(1000) .isLeader(true) .path("/path0") @@ -319,7 +319,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(1000) .isLeader(false) .path("/path0") @@ -327,17 +327,17 @@ private static ClusterInfo beforeClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::broker) + .map(Replica::brokerId) .distinct() .map( - broker -> + brokerId -> new Broker( - broker.id(), + brokerId, "", - broker.port(), + 2222, false, Config.EMPTY, - dataPath.get(broker.id()), + dataPath.get(brokerId), Set.of(), Set.of())) .collect(Collectors.toList()), @@ -360,7 +360,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(100) .isLeader(true) .path("/path1") @@ -368,7 +368,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(99) .isLeader(false) .path("/path0") @@ -376,7 +376,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(500) .isLeader(true) .path("/path0") @@ -384,7 +384,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(500) .isLeader(false) .path("/path1") @@ -392,7 +392,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(1000) .isLeader(true) .path("/path1") @@ -400,7 +400,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(1000) .isLeader(false) .path("/path0") @@ -408,17 +408,17 @@ private static ClusterInfo afterClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::broker) + .map(Replica::brokerId) .distinct() .map( - broker -> + brokerId -> new Broker( - broker.id(), + brokerId, "", - broker.port(), + 2222, false, Config.EMPTY, - dataPath.get(broker.id()), + dataPath.get(brokerId), Set.of(), Set.of())) .collect(Collectors.toList()), diff --git a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java index 6cac4f497d..2e5a12bc37 100644 --- a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java +++ b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java @@ -88,7 +88,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(100) .isLeader(true) .path("/path0") @@ -96,7 +96,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(99) .isLeader(false) .path("/path0") @@ -104,7 +104,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(500) .isLeader(true) .path("/path0") @@ -112,7 +112,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(499) .isLeader(false) .path("/path0") @@ -120,7 +120,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(1000) .isLeader(true) .path("/path0") @@ -128,7 +128,7 @@ private static ClusterInfo beforeClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(1000) .isLeader(false) .path("/path0") @@ -136,17 +136,17 @@ private static ClusterInfo beforeClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::broker) + .map(Replica::brokerId) .distinct() .map( - broker -> + brokerId -> new Broker( - broker.id(), + brokerId, "", - broker.port(), + 22222, false, Config.EMPTY, - dataPath.get(broker.id()), + dataPath.get(brokerId), Set.of(), Set.of())) .collect(Collectors.toList()), @@ -171,7 +171,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(100) .isLeader(true) .path("/path1") @@ -179,7 +179,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(99) .isLeader(false) .path("/path0") @@ -187,7 +187,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(500) .isLeader(true) .path("/path0") @@ -195,7 +195,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .size(500) .isLeader(false) .path("/path1") @@ -203,7 +203,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .size(1000) .isLeader(true) .path("/path1") @@ -211,7 +211,7 @@ private static ClusterInfo afterClusterInfo() { Replica.builder() .topic("topic1") .partition(2) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .size(1000) .isLeader(false) .path("/path0") @@ -219,17 +219,17 @@ private static ClusterInfo afterClusterInfo() { return ClusterInfo.of( "fake", replicas.stream() - .map(Replica::broker) + .map(Replica::brokerId) .distinct() .map( - broker -> + brokerId -> new Broker( - broker.id(), + brokerId, "", - broker.port(), + 22222, false, Config.EMPTY, - dataPath.get(broker.id()), + dataPath.get(brokerId), Set.of(), Set.of())) .collect(Collectors.toList()), diff --git a/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java b/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java index c7ccc33aa0..c900ad1eff 100644 --- a/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/MigrationCostTest.java @@ -21,7 +21,6 @@ import static org.astraea.common.cost.MigrationCost.replicaNumChanged; import java.util.List; -import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfoTest; import org.astraea.common.admin.Replica; import org.junit.jupiter.api.Assertions; @@ -36,7 +35,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .lag(-1) .size(-1) .isLeader(true) @@ -49,7 +48,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .lag(-1) .size(-1) .isLeader(false) @@ -62,7 +61,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .lag(-1) .size(-1) .isLeader(true) @@ -75,7 +74,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .lag(-1) .size(-1) .isLeader(false) @@ -90,7 +89,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .lag(-1) .size(-1) .isLeader(true) @@ -103,7 +102,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .lag(-1) .size(-1) .isLeader(false) @@ -116,7 +115,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .lag(-1) .size(-1) .isLeader(true) @@ -129,7 +128,7 @@ void testChangedReplicaLeaderNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .lag(-1) .size(-1) .isLeader(false) @@ -176,7 +175,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .lag(-1) .size(-1) .isLeader(true) @@ -189,7 +188,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .lag(-1) .size(-1) .isLeader(false) @@ -202,7 +201,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .lag(-1) .size(-1) .isLeader(true) @@ -215,7 +214,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .lag(-1) .size(-1) .isLeader(false) @@ -230,7 +229,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .lag(-1) .size(-1) .isLeader(true) @@ -243,7 +242,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(0) - .broker(Broker.of(1, "broker0", 1111)) + .brokerId(1) .lag(-1) .size(-1) .isLeader(false) @@ -256,7 +255,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(0, "broker0", 1111)) + .brokerId(0) .lag(-1) .size(-1) .isLeader(true) @@ -269,7 +268,7 @@ void testChangedReplicaNumber() { Replica.builder() .topic("topic1") .partition(1) - .broker(Broker.of(2, "broker0", 1111)) + .brokerId(2) .lag(-1) .size(-1) .isLeader(false) diff --git a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java index 80544becc8..98d7d61404 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java @@ -184,7 +184,7 @@ void testReplicationAware() { (short) 2, (r) -> Replica.builder(r) - .broker(base.node(iter1.next())) + .brokerId(base.node(iter1.next()).id()) .isPreferredLeader(iter2.next()) .isLeader(iter3.next()) .size(1) @@ -195,7 +195,7 @@ void testReplicationAware() { (short) 2, (r) -> Replica.builder(r) - .broker(base.node(iter4.next())) + .brokerId(base.node(iter4.next()).id()) .isPreferredLeader(iter5.next()) .isLeader(iter6.next()) .size(1) diff --git a/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java b/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java index 6d099b062d..5c43a8a12b 100644 --- a/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java @@ -40,21 +40,21 @@ public class RecordSizeCostTest { Replica.builder() .topic("topic") .partition(0) - .broker(Broker.of(1, "aa", 22)) + .brokerId(1) .size(100) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(0) - .broker(Broker.of(2, "aa", 22)) + .brokerId(2) .size(99) .path("/tmp/aa") .buildInSyncFollower(), Replica.builder() .topic("topic") .partition(1) - .broker(Broker.of(1, "aa", 22)) + .brokerId(1) .size(11) .path("/tmp/aa") .buildLeader())); diff --git a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java index caa18ea1e7..c9ebcb7e38 100644 --- a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderCostTest.java @@ -42,16 +42,28 @@ void testLeaderCount() { var sourceCluster = ClusterInfo.builder(baseCluster) .addTopic( - "topic1", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(1)).build()) + "topic1", + 3, + (short) 1, + r -> Replica.builder(r).brokerId(baseCluster.node(1).id()).build()) .addTopic( - "topic2", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(2)).build()) + "topic2", + 3, + (short) 1, + r -> Replica.builder(r).brokerId(baseCluster.node(2).id()).build()) .build(); var overFlowTargetCluster = ClusterInfo.builder(baseCluster) .addTopic( - "topic1", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(2)).build()) + "topic1", + 3, + (short) 1, + r -> Replica.builder(r).brokerId(baseCluster.node(2).id()).build()) .addTopic( - "topic2", 3, (short) 1, r -> Replica.builder(r).broker(baseCluster.node(1)).build()) + "topic2", + 3, + (short) 1, + r -> Replica.builder(r).brokerId(baseCluster.node(1).id()).build()) .build(); var overFlowMoveCost = @@ -76,7 +88,7 @@ void testNoMetrics() { .topic("topic") .partition(0) .isLeader(true) - .broker(Broker.of(10, "broker0", 1111)) + .brokerId(10) .path("/tmp/aa") .buildLeader(), Replica.builder() @@ -84,14 +96,14 @@ void testNoMetrics() { .partition(1) .isLeader(true) .partition(0) - .broker(Broker.of(10, "broker0", 1111)) + .brokerId(10) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(0) .isLeader(true) - .broker(Broker.of(11, "broker1", 1111)) + .brokerId(11) .path("/tmp/aa") .buildLeader()); var clusterInfo = diff --git a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java index d78ae5ad12..641618b0e1 100644 --- a/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/ReplicaLeaderSizeCostTest.java @@ -70,32 +70,14 @@ void testPartitionCost() { private ClusterInfo clusterInfo() { var replicas = List.of( - Replica.builder() - .topic("t") - .partition(10) - .isLeader(true) - .broker(Broker.of(0, "", -1)) - .size(777) - .build(), - Replica.builder() - .topic("t") - .partition(11) - .isLeader(true) - .broker(Broker.of(1, "", -1)) - .size(700) - .build(), - Replica.builder() - .topic("t") - .partition(12) - .isLeader(true) - .broker(Broker.of(2, "", -1)) - .size(500) - .build(), + Replica.builder().topic("t").partition(10).isLeader(true).brokerId(0).size(777).build(), + Replica.builder().topic("t").partition(11).isLeader(true).brokerId(1).size(700).build(), + Replica.builder().topic("t").partition(12).isLeader(true).brokerId(2).size(500).build(), Replica.builder() .topic("t") .partition(12) .isLeader(false) - .broker(Broker.of(0, "", -1)) + .brokerId(0) .size(499) .build()); return ClusterInfo.of( diff --git a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java index 94fbcc9b4a..bf3aa187a7 100644 --- a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java +++ b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java @@ -101,7 +101,7 @@ void testClusterInfoSensor() { TopicPartition.of(topic, 1), TopicPartition.of(topic, 2)))); // compare broker id - Assertions.assertTrue(info.replicaStream().allMatch(r -> r.broker().id() == aBroker.id())); + Assertions.assertTrue(info.replicaStream().allMatch(r -> r.brokerId() == aBroker.id())); // compare replica size var realCluster = admin.clusterInfo(Set.of(topic)).toCompletableFuture().join(); Assertions.assertTrue( diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java index cef2691d5b..7132d20adb 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerPerfTest.java @@ -66,19 +66,19 @@ void test() { Replica.builder() .topic("topic") .partition(0) - .broker(node0) + .brokerId(node0.id()) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(1) - .broker(node1) + .brokerId(node1.id()) .path("/tmp/aa") .buildLeader(), Replica.builder() .topic("topic") .partition(2) - .broker(node2) + .brokerId(node2.id()) .path("/tmp/aa") .buildLeader())); var admin = Mockito.mock(Admin.class); diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 8809be5932..524ca0987f 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -116,7 +116,7 @@ void testSingleBroker() { .topic("topic") .partition(10) .path("/tmp/aa") - .broker(nodeInfo) + .brokerId(nodeInfo.id()) .buildLeader(); try (var partitioner = new StrictCostPartitioner()) { partitioner.configure(Configuration.EMPTY); @@ -138,19 +138,9 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { @Test void testCostFunctionWithoutSensor() { var replicaInfo0 = - Replica.builder() - .topic("topic") - .partition(0) - .path("/tmp/aa") - .broker(Broker.of(10, "host", 11111)) - .buildLeader(); + Replica.builder().topic("topic").partition(0).path("/tmp/aa").brokerId(10).buildLeader(); var replicaInfo1 = - Replica.builder() - .topic("topic") - .partition(0) - .path("/tmp/aa") - .broker(Broker.of(12, "host2", 11111)) - .buildLeader(); + Replica.builder().topic("topic").partition(0).path("/tmp/aa").brokerId(12).buildLeader(); try (var partitioner = new StrictCostPartitioner()) { partitioner.configure( new Configuration( @@ -196,14 +186,14 @@ void testReturnedPartition() { .topic("topic") .partition(partitionId) .path("/tmp/aa") - .broker(Broker.of(brokerId, "host", 11111)) + .brokerId(brokerId) .buildLeader(); var replicaInfo1 = Replica.builder() .topic("topic") .partition(1) .path("/tmp/aa") - .broker(Broker.of(1111, "host2", 11111)) + .brokerId(1111) .buildLeader(); Assertions.assertEquals( partitionId, diff --git a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java index 563172af6e..af53d4d052 100644 --- a/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/health/BalancerNode.java @@ -124,21 +124,20 @@ static List> assignmentResult(Balancer.Plan plan) { previousAssignments.stream() .filter(Replica::isLeader) .findFirst() - .ifPresent( - r -> result.put(PREVIOUS_LEADER_KEY, r.broker().id() + ":" + r.path())); + .ifPresent(r -> result.put(PREVIOUS_LEADER_KEY, r.brokerId() + ":" + r.path())); newAssignments.stream() .filter(Replica::isLeader) .findFirst() - .ifPresent(r -> result.put(NEW_LEADER_KEY, r.broker().id() + ":" + r.path())); + .ifPresent(r -> result.put(NEW_LEADER_KEY, r.brokerId() + ":" + r.path())); var previousFollowers = previousAssignments.stream() .filter(r -> !r.isLeader()) - .map(r -> r.broker().id() + ":" + r.path()) + .map(r -> r.brokerId() + ":" + r.path()) .collect(Collectors.joining(",")); var newFollowers = newAssignments.stream() .filter(r -> !r.isLeader()) - .map(r -> r.broker().id() + ":" + r.path()) + .map(r -> r.brokerId() + ":" + r.path()) .collect(Collectors.joining(",")); if (!previousFollowers.isBlank()) result.put(PREVIOUS_FOLLOWER_KEY, previousFollowers); diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java index ff0026948b..e60a46546d 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java @@ -64,7 +64,7 @@ static List> allResult(ClusterInfo clusterInfo) { result.put(TOPIC_NAME_KEY, replica.topic()); result.put(PARTITION_KEY, replica.partition()); result.put("internal", replica.isInternal()); - result.put("broker", replica.broker().id()); + result.put("broker", replica.brokerId()); if (replica.path() != null) result.put(PATH_KEY, replica.path()); result.put("isLeader", replica.isLeader()); result.put("isPreferredLeader", replica.isPreferredLeader()); @@ -195,7 +195,7 @@ static List> allResult(ClusterInfo clusterInfo) { .allMatch( entry -> rs.replicas(entry.getKey()).stream() - .map(r -> r.broker().id()) + .map(r -> r.brokerId()) .collect(Collectors.toSet()) .containsAll(entry.getValue())), Duration.ofSeconds(10), diff --git a/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java b/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java index b9de226662..bdaaa07ec7 100644 --- a/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java +++ b/gui/src/test/java/org/astraea/gui/tab/health/BalancerNodeTest.java @@ -113,7 +113,7 @@ void testResult() { .isPreferredLeader(false) .topic(topic) .partition(0) - .broker(allNodes.get(0)) + .brokerId(allNodes.get(0).id()) .size(leaderSize) .path("/tmp/aaa") .build(), @@ -122,7 +122,7 @@ void testResult() { .isPreferredLeader(true) .topic(topic) .partition(0) - .broker(allNodes.get(1)) + .brokerId(allNodes.get(1).id()) .size(leaderSize) .path("/tmp/bbb") .build()); @@ -133,7 +133,7 @@ void testResult() { .isPreferredLeader(false) .topic(topic) .partition(0) - .broker(allNodes.get(2)) + .brokerId(allNodes.get(2).id()) .size(leaderSize) .path("/tmp/ddd") .build(), @@ -142,7 +142,7 @@ void testResult() { .isPreferredLeader(true) .topic(topic) .partition(0) - .broker(allNodes.get(1)) + .brokerId(allNodes.get(1).id()) .size(leaderSize) .path("/tmp/bbb") .build()); diff --git a/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java b/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java index 0530ff0b77..53d4226acf 100644 --- a/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java +++ b/gui/src/test/java/org/astraea/gui/tab/topic/ReplicaNodeTest.java @@ -124,8 +124,7 @@ void testTableAction() { .join() .replicas() .get(0) - .broker() - .id()); + .brokerId()); Assertions.assertEquals( path, admin @@ -150,7 +149,7 @@ void testResult() { .isLeader(true) .topic(topic) .partition(partition) - .broker(nodes.get(0)) + .brokerId(nodes.get(0).id()) .size(leaderSize) .path("/tmp/aaa") .build(), @@ -158,14 +157,14 @@ void testResult() { .isLeader(false) .topic(topic) .partition(partition) - .broker(nodes.get(1)) + .brokerId(nodes.get(1).id()) .size(20) .build(), Replica.builder() .isLeader(false) .topic(topic) .partition(partition) - .broker(nodes.get(2)) + .brokerId(nodes.get(2).id()) .size(30) .build()); var results = ReplicaNode.allResult(ClusterInfo.of("fake", nodes, Map.of(), replicas)); From f6ea4ed335463d933c0c44cafd67811461d33be5 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 30 May 2023 11:19:53 +0800 Subject: [PATCH 43/77] [COMMON] refine Broker structure to avoid duplicate large collection (#1792) --- .../org/astraea/app/web/BrokerHandler.java | 6 +- .../astraea/app/web/ReassignmentHandler.java | 9 +- .../java/org/astraea/common/ByteUtils.java | 50 +++++------ .../java/org/astraea/common/admin/Admin.java | 10 +-- .../org/astraea/common/admin/AdminImpl.java | 6 +- .../java/org/astraea/common/admin/Broker.java | 79 ++++------------ .../org/astraea/common/admin/ClusterInfo.java | 8 +- .../common/admin/ClusterInfoBuilder.java | 22 ++--- .../common/admin/TopicPartitionPath.java | 24 +++++ .../common/generated/admin/Broker.proto | 14 +-- .../org/astraea/common/admin/AdminTest.java | 44 +-------- .../admin/AdminWithOfflineBrokerTest.java | 9 +- .../common/admin/ClusterInfoBuilderTest.java | 2 +- .../astraea/common/admin/ClusterInfoTest.java | 9 +- .../common/balancer/FakeClusterInfo.java | 7 +- .../common/cost/BrokerDiskSpaceCostTest.java | 57 ++---------- .../astraea/common/cost/CostUtilsTest.java | 28 +----- .../java/org/astraea/gui/tab/BrokerNode.java | 90 +++++++++---------- .../org/astraea/gui/tab/topic/TopicNode.java | 9 +- 19 files changed, 158 insertions(+), 325 deletions(-) create mode 100644 common/src/main/java/org/astraea/common/admin/TopicPartitionPath.java diff --git a/app/src/main/java/org/astraea/app/web/BrokerHandler.java b/app/src/main/java/org/astraea/app/web/BrokerHandler.java index a59ad5536a..e735df1831 100644 --- a/app/src/main/java/org/astraea/app/web/BrokerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BrokerHandler.java @@ -26,7 +26,7 @@ import java.util.stream.Collectors; import org.astraea.common.admin.Admin; import org.astraea.common.admin.Broker; -import org.astraea.common.admin.TopicPartition; +import org.astraea.common.admin.TopicPartitionPath; class BrokerHandler implements Handler { @@ -95,8 +95,8 @@ static class Broker implements Response { Broker(org.astraea.common.admin.Broker broker) { this.id = broker.id(); this.topics = - broker.topicPartitions().stream() - .collect(Collectors.groupingBy(TopicPartition::topic)) + broker.topicPartitionPaths().stream() + .collect(Collectors.groupingBy(TopicPartitionPath::topic)) .entrySet() .stream() .map(e -> new Topic(e.getKey(), e.getValue().size())) diff --git a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java index cfd84680e6..af06031a2c 100644 --- a/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java +++ b/app/src/main/java/org/astraea/app/web/ReassignmentHandler.java @@ -35,6 +35,7 @@ import org.astraea.common.admin.ClusterInfo; import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; +import org.astraea.common.admin.TopicPartitionPath; import org.astraea.common.admin.TopicPartitionReplica; import org.astraea.common.json.TypeRef; @@ -92,7 +93,7 @@ public CompletionStage post(Channel channel) { var availableBrokers = brokers.stream().filter(b -> b.id() != exclude).toList(); var partitions = - excludedBroker.get().topicPartitions().stream() + excludedBroker.get().topicPartitionPaths().stream() .filter( tp -> excludeNode.topic.isEmpty() @@ -104,12 +105,14 @@ public CompletionStage post(Channel channel) { partitions.stream() .collect( Collectors.toMap( - tp -> tp, + TopicPartitionPath::topicPartition, tp -> { var ids = availableBrokers.stream() .filter( - b -> b.topicPartitions().contains(tp)) + b -> + b.topicPartitionPaths() + .contains(tp)) .map(Broker::id) .toList(); if (!ids.isEmpty()) return ids; diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index a97e93f14e..b6d2509a34 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -33,6 +33,7 @@ import org.astraea.common.admin.Replica; import org.astraea.common.admin.Topic; import org.astraea.common.admin.TopicPartition; +import org.astraea.common.admin.TopicPartitionPath; import org.astraea.common.generated.BeanObjectOuterClass; import org.astraea.common.generated.PrimitiveOuterClass; import org.astraea.common.generated.admin.BrokerOuterClass; @@ -305,15 +306,13 @@ public static ClusterInfo readClusterInfo(byte[] bytes) { // ---------------------------Serialize To ProtoBuf Outer Class------------------------------- // - private static BrokerOuterClass.Broker.DataFolder toOuterClass(Broker.DataFolder dataFolder) { - return BrokerOuterClass.Broker.DataFolder.newBuilder() - .setPath(dataFolder.path()) - .putAllPartitionSizes( - dataFolder.partitionSizes().entrySet().stream() - .collect(Collectors.toMap(entry -> entry.getKey().toString(), Map.Entry::getValue))) - .putAllOrphanPartitionSizes( - dataFolder.orphanPartitionSizes().entrySet().stream() - .collect(Collectors.toMap(entry -> entry.getKey().toString(), Map.Entry::getValue))) + private static BrokerOuterClass.Broker.TopicPartitionPath toOuterClass( + TopicPartitionPath topicPartitionPath) { + return BrokerOuterClass.Broker.TopicPartitionPath.newBuilder() + .setTopic(topicPartitionPath.topic()) + .setPartition(topicPartitionPath.partition()) + .setSize(topicPartitionPath.size()) + .setPath(topicPartitionPath.path()) .build(); } @@ -332,11 +331,9 @@ private static BrokerOuterClass.Broker toOuterClass(Broker broker) { .setPort(broker.port()) .setIsController(broker.isController()) .putAllConfig(broker.config().raw()) - .addAllDataFolder(broker.dataFolders().stream().map(ByteUtils::toOuterClass).toList()) - .addAllTopicPartitions( - broker.topicPartitions().stream().map(ByteUtils::toOuterClass).toList()) - .addAllTopicPartitionLeaders( - broker.topicPartitionLeaders().stream().map(ByteUtils::toOuterClass).toList()) + .addAllDataFolders(broker.dataFolders()) + .addAllTopicPartitionPaths( + broker.topicPartitionPaths().stream().map(ByteUtils::toOuterClass).toList()) .build(); } @@ -370,15 +367,13 @@ private static ReplicaOuterClass.Replica toOuterClass(Replica replica) { // -------------------------Deserialize From ProtoBuf Outer Class----------------------------- // - private static Broker.DataFolder toDataFolder(BrokerOuterClass.Broker.DataFolder dataFolder) { - return new Broker.DataFolder( - dataFolder.getPath(), - dataFolder.getPartitionSizesMap().entrySet().stream() - .collect( - Collectors.toMap(entry -> TopicPartition.of(entry.getKey()), Map.Entry::getValue)), - dataFolder.getOrphanPartitionSizesMap().entrySet().stream() - .collect( - Collectors.toMap(entry -> TopicPartition.of(entry.getKey()), Map.Entry::getValue))); + private static TopicPartitionPath toTopicPartitionPath( + BrokerOuterClass.Broker.TopicPartitionPath partitionPath) { + return new TopicPartitionPath( + partitionPath.getTopic(), + partitionPath.getPartition(), + partitionPath.getSize(), + partitionPath.getPath()); } private static TopicPartition toTopicPartition( @@ -393,13 +388,8 @@ private static Broker toBroker(BrokerOuterClass.Broker broker) { broker.getPort(), broker.getIsController(), new Config(broker.getConfigMap()), - broker.getDataFolderList().stream().map(ByteUtils::toDataFolder).toList(), - broker.getTopicPartitionsList().stream() - .map(ByteUtils::toTopicPartition) - .collect(Collectors.toSet()), - broker.getTopicPartitionLeadersList().stream() - .map(ByteUtils::toTopicPartition) - .collect(Collectors.toSet())); + Set.copyOf(broker.getDataFoldersList()), + broker.getTopicPartitionPathsList().stream().map(ByteUtils::toTopicPartitionPath).toList()); } private static Topic toTopic(TopicOuterClass.Topic topic) { diff --git a/common/src/main/java/org/astraea/common/admin/Admin.java b/common/src/main/java/org/astraea/common/admin/Admin.java index d6c7068ae2..71b160c313 100644 --- a/common/src/main/java/org/astraea/common/admin/Admin.java +++ b/common/src/main/java/org/astraea/common/admin/Admin.java @@ -178,15 +178,7 @@ default CompletionStage bootstrapServers() { default CompletionStage>> brokerFolders() { return brokers() .thenApply( - brokers -> - brokers.stream() - .collect( - Collectors.toMap( - Broker::id, - n -> - n.dataFolders().stream() - .map(Broker.DataFolder::path) - .collect(Collectors.toSet())))); + brokers -> brokers.stream().collect(Collectors.toMap(Broker::id, Broker::dataFolders))); } CompletionStage> consumerGroupIds(); diff --git a/common/src/main/java/org/astraea/common/admin/AdminImpl.java b/common/src/main/java/org/astraea/common/admin/AdminImpl.java index a29d57f56a..e6f858868b 100644 --- a/common/src/main/java/org/astraea/common/admin/AdminImpl.java +++ b/common/src/main/java/org/astraea/common/admin/AdminImpl.java @@ -455,7 +455,6 @@ private CompletionStage>> clusterIdAndBrokers() { return FutureUtils.combine( to(cluster.clusterId()), to(cluster.controller()), - topicNames(true).thenCompose(names -> to(kafkaAdmin.describeTopics(names).allTopicNames())), nodeFuture.thenCompose( nodes -> to( @@ -472,7 +471,7 @@ private CompletionStage>> clusterIdAndBrokers() { ConfigResource.Type.BROKER, String.valueOf(n.id()))) .collect(Collectors.toList()))), nodeFuture, - (id, controller, topics, logDirs, configs, nodes) -> + (id, controller, logDirs, configs, nodes) -> Map.entry( id, nodes.stream() @@ -482,8 +481,7 @@ private CompletionStage>> clusterIdAndBrokers() { node.id() == controller.id(), node, configs.get(String.valueOf(node.id())), - logDirs.get(node.id()), - topics.values())) + logDirs.get(node.id()))) .sorted(Comparator.comparing(Broker::id)) .collect(Collectors.toList()))); } diff --git a/common/src/main/java/org/astraea/common/admin/Broker.java b/common/src/main/java/org/astraea/common/admin/Broker.java index 3d2b042159..193df58fe7 100644 --- a/common/src/main/java/org/astraea/common/admin/Broker.java +++ b/common/src/main/java/org/astraea/common/admin/Broker.java @@ -16,11 +16,9 @@ */ package org.astraea.common.admin; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import org.apache.kafka.clients.admin.LogDirDescription; /** @@ -30,8 +28,6 @@ * @param isController * @param config config used by this node * @param dataFolders the disk folder used to stored data by this node - * @param topicPartitions - * @param topicPartitionLeaders partition leaders hosted by this broker */ public record Broker( int id, @@ -39,9 +35,8 @@ public record Broker( int port, boolean isController, Config config, - List dataFolders, - Set topicPartitions, - Set topicPartitionLeaders) { + Set dataFolders, + List topicPartitionPaths) { /** * @return true if the broker is offline. An offline node can't offer host or port information. @@ -51,7 +46,7 @@ public boolean offline() { } public static Broker of(int id, String host, int port) { - return new Broker(id, host, port, false, Config.EMPTY, List.of(), Set.of(), Set.of()); + return new Broker(id, host, port, false, Config.EMPTY, Set.of(), List.of()); } public static Broker of(org.apache.kafka.common.Node node) { @@ -62,66 +57,30 @@ static Broker of( boolean isController, org.apache.kafka.common.Node nodeInfo, Map configs, - Map dirs, - Collection topics) { + Map dirs) { var config = new Config(configs); - var partitionsFromTopicDesc = - topics.stream() - .flatMap( - t -> - t.partitions().stream() - .filter(p -> p.replicas().stream().anyMatch(n -> n.id() == nodeInfo.id())) - .map(p -> TopicPartition.of(t.name(), p.partition()))) - .collect(Collectors.toUnmodifiableSet()); - var folders = - dirs.entrySet().stream() - .map( - entry -> { - var path = entry.getKey(); - var allPartitionAndSize = - entry.getValue().replicaInfos().entrySet().stream() - .collect( - Collectors.toUnmodifiableMap( - e -> TopicPartition.from(e.getKey()), e -> e.getValue().size())); - var partitionSizes = - allPartitionAndSize.entrySet().stream() - .filter(tpAndSize -> partitionsFromTopicDesc.contains(tpAndSize.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - var orphanPartitionSizes = - allPartitionAndSize.entrySet().stream() - .filter( - tpAndSize -> !partitionsFromTopicDesc.contains(tpAndSize.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + var paths = Set.copyOf(dirs.keySet()); - return new DataFolder(path, partitionSizes, orphanPartitionSizes); - }) - .toList(); - var topicPartitionLeaders = - topics.stream() + var topicPartitionPaths = + dirs.entrySet().stream() .flatMap( - topic -> - topic.partitions().stream() - .filter(p -> p.leader() != null && p.leader().id() == nodeInfo.id()) - .map(p -> TopicPartition.of(topic.name(), p.partition()))) - .collect(Collectors.toUnmodifiableSet()); + entry -> + entry.getValue().replicaInfos().entrySet().stream() + .map( + tpEntry -> + new TopicPartitionPath( + tpEntry.getKey().topic(), + tpEntry.getKey().partition(), + tpEntry.getValue().size(), + entry.getKey()))) + .toList(); return new Broker( nodeInfo.id(), nodeInfo.host(), nodeInfo.port(), isController, config, - folders, - partitionsFromTopicDesc, - topicPartitionLeaders); + paths, + topicPartitionPaths); } - - /** - * @param path the path on the local disk - * @param partitionSizes topic partition hosed by this node and size of files - * @param orphanPartitionSizes topic partition located by this node but not traced by cluster - */ - public record DataFolder( - String path, - Map partitionSizes, - Map orphanPartitionSizes) {} } 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 33c7231608..54743a1f8f 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java @@ -319,13 +319,7 @@ default Broker node(int id) { */ default Map> brokerFolders() { return brokers().stream() - .collect( - Collectors.toUnmodifiableMap( - Broker::id, - node -> - node.dataFolders().stream() - .map(Broker.DataFolder::path) - .collect(Collectors.toUnmodifiableSet()))); + .collect(Collectors.toUnmodifiableMap(Broker::id, Broker::dataFolders)); } // ---------------------[streams methods]---------------------// diff --git a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java index f3d01f5a1c..58b2dcb4b8 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfoBuilder.java @@ -126,10 +126,8 @@ public ClusterInfoBuilder addFolders(Map> folders) { node.host(), node.port(), Stream.concat( - node.dataFolders().stream(), - folders.get(node.id()).stream() - .map(ClusterInfoBuilder::fakeDataFolder)) - .toList()); + node.dataFolders().stream(), folders.get(node.id()).stream()) + .collect(Collectors.toUnmodifiableSet())); else return node; }) .toList(); @@ -177,9 +175,7 @@ public ClusterInfoBuilder addTopic( Broker::id, node -> node.dataFolders().stream() - .collect( - Collectors.toMap( - Broker.DataFolder::path, x -> new AtomicInteger())))); + .collect(Collectors.toMap(t -> t, x -> new AtomicInteger())))); replicas.forEach( replica -> folderLogCounter.get(replica.brokerId()).get(replica.path()).incrementAndGet()); @@ -330,16 +326,10 @@ public ClusterInfo build() { private static Broker fakeNode(int brokerId) { var host = "fake-node-" + brokerId; var port = new Random(brokerId).nextInt(65535) + 1; - var folders = List.of(); - - return fakeBroker(brokerId, host, port, folders); - } - - static Broker fakeBroker(int Id, String host, int port, List dataFolders) { - return new Broker(Id, host, port, false, Config.EMPTY, dataFolders, Set.of(), Set.of()); + return fakeBroker(brokerId, host, port, Set.of()); } - private static Broker.DataFolder fakeDataFolder(String path) { - return new Broker.DataFolder(path, Map.of(), Map.of()); + static Broker fakeBroker(int Id, String host, int port, Set dataFolders) { + return new Broker(Id, host, port, false, Config.EMPTY, dataFolders, List.of()); } } diff --git a/common/src/main/java/org/astraea/common/admin/TopicPartitionPath.java b/common/src/main/java/org/astraea/common/admin/TopicPartitionPath.java new file mode 100644 index 0000000000..6d017f6f6d --- /dev/null +++ b/common/src/main/java/org/astraea/common/admin/TopicPartitionPath.java @@ -0,0 +1,24 @@ +/* + * 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.admin; + +public record TopicPartitionPath(String topic, int partition, long size, String path) { + + public TopicPartition topicPartition() { + return new TopicPartition(topic, partition); + } +} diff --git a/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto b/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto index cf463e193c..f08b1af7d6 100644 --- a/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto +++ b/common/src/main/proto/org/astraea/common/generated/admin/Broker.proto @@ -10,13 +10,13 @@ message Broker { int32 port = 3; bool isController = 4; map config = 5; - repeated DataFolder dataFolder = 6; - repeated TopicPartition topicPartitions = 7; - repeated TopicPartition topicPartitionLeaders = 8; + repeated string dataFolders = 6; + repeated TopicPartitionPath topicPartitionPaths = 7; - message DataFolder { - string path = 1; - map partitionSizes = 2; - map orphanPartitionSizes = 3; + message TopicPartitionPath { + string topic = 1; + int32 partition = 2; + int64 size = 3; + string path = 4; } } \ No newline at end of file diff --git a/common/src/test/java/org/astraea/common/admin/AdminTest.java b/common/src/test/java/org/astraea/common/admin/AdminTest.java index eeaccaa638..7aae77d860 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminTest.java @@ -1275,38 +1275,7 @@ void testBrokers() { Assertions.assertEquals(3, brokers.size()); brokers.forEach(broker -> Assertions.assertNotEquals(0, broker.config().raw().size())); Assertions.assertEquals(1, brokers.stream().filter(Broker::isController).count()); - brokers.forEach(broker -> Assertions.assertNotEquals(0, broker.topicPartitions().size())); - brokers.forEach( - broker -> Assertions.assertNotEquals(0, broker.topicPartitionLeaders().size())); - } - } - - @Test - void testBrokerFolders() { - try (var admin = Admin.of(SERVICE.bootstrapServers())) { - Assertions.assertEquals( - SERVICE.dataFolders().keySet().size(), - admin.brokers().toCompletableFuture().join().size()); - // list all - SERVICE - .dataFolders() - .forEach( - (id, ds) -> - Assertions.assertEquals( - admin.brokerFolders().toCompletableFuture().join().get(id).size(), - ds.size())); - - admin - .brokers() - .toCompletableFuture() - .join() - .forEach( - broker -> - Assertions.assertEquals( - broker.topicPartitions().size(), - broker.dataFolders().stream() - .mapToInt(e -> e.partitionSizes().size()) - .sum())); + brokers.forEach(broker -> Assertions.assertNotEquals(0, broker.topicPartitionPaths().size())); } } @@ -1523,14 +1492,9 @@ void testSizeOfNoDataTopic() { .join() .forEach( broker -> - broker - .dataFolders() - .forEach( - d -> - d.partitionSizes().entrySet().stream() - .filter(e -> e.getKey().topic().equals(topic)) - .map(Map.Entry::getValue) - .forEach(v -> Assertions.assertEquals(0, v)))); + broker.topicPartitionPaths().stream() + .filter(tp -> tp.topic().equals(topic)) + .forEach(d -> Assertions.assertEquals(0, d.size()))); } } diff --git a/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java b/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java index 2595b0e8d1..a66ef906ac 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminWithOfflineBrokerTest.java @@ -112,11 +112,7 @@ void testBrokers() { try (var admin = Admin.of(SERVICE.bootstrapServers())) { var brokers = admin.brokers().toCompletableFuture().join(); Assertions.assertEquals(2, brokers.size()); - brokers.forEach( - b -> - b.dataFolders() - .forEach(d -> Assertions.assertEquals(0, d.orphanPartitionSizes().size()))); - var offlineBrokers = brokers.stream().filter(Broker::offline).collect(Collectors.toList()); + var offlineBrokers = brokers.stream().filter(Broker::offline).toList(); Assertions.assertEquals(0, offlineBrokers.size()); } } @@ -127,8 +123,7 @@ void testPartitions() { try (var admin = Admin.of(SERVICE.bootstrapServers())) { var partitions = admin.partitions(Set.of(TOPIC_NAME)).toCompletableFuture().join(); Assertions.assertEquals(PARTITIONS, partitions.size()); - var offlinePartitions = - partitions.stream().filter(p -> p.leaderId().isEmpty()).collect(Collectors.toList()); + var offlinePartitions = partitions.stream().filter(p -> p.leaderId().isEmpty()).toList(); offlinePartitions.forEach( p -> { Assertions.assertEquals(1, p.replicas().size()); diff --git a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java index 85861e188b..44f020f011 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoBuilderTest.java @@ -332,7 +332,7 @@ void setPreferredLeader() { "300, host3, 3000", }) void testFakeBrokerInteraction(int id, String host, int port) { - var node0 = ClusterInfoBuilder.fakeBroker(id, host, port, List.of()); + var node0 = ClusterInfoBuilder.fakeBroker(id, host, port, Set.of()); var node1 = Broker.of(id, host, port); var node2 = Broker.of(id + 1, host, port); 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 716ab3b4fc..e50b4530ab 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterInfoTest.java @@ -51,14 +51,7 @@ public static ClusterInfo of(List replicas) { .map( r -> new Broker( - r.brokerId(), - "hpost", - 22222, - false, - Config.EMPTY, - List.of(), - Set.of(), - Set.of())) + r.brokerId(), "hpost", 22222, false, Config.EMPTY, Set.of(), List.of())) .collect(Collectors.groupingBy(Broker::id, Collectors.reducing((x, y) -> x))) .values() .stream() diff --git a/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java b/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java index 460888447c..0146c7076c 100644 --- a/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java +++ b/common/src/test/java/org/astraea/common/balancer/FakeClusterInfo.java @@ -84,11 +84,8 @@ public static ClusterInfo of( node.port(), false, new Config(Map.of()), - dataDirectories.stream() - .map(path -> new Broker.DataFolder(path, Map.of(), Map.of())) - .toList(), - Set.of(), - Set.of())) + dataDirectories, + List.of())) .toList(); final var dataDirectoryList = List.copyOf(dataDirectories); final var topics = topicNameGenerator.apply(topicCount); diff --git a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java index 773f24f955..c985268faf 100644 --- a/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/BrokerDiskSpaceCostTest.java @@ -219,16 +219,7 @@ void testBrokerPathDiskUsageSizeOverflow() { } public static ClusterInfo of(List replicas) { - var dataPath = - Map.of( - 0, - List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 1, - List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 2, - List.of( - new Broker.DataFolder("/path0", Map.of(), Map.of()), - new Broker.DataFolder("/path1", Map.of(), Map.of()))); + var dataPath = Map.of(0, Set.of("/path0"), 1, Set.of("/path0"), 2, Set.of("/path0", "/path1")); return ClusterInfo.of( "fake", replicas.stream() @@ -237,14 +228,7 @@ public static ClusterInfo of(List replicas) { .map( brokerId -> new Broker( - brokerId, - "", - 2222, - false, - Config.EMPTY, - dataPath.get(brokerId), - Set.of(), - Set.of())) + brokerId, "", 2222, false, Config.EMPTY, dataPath.get(brokerId), List.of())) .collect(Collectors.toList()), Map.of(), replicas); @@ -266,14 +250,7 @@ public static ClusterInfo of(List replicas) { */ private static ClusterInfo beforeClusterInfo() { - var dataPath = - Map.of( - 0, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 1, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 2, - List.of( - new Broker.DataFolder("/path0", Map.of(), Map.of()), - new Broker.DataFolder("/path1", Map.of(), Map.of()))); + var dataPath = Map.of(0, Set.of("/path0"), 1, Set.of("/path0"), 2, Set.of("/path0", "/path1")); var replicas = List.of( Replica.builder() @@ -332,29 +309,14 @@ private static ClusterInfo beforeClusterInfo() { .map( brokerId -> new Broker( - brokerId, - "", - 2222, - false, - Config.EMPTY, - dataPath.get(brokerId), - Set.of(), - Set.of())) + brokerId, "", 2222, false, Config.EMPTY, dataPath.get(brokerId), List.of())) .collect(Collectors.toList()), Map.of(), replicas); } private static ClusterInfo afterClusterInfo() { - - var dataPath = - Map.of( - 0, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 1, List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 2, - List.of( - new Broker.DataFolder("/path0", Map.of(), Map.of()), - new Broker.DataFolder("/path1", Map.of(), Map.of()))); + var dataPath = Map.of(0, Set.of("/path0"), 1, Set.of("/path0"), 2, Set.of("/path0", "/path1")); var replicas = List.of( Replica.builder() @@ -413,14 +375,7 @@ private static ClusterInfo afterClusterInfo() { .map( brokerId -> new Broker( - brokerId, - "", - 2222, - false, - Config.EMPTY, - dataPath.get(brokerId), - Set.of(), - Set.of())) + brokerId, "", 2222, false, Config.EMPTY, dataPath.get(brokerId), List.of())) .collect(Collectors.toList()), Map.of(), replicas); diff --git a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java index 2e5a12bc37..b3da81bc6f 100644 --- a/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java +++ b/common/src/test/java/org/astraea/common/cost/CostUtilsTest.java @@ -73,16 +73,7 @@ void testChangedRecordSizeOverflow() { */ private static ClusterInfo beforeClusterInfo() { - var dataPath = - Map.of( - 0, - List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 1, - List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 2, - List.of( - new Broker.DataFolder("/path0", Map.of(), Map.of()), - new Broker.DataFolder("/path1", Map.of(), Map.of()))); + var dataPath = Map.of(0, Set.of("/path0"), 1, Set.of("/path0"), 2, Set.of("/path0", "/path1")); var replicas = List.of( Replica.builder() @@ -147,8 +138,7 @@ private static ClusterInfo beforeClusterInfo() { false, Config.EMPTY, dataPath.get(brokerId), - Set.of(), - Set.of())) + List.of())) .collect(Collectors.toList()), Map.of(), replicas); @@ -156,16 +146,7 @@ private static ClusterInfo beforeClusterInfo() { private static ClusterInfo afterClusterInfo() { - var dataPath = - Map.of( - 0, - List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 1, - List.of(new Broker.DataFolder("/path0", Map.of(), Map.of())), - 2, - List.of( - new Broker.DataFolder("/path0", Map.of(), Map.of()), - new Broker.DataFolder("/path1", Map.of(), Map.of()))); + var dataPath = Map.of(0, Set.of("/path0"), 1, Set.of("/path0"), 2, Set.of("/path0", "/path1")); var replicas = List.of( Replica.builder() @@ -230,8 +211,7 @@ private static ClusterInfo afterClusterInfo() { false, Config.EMPTY, dataPath.get(brokerId), - Set.of(), - Set.of())) + List.of())) .collect(Collectors.toList()), Map.of(), replicas); diff --git a/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java b/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java index 67dc8930fb..c7ddce82f6 100644 --- a/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/BrokerNode.java @@ -35,10 +35,12 @@ import javafx.geometry.Side; import javafx.scene.Node; import org.astraea.common.DataSize; +import org.astraea.common.FutureUtils; import org.astraea.common.MapUtils; import org.astraea.common.admin.Broker; import org.astraea.common.admin.BrokerConfigs; -import org.astraea.common.admin.TopicPartition; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.TopicPartitionPath; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.broker.ControllerMetrics; import org.astraea.common.metrics.broker.HasGauge; @@ -257,7 +259,8 @@ static Node metricsNode(Context context) { return PaneBuilder.of().firstPart(firstPart).build(); } - private static List> basicResult(List brokers) { + private static List> basicResult( + List brokers, ClusterInfo clusterInfo) { return brokers.stream() .map( broker -> @@ -271,37 +274,32 @@ private static List> basicResult(List brokers) { "controller", broker.isController(), "topics", - broker.dataFolders().stream() - .flatMap( - d -> d.partitionSizes().keySet().stream().map(TopicPartition::topic)) + broker.topicPartitionPaths().stream() + .map(TopicPartitionPath::topic) .distinct() .count(), "partitions", - broker.dataFolders().stream() - .flatMap(d -> d.partitionSizes().keySet().stream()) + broker.topicPartitionPaths().stream() + .map(TopicPartitionPath::topicPartition) .distinct() .count(), "leaders", - broker.topicPartitionLeaders().size(), - "size", - DataSize.Byte.of( - broker.dataFolders().stream() - .mapToLong( - d -> d.partitionSizes().values().stream().mapToLong(v -> v).sum()) - .sum()), - "orphan partitions", - broker.dataFolders().stream() - .flatMap(d -> d.orphanPartitionSizes().keySet().stream()) - .distinct() + broker.topicPartitionPaths().stream() + .map(TopicPartitionPath::topicPartition) + .filter( + tp -> + clusterInfo + .replicaStream() + .anyMatch( + r -> + r.topicPartition().equals(tp) + && r.isLeader() + && r.brokerId() == broker.id())) .count(), - "orphan size", + "size", DataSize.Byte.of( - broker.dataFolders().stream() - .mapToLong( - d -> - d.orphanPartitionSizes().values().stream() - .mapToLong(v -> v) - .sum()) + broker.topicPartitionPaths().stream() + .mapToLong(TopicPartitionPath::size) .sum()))) .collect(Collectors.toList()); } @@ -311,7 +309,14 @@ private static Node basicNode(Context context) { FirstPart.builder() .clickName("REFRESH") .tableRefresher( - (argument, logger) -> context.admin().brokers().thenApply(BrokerNode::basicResult)) + (argument, logger) -> + FutureUtils.combine( + context.admin().brokers(), + context + .admin() + .topicNames(true) + .thenCompose(names -> context.admin().clusterInfo(names)), + BrokerNode::basicResult)) .build(); return PaneBuilder.of().firstPart(firstPart).build(); } @@ -438,36 +443,29 @@ private static Node folderNode(Context context) { .flatMap( broker -> broker.dataFolders().stream() - .sorted( - Comparator.comparing(Broker.DataFolder::path)) + .sorted() .map( - d -> { + path -> { Map result = new LinkedHashMap<>(); result.put("broker id", broker.id()); - result.put("path", d.path()); + result.put("path", path); result.put( - "partitions", d.partitionSizes().size()); + "partitions", + broker.topicPartitionPaths().stream() + .filter(tp -> tp.path().equals(path)) + .count()); result.put( "size", DataSize.Byte.of( - d.partitionSizes().values().stream() - .mapToLong(s -> s) - .sum())); - result.put( - "orphan partitions", - d.orphanPartitionSizes().size()); - result.put( - "orphan size", - DataSize.Byte.of( - d - .orphanPartitionSizes() - .values() - .stream() - .mapToLong(s -> s) + broker.topicPartitionPaths().stream() + .filter( + tp -> tp.path().equals(path)) + .mapToLong( + TopicPartitionPath::size) .sum())); result.putAll( - metrics.apply(broker.id(), d.path())); + metrics.apply(broker.id(), path)); return result; })) .collect(Collectors.toList()))) diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java index 7fcdf3e730..d0a4a53ca0 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java @@ -41,6 +41,7 @@ import org.astraea.common.admin.Partition; import org.astraea.common.admin.ProducerState; import org.astraea.common.admin.TopicConfigs; +import org.astraea.common.admin.TopicPartitionPath; import org.astraea.common.metrics.broker.HasRate; import org.astraea.common.metrics.broker.ServerMetrics; import org.astraea.gui.Context; @@ -363,12 +364,12 @@ private static List> basicResult( List producerStates) { var topicSize = brokers.stream() - .flatMap( - n -> n.dataFolders().stream().flatMap(d -> d.partitionSizes().entrySet().stream())) + .flatMap(n -> n.topicPartitionPaths().stream()) .collect( Collectors.groupingBy( - e -> e.getKey().topic(), - Collectors.mapping(Map.Entry::getValue, Collectors.reducing(0L, Long::sum)))); + TopicPartitionPath::topic, + Collectors.mapping( + TopicPartitionPath::size, Collectors.reducing(0L, Long::sum)))); var topicPartitions = partitions.stream().collect(Collectors.groupingBy(Partition::topic)); var topicGroups = From cba77245ed7ab621afa582d0378b33988d78c66b Mon Sep 17 00:00:00 2001 From: Haser Date: Tue, 30 May 2023 13:32:37 +0800 Subject: [PATCH 44/77] [SERIALIZATION] add serialization and deserialization functionality for ClusterBean (#1769) * add serialization and deserialization functionality for ClusterBean * rename name of unit test * rename the ClusterBean to MapOfBeanObjects, also remove the unnecessary variables in test. * fix format violations * add comments for setCreatedTimestamp, also rename the readClusterBean to readBeanObjects * fix format violations * extract method from BeanObject toByte * add the comment for readBeanObjects --- .../java/org/astraea/common/ByteUtils.java | 104 +++++++++++++++--- .../astraea/common/generated/BeanObject.proto | 9 ++ .../org/astraea/common/ByteUtilsTest.java | 38 +++++++ 3 files changed, 133 insertions(+), 18 deletions(-) diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index b6d2509a34..a126e688c2 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -22,6 +22,8 @@ import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -42,6 +44,7 @@ import org.astraea.common.generated.admin.TopicOuterClass; import org.astraea.common.generated.admin.TopicPartitionOuterClass; import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.HasBeanObject; public final class ByteUtils { @@ -171,24 +174,7 @@ public static byte[] toBytes(boolean value) { /** Serialize BeanObject by protocol buffer. The unsupported value will be ignored. */ public static byte[] toBytes(BeanObject value) { - var beanBuilder = BeanObjectOuterClass.BeanObject.newBuilder(); - beanBuilder.setDomain(value.domainName()); - beanBuilder.putAllProperties(value.properties()); - value - .attributes() - .forEach( - (key, val) -> { - try { - beanBuilder.putAttributes(key, primitive(val)); - } catch (SerializationException ignore) { - // Bean attribute may contain non-primitive value. e.g. TimeUnit, Byte. - } - }); - beanBuilder.setCreatedTimestamp( - Timestamp.newBuilder() - .setSeconds(value.createdTimestamp() / 1000) - .setNanos((int) (value.createdTimestamp() % 1000) * 1000000)); - return beanBuilder.build().toByteArray(); + return toOuterClass(value).toByteArray(); } /** Serialize ClusterInfo by protocol buffer. */ @@ -202,6 +188,33 @@ public static byte[] toBytes(ClusterInfo value) { .toByteArray(); } + public static byte[] toBytes(Map> values) { + var mapOfBeanObjects = + values.entrySet().stream() + .collect( + Collectors.toUnmodifiableMap( + Map.Entry::getKey, + e -> + e.getValue().stream() + .map(HasBeanObject::beanObject) + // convert BeanObject to protocol buffer + .map(ByteUtils::toOuterClass) + .toList())); + + return BeanObjectOuterClass.MapOfBeanObjects.newBuilder() + .putAllAllBeans( + mapOfBeanObjects.entrySet().stream() + .collect( + Collectors.toUnmodifiableMap( + Map.Entry::getKey, + Objects -> + BeanObjectOuterClass.MapOfBeanObjects.BeanObjects.newBuilder() + .addAllBeanObjects(Objects.getValue()) + .build()))) + .build() + .toByteArray(); + } + public static int readInt(ReadableByteChannel channel) { return Utils.packException( () -> { @@ -288,6 +301,36 @@ public static BeanObject readBeanObject(byte[] bytes) throws SerializationExcept } } + /** Deserialize to a map with Integer keys and list of BeanObject values using protocol buffer */ + public static Map> readBeanObjects(byte[] bytes) { + try { + var outerClusterBean = BeanObjectOuterClass.MapOfBeanObjects.parseFrom(bytes); + return outerClusterBean.getAllBeansMap().entrySet().stream() + .collect( + Collectors.toUnmodifiableMap( + k -> k.getKey(), + v -> + v.getValue().getBeanObjectsList().stream() + .map( + i -> + new BeanObject( + i.getDomain(), + i.getPropertiesMap(), + i.getAttributesMap().entrySet().stream() + .collect( + Collectors.toUnmodifiableMap( + Map.Entry::getKey, + e -> + Objects.requireNonNull( + toObject(e.getValue())))), + i.getCreatedTimestamp().getSeconds() * 1000 + + i.getCreatedTimestamp().getNanos() / 1000000)) + .toList())); + } catch (InvalidProtocolBufferException ex) { + throw new SerializationException(ex); + } + } + /** Deserialize to ClusterInfo with protocol buffer */ public static ClusterInfo readClusterInfo(byte[] bytes) { try { @@ -365,6 +408,31 @@ private static ReplicaOuterClass.Replica toOuterClass(Replica replica) { .build(); } + private static BeanObjectOuterClass.BeanObject toOuterClass(BeanObject beanObject) { + var beanBuilder = BeanObjectOuterClass.BeanObject.newBuilder(); + beanBuilder.setDomain(beanObject.domainName()); + beanBuilder.putAllProperties(beanObject.properties()); + beanObject + .attributes() + .forEach( + (key, val) -> { + try { + beanBuilder.putAttributes(key, primitive(val)); + } catch (SerializationException ignore) { + // Bean attribute may contain non-primitive value. e.g. TimeUnit, Byte. + } + }); + return beanBuilder + // the following code sets the created timestamp field using + // the recommended + // style by protobuf documentation. + .setCreatedTimestamp( + Timestamp.newBuilder() + .setSeconds(beanObject.createdTimestamp() / 1000) + .setNanos((int) (beanObject.createdTimestamp() % 1000 * 1000000))) + .build(); + } + // -------------------------Deserialize From ProtoBuf Outer Class----------------------------- // private static TopicPartitionPath toTopicPartitionPath( diff --git a/common/src/main/proto/org/astraea/common/generated/BeanObject.proto b/common/src/main/proto/org/astraea/common/generated/BeanObject.proto index a1669bb67b..0577576b42 100644 --- a/common/src/main/proto/org/astraea/common/generated/BeanObject.proto +++ b/common/src/main/proto/org/astraea/common/generated/BeanObject.proto @@ -11,3 +11,12 @@ message BeanObject { map attributes = 3; google.protobuf.Timestamp createdTimestamp = 4; } + + +message MapOfBeanObjects { + message BeanObjects { + repeated BeanObject beanObjects = 1; + } + + map allBeans = 1; +} \ No newline at end of file diff --git a/common/src/test/java/org/astraea/common/ByteUtilsTest.java b/common/src/test/java/org/astraea/common/ByteUtilsTest.java index 03e7a0cc76..862d04cc49 100644 --- a/common/src/test/java/org/astraea/common/ByteUtilsTest.java +++ b/common/src/test/java/org/astraea/common/ByteUtilsTest.java @@ -18,9 +18,15 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; +import java.util.List; +import java.util.Map; import java.util.Set; import org.astraea.common.admin.Admin; import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.ClusterBean; +import org.astraea.common.metrics.broker.HasGauge; +import org.astraea.common.metrics.broker.LogMetrics; import org.astraea.it.Service; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -113,4 +119,36 @@ void testReadAndToBytesEmptyClusterInfo() { Assertions.assertEquals(clusterInfo.topics(), deserializedClusterInfo.topics()); Assertions.assertEquals(clusterInfo.replicas(), deserializedClusterInfo.replicas()); } + + @Test + void testReadAndToBytesBeanObjects() { + BeanObject testBeanObject = + new BeanObject( + "kafka.log", + Map.of( + "name", + LogMetrics.Log.SIZE.metricName(), + "type", + "Log", + "topic", + "testBeans", + "partition", + "0"), + Map.of("Value", 100)); + var clusterBean = ClusterBean.of(Map.of(1, List.of(HasGauge.of(testBeanObject)))); + + var bytes = ByteUtils.toBytes(clusterBean.all()); + var deserializedClusterBean = ByteUtils.readBeanObjects(bytes); + + Assertions.assertEquals(1, deserializedClusterBean.size()); + Assertions.assertEquals( + testBeanObject.domainName(), deserializedClusterBean.get(1).get(0).domainName()); + Assertions.assertEquals( + testBeanObject.createdTimestamp(), + deserializedClusterBean.get(1).get(0).createdTimestamp()); + Assertions.assertEquals( + testBeanObject.properties(), deserializedClusterBean.get(1).get(0).properties()); + Assertions.assertEquals( + testBeanObject.attributes(), deserializedClusterBean.get(1).get(0).attributes()); + } } From 9c04e0891ac3fd4e790d1fd05e2629c13642c84c Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Tue, 30 May 2023 15:49:40 +0800 Subject: [PATCH 45/77] [COMMON] fix warnings of AdminTest (#1793) --- .../org/astraea/common/admin/AdminTest.java | 29 +++++++------------ 1 file changed, 11 insertions(+), 18 deletions(-) diff --git a/common/src/test/java/org/astraea/common/admin/AdminTest.java b/common/src/test/java/org/astraea/common/admin/AdminTest.java index 7aae77d860..6eb766ecdf 100644 --- a/common/src/test/java/org/astraea/common/admin/AdminTest.java +++ b/common/src/test/java/org/astraea/common/admin/AdminTest.java @@ -359,7 +359,7 @@ void testOrder() { .sorted( Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) - .thenComparing(r -> r.brokerId())) + .thenComparing(Replica::brokerId)) .collect(Collectors.toList()), replicas); } @@ -849,8 +849,7 @@ void testCreateTopicWithReplicasAssignment() { var config = admin.topics(Set.of(topic)).toCompletableFuture().join().get(0).config(); var partitions = - admin.partitions(Set.of(topic)).toCompletableFuture().join().stream() - .collect(Collectors.toUnmodifiableList()); + admin.partitions(Set.of(topic)).toCompletableFuture().join().stream().toList(); Assertions.assertTrue(config.raw().containsValue("lz4")); Assertions.assertEquals( List.of(0, 2), @@ -999,7 +998,6 @@ void testConsumerGroups() { .forEach( c -> { Assertions.assertNotNull(c.groupId()); - Assertions.assertNotNull(c.coordinatorId()); Assertions.assertNotNull(c.assignor()); Assertions.assertNotNull(c.state()); }); @@ -1249,7 +1247,7 @@ void testCompact() { IntStream.range(0, 5) .mapToObj(i -> consumer.poll(Duration.ofSeconds(1))) .flatMap(FixedIterable::stream) - .collect(Collectors.toList()); + .toList(); Assertions.assertEquals( 1, records.stream().filter(record -> record.key().equals(key)).count()); @@ -1354,8 +1352,7 @@ void testReplicasPreferredLeaderFlag() { .collect( Collectors.groupingBy( replica -> TopicPartition.of(replica.topic(), replica.partition()), - Collectors.mapping( - replica -> replica.brokerId(), Collectors.toList()))); + Collectors.mapping(Replica::brokerId, Collectors.toList()))); IntStream.range(0, partitionCount) .forEach(p -> admin.moveToBrokers(Map.of(TopicPartition.of(topic, p), List.of(0, 1, 2)))); @@ -1381,10 +1378,7 @@ void testProducerStates() { .toCompletableFuture() .join(); Assertions.assertNotEquals(0, states.size()); - var producerState = - states.stream() - .filter(s -> s.topic().equals(topic)) - .collect(Collectors.toUnmodifiableList()); + var producerState = states.stream().filter(s -> s.topic().equals(topic)).toList(); Assertions.assertEquals(1, producerState.size()); } } @@ -1397,7 +1391,7 @@ void testConnectionQuotas() { var quotas = admin.quotas(Set.of(QuotaConfigs.IP)).toCompletableFuture().join().stream() .filter(q -> q.targetValue().equals(Utils.hostname())) - .collect(Collectors.toList()); + .toList(); Assertions.assertNotEquals(0, quotas.size()); quotas.forEach( quota -> { @@ -1431,7 +1425,7 @@ void testProducerQuotas() { admin.quotas(Set.of(QuotaConfigs.CLIENT_ID)).toCompletableFuture().join().stream() .filter(q -> q.targetValue().equals(Utils.hostname())) .filter(q -> q.limitKey().equals(QuotaConfigs.PRODUCER_BYTE_RATE_CONFIG)) - .collect(Collectors.toList()); + .toList(); Assertions.assertNotEquals(0, quotas.size()); quotas.forEach( quota -> Assertions.assertEquals(DataRate.Byte.of(100).byteRate(), quota.limitValue())); @@ -1461,7 +1455,7 @@ void testConsumerQuotas() { admin.quotas(Set.of(QuotaConfigs.CLIENT_ID)).toCompletableFuture().join().stream() .filter(q -> q.targetValue().equals(Utils.hostname())) .filter(q -> q.limitKey().equals(QuotaConfigs.CONSUMER_BYTE_RATE_CONFIG)) - .collect(Collectors.toList()); + .toList(); Assertions.assertNotEquals(0, quotas.size()); quotas.forEach( quota -> Assertions.assertEquals(DataRate.Byte.of(1000).byteRate(), quota.limitValue())); @@ -1564,8 +1558,7 @@ void testDeleteRecord() { @Test void testDeleteTopic() { - var topic = - IntStream.range(0, 4).mapToObj(x -> Utils.randomString()).collect(Collectors.toList()); + var topic = IntStream.range(0, 4).mapToObj(x -> Utils.randomString()).toList(); try (var admin = Admin.of(SERVICE.bootstrapServers())) { topic.forEach( @@ -1645,7 +1638,7 @@ void testDeleteMembers() { .toCompletableFuture() .join(); } - String groupId = null; + String groupId; try (var consumer = Consumer.forTopics(Set.of(topic)) .bootstrapServers(SERVICE.bootstrapServers()) @@ -1675,7 +1668,7 @@ void testDeleteGroups() { .toCompletableFuture() .join(); } - String groupId = null; + String groupId; try (var consumer = Consumer.forTopics(Set.of(topic)) .bootstrapServers(SERVICE.bootstrapServers()) From baea43d0298a06ddd096d831f41cef53b21b4f16 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Wed, 31 May 2023 02:21:57 +0800 Subject: [PATCH 46/77] [METRICS] Implementation of fixed receiver (#1795) --- .../common/metrics/collector/MetricStore.java | 16 ++++ .../metrics/collector/FixedMetricsTest.java | 82 +++++++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 common/src/test/java/org/astraea/common/metrics/collector/FixedMetricsTest.java diff --git a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java index d4e17d6ed3..67ac8fc01f 100644 --- a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java +++ b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java @@ -82,6 +82,22 @@ static MetricFetcher.Sender local() { return LocalSenderReceiver.of(); } + static Receiver fixed(Map> beans) { + return new Receiver() { + private final AtomicBoolean done = new AtomicBoolean(false); + + @Override + public Map> receive(Duration timeout) { + return done.compareAndSet(false, true) ? beans : Map.of(); + } + + @Override + public void close() { + done.set(true); + } + }; + } + /** * Using an embedded fetcher build the receiver. The fetcher will keep fetching beans * background, and it pushes all beans to store internally. diff --git a/common/src/test/java/org/astraea/common/metrics/collector/FixedMetricsTest.java b/common/src/test/java/org/astraea/common/metrics/collector/FixedMetricsTest.java new file mode 100644 index 0000000000..f9b2bc15c7 --- /dev/null +++ b/common/src/test/java/org/astraea/common/metrics/collector/FixedMetricsTest.java @@ -0,0 +1,82 @@ +/* + * 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.metrics.collector; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.BeanQuery; +import org.astraea.common.metrics.HasBeanObject; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class FixedMetricsTest { + + @Test + void testFixedReceiverOnce() { + var beans = + Map.of(1, (Collection) List.of(new BeanObject("domain", Map.of(), Map.of()))); + try (var receiver = MetricStore.Receiver.fixed(beans)) { + Assertions.assertEquals(beans, receiver.receive(Duration.ZERO)); + Assertions.assertEquals(Map.of(), receiver.receive(Duration.ZERO)); + Assertions.assertEquals(Map.of(), receiver.receive(Duration.ZERO)); + Assertions.assertEquals(Map.of(), receiver.receive(Duration.ZERO)); + } + + var receiver = MetricStore.Receiver.fixed(beans); + receiver.close(); + Assertions.assertEquals(Map.of(), receiver.receive(Duration.ZERO)); + } + + @Test + void testFixedReceiver() { + interface MyBeanObject extends HasBeanObject {} + + var beans = + Map.of( + 1, + (Collection) + List.of(new BeanObject("domain", Map.of("Hello", "World"), Map.of()))); + var sensor = + (MetricSensor) + (client, cb) -> + List.of( + (MyBeanObject) + () -> + client.bean( + BeanQuery.builder() + .domainName("domain") + .property("Hello", "World") + .build())); + + try (var store = + MetricStore.builder() + .receivers(List.of(MetricStore.Receiver.fixed(beans))) + .beanExpiration(Duration.ofDays(3)) + .sensorsSupplier(() -> Map.of(sensor, (a, x) -> {})) + .build()) { + store.wait(cb -> cb.all().containsKey(1), Duration.ofSeconds(1)); + Assertions.assertEquals(Set.of(1), store.clusterBean().all().keySet()); + Assertions.assertEquals(1, store.clusterBean().all().get(1).size()); + Assertions.assertInstanceOf( + MyBeanObject.class, store.clusterBean().all().get(1).iterator().next()); + } + } +} From 0118432fd273bad38953729176018b02c6c070d1 Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Wed, 31 May 2023 10:02:14 +0800 Subject: [PATCH 47/77] [METRICS] Add mbean to metric store and metric fetcher (#1794) --- .../metrics/collector/MetricFetcher.java | 32 ++++++++++++--- .../common/metrics/collector/MetricStore.java | 41 ++++++++++++++++++- 2 files changed, 67 insertions(+), 6 deletions(-) diff --git a/common/src/main/java/org/astraea/common/metrics/collector/MetricFetcher.java b/common/src/main/java/org/astraea/common/metrics/collector/MetricFetcher.java index b0d4908122..d6517f1e99 100644 --- a/common/src/main/java/org/astraea/common/metrics/collector/MetricFetcher.java +++ b/common/src/main/java/org/astraea/common/metrics/collector/MetricFetcher.java @@ -33,7 +33,6 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Supplier; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.FutureUtils; @@ -41,6 +40,8 @@ import org.astraea.common.metrics.BeanObject; import org.astraea.common.metrics.BeanQuery; import org.astraea.common.metrics.MBeanClient; +import org.astraea.common.metrics.MBeanRegister; +import org.astraea.common.metrics.Sensor; import org.astraea.common.metrics.broker.ClusterMetrics; import org.astraea.common.metrics.broker.ControllerMetrics; import org.astraea.common.metrics.broker.LogMetrics; @@ -51,6 +52,7 @@ import org.astraea.common.metrics.client.producer.ProducerMetrics; import org.astraea.common.metrics.connector.ConnectorMetrics; import org.astraea.common.metrics.platform.HostMetrics; +import org.astraea.common.metrics.stats.Sum; import org.astraea.common.producer.Producer; import org.astraea.common.producer.Record; import org.astraea.common.producer.Serializer; @@ -70,7 +72,7 @@ public interface MetricFetcher extends AutoCloseable { ConnectorMetrics.QUERIES.stream(), HostMetrics.QUERIES.stream()) .flatMap(s -> s) - .collect(Collectors.toUnmodifiableList()); + .toList(); static Builder builder() { return new Builder(); @@ -109,11 +111,11 @@ public CompletionStage send(int id, Collection beans) { var records = beans.stream() .map(bean -> Record.builder().topic(METRIC_TOPIC).key(id).value(bean).build()) - .collect(Collectors.toUnmodifiableList()); + .toList(); return FutureUtils.sequence( producer.send(records).stream() .map(CompletionStage::toCompletableFuture) - .collect(Collectors.toUnmodifiableList())) + .toList()) .thenAccept(ignored -> {}); } @@ -178,6 +180,13 @@ public MetricFetcher build() { } class MetricFetcherImpl implements MetricFetcher { + public static final String DOMAIN_NAME = "org.astraea"; + public static final String TYPE_PROPERTY = "type"; + public static final String TYPE_VALUE = "metricFetcher"; + public static final String NAME_PROPERTY = "name"; + public static final String BEAN_FETCHED_NAME = "BeanFetched"; + public static final String ID_PROPERTY = "id"; + public static final String SUM_PROPERTY = "sum"; private volatile Map clients = new HashMap<>(); private final Map> latest = new ConcurrentHashMap<>(); @@ -195,6 +204,8 @@ class MetricFetcherImpl implements MetricFetcher { private final Supplier>> clientSupplier; private final Duration fetchBeanDelay; + private final Sensor beanFetchedSensor = + Sensor.builder().addStat(SUM_PROPERTY, Sum.ofLong()).build(); private MetricFetcherImpl( int threads, @@ -235,6 +246,16 @@ private MetricFetcherImpl( }; IntStream.range(0, threads).forEach(ignored -> executor.execute(job)); + + // MBean register + MBeanRegister.local() + .domainName(DOMAIN_NAME) + .property(TYPE_PROPERTY, TYPE_VALUE) + .property(ID_PROPERTY, Utils.randomString()) + .property(NAME_PROPERTY, BEAN_FETCHED_NAME) + .attribute(SUM_PROPERTY, Long.class, () -> beanFetchedSensor.measure(SUM_PROPERTY)) + .description("The number of fetched beans") + .register(); } private void updateMetadata() { @@ -269,7 +290,8 @@ private void updateData(DelayedIdentity identity) { beans = QUERIES.stream() .flatMap(q -> clients.get(identity.id).beans(q, e -> {}).stream()) - .collect(Collectors.toUnmodifiableList()); + .toList(); + beanFetchedSensor.record((long) beans.size()); } finally { lock.readLock().unlock(); } diff --git a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java index 67ac8fc01f..a3d9f79ce2 100644 --- a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java +++ b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java @@ -47,6 +47,9 @@ import org.astraea.common.metrics.ClusterBean; import org.astraea.common.metrics.HasBeanObject; import org.astraea.common.metrics.MBeanClient; +import org.astraea.common.metrics.MBeanRegister; +import org.astraea.common.metrics.Sensor; +import org.astraea.common.metrics.stats.Sum; public interface MetricStore extends AutoCloseable { @@ -163,7 +166,7 @@ class Builder { (client, bean) -> client.beans(BeanQuery.all()).stream() .map(bs -> (HasBeanObject) () -> bs) - .collect(Collectors.toUnmodifiableList()), + .toList(), (id, ignored) -> {}); private Collection receivers; @@ -194,6 +197,15 @@ public MetricStore build() { } class MetricStoreImpl implements MetricStore { + public static final String DOMAIN_NAME = "org.astraea"; + public static final String TYPE_PROPERTY = "type"; + public static final String TYPE_VALUE = "metricStore"; + public static final String NAME_PROPERTY = "name"; + public static final String BEAN_COUNT_NAME = "BeanCount"; + public static final String BEAN_RECEIVE_NAME = "BeanReceived"; + public static final String ID_PROPERTY = "id"; + public static final String COUNT_PROPERTY = "count"; + public static final String SUM_PROPERTY = "sum"; private final Map> beans = new ConcurrentHashMap<>(); @@ -212,6 +224,10 @@ class MetricStoreImpl implements MetricStore { private volatile Map> lastSensors = Map.of(); private final Map> waitingList = new ConcurrentHashMap<>(); + // For mbean register. To distinguish mbeans of different metricStore. + private final String uid = Utils.randomString(); + private final Sensor beanReceivedSensor = + Sensor.builder().addStat(SUM_PROPERTY, Sum.ofLong()).build(); private MetricStoreImpl( Supplier>> sensorsSupplier, @@ -248,6 +264,8 @@ private MetricStoreImpl( .map(r -> r.receive(Duration.ofSeconds(3))) .forEach( allBeans -> { + beanReceivedSensor.record( + allBeans.values().stream().mapToLong(Collection::size).sum()); identities.addAll(allBeans.keySet()); lastSensors = sensorsSupplier.get(); allBeans.forEach( @@ -280,6 +298,27 @@ private MetricStoreImpl( }; executor.execute(cleanerJob); executor.execute(receiverJob); + + // ------------ MBean register ------------ + MBeanRegister.local() + .domainName(DOMAIN_NAME) + .property(TYPE_PROPERTY, TYPE_VALUE) + .property(ID_PROPERTY, uid) + .property(NAME_PROPERTY, BEAN_COUNT_NAME) + .attribute( + COUNT_PROPERTY, + Long.class, + () -> beans.values().stream().mapToLong(Collection::size).sum()) + .description("The number of beans stored in this metricStore.") + .register(); + MBeanRegister.local() + .domainName(DOMAIN_NAME) + .property(TYPE_PROPERTY, TYPE_VALUE) + .property(ID_PROPERTY, uid) + .property(NAME_PROPERTY, BEAN_RECEIVE_NAME) + .attribute(SUM_PROPERTY, Long.class, () -> beanReceivedSensor.measure(SUM_PROPERTY)) + .description("The total number of beans received.") + .register(); } @Override From a5561801187ffbafd8561aac3dc005f05d6d30c6 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 31 May 2023 11:21:22 +0800 Subject: [PATCH 48/77] [COMMON] apply Configuration#EMPTY to code base (#1797) --- .../src/main/java/org/astraea/common/cost/RecordSizeCost.java | 3 +-- .../main/java/org/astraea/common/cost/ReplicaLeaderCost.java | 2 +- .../java/org/astraea/common/cost/ReplicaLeaderSizeCost.java | 3 +-- .../main/java/org/astraea/common/cost/ReplicaNumberCost.java | 3 +-- common/src/test/java/org/astraea/common/UtilsTest.java | 4 ++-- .../common/balancer/algorithms/GreedyBalancerTest.java | 2 +- .../astraea/common/partitioner/StrictCostPartitionerTest.java | 4 ++-- .../src/test/java/org/astraea/connector/ConnectorTest.java | 4 ++-- 8 files changed, 11 insertions(+), 14 deletions(-) diff --git a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java index 6a1085f792..f227d06508 100644 --- a/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/RecordSizeCost.java @@ -18,7 +18,6 @@ import static org.astraea.common.cost.CostUtils.changedRecordSizeOverflow; -import java.util.Map; import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.DataSize; @@ -33,7 +32,7 @@ public class RecordSizeCost public static final String MAX_MIGRATE_SIZE_KEY = "max.migrated.size"; public RecordSizeCost() { - this.config = new Configuration(Map.of()); + this.config = Configuration.EMPTY; } public RecordSizeCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java index 0ea33d71d2..c37f0b06aa 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderCost.java @@ -34,7 +34,7 @@ public class ReplicaLeaderCost implements HasBrokerCost, HasClusterCost, HasMove public static final String MAX_MIGRATE_LEADER_KEY = "max.migrated.leader.number"; public ReplicaLeaderCost() { - this.config = new Configuration(Map.of()); + this.config = Configuration.EMPTY; } public ReplicaLeaderCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index f2586401e8..e02ff2e6b1 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -18,7 +18,6 @@ import static org.astraea.common.cost.CostUtils.changedRecordSizeOverflow; -import java.util.Map; import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.DataSize; @@ -39,7 +38,7 @@ public class ReplicaLeaderSizeCost public static final String MOVED_LEADER_SIZE = "moved leader size (bytes)"; public ReplicaLeaderSizeCost() { - this.config = new Configuration(Map.of()); + this.config = Configuration.EMPTY; } public ReplicaLeaderSizeCost(Configuration config) { diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java index 720c46f6f0..299be20bdf 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java @@ -18,7 +18,6 @@ import static org.astraea.common.cost.MigrationCost.replicaNumChanged; -import java.util.Map; import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.admin.Broker; @@ -32,7 +31,7 @@ public class ReplicaNumberCost implements HasClusterCost, HasMoveCost { private final Configuration config; public ReplicaNumberCost() { - this.config = new Configuration(Map.of()); + this.config = Configuration.EMPTY; } public ReplicaNumberCost(Configuration config) { diff --git a/common/src/test/java/org/astraea/common/UtilsTest.java b/common/src/test/java/org/astraea/common/UtilsTest.java index dc3ba07d45..9119c7441c 100644 --- a/common/src/test/java/org/astraea/common/UtilsTest.java +++ b/common/src/test/java/org/astraea/common/UtilsTest.java @@ -206,7 +206,7 @@ public int value() { @ParameterizedTest @ValueSource(classes = {TestCostFunction.class, TestConfigCostFunction.class}) void testConstruct(Class aClass) { - var config = new Configuration(Map.of()); + var config = Configuration.EMPTY; var costFunction = Utils.construct(aClass, config); Assertions.assertInstanceOf(CostFunction.class, costFunction); @@ -226,7 +226,7 @@ void testConstruct(Class aClass) { void testConstructException() { // arrange var aClass = TestBadCostFunction.class; - var config = new Configuration(Map.of()); + var config = Configuration.EMPTY; // act, assert Assertions.assertThrows(RuntimeException.class, () -> Utils.construct(aClass, config)); diff --git a/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java b/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java index 3cc236f67c..fe28ddde1c 100644 --- a/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java @@ -58,7 +58,7 @@ void testConfig() { @Test void testJmx() { - var cost = new DecreasingCost(new Configuration(Map.of())); + var cost = new DecreasingCost(Configuration.EMPTY); var id = "TestJmx-" + UUID.randomUUID(); var clusterInfo = FakeClusterInfo.of(5, 5, 5, 2); var balancer = diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 524ca0987f..87a00930d1 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -49,7 +49,7 @@ public class StrictCostPartitionerTest { @Test void testJmxPort() { try (var partitioner = new StrictCostPartitioner()) { - partitioner.configure(new Configuration(Map.of())); + partitioner.configure(Configuration.EMPTY); Assertions.assertThrows( NoSuchElementException.class, () -> partitioner.jmxPortGetter.apply(0)); partitioner.configure(new Configuration(Map.of(StrictCostPartitioner.JMX_PORT, "12345"))); @@ -208,7 +208,7 @@ void testReturnedPartition() { @Test void testDefaultFunction() { try (var partitioner = new StrictCostPartitioner()) { - partitioner.configure(new Configuration(Map.of())); + partitioner.configure(Configuration.EMPTY); Assertions.assertNotEquals(HasBrokerCost.EMPTY, partitioner.costFunction); Utils.waitFor(() -> partitioner.metricStore.sensors().size() == 1); } diff --git a/connector/src/test/java/org/astraea/connector/ConnectorTest.java b/connector/src/test/java/org/astraea/connector/ConnectorTest.java index 7e134bd8fb..4dfa1cdffd 100644 --- a/connector/src/test/java/org/astraea/connector/ConnectorTest.java +++ b/connector/src/test/java/org/astraea/connector/ConnectorTest.java @@ -126,7 +126,7 @@ protected Class task() { @Override protected List takeConfiguration(int maxTasks) { return IntStream.range(0, maxTasks) - .mapToObj(i -> new Configuration(Map.of())) + .mapToObj(i -> Configuration.EMPTY) .collect(Collectors.toList()); } @@ -166,7 +166,7 @@ protected Class task() { @Override protected List takeConfiguration(int maxTasks) { return IntStream.range(0, maxTasks) - .mapToObj(i -> new Configuration(Map.of())) + .mapToObj(i -> Configuration.EMPTY) .collect(Collectors.toList()); } From 1c932fbbe05e767c1bab71689c7e569265c321c7 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Wed, 31 May 2023 21:45:09 +0800 Subject: [PATCH 49/77] [SCRIPT] Resolve kraft container restart issue (#1799) --- docker/start_broker.sh | 2 +- docker/start_controller.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/start_broker.sh b/docker/start_broker.sh index 719054076c..cb3a6d346a 100755 --- a/docker/start_broker.sh +++ b/docker/start_broker.sh @@ -314,7 +314,7 @@ if [[ "$quorum" == "kraft" ]]; then setPropertyIfEmpty "node.id" "$NODE_ID" setPropertyIfEmpty "process.roles" "broker" setPropertyIfEmpty "controller.listener.names" "CONTROLLER" - command="./bin/kafka-storage.sh format -t $CLUSTER_ID -c /tmp/broker.properties && ./bin/kafka-server-start.sh /tmp/broker.properties" + command="./bin/kafka-storage.sh format -t $CLUSTER_ID -c /tmp/broker.properties --ignore-formatted && ./bin/kafka-server-start.sh /tmp/broker.properties" fi docker run -d --init \ diff --git a/docker/start_controller.sh b/docker/start_controller.sh index ca8f04ebf2..0fab50d5ef 100755 --- a/docker/start_controller.sh +++ b/docker/start_controller.sh @@ -233,7 +233,7 @@ docker run -d --init \ -p $CONTROLLER_PORT:9093 \ -p $CONTROLLER_JMX_PORT:$CONTROLLER_JMX_PORT \ -p $EXPORTER_PORT:$EXPORTER_PORT \ - "$IMAGE_NAME" sh -c "./bin/kafka-storage.sh format -t $CLUSTER_ID -c /tmp/controller.properties && ./bin/kafka-server-start.sh /tmp/controller.properties" + "$IMAGE_NAME" sh -c "./bin/kafka-storage.sh format -t $CLUSTER_ID -c /tmp/controller.properties --ignore-formatted && ./bin/kafka-server-start.sh /tmp/controller.properties" echo "=================================================" [[ -n "$META_FOLDER" ]] && echo "mount $META_FOLDER to container: $CONTAINER_NAME" From 69c29a37021bd27ddb1fdcaf1d5c1a113d2aacea Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Thu, 1 Jun 2023 09:27:15 +0800 Subject: [PATCH 50/77] [METRICS] Configurable metric collector for Assignor (#1796) --- .../org/astraea/common/assignor/Assignor.java | 56 +++++++++++++------ .../common/consumer/ConsumerConfigs.java | 3 + 2 files changed, 43 insertions(+), 16 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 4d75189b22..1cee48e8e4 100644 --- a/common/src/main/java/org/astraea/common/assignor/Assignor.java +++ b/common/src/main/java/org/astraea/common/assignor/Assignor.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.function.Function; import java.util.function.Supplier; @@ -41,6 +42,7 @@ import org.astraea.common.metrics.MBeanClient; import org.astraea.common.metrics.collector.MetricStore; import org.astraea.common.partitioner.PartitionerUtils; +import org.astraea.common.producer.ProducerConfigs; /** Abstract assignor implementation which does some common work (e.g., configuration). */ public abstract class Assignor implements ConsumerPartitionAssignor, Configurable { @@ -99,24 +101,46 @@ private void establishResource() { () -> new IllegalArgumentException( ConsumerConfigs.BOOTSTRAP_SERVERS_CONFIG + " must be defined")); - Supplier>> clientSupplier = - () -> - admin - .brokers() - .thenApply( - brokers -> { - var map = new HashMap(); - brokers.forEach( - b -> - map.put( - b.id(), JndiClient.of(b.host(), jmxPortGetter.apply(b.id())))); - // add local client to fetch consumer metrics - map.put(-1, JndiClient.local()); - return Collections.unmodifiableMap(map); - }); + + List receivers = + switch (config + .string(ConsumerConfigs.METRIC_STORE_KEY) + .orElse(ConsumerConfigs.METRIC_STORE_LOCAL)) { + case ConsumerConfigs.METRIC_STORE_TOPIC -> List.of( + MetricStore.Receiver.topic( + config.requireString(ProducerConfigs.BOOTSTRAP_SERVERS_CONFIG)), + MetricStore.Receiver.local( + () -> CompletableFuture.completedStage(Map.of(-1, JndiClient.local())))); + case ConsumerConfigs.METRIC_STORE_LOCAL -> { + Supplier>> clientSupplier = + () -> + admin + .brokers() + .thenApply( + brokers -> { + var map = new HashMap(); + brokers.forEach( + b -> + map.put( + b.id(), + JndiClient.of(b.host(), jmxPortGetter.apply(b.id())))); + // add local client to fetch consumer metrics + map.put(-1, JndiClient.local()); + return Collections.unmodifiableMap(map); + }); + yield List.of(MetricStore.Receiver.local(clientSupplier)); + } + default -> throw new IllegalArgumentException( + "unknown metric store type: " + + config.string(ConsumerConfigs.METRIC_STORE_KEY) + + ". Use " + + ConsumerConfigs.METRIC_STORE_TOPIC + + " or " + + ConsumerConfigs.METRIC_STORE_LOCAL); + }; metricStore = MetricStore.builder() - .receivers(List.of(MetricStore.Receiver.local(clientSupplier))) + .receivers(receivers) .sensorsSupplier(() -> Map.of(this.costFunction.metricSensor(), (integer, e) -> {})) .build(); } diff --git a/common/src/main/java/org/astraea/common/consumer/ConsumerConfigs.java b/common/src/main/java/org/astraea/common/consumer/ConsumerConfigs.java index 36c2d16bb1..702d08c5c9 100644 --- a/common/src/main/java/org/astraea/common/consumer/ConsumerConfigs.java +++ b/common/src/main/java/org/astraea/common/consumer/ConsumerConfigs.java @@ -65,6 +65,7 @@ public final class ConsumerConfigs { public static final String EXCLUDE_INTERNAL_TOPICS_CONFIG = "exclude.internal.topics"; public static final String ISOLATION_LEVEL_CONFIG = "isolation.level"; public static final String ALLOW_AUTO_CREATE_TOPICS_CONFIG = "allow.auto.create.topics"; + public static final String METRIC_STORE_KEY = "metric.store"; public static final Set ALL_CONFIGS = new TreeSet<>( @@ -76,6 +77,8 @@ public final class ConsumerConfigs { public static final String AUTO_OFFSET_RESET_NONE = "none"; public static final String ISOLATION_LEVEL_UNCOMMITTED = "read_uncommitted"; public static final String ISOLATION_LEVEL_COMMITTED = "read_committed"; + public static final String METRIC_STORE_LOCAL = "local"; + public static final String METRIC_STORE_TOPIC = "topic"; private ConsumerConfigs() {} } From 33e72f73b4a470fa7eec57bf5799205ca88f7cc8 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sat, 3 Jun 2023 17:42:35 +0800 Subject: [PATCH 51/77] [CONNECTOR] add contexts for source/sink (#1802) --- .../org/astraea/connector/SinkConnector.java | 4 +- .../org/astraea/connector/SinkContext.java | 29 ++++++++++ ...{TaskContext.java => SinkTaskContext.java} | 7 +-- .../astraea/connector/SourceConnector.java | 4 +- .../org/astraea/connector/SourceContext.java | 56 +++++++++++++++++++ .../org/astraea/connector/SourceTask.java | 5 +- ...ataStorage.java => SourceTaskContext.java} | 9 ++- .../astraea/connector/backup/Exporter.java | 3 +- .../astraea/connector/backup/Importer.java | 7 ++- .../org/astraea/connector/perf/PerfSink.java | 3 +- .../astraea/connector/perf/PerfSource.java | 7 ++- .../org/astraea/connector/ConnectorTest.java | 4 +- .../connector/MetadataStorageTest.java | 6 +- .../org/astraea/connector/SourceDataTest.java | 4 +- .../connector/backup/ImporterTest.java | 4 +- .../connector/perf/PerfSourceTest.java | 13 +++-- 16 files changed, 127 insertions(+), 38 deletions(-) create mode 100644 connector/src/main/java/org/astraea/connector/SinkContext.java rename connector/src/main/java/org/astraea/connector/{TaskContext.java => SinkTaskContext.java} (94%) create mode 100644 connector/src/main/java/org/astraea/connector/SourceContext.java rename connector/src/main/java/org/astraea/connector/{MetadataStorage.java => SourceTaskContext.java} (83%) diff --git a/connector/src/main/java/org/astraea/connector/SinkConnector.java b/connector/src/main/java/org/astraea/connector/SinkConnector.java index 662902de45..705321a8d2 100644 --- a/connector/src/main/java/org/astraea/connector/SinkConnector.java +++ b/connector/src/main/java/org/astraea/connector/SinkConnector.java @@ -25,7 +25,7 @@ public abstract class SinkConnector extends org.apache.kafka.connect.sink.SinkConnector { public static final String TOPICS_KEY = TOPICS_CONFIG; - protected void init(Configuration configuration) { + protected void init(Configuration configuration, SinkContext context) { // empty } @@ -42,7 +42,7 @@ protected void close() { // -------------------------[final]-------------------------// @Override public final void start(Map props) { - init(new Configuration(props)); + init(new Configuration(props), SinkContext.of(context())); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SinkContext.java b/connector/src/main/java/org/astraea/connector/SinkContext.java new file mode 100644 index 0000000000..8a4b4eb5aa --- /dev/null +++ b/connector/src/main/java/org/astraea/connector/SinkContext.java @@ -0,0 +1,29 @@ +/* + * 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.connector; + +public interface SinkContext { + + SinkContext EMPTY = e -> {}; + + static SinkContext of(org.apache.kafka.connect.sink.SinkConnectorContext context) { + return context::raiseError; + } + + /** {@link org.apache.kafka.connect.sink.SinkConnectorContext#raiseError(Exception)} */ + void raiseError(Exception e); +} diff --git a/connector/src/main/java/org/astraea/connector/TaskContext.java b/connector/src/main/java/org/astraea/connector/SinkTaskContext.java similarity index 94% rename from connector/src/main/java/org/astraea/connector/TaskContext.java rename to connector/src/main/java/org/astraea/connector/SinkTaskContext.java index 6e2e69171b..a4ad097076 100644 --- a/connector/src/main/java/org/astraea/connector/TaskContext.java +++ b/connector/src/main/java/org/astraea/connector/SinkTaskContext.java @@ -19,10 +19,9 @@ import java.util.Collection; import java.util.Map; import java.util.stream.Collectors; -import org.apache.kafka.connect.sink.SinkTaskContext; import org.astraea.common.admin.TopicPartition; -public interface TaskContext { +public interface SinkTaskContext { /** * Reset the consumer offsets for the specified partitions. @@ -52,8 +51,8 @@ public interface TaskContext { */ void requestCommit(); - static TaskContext of(SinkTaskContext context) { - return new TaskContext() { + static SinkTaskContext of(org.apache.kafka.connect.sink.SinkTaskContext context) { + return new SinkTaskContext() { @Override public void offset(Map offsets) { context.offset( diff --git a/connector/src/main/java/org/astraea/connector/SourceConnector.java b/connector/src/main/java/org/astraea/connector/SourceConnector.java index 89ac37531d..57a0102ca6 100644 --- a/connector/src/main/java/org/astraea/connector/SourceConnector.java +++ b/connector/src/main/java/org/astraea/connector/SourceConnector.java @@ -25,7 +25,7 @@ public abstract class SourceConnector extends org.apache.kafka.connect.source.SourceConnector { public static final String TOPICS_KEY = "topics"; - protected abstract void init(Configuration configuration, MetadataStorage storage); + protected abstract void init(Configuration configuration, SourceContext context); protected abstract Class task(); @@ -40,7 +40,7 @@ protected void close() { // -------------------------[final]-------------------------// @Override public final void start(Map props) { - init(new Configuration(props), MetadataStorage.of(context().offsetStorageReader())); + init(new Configuration(props), SourceContext.of(context())); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SourceContext.java b/connector/src/main/java/org/astraea/connector/SourceContext.java new file mode 100644 index 0000000000..f85410ac7f --- /dev/null +++ b/connector/src/main/java/org/astraea/connector/SourceContext.java @@ -0,0 +1,56 @@ +/* + * 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.connector; + +import java.util.Map; +import java.util.stream.Collectors; + +public interface SourceContext { + + SourceContext EMPTY = + new SourceContext() { + @Override + public void raiseError(Exception e) {} + + @Override + public Map metadata(Map index) { + return Map.of(); + } + }; + + static SourceContext of(org.apache.kafka.connect.source.SourceConnectorContext context) { + return new SourceContext() { + @Override + public void raiseError(Exception e) { + context.raiseError(e); + } + + @Override + public Map metadata(Map index) { + var v = context.offsetStorageReader().offset(index); + if (v == null) return Map.of(); + return v.entrySet().stream() + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, e -> e.getValue().toString())); + } + }; + } + + /** {@link org.apache.kafka.connect.source.SourceConnectorContext#raiseError(Exception)} */ + void raiseError(Exception e); + + Map metadata(Map index); +} diff --git a/connector/src/main/java/org/astraea/connector/SourceTask.java b/connector/src/main/java/org/astraea/connector/SourceTask.java index 99ef9a8cb5..dbdbaa94a5 100644 --- a/connector/src/main/java/org/astraea/connector/SourceTask.java +++ b/connector/src/main/java/org/astraea/connector/SourceTask.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.kafka.connect.data.Schema; import org.astraea.common.Configuration; @@ -28,7 +29,7 @@ public abstract class SourceTask extends org.apache.kafka.connect.source.SourceTask { - protected abstract void init(Configuration configuration, MetadataStorage storage); + protected abstract void init(Configuration configuration, SourceTaskContext storage); /** * use {@link Record#builder()} or {@link SourceRecord#builder()} to construct the returned @@ -52,7 +53,7 @@ public final String version() { @Override public final void start(Map props) { - init(new Configuration(props), MetadataStorage.of(context.offsetStorageReader())); + init(new Configuration(props), SourceTaskContext.of(Objects.requireNonNull(context))); } @Override diff --git a/connector/src/main/java/org/astraea/connector/MetadataStorage.java b/connector/src/main/java/org/astraea/connector/SourceTaskContext.java similarity index 83% rename from connector/src/main/java/org/astraea/connector/MetadataStorage.java rename to connector/src/main/java/org/astraea/connector/SourceTaskContext.java index 5ed7703033..a955299e41 100644 --- a/connector/src/main/java/org/astraea/connector/MetadataStorage.java +++ b/connector/src/main/java/org/astraea/connector/SourceTaskContext.java @@ -18,15 +18,14 @@ import java.util.Map; import java.util.stream.Collectors; -import org.apache.kafka.connect.storage.OffsetStorageReader; -public interface MetadataStorage { +public interface SourceTaskContext { - MetadataStorage EMPTY = ignored -> Map.of(); + SourceTaskContext EMPTY = ignored -> Map.of(); - static MetadataStorage of(OffsetStorageReader reader) { + static SourceTaskContext of(org.apache.kafka.connect.source.SourceTaskContext context) { return index -> { - var v = reader.offset(index); + var v = context.offsetStorageReader().offset(index); if (v == null) return Map.of(); return v.entrySet().stream() .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, e -> e.getValue().toString())); diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 4448e750b9..8bd2f83c6d 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -37,6 +37,7 @@ import org.astraea.common.consumer.Record; import org.astraea.connector.Definition; import org.astraea.connector.SinkConnector; +import org.astraea.connector.SinkContext; import org.astraea.connector.SinkTask; import org.astraea.fs.FileSystem; @@ -123,7 +124,7 @@ public class Exporter extends SinkConnector { private Configuration configs; @Override - protected void init(Configuration configuration) { + protected void init(Configuration configuration, SinkContext context) { this.configs = configuration; } diff --git a/connector/src/main/java/org/astraea/connector/backup/Importer.java b/connector/src/main/java/org/astraea/connector/backup/Importer.java index 880f50e71f..bf9df8a438 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Importer.java +++ b/connector/src/main/java/org/astraea/connector/backup/Importer.java @@ -32,10 +32,11 @@ import org.astraea.common.backup.RecordReader; import org.astraea.common.backup.RecordWriter; import org.astraea.connector.Definition; -import org.astraea.connector.MetadataStorage; import org.astraea.connector.SourceConnector; +import org.astraea.connector.SourceContext; import org.astraea.connector.SourceRecord; import org.astraea.connector.SourceTask; +import org.astraea.connector.SourceTaskContext; import org.astraea.fs.FileSystem; import org.astraea.fs.Type; @@ -98,7 +99,7 @@ public class Importer extends SourceConnector { private Configuration config; @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceContext context) { this.config = configuration; } @@ -143,7 +144,7 @@ public static class Task extends SourceTask { private String cleanSource; private Optional archiveDir; - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceTaskContext storage) { this.Client = FileSystem.of(configuration.requireString(SCHEMA_KEY.name()), configuration); this.fileSet = configuration.requireInteger(FILE_SET_KEY); this.addedPaths = new HashSet<>(); diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java index 898b1b3f78..cf50544470 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java @@ -24,6 +24,7 @@ import org.astraea.common.consumer.Record; import org.astraea.connector.Definition; import org.astraea.connector.SinkConnector; +import org.astraea.connector.SinkContext; import org.astraea.connector.SinkTask; public class PerfSink extends SinkConnector { @@ -41,7 +42,7 @@ public class PerfSink extends SinkConnector { private Configuration config; @Override - protected void init(Configuration configuration) { + protected void init(Configuration configuration, SinkContext context) { this.config = configuration; } diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java index 919330be7c..406470160d 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSource.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSource.java @@ -31,10 +31,11 @@ import org.astraea.common.admin.TopicPartition; import org.astraea.common.producer.RecordGenerator; import org.astraea.connector.Definition; -import org.astraea.connector.MetadataStorage; import org.astraea.connector.SourceConnector; +import org.astraea.connector.SourceContext; import org.astraea.connector.SourceRecord; import org.astraea.connector.SourceTask; +import org.astraea.connector.SourceTaskContext; public class PerfSource extends SourceConnector { @@ -146,7 +147,7 @@ public class PerfSource extends SourceConnector { private Configuration config; @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceContext context) { this.config = configuration; } @@ -198,7 +199,7 @@ public static class Task extends SourceTask { RecordGenerator recordGenerator = null; @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceTaskContext storage) { var throughput = configuration.string(THROUGHPUT_DEF.name()).map(DataSize::of).orElse(THROUGHPUT_DEFAULT); var KeySize = diff --git a/connector/src/test/java/org/astraea/connector/ConnectorTest.java b/connector/src/test/java/org/astraea/connector/ConnectorTest.java index 4dfa1cdffd..b8d2aef1fb 100644 --- a/connector/src/test/java/org/astraea/connector/ConnectorTest.java +++ b/connector/src/test/java/org/astraea/connector/ConnectorTest.java @@ -109,7 +109,7 @@ public static class MySource extends SourceConnector { private static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceContext context) { INIT_COUNT.incrementAndGet(); } @@ -141,7 +141,7 @@ public static class MySourceTask extends SourceTask { private static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceTaskContext storage) { INIT_COUNT.incrementAndGet(); } diff --git a/connector/src/test/java/org/astraea/connector/MetadataStorageTest.java b/connector/src/test/java/org/astraea/connector/MetadataStorageTest.java index 16659b831c..c3ead8e045 100644 --- a/connector/src/test/java/org/astraea/connector/MetadataStorageTest.java +++ b/connector/src/test/java/org/astraea/connector/MetadataStorageTest.java @@ -103,9 +103,9 @@ public static class MySource extends SourceConnector { private Configuration configuration; @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceContext context) { this.configuration = configuration; - FETCHED_METADATA = storage.metadata(KEY); + FETCHED_METADATA = context.metadata(KEY); } @Override @@ -132,7 +132,7 @@ public static class MyTask extends SourceTask { private Set topics = Set.of(); @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceTaskContext storage) { topics = Set.copyOf(configuration.list(ConnectorConfigs.TOPICS_KEY, ",")); FETCHED_METADATA = storage.metadata(KEY); } diff --git a/connector/src/test/java/org/astraea/connector/SourceDataTest.java b/connector/src/test/java/org/astraea/connector/SourceDataTest.java index 1e9ea36290..bec16a752a 100644 --- a/connector/src/test/java/org/astraea/connector/SourceDataTest.java +++ b/connector/src/test/java/org/astraea/connector/SourceDataTest.java @@ -116,7 +116,7 @@ public static class MySource extends SourceConnector { private Configuration configuration = Configuration.EMPTY; @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceContext context) { this.configuration = configuration; } @@ -142,7 +142,7 @@ public static class MyTask extends SourceTask { private boolean isDone = false; @Override - protected void init(Configuration configuration, MetadataStorage storage) { + protected void init(Configuration configuration, SourceTaskContext context) { topics = Set.copyOf(configuration.list(ConnectorConfigs.TOPICS_KEY, ",")); } diff --git a/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java index fbea1aed67..9c39e72fd1 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ImporterTest.java @@ -29,7 +29,7 @@ import org.astraea.common.connector.ConnectorClient; import org.astraea.common.connector.Value; import org.astraea.common.consumer.Record; -import org.astraea.connector.MetadataStorage; +import org.astraea.connector.SourceTaskContext; import org.astraea.fs.FileSystem; import org.astraea.it.FtpServer; import org.astraea.it.Service; @@ -150,7 +150,7 @@ void testFtpSourceTask() { records.forEach(writer::append); writer.close(); - task.init(new Configuration(configs), MetadataStorage.EMPTY); + task.init(new Configuration(configs), SourceTaskContext.EMPTY); var returnRecords = new ArrayList<>(task.take()); for (int i = 0; i < records.size(); i++) { diff --git a/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java b/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java index ef4f6dcfcb..642b548cdd 100644 --- a/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java +++ b/connector/src/test/java/org/astraea/connector/perf/PerfSourceTest.java @@ -30,8 +30,9 @@ import org.astraea.common.connector.ConnectorConfigs; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.connector.ConnectorMetrics; -import org.astraea.connector.MetadataStorage; import org.astraea.connector.SourceConnector; +import org.astraea.connector.SourceContext; +import org.astraea.connector.SourceTaskContext; import org.astraea.it.Service; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; @@ -51,7 +52,7 @@ static void closeService() { void testDistributeConfigs() { var s = new PerfSource(); var config = new Configuration(Map.of(SourceConnector.TOPICS_KEY, "a,b,c,d")); - s.init(config, MetadataStorage.EMPTY); + s.init(config, SourceContext.EMPTY); var configs = s.takeConfiguration(10); Assertions.assertEquals(4, configs.size()); Assertions.assertEquals("a", configs.get(0).requireString(SourceConnector.TOPICS_KEY)); @@ -331,7 +332,7 @@ void testMetrics() { @Test void testInit() { var task = new PerfSource.Task(); - task.init(new Configuration(Map.of(ConnectorConfigs.TOPICS_KEY, "a")), MetadataStorage.EMPTY); + task.init(new Configuration(Map.of(ConnectorConfigs.TOPICS_KEY, "a")), SourceTaskContext.EMPTY); Assertions.assertNotNull(task.recordGenerator); Assertions.assertEquals(1, task.specifyPartitions.size()); } @@ -348,7 +349,7 @@ void testKeyAndValue() { "uniform", PerfSource.VALUE_DISTRIBUTION_DEF.name(), "uniform")), - MetadataStorage.EMPTY); + SourceTaskContext.EMPTY); var records = task.take(); var keySizes = records.stream().map(r -> r.key().length).collect(Collectors.toUnmodifiableSet()); @@ -364,7 +365,7 @@ void testZeroKeySize() { task.init( new Configuration( Map.of(ConnectorConfigs.TOPICS_KEY, "a", PerfSource.KEY_SIZE_DEF.name(), "0Byte")), - MetadataStorage.EMPTY); + SourceTaskContext.EMPTY); var records = task.take(); Assertions.assertNotEquals(0, records.size()); records.forEach(r -> Assertions.assertNull(r.key())); @@ -376,7 +377,7 @@ void testZeroValueSize() { task.init( new Configuration( Map.of(ConnectorConfigs.TOPICS_KEY, "a", PerfSource.VALUE_SIZE_DEF.name(), "0Byte")), - MetadataStorage.EMPTY); + SourceTaskContext.EMPTY); var records = task.take(); Assertions.assertNotEquals(0, records.size()); records.forEach(r -> Assertions.assertNull(r.value())); From 61c158cfaf4926dc1bbabc7aad43207ed9585e93 Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Sat, 3 Jun 2023 19:27:18 +0800 Subject: [PATCH 52/77] [BALANCER] Implementation of `BalancerBenchmarkApp` (#1736) --- app/src/main/java/org/astraea/app/App.java | 5 +- .../app/benchmark/BalancerBenchmarkApp.java | 542 ++++++++++++++++++ .../balancer}/BalancerBenchmark.java | 2 +- .../balancer}/CostProfilingImpl.java | 2 +- .../balancer}/ExperimentBuilderImpl.java | 2 +- .../benchmark/BalancerBenchmarkAppTest.java | 179 ++++++ 6 files changed, 728 insertions(+), 4 deletions(-) create mode 100644 app/src/main/java/org/astraea/app/benchmark/BalancerBenchmarkApp.java rename app/src/main/java/org/astraea/{balancer/bench => app/benchmark/balancer}/BalancerBenchmark.java (98%) rename app/src/main/java/org/astraea/{balancer/bench => app/benchmark/balancer}/CostProfilingImpl.java (99%) rename app/src/main/java/org/astraea/{balancer/bench => app/benchmark/balancer}/ExperimentBuilderImpl.java (98%) create mode 100644 app/src/test/java/org/astraea/app/benchmark/BalancerBenchmarkAppTest.java diff --git a/app/src/main/java/org/astraea/app/App.java b/app/src/main/java/org/astraea/app/App.java index 118b048af1..53c3b0b36f 100644 --- a/app/src/main/java/org/astraea/app/App.java +++ b/app/src/main/java/org/astraea/app/App.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import org.astraea.app.automation.Automation; +import org.astraea.app.benchmark.BalancerBenchmarkApp; import org.astraea.app.performance.Performance; import org.astraea.app.publisher.MetricPublisher; import org.astraea.app.version.Version; @@ -39,7 +40,9 @@ public class App { "version", Version.class, "metric_publisher", - MetricPublisher.class); + MetricPublisher.class, + "balancer_benchmark", + BalancerBenchmarkApp.class); static void execute(Map> mains, List args) throws Throwable { diff --git a/app/src/main/java/org/astraea/app/benchmark/BalancerBenchmarkApp.java b/app/src/main/java/org/astraea/app/benchmark/BalancerBenchmarkApp.java new file mode 100644 index 0000000000..05004673b6 --- /dev/null +++ b/app/src/main/java/org/astraea/app/benchmark/BalancerBenchmarkApp.java @@ -0,0 +1,542 @@ +/* + * 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.app.benchmark; + +import com.beust.jcommander.Parameter; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.astraea.app.argument.Argument; +import org.astraea.app.argument.PathField; +import org.astraea.app.argument.PositiveIntegerField; +import org.astraea.app.benchmark.balancer.BalancerBenchmark; +import org.astraea.common.ByteUtils; +import org.astraea.common.Configuration; +import org.astraea.common.Utils; +import org.astraea.common.VersionUtils; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.balancer.Balancer; +import org.astraea.common.balancer.BalancerProblemFormat; +import org.astraea.common.cost.ClusterCost; +import org.astraea.common.json.JsonConverter; +import org.astraea.common.json.TypeRef; +import org.astraea.common.metrics.BeanObject; +import org.astraea.common.metrics.ClusterBean; +import org.astraea.common.metrics.HasBeanObject; +import org.astraea.common.metrics.collector.MetricSensor; +import org.astraea.common.metrics.collector.MetricStore; + +public class BalancerBenchmarkApp { + + public void execute(String[] args) { + var name = args.length > 0 ? args[0] : ""; + var arguments = Arrays.stream(args).skip(1).toArray(String[]::new); + var benchmarks = + Map.of( + "experiment", + () -> runExperiment(Argument.parse(new ExperimentArgument(), arguments)), + "cost_profiling", + () -> runCostProfiling(Argument.parse(new CostProfilingArgument(), arguments))); + + Runnable help = + () -> { + System.out.printf("Usage: %s [args ...]%n", benchmarks.keySet()); + if (name.isEmpty()) throw new IllegalArgumentException("No argument specified"); + else if (!name.equalsIgnoreCase("help")) + throw new IllegalArgumentException("Unknown benchmark name: " + name); + }; + + // try to run the specified benchmark or cry for help + benchmarks.getOrDefault(name, help).run(); + } + + void runExperiment(ExperimentArgument argument) { + var cluster = argument.fetchClusterInfo(); + var beans = argument.fetchBeanObjects(); + var problem = argument.fetchBalancerProblem(); + var configs = problem.parse(); + var balancer = + Utils.construct( + (Class) Utils.packException(() -> Class.forName(problem.balancer)), + new Configuration(problem.balancerConfig)); + var clusterBean = + toClusterBean( + List.of( + configs.clusterCostFunction().metricSensor(), + configs.moveCostFunction().metricSensor()), + beans); + + System.out.println("Running Experiment..."); + System.out.println(); + + var result = + BalancerBenchmark.experiment() + .setBalancer(balancer) + .setClusterInfo(cluster) + .setClusterBean(clusterBean) + .setAlgorithmConfig(configs) + .setExperimentTrials(argument.trials) + .setExecutionTimeout(problem.timeout) + .start() + .toCompletableFuture() + .join(); + + System.out.println(optimizationSummary(argument, cluster, clusterBean, problem)); + System.out.println(experimentSummary(result)); + } + + void runCostProfiling(CostProfilingArgument argument) { + var cluster = argument.fetchClusterInfo(); + var beans = argument.fetchBeanObjects(); + var problem = argument.fetchBalancerProblem(); + var configs = problem.parse(); + var balancer = + Utils.construct( + (Class) Utils.packException(() -> Class.forName(problem.balancer)), + new Configuration(problem.balancerConfig)); + var clusterBean = + toClusterBean( + List.of( + configs.clusterCostFunction().metricSensor(), + configs.moveCostFunction().metricSensor()), + beans); + + System.out.println("Running CostProfiling..."); + System.out.println(); + + var result = + BalancerBenchmark.costProfiling() + .setBalancer(balancer) + .setClusterInfo(cluster) + .setClusterBean(clusterBean) + .setAlgorithmConfig(problem.parse()) + .setExecutionTimeout(problem.timeout) + .start() + .toCompletableFuture() + .join(); + + System.out.println(optimizationSummary(argument, cluster, clusterBean, problem)); + System.out.println(costProfilingSummary(argument, result)); + } + + private String optimizationSummary( + CommonArgument arg, ClusterInfo info, ClusterBean bean, BalancerProblemFormat optimization) { + + var metricStart = + bean.all().values().stream() + .flatMap(Collection::stream) + .map(HasBeanObject::beanObject) + .mapToLong(BeanObject::createdTimestamp) + .min() + .stream() + .mapToObj( + time -> + ZonedDateTime.ofInstant(Instant.ofEpochMilli(time), ZoneId.systemDefault()) + .toLocalDateTime()) + .findFirst() + .orElse(null); + var metricEnd = + bean.all().values().stream() + .flatMap(Collection::stream) + .map(HasBeanObject::beanObject) + .mapToLong(BeanObject::createdTimestamp) + .max() + .stream() + .mapToObj( + time -> + ZonedDateTime.ofInstant(Instant.ofEpochMilli(time), ZoneId.systemDefault()) + .toLocalDateTime()) + .findFirst() + .orElse(null); + var duration = + metricStart != null && metricEnd != null + ? Duration.between(metricStart, metricEnd) + : Duration.ZERO; + + return """ + Balancer Benchmark + =============================== + + * Version: %s + * Build Time: %s + * Revision: %s + * Author: %s + + ## Balancing Problem + + ```json + %s + ``` + + * Execution: %s + * Balancer: %s + * Balancer Configuration: + %s + * Cluster Cost Function: %s + * Move Cost Function: %s + * Cost Function Configuration: + %s + + ## ClusterInfo Summary + + * ClusterId: %s + * Topics: %d + * Partition: %d + * Replicas: %d + * Broker Count: %d + + ## ClusterBean Summary + + * Total Metrics: %d + * Avg Metrics Per Broker: %f + * Broker Count: %d + * Metrics Start From: %s + * Metrics End at: %s + * Recorded Duration: %s + + """ + .formatted( + // astraea version + VersionUtils.VERSION, + VersionUtils.DATE, + VersionUtils.REVISION, + VersionUtils.BUILDER, + // Balancer Problem Summary + arg.fetchBalancerProblemJson(), + optimization.timeout, + optimization.balancer, + Optional.of( + optimization.balancerConfig.entrySet().stream() + .map(e -> String.format(" * \"%s\": %s", e.getKey(), e.getValue())) + .collect(Collectors.joining(System.lineSeparator()))) + .filter(Predicate.not(String::isEmpty)) + .orElse(" * no config"), + optimization.parse().clusterCostFunction().toString(), + optimization.parse().moveCostFunction().toString(), + Optional.of( + optimization.costConfig.entrySet().stream() + .map(e -> String.format(" * \"%s\": %s", e.getKey(), e.getValue())) + .collect(Collectors.joining(System.lineSeparator()))) + .filter(Predicate.not(String::isEmpty)) + .orElse(" * no config"), + // ClusterInfo Summary + info.clusterId(), + info.topicNames().size(), + info.topicPartitions().size(), + info.replicas().size(), + info.brokers().size(), + // ClusterBean Summary + bean.all().values().stream().mapToInt(Collection::size).sum(), + (double) bean.all().values().stream().mapToInt(Collection::size).sum() + / bean.brokerIds().size(), + bean.brokerIds().size(), + metricStart != null ? metricStart : "no metric", + metricEnd != null ? metricEnd : "no metric", + duration); + } + + private String experimentSummary(BalancerBenchmark.ExperimentResult result) { + + var count = result.costSummary().getCount(); + + return """ + Balancer Experiment Result + =============================== + + * Attempted Trials: %d + * Solution Found Trials: %d + * No Solution Found Trials: %d + + ## ClusterCost Detail + + * Initial ClusterCost: %f + > %s + * Best ClusterCost: %s + > %s + + ## Statistics + + * Initial Cost: %f + * Min Cost: %s + * Average Cost: %s + * Max Cost: %s + * Cost Variance: %s + + ## All Cost Values + + ``` + %s + ``` + + """ + .formatted( + // Trials + result.trials(), + result.costs().size(), + result.trials() - result.costs().size(), + // Cost Detail + result.initial().value(), + result.initial(), + result + .bestCost() + .map(ClusterCost::value) + .map(Object::toString) + .orElse("no usable solution found"), + result.bestCost().map(Object::toString).orElse("no usable solution found"), + // Cost Statistics + result.initial().value(), + count > 0 ? result.costSummary().getMin() : -1, + count > 0 ? result.costSummary().getAverage() : -1, + count > 0 ? result.costSummary().getMax() : -1, + result.variance().orElse(-1), + // All values + result.costs().stream() + .mapToDouble(ClusterCost::value) + .sorted() + .mapToObj(Double::toString) + .collect(Collectors.joining(System.lineSeparator()))); + } + + private String costProfilingSummary( + CostProfilingArgument arg, BalancerBenchmark.CostProfilingResult result) { + + // use the move cost evaluation count as the number of iteration(an optimization attempt) been + // performed. we are not using cluster cost since some balancer implementation won't perform + // cluster cost evaluation if it knows the solution is infeasible. + var iterations = Math.max(1, result.moveCostProcessingTimeNs().getCount()); + var time = System.currentTimeMillis(); + var randomName = Utils.randomString(4); + + var csvClusterCostFilename = "cost-profiling-" + time + "-" + randomName + ".csv"; + var csv0 = Path.of(arg.exportFolder.toAbsolutePath().toString(), csvClusterCostFilename); + exportCsv( + csv0, + result.costTimeSeries().entrySet().stream() + .sorted(Map.Entry.comparingByKey(Comparator.comparingLong(x -> x))) + .map(e -> List.of(e.getKey(), e.getValue().value()))); + + var csvVerboseFilename = "cost-profiling-" + time + "-" + randomName + "-verbose.csv"; + var csv1 = Path.of(arg.exportFolder.toAbsolutePath().toString(), csvVerboseFilename); + exportCsv( + csv1, + Stream.concat( + result.costTimeSeries().entrySet().stream() + // time, cluster-cost-value, move-cost-overflow, cluster-cost, move-cost + .map(e -> List.of(e.getKey(), e.getValue().value(), "", e.getValue(), "")), + result.moveCostTimeSeries().entrySet().stream() + // time, cluster-cost-value, move-cost-overflow, cluster-cost, move-cost + .map( + e -> + List.of( + e.getKey(), "", e.getValue().overflow() ? 1 : 0, "", e.getValue()))) + .sorted(Comparator.comparingLong(x -> (long) x.get(0)))); + + return """ + Balancer Cost Profiling Result + =============================== + + * Initial Cost Value: %f + > %s + + * Best Cost Value: %s + > %s + + ## Runtime Statistics + + * Execution Time: %s + * Average Iteration Time: %.3f ms + * Average Balancer Operation Time: %.3f ms + * Average ClusterCost Processing Time: %.3f ms + * Average MoveCost Processing Time: %.3f ms + * Total ClusterCost Evaluation: %d + * Total MoveCost Evaluation: %d + + ## Detail + + * Cost Profiling Result (ClusterCost Only) in CSV: %s + * Cost Profiling Result (All) in CSV: %s + """ + .formatted( + // summary + result.initial().value(), + result.initial(), + result + .plan() + .map(Balancer.Plan::proposalClusterCost) + .map(ClusterCost::value) + .map(Object::toString) + .orElse("no solution found"), + result + .plan() + .map(Balancer.Plan::proposalClusterCost) + .map(Object::toString) + .orElse("no solution found"), + // runtime statistics + result.executionTime(), + result.executionTime().dividedBy(iterations).toNanos() / 1e6, + result + .executionTime() + .minusNanos(result.clusterCostProcessingTimeNs().getSum()) + .minusNanos(result.moveCostProcessingTimeNs().getSum()) + .dividedBy(iterations) + .toNanos() + / 1e6, + result.clusterCostProcessingTimeNs().getAverage() / 1e6, + result.moveCostProcessingTimeNs().getAverage() / 1e6, + result.clusterCostProcessingTimeNs().getCount(), + result.moveCostProcessingTimeNs().getCount(), + // details + csv0, + csv1); + } + + static void exportCsv(Path location, Stream> timeSeries) { + try (var writer = Utils.packException(() -> Files.newBufferedWriter(location))) { + var iterator = timeSeries.iterator(); + + while (iterator.hasNext()) { + boolean commas = false; + for (var item : iterator.next()) { + if (commas) writer.write(","); + var s = item.toString(); + // deal with commas inside the field according to RFC 4180 + if (s.contains(",")) { + // the field should be wrapped by double-quotes + writer.write("\""); + // any double-quote inside the field will be covert into two double-quotes + writer.write(s.replace("\"", "\"\"")); + // the field should be wrapped by double-quotes + writer.write("\""); + } else { + writer.write(s); + } + commas = true; + } + writer.newLine(); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + static ClusterBean toClusterBean( + Collection sensors, Map> beans) { + try (var store = + MetricStore.builder() + .receivers( + List.of( + MetricStore.Receiver.fixed( + beans.entrySet().stream() + .collect( + Collectors.toUnmodifiableMap( + Map.Entry::getKey, Map.Entry::getValue))))) + .sensorsSupplier( + () -> + sensors.stream() + .collect(Collectors.toUnmodifiableMap(x -> x, x -> (id, err) -> {}))) + .build()) { + for (int i = 0; i < 3 && store.clusterBean().all().isEmpty(); i++) + Utils.sleep(Duration.ofSeconds(1)); + return store.clusterBean(); + } + } + + public static void main(String[] args) { + new BalancerBenchmarkApp().execute(args); + } + + static class CommonArgument { + @Parameter( + names = {"--cluster.info"}, + description = "String: path to the serialized cluster info file", + required = true, + converter = PathField.class) + Path serializedClusterInfo; + + @Parameter( + names = {"--cluster.bean"}, + description = "String: path to the serialized cluster bean file", + required = true, + converter = PathField.class) + Path serializedClusterBean; + + @Parameter( + names = {"--optimization.config"}, + description = + "String: path to the json file containing the optimization problem definition.", + required = true, + converter = PathField.class) + Path optimizationConfig; + + ClusterInfo fetchClusterInfo() { + try (var stream = Files.newInputStream(serializedClusterInfo)) { + return ByteUtils.readClusterInfo(stream.readAllBytes()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + Map> fetchBeanObjects() { + try (var stream = Files.newInputStream(serializedClusterBean)) { + return ByteUtils.readBeanObjects(stream.readAllBytes()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + String fetchBalancerProblemJson() { + var bytes = Utils.packException(() -> Files.readAllBytes(optimizationConfig)); + return new String(bytes); + } + + BalancerProblemFormat fetchBalancerProblem() { + return JsonConverter.defaultConverter() + .fromJson(fetchBalancerProblemJson(), TypeRef.of(BalancerProblemFormat.class)); + } + } + + static class ExperimentArgument extends CommonArgument { + @Parameter( + names = {"--trials"}, + description = "Integer: the number of experiments to perform.", + required = true, + converter = PositiveIntegerField.class) + int trials; + } + + static class CostProfilingArgument extends CommonArgument { + @Parameter( + names = {"--export.folder"}, + description = "String: the directory to store experiment result.", + converter = PathField.class) + Path exportFolder = Path.of(System.getProperty("java.io.tmpdir")); + } +} diff --git a/app/src/main/java/org/astraea/balancer/bench/BalancerBenchmark.java b/app/src/main/java/org/astraea/app/benchmark/balancer/BalancerBenchmark.java similarity index 98% rename from app/src/main/java/org/astraea/balancer/bench/BalancerBenchmark.java rename to app/src/main/java/org/astraea/app/benchmark/balancer/BalancerBenchmark.java index de516066da..cd2b1e8d9d 100644 --- a/app/src/main/java/org/astraea/balancer/bench/BalancerBenchmark.java +++ b/app/src/main/java/org/astraea/app/benchmark/balancer/BalancerBenchmark.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.astraea.balancer.bench; +package org.astraea.app.benchmark.balancer; import java.time.Duration; import java.util.Comparator; diff --git a/app/src/main/java/org/astraea/balancer/bench/CostProfilingImpl.java b/app/src/main/java/org/astraea/app/benchmark/balancer/CostProfilingImpl.java similarity index 99% rename from app/src/main/java/org/astraea/balancer/bench/CostProfilingImpl.java rename to app/src/main/java/org/astraea/app/benchmark/balancer/CostProfilingImpl.java index c9ecfedfa2..6ba9bc6329 100644 --- a/app/src/main/java/org/astraea/balancer/bench/CostProfilingImpl.java +++ b/app/src/main/java/org/astraea/app/benchmark/balancer/CostProfilingImpl.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.astraea.balancer.bench; +package org.astraea.app.benchmark.balancer; import java.time.Duration; import java.util.Collections; diff --git a/app/src/main/java/org/astraea/balancer/bench/ExperimentBuilderImpl.java b/app/src/main/java/org/astraea/app/benchmark/balancer/ExperimentBuilderImpl.java similarity index 98% rename from app/src/main/java/org/astraea/balancer/bench/ExperimentBuilderImpl.java rename to app/src/main/java/org/astraea/app/benchmark/balancer/ExperimentBuilderImpl.java index 05c3ba3d46..d62d1c32ed 100644 --- a/app/src/main/java/org/astraea/balancer/bench/ExperimentBuilderImpl.java +++ b/app/src/main/java/org/astraea/app/benchmark/balancer/ExperimentBuilderImpl.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.astraea.balancer.bench; +package org.astraea.app.benchmark.balancer; import java.time.Duration; import java.util.Set; diff --git a/app/src/test/java/org/astraea/app/benchmark/BalancerBenchmarkAppTest.java b/app/src/test/java/org/astraea/app/benchmark/BalancerBenchmarkAppTest.java new file mode 100644 index 0000000000..9e226a357f --- /dev/null +++ b/app/src/test/java/org/astraea/app/benchmark/BalancerBenchmarkAppTest.java @@ -0,0 +1,179 @@ +/* + * 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.app.benchmark; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Pattern; +import org.astraea.common.Configuration; +import org.astraea.common.VersionUtils; +import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.balancer.AlgorithmConfig; +import org.astraea.common.balancer.Balancer; +import org.astraea.common.balancer.BalancerProblemFormat; +import org.astraea.common.cost.ClusterCost; +import org.astraea.common.cost.ReplicaLeaderCost; +import org.astraea.common.metrics.BeanObject; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class BalancerBenchmarkAppTest { + + private PrintStream original; + private ByteArrayOutputStream output = new ByteArrayOutputStream(); + + @BeforeEach + void setOutput() { + original = System.out; + System.setOut(new PrintStream(output)); + } + + @AfterEach + void recoverOutput() { + System.setOut(original); + } + + @Test + void testExecuteHelp() { + var help = new String[] {"help"}; + var noBench = new String[] {"no_such"}; + Assertions.assertDoesNotThrow(() -> new BalancerBenchmarkApp().execute(help)); + Assertions.assertThrows( + IllegalArgumentException.class, () -> new BalancerBenchmarkApp().execute(noBench)); + } + + @Test + void testExecuteExperiment() { + var args = + new BalancerBenchmarkApp.ExperimentArgument() { + + @Override + ClusterInfo fetchClusterInfo() { + return ClusterInfo.empty(); + } + + @Override + Map> fetchBeanObjects() { + return Map.of(); + } + + @Override + String fetchBalancerProblemJson() { + return "BALANCER_PROBLEM"; + } + + @Override + BalancerProblemFormat fetchBalancerProblem() { + var bpf = new BalancerProblemFormat(); + bpf.balancer = NoOpBalancer.class.getName(); + bpf.clusterCosts = List.of(costWeight(ReplicaLeaderCost.class.getName(), 1)); + return bpf; + } + }; + args.trials = 1; + new BalancerBenchmarkApp().runExperiment(args); + + var stdout = output.toString(); + Assertions.assertTrue(stdout.contains("Version: " + VersionUtils.VERSION)); + Assertions.assertTrue(stdout.contains("Balancer: " + NoOpBalancer.class.getName())); + Assertions.assertTrue(stdout.contains("BALANCER_PROBLEM")); + Assertions.assertTrue(stdout.contains("Attempted Trials: 1")); + Assertions.assertTrue(stdout.contains("MOCKED_RESULT")); + } + + @Test + void testExecuteCostProfiling() { + var args = + new BalancerBenchmarkApp.CostProfilingArgument() { + + @Override + ClusterInfo fetchClusterInfo() { + return ClusterInfo.empty(); + } + + @Override + Map> fetchBeanObjects() { + return Map.of(); + } + + @Override + String fetchBalancerProblemJson() { + return "BALANCER_PROBLEM"; + } + + @Override + BalancerProblemFormat fetchBalancerProblem() { + var bpf = new BalancerProblemFormat(); + bpf.balancer = NoOpBalancer.class.getName(); + bpf.clusterCosts = List.of(costWeight(ReplicaLeaderCost.class.getName(), 1)); + return bpf; + } + }; + new BalancerBenchmarkApp().runCostProfiling(args); + + var stdout = output.toString(); + Assertions.assertTrue(stdout.contains("Version: " + VersionUtils.VERSION)); + Assertions.assertTrue(stdout.contains("Balancer: " + NoOpBalancer.class.getName())); + Assertions.assertTrue(stdout.contains("BALANCER_PROBLEM")); + Assertions.assertTrue(stdout.contains("MOCKED_RESULT")); + + var matcher = Pattern.compile(": (.+\\.csv)").matcher(stdout); + Assertions.assertTrue(matcher.find()); + Assertions.assertTrue(Files.exists(Path.of(matcher.group(1)))); + Assertions.assertTrue(matcher.find()); + Assertions.assertTrue(Files.exists(Path.of(matcher.group(1)))); + } + + @Test + void testArgument() { + Assertions.assertThrows( + IllegalArgumentException.class, () -> BalancerBenchmarkApp.main(new String[] {})); + Assertions.assertThrows( + IllegalArgumentException.class, () -> BalancerBenchmarkApp.main(new String[] {"Bad"})); + Assertions.assertDoesNotThrow(() -> BalancerBenchmarkApp.main(new String[] {"help"})); + } + + private static BalancerProblemFormat.CostWeight costWeight(String cost, double weight) { + var cw = new BalancerProblemFormat.CostWeight(); + cw.cost = cost; + cw.weight = weight; + return cw; + } + + private static class NoOpBalancer implements Balancer { + public NoOpBalancer(Configuration config) {} + + @Override + public Optional offer(AlgorithmConfig config) { + return Optional.of( + new Plan( + config.clusterBean(), + config.clusterInfo(), + config.clusterCostFunction().clusterCost(config.clusterInfo(), config.clusterBean()), + config.clusterInfo(), + ClusterCost.of(ThreadLocalRandom.current().nextDouble(), () -> "MOCKED_RESULT"))); + } + } +} From c8ec30ab533f1bd1d2515d7d3cd223ef85f8d003 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sat, 3 Jun 2023 22:46:18 +0800 Subject: [PATCH 53/77] [COMMON] tweak code base (#1803) --- .../java/org/astraea/app/backup/Backup.java | 2 +- .../app/publisher/MetricPublisher.java | 51 ++++++----- .../org/astraea/app/web/BalancerHandler.java | 35 ++++---- .../org/astraea/app/web/BrokerHandler.java | 1 - .../org/astraea/app/web/RecordHandler.java | 6 -- .../app/performance/PerformanceTest.java | 2 +- .../astraea/app/web/BalancerHandlerTest.java | 2 +- .../java/org/astraea/common/ByteUtils.java | 2 +- .../org/astraea/common/admin/AdminImpl.java | 2 +- .../common/admin/OptimizedClusterInfo.java | 2 +- .../common/balancer/BalancerConsoleImpl.java | 8 +- .../executor/StraightPlanExecutor.java | 2 +- .../astraea/common/cost/HasPartitionCost.java | 18 ++-- .../astraea/common/cost/MigrationCost.java | 2 +- .../common/cost/NetworkIngressCost.java | 2 +- .../common/cost/ReplicaLeaderSizeCost.java | 2 +- .../common/cost/ReplicaNumberCost.java | 3 +- .../java/org/astraea/common/json/TypeRef.java | 4 +- .../common/partitioner/Partitioner.java | 6 +- .../astraea/common/admin/ClusterBeanTest.java | 86 +++++++++--------- .../balancer/BalancerConfigTestSuite.java | 28 +++--- .../astraea/common/consumer/ConsumerTest.java | 1 - .../org/astraea/common/cost/MoveCostTest.java | 2 +- .../astraea/common/cost/NetworkCostTest.java | 88 ++++++++----------- .../common/cost/RecordSizeCostTest.java | 1 - fs/src/main/java/org/astraea/fs/Type.java | 2 +- gui/src/main/java/org/astraea/gui/Query.java | 24 ++--- .../astraea/gui/tab/topic/ReplicaNode.java | 3 +- .../org/astraea/gui/tab/topic/TopicNode.java | 10 +-- 29 files changed, 179 insertions(+), 218 deletions(-) diff --git a/app/src/main/java/org/astraea/app/backup/Backup.java b/app/src/main/java/org/astraea/app/backup/Backup.java index 71225e369c..a713b5d4c8 100644 --- a/app/src/main/java/org/astraea/app/backup/Backup.java +++ b/app/src/main/java/org/astraea/app/backup/Backup.java @@ -50,7 +50,7 @@ public void restoreDistribution(ClusterInfo clusterInfo, String bootstrapServers .sorted( Comparator.comparing( replica -> !replica.isLeader())) - .map(replica -> replica.brokerId()) + .map(Replica::brokerId) .toList()))))) .configs(topic.config().raw()) .run() diff --git a/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java b/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java index e260bf42ac..e1adf376c1 100644 --- a/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java +++ b/app/src/main/java/org/astraea/app/publisher/MetricPublisher.java @@ -43,33 +43,32 @@ public static void main(String[] args) { // Valid for testing static void execute(Arguments arguments) { - var admin = Admin.of(arguments.bootstrapServers()); - var topicSender = MetricFetcher.Sender.topic(arguments.bootstrapServers()); - try (var metricFetcher = - MetricFetcher.builder() - .clientSupplier( - () -> - admin - .brokers() - .thenApply( - brokers -> - brokers.stream() - .collect( - Collectors.toUnmodifiableMap( - Broker::id, - broker -> - JndiClient.of( - broker.host(), - arguments.idToJmxPort().apply(broker.id())))))) - .fetchBeanDelay(arguments.period) - .fetchMetadataDelay(Duration.ofMinutes(5)) - .threads(3) - .sender(topicSender) - .build()) { + try (var admin = Admin.of(arguments.bootstrapServers()); + var topicSender = MetricFetcher.Sender.topic(arguments.bootstrapServers()); + var metricFetcher = + MetricFetcher.builder() + .clientSupplier( + () -> + admin + .brokers() + .thenApply( + brokers -> + brokers.stream() + .collect( + Collectors.toUnmodifiableMap( + Broker::id, + broker -> + JndiClient.of( + broker.host(), + arguments + .idToJmxPort() + .apply(broker.id())))))) + .fetchBeanDelay(arguments.period) + .fetchMetadataDelay(Duration.ofMinutes(5)) + .threads(3) + .sender(topicSender) + .build()) { Utils.sleep(arguments.ttl); - } finally { - admin.close(); - topicSender.close(); } } diff --git a/app/src/main/java/org/astraea/app/web/BalancerHandler.java b/app/src/main/java/org/astraea/app/web/BalancerHandler.java index 45cf1626fc..362b1d2efa 100644 --- a/app/src/main/java/org/astraea/app/web/BalancerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BalancerHandler.java @@ -146,24 +146,23 @@ private PlanExecutionProgress progress(String taskId) { var contextCluster = taskMetadata.get(taskId).clusterInfo; var exception = (Function) - (phase) -> { - return switch (phase) { - case Searching, Searched, Executing, Executed -> - // No error message during the search & execution - null; - case SearchFailed -> planGenerations - .get(taskId) - .handle((plan, err) -> err != null ? err.toString() : null) - .toCompletableFuture() - .getNow(null); - case ExecutionFailed -> planExecutions - .get(taskId) - .handle((ignore, err) -> err != null ? err.toString() : null) - .toCompletableFuture() - .getNow(null); - default -> throw new IllegalStateException("Unknown state: " + phase); - }; - }; + (phase) -> + switch (phase) { + case Searching, Searched, Executing, Executed -> + // No error message during the search & execution + null; + case SearchFailed -> planGenerations + .get(taskId) + .handle((plan, err) -> err != null ? err.toString() : null) + .toCompletableFuture() + .getNow(null); + case ExecutionFailed -> planExecutions + .get(taskId) + .handle((ignore, err) -> err != null ? err.toString() : null) + .toCompletableFuture() + .getNow(null); + default -> throw new IllegalStateException("Unknown state: " + phase); + }; var changes = (Function>) (solution) -> diff --git a/app/src/main/java/org/astraea/app/web/BrokerHandler.java b/app/src/main/java/org/astraea/app/web/BrokerHandler.java index e735df1831..3fbc6ce992 100644 --- a/app/src/main/java/org/astraea/app/web/BrokerHandler.java +++ b/app/src/main/java/org/astraea/app/web/BrokerHandler.java @@ -25,7 +25,6 @@ import java.util.concurrent.CompletionStage; import java.util.stream.Collectors; import org.astraea.common.admin.Admin; -import org.astraea.common.admin.Broker; import org.astraea.common.admin.TopicPartitionPath; class BrokerHandler implements Handler { diff --git a/app/src/main/java/org/astraea/app/web/RecordHandler.java b/app/src/main/java/org/astraea/app/web/RecordHandler.java index a2eda70c95..77f5431b64 100644 --- a/app/src/main/java/org/astraea/app/web/RecordHandler.java +++ b/app/src/main/java/org/astraea/app/web/RecordHandler.java @@ -44,7 +44,6 @@ import org.astraea.common.consumer.Deserializer; import org.astraea.common.consumer.SeekStrategy; import org.astraea.common.consumer.SubscribedConsumer; -import org.astraea.common.json.JsonConverter; import org.astraea.common.json.TypeRef; import org.astraea.common.producer.Producer; import org.astraea.common.producer.ProducerConfigs; @@ -398,11 +397,6 @@ private GetResponse(Consumer consumer, Collection record this.records = records; } - @Override - public String json() { - return JsonConverter.defaultConverter().toJson(this); - } - @Override public void onComplete(Throwable error) { try { diff --git a/app/src/test/java/org/astraea/app/performance/PerformanceTest.java b/app/src/test/java/org/astraea/app/performance/PerformanceTest.java index 2150b5c979..d5e1579c8c 100644 --- a/app/src/test/java/org/astraea/app/performance/PerformanceTest.java +++ b/app/src/test/java/org/astraea/app/performance/PerformanceTest.java @@ -259,7 +259,7 @@ void testPartitionSupplier() { .get() .partition(); // no specify broker - Assertions.assertTrue(-1 == partition); + Assertions.assertEquals(-1, partition); // Test no partition in specified broker var topicName3 = Utils.randomString(10); diff --git a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java index 03bdd829a8..5f7158e532 100644 --- a/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/BalancerHandlerTest.java @@ -153,7 +153,7 @@ void testReport() { c.before.forEach( p -> { // if the topic is generated by this test, it should have data - if (topics.contains(c.topic)) Assertions.assertNotEquals(0, p.size); + if (topics.contains(c.topic)) Assertions.assertNotEquals(0, p.size.get()); // otherwise, we just check non-null else Assertions.assertNotNull(p.size); })); diff --git a/common/src/main/java/org/astraea/common/ByteUtils.java b/common/src/main/java/org/astraea/common/ByteUtils.java index a126e688c2..ec568e660e 100644 --- a/common/src/main/java/org/astraea/common/ByteUtils.java +++ b/common/src/main/java/org/astraea/common/ByteUtils.java @@ -308,7 +308,7 @@ public static Map> readBeanObjects(byte[] bytes) { return outerClusterBean.getAllBeansMap().entrySet().stream() .collect( Collectors.toUnmodifiableMap( - k -> k.getKey(), + Map.Entry::getKey, v -> v.getValue().getBeanObjectsList().stream() .map( diff --git a/common/src/main/java/org/astraea/common/admin/AdminImpl.java b/common/src/main/java/org/astraea/common/admin/AdminImpl.java index e6f858868b..716204bf56 100644 --- a/common/src/main/java/org/astraea/common/admin/AdminImpl.java +++ b/common/src/main/java/org/astraea/common/admin/AdminImpl.java @@ -715,7 +715,7 @@ private CompletionStage> replicas(Set topics) { .sorted( Comparator.comparing(Replica::topic) .thenComparing(Replica::partition) - .thenComparing(r -> r.brokerId())) + .thenComparing(Replica::brokerId)) .toList()); } diff --git a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java index cf874fc9d0..ac39457194 100644 --- a/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/OptimizedClusterInfo.java @@ -91,7 +91,7 @@ class OptimizedClusterInfo implements ClusterInfo { () -> all.stream() .collect( - Collectors.groupingBy(r -> r.brokerId(), Collectors.toUnmodifiableList()))); + Collectors.groupingBy(Replica::brokerId, Collectors.toUnmodifiableList()))); this.byTopic = Lazy.of( diff --git a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java index fe4db19073..59d412430e 100644 --- a/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java +++ b/common/src/main/java/org/astraea/common/balancer/BalancerConsoleImpl.java @@ -270,9 +270,9 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(x -> x.brokerId())) + .thenComparing(Replica::brokerId)) .map(x -> Map.entry(x.brokerId(), x.path())) - .collect(Collectors.toUnmodifiableList()))); + .toList())); return admin .topicNames(true) .thenCompose(admin::clusterInfo) @@ -292,9 +292,9 @@ private CompletionStage checkPlanConsistency(Balancer.Plan plan) { .sorted( Comparator.comparing(Replica::isPreferredLeader) .reversed() - .thenComparing(x -> x.brokerId())) + .thenComparing(Replica::brokerId)) .map(x -> Map.entry(x.brokerId(), x.path())) - .collect(Collectors.toUnmodifiableList()))); + .toList())); var mismatchPartitions = before.entrySet().stream() .filter( diff --git a/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java b/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java index 796fb89b14..5bbecde7c2 100644 --- a/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java +++ b/common/src/main/java/org/astraea/common/balancer/executor/StraightPlanExecutor.java @@ -79,7 +79,7 @@ public CompletionStage run(Admin admin, ClusterInfo logAllocation, Duratio .collect( Collectors.groupingBy( Replica::topicPartition, - Collectors.mapping(r -> r.brokerId(), Collectors.toList())))) + Collectors.mapping(Replica::brokerId, Collectors.toList())))) .thenApply(ignored -> replicas)) // step 2: wait replicas get reassigned .thenCompose( diff --git a/common/src/main/java/org/astraea/common/cost/HasPartitionCost.java b/common/src/main/java/org/astraea/common/cost/HasPartitionCost.java index f396f4906b..6400ba7bbd 100644 --- a/common/src/main/java/org/astraea/common/cost/HasPartitionCost.java +++ b/common/src/main/java/org/astraea/common/cost/HasPartitionCost.java @@ -61,16 +61,14 @@ public Map value() { @Override public Map> incompatibility() { - var incompatibleSet = - partitionCost.values().stream() - .map(PartitionCost::incompatibility) - .flatMap(m -> m.entrySet().stream()) - .collect( - Collectors.groupingBy( - Map.Entry::getKey, - Collectors.flatMapping( - e -> e.getValue().stream(), Collectors.toUnmodifiableSet()))); - return incompatibleSet; + return partitionCost.values().stream() + .map(PartitionCost::incompatibility) + .flatMap(m -> m.entrySet().stream()) + .collect( + Collectors.groupingBy( + Map.Entry::getKey, + Collectors.flatMapping( + e -> e.getValue().stream(), Collectors.toUnmodifiableSet()))); } }; } diff --git a/common/src/main/java/org/astraea/common/cost/MigrationCost.java b/common/src/main/java/org/astraea/common/cost/MigrationCost.java index 9bb6b92274..b2bbbcac09 100644 --- a/common/src/main/java/org/astraea/common/cost/MigrationCost.java +++ b/common/src/main/java/org/astraea/common/cost/MigrationCost.java @@ -109,7 +109,7 @@ private static Map migratedChanged( }) .collect( Collectors.groupingBy( - r -> r.brokerId(), + Replica::brokerId, Collectors.mapping( Function.identity(), Collectors.summingLong(replicaFunction::apply)))); return Stream.concat(dest.brokers().stream(), source.brokers().stream()) diff --git a/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java b/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java index d633a70774..4250900ddf 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkIngressCost.java @@ -60,7 +60,7 @@ public PartitionCost partitionCost(ClusterInfo clusterInfo, ClusterBean clusterB .filter(Replica::isOnline) .collect( Collectors.groupingBy( - replica -> replica.brokerId(), + Replica::brokerId, Collectors.toMap( Replica::topicPartition, r -> partitionTraffic.get(r.topicPartition())))); diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java index e02ff2e6b1..e42dc5763a 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaLeaderSizeCost.java @@ -66,7 +66,7 @@ public BrokerCost brokerCost(ClusterInfo clusterInfo, ClusterBean clusterBean) { clusterInfo.replicas().stream() .collect( Collectors.groupingBy( - r -> r.brokerId(), + Replica::brokerId, Collectors.mapping( r -> clusterInfo diff --git a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java index 299be20bdf..d4bf7f53e5 100644 --- a/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java +++ b/common/src/main/java/org/astraea/common/cost/ReplicaNumberCost.java @@ -22,6 +22,7 @@ import org.astraea.common.Configuration; import org.astraea.common.admin.Broker; import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Replica; import org.astraea.common.metrics.ClusterBean; /** more replicas migrate -> higher cost */ @@ -58,7 +59,7 @@ public ClusterCost clusterCost(ClusterInfo clusterInfo, ClusterBean clusterBean) var replicaPerBroker = clusterInfo .replicaStream() - .collect(Collectors.groupingBy(r -> r.brokerId(), Collectors.counting())); + .collect(Collectors.groupingBy(Replica::brokerId, Collectors.counting())); var summary = replicaPerBroker.values().stream().mapToLong(x -> x).summaryStatistics(); var anyBrokerEmpty = diff --git a/common/src/main/java/org/astraea/common/json/TypeRef.java b/common/src/main/java/org/astraea/common/json/TypeRef.java index c0d9532133..3076fe03d6 100644 --- a/common/src/main/java/org/astraea/common/json/TypeRef.java +++ b/common/src/main/java/org/astraea/common/json/TypeRef.java @@ -77,12 +77,10 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof TypeRef)) { + if (!(o instanceof TypeRef typeRef)) { return false; } - TypeRef typeRef = (TypeRef) o; - return getType() != null ? getType().equals(typeRef.getType()) : typeRef.getType() == null; } } diff --git a/common/src/main/java/org/astraea/common/partitioner/Partitioner.java b/common/src/main/java/org/astraea/common/partitioner/Partitioner.java index ab5be8282a..4593914e41 100644 --- a/common/src/main/java/org/astraea/common/partitioner/Partitioner.java +++ b/common/src/main/java/org/astraea/common/partitioner/Partitioner.java @@ -166,8 +166,8 @@ public final int partition( return target; } - boolean tryToUpdate() { - if (admin == null) return false; + void tryToUpdate() { + if (admin == null) return; var now = System.nanoTime(); // need to refresh cluster info if lease expires if (lastUpdated.updateAndGet(last -> now - last >= CLUSTER_INFO_LEASE.toNanos() ? now : last) @@ -182,9 +182,7 @@ boolean tryToUpdate() { lastUpdated.set(System.nanoTime()); } }); - return true; } - return false; } @Override diff --git a/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java b/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java index 14aa123109..9d421096c3 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java @@ -116,50 +116,48 @@ Stream random(int seed) { var random = new Random(seed); return Stream.generate( () -> { - switch (random.nextInt(5)) { - case 0: // topic metrics - case 1: // broker topic metrics - { - var domainName = ServerMetrics.DOMAIN_NAME; - var properties = - Map.of( - "type", "BrokerTopicMetrics", - "topic", fakeTopics.get(random.nextInt(10)), - "name", "BytesInPerSec"); - return new ServerMetrics.Topic.Meter( - new BeanObject(domainName, properties, Map.of())); - } - case 2: // topic/partition metrics - case 3: // topic/partition/replica metrics - { - var domainName = LogMetrics.DOMAIN_NAME; - var properties = - Map.of( - "type", - "Log", - "topic", - fakeTopics.get(random.nextInt(10)), - "partition", - String.valueOf(random.nextInt(3)), - "name", - "Size"); - return new LogMetrics.Log.Gauge(new BeanObject(domainName, properties, Map.of())); - } - case 4: // noise - { - var domainName = "RandomMetrics"; - var properties = new HashMap(); - properties.put("name", "whatever-" + (random.nextInt(100))); - properties.put("type", "something"); - if (random.nextInt(2) == 0) - properties.put("topic", fakeTopics.get(random.nextInt(3))); - if (random.nextInt(2) == 0) - properties.put("partition", String.valueOf(random.nextInt(3))); - var beanObject = new BeanObject(domainName, properties, Map.of()); - return () -> beanObject; - } - default: - throw new RuntimeException(); + switch (random.nextInt(5)) { // topic metrics + case 0, 1 -> // broker topic metrics + { + var domainName = ServerMetrics.DOMAIN_NAME; + var properties = + Map.of( + "type", "BrokerTopicMetrics", + "topic", fakeTopics.get(random.nextInt(10)), + "name", "BytesInPerSec"); + return new ServerMetrics.Topic.Meter( + new BeanObject(domainName, properties, Map.of())); + } + // topic/partition metrics + case 2, 3 -> // topic/partition/replica metrics + { + var domainName = LogMetrics.DOMAIN_NAME; + var properties = + Map.of( + "type", + "Log", + "topic", + fakeTopics.get(random.nextInt(10)), + "partition", + String.valueOf(random.nextInt(3)), + "name", + "Size"); + return new LogMetrics.Log.Gauge(new BeanObject(domainName, properties, Map.of())); + } + case 4 -> // noise + { + var domainName = "RandomMetrics"; + var properties = new HashMap(); + properties.put("name", "whatever-" + (random.nextInt(100))); + properties.put("type", "something"); + if (random.nextInt(2) == 0) + properties.put("topic", fakeTopics.get(random.nextInt(3))); + if (random.nextInt(2) == 0) + properties.put("partition", String.valueOf(random.nextInt(3))); + var beanObject = new BeanObject(domainName, properties, Map.of()); + return () -> beanObject; + } + default -> throw new RuntimeException(); } }); } diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java index 83407490fa..57e1b6ef3b 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConfigTestSuite.java @@ -567,21 +567,19 @@ static void assertOnlyAllowedBrokerMovement( .filter(r -> !allowedBroker.test(r.brokerId())) // they should exist as-is in the target allocation .forEach( - fixedReplica -> { - target - .replicaStream() - .filter(targetReplica -> targetReplica.equals(fixedReplica)) - .findFirst() - .ifPresentOrElse( - (r) -> {}, - () -> { - Assertions.fail( - name - + ": Expect replica " - + fixedReplica - + " not moved, but it appears to disappear from the target allocation"); - }); - }); + fixedReplica -> + target + .replicaStream() + .filter(targetReplica -> targetReplica.equals(fixedReplica)) + .findFirst() + .ifPresentOrElse( + (r) -> {}, + () -> + Assertions.fail( + name + + ": Expect replica " + + fixedReplica + + " not moved, but it appears to disappear from the target allocation"))); } static void assertBrokerEmpty(ClusterInfo target, Predicate clearBroker, String name) { diff --git a/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java b/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java index ce0d8e6ae8..cb945b82ca 100644 --- a/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java +++ b/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java @@ -557,7 +557,6 @@ void testIterator(boolean isAssigned) { ConsumerConfigs.AUTO_OFFSET_RESET_CONFIG, ConsumerConfigs.AUTO_OFFSET_RESET_EARLIEST) .iterator(List.of(limit)); - ; return StreamSupport.stream( Spliterators.spliteratorUnknownSize(iter, Spliterator.ORDERED), false); }; diff --git a/common/src/test/java/org/astraea/common/cost/MoveCostTest.java b/common/src/test/java/org/astraea/common/cost/MoveCostTest.java index f5a3dba6cc..81ab366e88 100644 --- a/common/src/test/java/org/astraea/common/cost/MoveCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/MoveCostTest.java @@ -79,7 +79,7 @@ void testSensor() { .equals(ServerMetrics.BrokerTopic.BYTES_IN_PER_SEC.metricName()))); } - class FakeCf implements HasMoveCost { + static class FakeCf implements HasMoveCost { @Override public MetricSensor metricSensor() { return MetricSensor.of( diff --git a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java index 98d7d61404..2545947676 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkCostTest.java @@ -364,40 +364,39 @@ void testExpectedImprovement(int seed) { var originalCost = costFunction.clusterCost(clusterInfo, clusterBean); Function experiment = - (tweaker) -> { - return IntStream.range(0, 10) - .mapToDouble( - (ignore) -> { - var end = System.currentTimeMillis() + Duration.ofMillis(1000).toMillis(); - var timeUp = (Supplier) () -> (System.currentTimeMillis() > end); - var counting = 0; - var next = clusterInfo; - while (!timeUp.get()) { - next = - tweaker - .generate(next) - .parallel() - .limit(30) - .takeWhile(i -> !timeUp.get()) - .map( - cluster -> - Map.entry( - cluster, - costFunction.clusterCost(cluster, clusterBean).value())) - .filter(e -> originalCost.value() > e.getValue()) - .min(Map.Entry.comparingByValue()) - .map(Map.Entry::getKey) - .orElse(next); - counting++; - } - var a = originalCost.value(); - var b = costFunction.clusterCost(next, clusterBean).value(); - System.out.println(counting); - return a - b; - }) - .average() - .orElseThrow(); - }; + (tweaker) -> + IntStream.range(0, 10) + .mapToDouble( + (ignore) -> { + var end = System.currentTimeMillis() + Duration.ofMillis(1000).toMillis(); + var timeUp = (Supplier) () -> (System.currentTimeMillis() > end); + var counting = 0; + var next = clusterInfo; + while (!timeUp.get()) { + next = + tweaker + .generate(next) + .parallel() + .limit(30) + .takeWhile(i -> !timeUp.get()) + .map( + cluster -> + Map.entry( + cluster, + costFunction.clusterCost(cluster, clusterBean).value())) + .filter(e -> originalCost.value() > e.getValue()) + .min(Map.Entry.comparingByValue()) + .map(Map.Entry::getKey) + .orElse(next); + counting++; + } + var a = originalCost.value(); + var b = costFunction.clusterCost(next, clusterBean).value(); + System.out.println(counting); + return a - b; + }) + .average() + .orElseThrow(); long s0 = System.currentTimeMillis(); double small = experiment.apply(smallShuffle); @@ -580,21 +579,12 @@ public LargeTestCase(int brokers, int partitions, int seed) { var random = new Random(seed); this.dataRateSupplier = () -> { - switch (1 + random.nextInt(8)) { - case 1: - case 2: - case 3: - case 4: - case 5: - return DataRate.MB.of((long) (random.nextInt(50))); - case 6: - case 7: - return DataRate.MB.of(3 * (long) (random.nextInt(50))); - case 8: - return DataRate.MB.of(5 * (long) (random.nextInt(50))); - default: - throw new RuntimeException(); - } + return switch (1 + random.nextInt(8)) { + case 1, 2, 3, 4, 5 -> DataRate.MB.of((long) (random.nextInt(50))); + case 6, 7 -> DataRate.MB.of(3 * (long) (random.nextInt(50))); + case 8 -> DataRate.MB.of(5 * (long) (random.nextInt(50))); + default -> throw new RuntimeException(); + }; }; this.clusterInfo = ClusterInfo.builder() diff --git a/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java b/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java index 5c43a8a12b..3a6180abc7 100644 --- a/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/RecordSizeCostTest.java @@ -29,7 +29,6 @@ public class RecordSizeCostTest { private final RecordSizeCost function = new RecordSizeCost(); - ; private final ClusterInfo clusterInfo = ClusterInfo.of( diff --git a/fs/src/main/java/org/astraea/fs/Type.java b/fs/src/main/java/org/astraea/fs/Type.java index e7e367b63a..17722e10ed 100644 --- a/fs/src/main/java/org/astraea/fs/Type.java +++ b/fs/src/main/java/org/astraea/fs/Type.java @@ -19,5 +19,5 @@ public enum Type { NONEXISTENT, FILE, - FOLDER; + FOLDER } diff --git a/gui/src/main/java/org/astraea/gui/Query.java b/gui/src/main/java/org/astraea/gui/Query.java index d10a7b3041..5c752d7115 100644 --- a/gui/src/main/java/org/astraea/gui/Query.java +++ b/gui/src/main/java/org/astraea/gui/Query.java @@ -123,15 +123,13 @@ public boolean required(Map item) { // swallow } } - if (e.getValue() instanceof DataSize) { - var size = ((DataSize) e.getValue()); + if (e.getValue() instanceof DataSize size) { return keyPattern.matcher(e.getKey()).matches() && size.smallerThan(DataSize.of(valueString)); } - if (e.getValue() instanceof LocalDateTime) { - var time = ((LocalDateTime) e.getValue()); + if (e.getValue() instanceof LocalDateTime time) { return keyPattern.matcher(e.getKey()).matches() - && time.compareTo(LocalDateTime.parse(valueString)) < 0; + && time.isBefore(LocalDateTime.parse(valueString)); } return false; }); @@ -160,15 +158,13 @@ public boolean required(Map item) { // swallow } } - if (e.getValue() instanceof DataSize) { - var size = ((DataSize) e.getValue()); + if (e.getValue() instanceof DataSize size) { return keyPattern.matcher(e.getKey()).matches() && size.equals(DataSize.of(valueString)); } - if (e.getValue() instanceof LocalDateTime) { - var time = ((LocalDateTime) e.getValue()); + if (e.getValue() instanceof LocalDateTime time) { return keyPattern.matcher(e.getKey()).matches() - && time.compareTo(LocalDateTime.parse(valueString)) == 0; + && time.isEqual(LocalDateTime.parse(valueString)); } return false; }); @@ -197,15 +193,13 @@ public boolean required(Map item) { // swallow } } - if (e.getValue() instanceof DataSize) { - var size = ((DataSize) e.getValue()); + if (e.getValue() instanceof DataSize size) { return keyPattern.matcher(e.getKey()).matches() && size.greaterThan(DataSize.of(valueString)); } - if (e.getValue() instanceof LocalDateTime) { - var time = ((LocalDateTime) e.getValue()); + if (e.getValue() instanceof LocalDateTime time) { return keyPattern.matcher(e.getKey()).matches() - && time.compareTo(LocalDateTime.parse(valueString)) > 0; + && time.isAfter(LocalDateTime.parse(valueString)); } return false; }); diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java index e60a46546d..8cea538c0c 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/ReplicaNode.java @@ -30,6 +30,7 @@ import org.astraea.common.DataSize; import org.astraea.common.MapUtils; import org.astraea.common.admin.ClusterInfo; +import org.astraea.common.admin.Replica; import org.astraea.common.admin.TopicPartition; import org.astraea.common.admin.TopicPartitionReplica; import org.astraea.common.function.Bi3Function; @@ -195,7 +196,7 @@ static List> allResult(ClusterInfo clusterInfo) { .allMatch( entry -> rs.replicas(entry.getKey()).stream() - .map(r -> r.brokerId()) + .map(Replica::brokerId) .collect(Collectors.toSet()) .containsAll(entry.getValue())), Duration.ofSeconds(10), diff --git a/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java b/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java index d0a4a53ca0..5abe5ddb27 100644 --- a/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java +++ b/gui/src/main/java/org/astraea/gui/tab/topic/TopicNode.java @@ -108,13 +108,9 @@ private static Node metricsNode(Context context) { .mapToDouble(HasRate::fiveMinuteRate) .sum(); switch (metric) { - case BYTES_IN_PER_SEC: - case BYTES_OUT_PER_SEC: - map.put(key, DataSize.Byte.of((long) value)); - break; - default: - map.put(key, value); - break; + case BYTES_IN_PER_SEC, BYTES_OUT_PER_SEC -> map.put( + key, DataSize.Byte.of((long) value)); + default -> map.put(key, value); } }); return map; From f7306b49ca94e68ac2f821636e33dd0fa267d4fc Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sun, 4 Jun 2023 00:22:03 +0800 Subject: [PATCH 54/77] [CONNECTOR] add SinkTaskContext to Connector (#1805) --- .../java/org/astraea/connector/SinkTask.java | 4 ++-- .../org/astraea/connector/SinkTaskContext.java | 16 ++++++++++++++++ .../org/astraea/connector/backup/Exporter.java | 3 ++- .../org/astraea/connector/perf/PerfSink.java | 3 ++- .../org/astraea/connector/perf/PerfSinkTest.java | 4 +++- 5 files changed, 25 insertions(+), 5 deletions(-) diff --git a/connector/src/main/java/org/astraea/connector/SinkTask.java b/connector/src/main/java/org/astraea/connector/SinkTask.java index 6f9533aed0..df11e34dd6 100644 --- a/connector/src/main/java/org/astraea/connector/SinkTask.java +++ b/connector/src/main/java/org/astraea/connector/SinkTask.java @@ -32,7 +32,7 @@ public abstract class SinkTask extends org.apache.kafka.connect.sink.SinkTask { - protected void init(Configuration configuration) { + protected void init(Configuration configuration, SinkTaskContext context) { // empty } @@ -50,7 +50,7 @@ public final String version() { @Override public final void start(Map props) { - init(new Configuration(props)); + init(new Configuration(props), SinkTaskContext.of(context)); } @Override diff --git a/connector/src/main/java/org/astraea/connector/SinkTaskContext.java b/connector/src/main/java/org/astraea/connector/SinkTaskContext.java index a4ad097076..fc2c069c57 100644 --- a/connector/src/main/java/org/astraea/connector/SinkTaskContext.java +++ b/connector/src/main/java/org/astraea/connector/SinkTaskContext.java @@ -23,6 +23,22 @@ public interface SinkTaskContext { + SinkTaskContext EMPTY = + new SinkTaskContext() { + + @Override + public void offset(Map offsets) {} + + @Override + public void offset(TopicPartition topicPartition, long offset) {} + + @Override + public void pause(Collection partitions) {} + + @Override + public void requestCommit() {} + }; + /** * Reset the consumer offsets for the specified partitions. * diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 8bd2f83c6d..49986adc97 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -39,6 +39,7 @@ import org.astraea.connector.SinkConnector; import org.astraea.connector.SinkContext; import org.astraea.connector.SinkTask; +import org.astraea.connector.SinkTaskContext; import org.astraea.fs.FileSystem; public class Exporter extends SinkConnector { @@ -259,7 +260,7 @@ List> recordsFromBuffer() { } @Override - protected void init(Configuration configuration) { + protected void init(Configuration configuration, SinkTaskContext context) { this.topicName = configuration.requireString(TOPICS_KEY); this.path = configuration.requireString(PATH_KEY.name()); this.size = configuration.string(SIZE_KEY.name()).map(DataSize::of).orElse(SIZE_DEFAULT); diff --git a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java index cf50544470..579f90e99f 100644 --- a/connector/src/main/java/org/astraea/connector/perf/PerfSink.java +++ b/connector/src/main/java/org/astraea/connector/perf/PerfSink.java @@ -26,6 +26,7 @@ import org.astraea.connector.SinkConnector; import org.astraea.connector.SinkContext; import org.astraea.connector.SinkTask; +import org.astraea.connector.SinkTaskContext; public class PerfSink extends SinkConnector { @@ -68,7 +69,7 @@ public static class Task extends SinkTask { private volatile long lastPut = System.currentTimeMillis(); @Override - protected void init(Configuration configuration) { + protected void init(Configuration configuration, SinkTaskContext context) { frequency = configuration.string(FREQUENCY_DEF.name()).map(Utils::toDuration).orElse(frequency); } diff --git a/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java b/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java index 9dad45cf99..bc0cef092e 100644 --- a/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java +++ b/connector/src/test/java/org/astraea/connector/perf/PerfSinkTest.java @@ -27,6 +27,7 @@ import org.astraea.common.consumer.Record; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.connector.ConnectorMetrics; +import org.astraea.connector.SinkTaskContext; import org.astraea.it.Service; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; @@ -103,7 +104,8 @@ void testFrequency() { @Test void testTask() { var task = new PerfSink.Task(); - task.init(new Configuration(Map.of(PerfSink.FREQUENCY_DEF.name(), "1s"))); + task.init( + new Configuration(Map.of(PerfSink.FREQUENCY_DEF.name(), "1s")), SinkTaskContext.EMPTY); var now = System.currentTimeMillis(); task.put(List.>of()); From 3d1ebd9c81c0781aec323ade17b4fb8a7c7cb7ed Mon Sep 17 00:00:00 2001 From: Zheng-Xian Li Date: Mon, 5 Jun 2023 22:07:20 +0800 Subject: [PATCH 55/77] [DOCS] Fix typo in `web_api_balancer_chinese.md` (#1806) --- docs/web_server/web_api_balancer_chinese.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/web_server/web_api_balancer_chinese.md b/docs/web_server/web_api_balancer_chinese.md index 432da98212..1eb036c488 100644 --- a/docs/web_server/web_api_balancer_chinese.md +++ b/docs/web_server/web_api_balancer_chinese.md @@ -12,14 +12,14 @@ POST /balancer 參數 -| 名稱 | 說明 | 預設值 | -|----------------|-------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| clusterCosts | (必填) 指定要優化的目標以及對應權重 | 無 | -| timeout | (選填) 指定產生時間 | 3s | -| balancer | (選填) 欲使用的負載優化計劃搜尋演算法 | org.astraea.common.balancer.algorithms.GreedyBalancer | -| balancerConfig | (選填) 搜尋演算法的實作細節參數,此為一個 JSON Object 內含一系列的 key/value String | 無 | -| moveCosts | (必填) 指定要計算的各項搬移成本 | "org.astraea.common.cost.ReplicaLeaderCost", "org.astraea.common.cost.RecordSizeCost", "org.astraea.common.cost.ReplicaNumberCost", "org.astraea.common.cost.ReplicaLeaderSizeCost" | -| costConfig | (選填) 針對各個搬移成本做限制,讓,此為一個 JSON Object 內含一系列的 key/value String | 無 | +| 名稱 | 說明 | 預設值 | +|----------------|------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| clusterCosts | (必填) 指定要優化的目標以及對應權重 | 無 | +| timeout | (選填) 指定產生時間 | 3s | +| balancer | (選填) 欲使用的負載優化計劃搜尋演算法 | org.astraea.common.balancer.algorithms.GreedyBalancer | +| balancerConfig | (選填) 搜尋演算法的實作細節參數,此為一個 JSON Object 內含一系列的 key/value String | 無 | +| moveCosts | (必填) 指定要計算的各項搬移成本 | "org.astraea.common.cost.ReplicaLeaderCost", "org.astraea.common.cost.RecordSizeCost", "org.astraea.common.cost.ReplicaNumberCost", "org.astraea.common.cost.ReplicaLeaderSizeCost" | +| costConfig | (選填) 針對各個搬移成本做限制,此為一個 JSON Object 內含一系列的 key/value String | 無 | 官方記錄的 balancerConfig: From 176fd4a4956dabb4d719c93d4bc32892e43bc9c6 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 7 Jun 2023 22:06:53 +0800 Subject: [PATCH 56/77] [COMMON] upgrade kafka from 3.4.0 to 3.4.1 (#1809) --- docker/start_broker.sh | 4 ++-- docker/start_controller.sh | 4 ++-- docker/start_worker.sh | 4 ++-- gradle/dependencies.gradle | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/start_broker.sh b/docker/start_broker.sh index cb3a6d346a..42dd600702 100755 --- a/docker/start_broker.sh +++ b/docker/start_broker.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.4.0}} +declare -r VERSION=${REVISION:-${VERSION:-3.4.1}} declare -r DOCKERFILE=$DOCKER_FOLDER/broker.dockerfile declare -r DATA_FOLDER_IN_CONTAINER_PREFIX="/tmp/log-folder" declare -r EXPORTER_VERSION="0.16.1" @@ -61,7 +61,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set broker JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.4.0 set version of kafka distribution" + echo " VERSION=3.4.1 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " DATA_FOLDERS=/tmp/folder1 set host folders used by broker" diff --git a/docker/start_controller.sh b/docker/start_controller.sh index 0fab50d5ef..09e462fa56 100755 --- a/docker/start_controller.sh +++ b/docker/start_controller.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.4.0}} +declare -r VERSION=${REVISION:-${VERSION:-3.4.1}} declare -r DOCKERFILE=$DOCKER_FOLDER/controller.dockerfile declare -r EXPORTER_VERSION="0.16.1" declare -r CLUSTER_ID=${CLUSTER_ID:-"$(randomString)"} @@ -52,7 +52,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set controller JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.4.0 set version of kafka distribution" + echo " VERSION=3.4.1 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " META_FOLDER=/tmp/folder1 set host folder used by controller" diff --git a/docker/start_worker.sh b/docker/start_worker.sh index 566a4930e6..201187adca 100755 --- a/docker/start_worker.sh +++ b/docker/start_worker.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.4.0}} +declare -r VERSION=${REVISION:-${VERSION:-3.4.1}} declare -r DOCKERFILE=$DOCKER_FOLDER/worker.dockerfile declare -r WORKER_PORT=${WORKER_PORT:-"$(getRandomPort)"} declare -r CONTAINER_NAME="worker-$WORKER_PORT" @@ -50,7 +50,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set worker JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.4.0 set version of kafka distribution" + echo " VERSION=3.4.1 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " WORKER_PLUGIN_PATH=/tmp/worker-plugins set plugin path to kafka worker" diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index a46e0cff21..e4aa608e24 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -28,7 +28,7 @@ def versions = [ jackson : project.properties['jackson.version'] ?: "2.14.2", jcommander : project.properties['jcommander.version'] ?: "1.82", junit : project.properties['junit.version'] ?: "5.9.2", - kafka : project.properties['kafka.version'] ?: "3.4.0", + kafka : project.properties['kafka.version'] ?: "3.4.1", mockito : project.properties['mockito.version'] ?: "5.2.0", "opencsv" : project.properties['opencsv.version'] ?: "5.7.1", scala : project.properties['scala.version'] ?: "2.13.10", From 6da4e09e4add26b48d26767ede19760abc801d86 Mon Sep 17 00:00:00 2001 From: Haser Date: Fri, 9 Jun 2023 20:34:35 +0800 Subject: [PATCH 57/77] [BACKUP] Make user can specify offset targets to backup (#1709) * remove topicName in task making topics can use regex. * add function to update target ranges for topic partitions based on assigned target parameters * make the task can seek to the next target offset. * seek the offset after cleaning all records in put * remove entrySet * spotlessJavaApply * resolve conflict * rename unit test name * make some methods in targetStatus return Optional, also the range in targets is not necessary * the task will pause consumption from the topicPartition which subsequent messages are useless * update the constructor of TargetStatus for deep copying * fix format violations * replace all parts that use kafka context with TaskContext in Exporter * move taskContext to super class SinkTask * remove class TargetStatus and simplify the functionality. the user can now only specify the "from" parameter and also added the method requireStringByRegex in Configuration. * uncomment for unnecessary dependency * change name to requireRegex, also return an empty map instead optional empty * fix format violations * prevent reset the offset infinitely if we seek to the offset which exceeded the latest offset * add new map for checking the target offset for a whole topic instead of topic partition, also add a definition for FROM_OFFSET_REGEX_KEY. * update doc file to specify the offset for the backup * extract methods checkSeekOffset and targetOffset with unit test * replace Long with Optional * simplify the if else statement * remove checking seeked topic partition * add docs about setting offset from warning * fix format violations * remove unnecessary field --- .../org/astraea/common/Configuration.java | 11 ++ .../astraea/connector/backup/Exporter.java | 106 +++++++++++++++++- .../connector/backup/ExporterTest.java | 90 +++++++++++++++ docs/connector/exporter.md | 60 ++++++++-- 4 files changed, 253 insertions(+), 14 deletions(-) diff --git a/common/src/main/java/org/astraea/common/Configuration.java b/common/src/main/java/org/astraea/common/Configuration.java index d1e27ff877..e84bb8b3a5 100644 --- a/common/src/main/java/org/astraea/common/Configuration.java +++ b/common/src/main/java/org/astraea/common/Configuration.java @@ -108,4 +108,15 @@ public Configuration filteredPrefixConfigs(String prefix) { public List list(String key, String separator) { return string(key).map(s -> Arrays.asList(s.split(separator))).orElseGet(List::of); } + + /** + * @param regex the regex pattern to apply to the keys of the raw map + * @return an Optional containing a Map of key-value entries that match the specified regex + * pattern. + */ + public Map requireRegex(String regex) { + return raw.entrySet().stream() + .filter(entry -> Pattern.compile(regex).matcher(entry.getKey()).matches()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } } diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 49986adc97..9b537911c6 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -20,7 +20,9 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; @@ -122,6 +124,16 @@ public class Exporter extends SinkConnector { "a value that represents the capacity of a blocking queue from which the writer can take records.") .defaultValue(BUFFER_SIZE_DEFAULT.toString()) .build(); + + static Definition FROM_OFFSET_REGEX_KEY = + Definition.builder() + .name(".*offset.from") + .type(Definition.Type.STRING) + .documentation( + "a value that specifies the starting offset value for the " + + "backups of a particular topic or topic partition. it can be used in 2 ways: " + + "'.offset.from' or '..offset.from'.") + .build(); private Configuration configs; @Override @@ -168,15 +180,25 @@ public static class Task extends SinkTask { private long bufferSizeLimit; FileSystem fs; - String topicName; String path; DataSize size; long interval; + // a map of > + private final Map> offsetForTopicPartition = new HashMap<>(); + + private final Map offsetForTopic = new HashMap<>(); + + // visible for test + protected final Map seekOffset = new HashMap<>(); + + private SinkTaskContext taskContext; + RecordWriter createRecordWriter(TopicPartition tp, long offset) { var fileName = String.valueOf(offset); return RecordWriter.builder( - fs.write(String.join("/", path, topicName, String.valueOf(tp.partition()), fileName))) + fs.write( + String.join("/", path, tp.topic(), String.valueOf(tp.partition()), fileName))) .build(); } @@ -261,7 +283,6 @@ List> recordsFromBuffer() { @Override protected void init(Configuration configuration, SinkTaskContext context) { - this.topicName = configuration.requireString(TOPICS_KEY); this.path = configuration.requireString(PATH_KEY.name()); this.size = configuration.string(SIZE_KEY.name()).map(DataSize::of).orElse(SIZE_DEFAULT); this.interval = @@ -277,6 +298,25 @@ protected void init(Configuration configuration, SinkTaskContext context) { .map(DataSize::of) .orElse(BUFFER_SIZE_DEFAULT) .bytes(); + this.taskContext = context; + + // fetches key-value pairs from the configuration's variable matching the regular expression + // '.*offset.from', updates the values of 'offsetForTopic' or 'offsetForTopicPartition' based + // on the + // key's prefix. + configuration + .requireRegex(FROM_OFFSET_REGEX_KEY.name()) + .forEach( + (k, v) -> { + var splitKey = k.split("\\."); + if (splitKey.length == 3) { + this.offsetForTopic.put(splitKey[0], Long.valueOf(v)); + } else { + this.offsetForTopicPartition.put( + splitKey[0], Map.of(splitKey[1], Long.valueOf(v))); + } + }); + this.fs = FileSystem.of(configuration.requireString(SCHEMA_KEY.name()), configuration); this.writerFuture = CompletableFuture.runAsync(createWriter()); } @@ -287,6 +327,8 @@ protected void put(List> records) { r -> Utils.packException( () -> { + if (!isValid(r)) return; + int recordLength = Stream.of(r.key(), r.value()) .filter(Objects::nonNull) @@ -302,6 +344,64 @@ protected void put(List> records) { this.bufferSize.add(recordLength); })); + this.seekOffset + .entrySet() + .iterator() + .forEachRemaining( + entry -> { + this.taskContext.requestCommit(); + this.taskContext.offset(entry.getKey(), entry.getValue()); + this.seekOffset.remove(entry.getKey()); + }); + } + + protected boolean isValid(Record r) { + var targetOffset = targeOffset(r); + + // If the target offset exists and the record's offset is less than the target offset, + // set the seek offset to the target offset and return false. + if (targetOffset.isPresent() && r.offset() < targetOffset.get()) { + this.seekOffset.put(r.topicPartition(), targetOffset.get()); + return false; + } + + checkSeekOffset(r); + return true; + } + + /** + * Retrieves the target offset for the specified topic and partition. + * + * @param r {@link Record} + * @return the target offset for the specified topic and partition, or null if the target offset + * is not found. + */ + // visible for test + protected Optional targeOffset(Record r) { + var topicMap = this.offsetForTopicPartition.get(r.topic()); + + // If we are unable to obtain the target offset from the 'offsetForTopicPartition' map, + // we will attempt to retrieve it from another map called 'offsetForTopic'. + if (topicMap != null && topicMap.get(String.valueOf(r.partition())) != null) + return Optional.ofNullable(topicMap.get(String.valueOf(r.partition()))); + return Optional.ofNullable(this.offsetForTopic.get(r.topic())); + } + + /** + * Checks if the record's offset is greater or equal to the seek offset for this topicPartition. + * If the record's offset is greater or equal to the seek offset for this topicPartition, the + * seek offset will be removed. + * + *

    This method is used to prevent reset offset infinitely. + * + *

    This method is called by {@link #isValid(Record)} + * + * @param r {@link Record} + */ + private void checkSeekOffset(Record r) { + var seekOffset = this.seekOffset.get(r.topicPartition()); + if (seekOffset != null && seekOffset <= r.offset()) + this.seekOffset.remove(r.topicPartition()); } @Override diff --git a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java index dcfe2bb61b..0f88ab4435 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java @@ -842,4 +842,94 @@ void testWriteRecords() { Assertions.assertEquals(0, task.recordsQueue.size()); } } + + @Test + void testIsValid() { + try (var server = HdfsServer.local()) { + var topicName = Utils.randomString(10); + var path = "/test"; + var configs = + Map.of( + "fs.schema", + "hdfs", + "topics", + topicName, + "fs.hdfs.hostname", + String.valueOf(server.hostname()), + "fs.hdfs.port", + String.valueOf(server.port()), + "fs.hdfs.user", + String.valueOf(server.user()), + "path", + path, + "test1.offset.from", + "100", + "test2.0.offset.from", + "200"); + + var task = new Exporter.Task(); + + task.start(configs); + + var record1 = + Record.builder() + .topic("test0") + .key("test".getBytes()) + .value("test".getBytes()) + .partition(0) + .offset(1) + .timestamp(System.currentTimeMillis()) + .build(); + var record2 = + Record.builder() + .topic("test1") + .key("test".getBytes()) + .value("test".getBytes()) + .partition(0) + .offset(1) + .timestamp(System.currentTimeMillis()) + .build(); + var record3 = + Record.builder() + .topic("test2") + .key("test".getBytes()) + .value("test".getBytes()) + .partition(0) + .offset(1) + .timestamp(System.currentTimeMillis()) + .build(); + var record4 = + Record.builder() + .topic("test2") + .key("test".getBytes()) + .value("test".getBytes()) + .partition(0) + .offset(200) + .timestamp(System.currentTimeMillis()) + .build(); + var record5 = + Record.builder() + .topic("test2") + .key("test".getBytes()) + .value("test".getBytes()) + .partition(1) + .offset(1) + .timestamp(System.currentTimeMillis()) + .build(); + + Assertions.assertFalse(task.targeOffset(record1).isPresent()); + Assertions.assertEquals(100, task.targeOffset(record2).orElse(null)); + Assertions.assertEquals(200, task.targeOffset(record3).orElse(null)); + Assertions.assertEquals(200, task.targeOffset(record4).orElse(null)); + Assertions.assertFalse(task.targeOffset(record5).isPresent()); + + Assertions.assertTrue(task.isValid(record1)); + Assertions.assertFalse(task.isValid(record2)); + Assertions.assertFalse(task.isValid(record3)); + Assertions.assertEquals(2, task.seekOffset.size()); + Assertions.assertTrue(task.isValid(record4)); + Assertions.assertTrue(task.isValid(record5)); + Assertions.assertEquals(1, task.seekOffset.size()); + } + } } diff --git a/docs/connector/exporter.md b/docs/connector/exporter.md index 55ff94e4d2..371d17e186 100644 --- a/docs/connector/exporter.md +++ b/docs/connector/exporter.md @@ -13,26 +13,30 @@ 執行 Exporter 時所需要的參數及說明如下 -| 參數名稱 | 說明 | 預設值 | -|:--------------------------|--------------------------------------------------------------------------------------------------------------|-------| -| fs.schema | (必填) 決定儲存目標為何種檔案系統,例如: `local`, `ftp`等 | 無 | -| path | (必填) 填入目標檔案系統要儲存的資料夾目錄之目標位置 | 無 | -| fs.{file System}.hostname | (選填) 如果最初的 `fs.schema` 選定為非 `local` 之項目,需要填入目標 `host name`, `file System` 取決於前者之內容 | 無 | -| fs.{file System}.port | (選填) 填入目標檔案系統之 `port` | 無 | -| fs.{file System}.user | (選填) 填入目標檔案系統之登入 `user` | 無 | -| fs.{file System}.password | (選填) 填入目標檔案系統之登入 `password` | 無 | -| size | (選填) 寫入檔案目標超過此設定之大小上限時會創見新檔案,並且寫入目標改為新創建之檔案。
    檔案大小單位: `Bit`, `Kb`, `KiB`, `Mb`, etc. | 100MB | -| roll.duration | (選填) 如果 `connector` 在超過此時間沒有任何資料流入,會把當下所有已創建之檔案關閉,並在之後有新資料時會創建新檔案並寫入。
    時間單位: `s`, `m`, `h`, `day`, etc. | 3s | - +| 參數名稱 | 說明 | 預設值 | +|:--------------------------------|-----------------------------------------------------------------------------------------------------------------|-------| +| fs.schema | (必填) 決定儲存目標為何種檔案系統,例如: `local`, `ftp`等 | 無 | +| path | (必填) 填入目標檔案系統要儲存的資料夾目錄之目標位置 | 無 | +| fs.{file System}.hostname | (選填) 如果最初的 `fs.schema` 選定為非 `local` 之項目,需要填入目標 `host name`, `file System` 取決於前者之內容 | 無 | +| fs.{file System}.port | (選填) 填入目標檔案系統之 `port` | 無 | +| fs.{file System}.user | (選填) 填入目標檔案系統之登入 `user` | 無 | +| fs.{file System}.password | (選填) 填入目標檔案系統之登入 `password` | 無 | +| size | (選填) 寫入檔案目標超過此設定之大小上限時會創見新檔案,並且寫入目標改為新創建之檔案。
    檔案大小單位: `Bit`, `Kb`, `KiB`, `Mb`, etc. | 100MB | +| roll.duration | (選填) 如果 `connector` 在超過此時間沒有任何資料流入,會把當下所有已創建之檔案關閉,並在之後有新資料時會創建新檔案並寫入。
    時間單位: `s`, `m`, `h`, `day`, etc. | 3s | +| ..offset.from | (選填) 針對想要的 `topic` 或是 `topicPartition` 指定備份 offset 的起點,如果要針對整個 topic 指定需將 `partition` 留空。ex: `test.offset.from` | 無 | ##### 注意 + +###### converter 如果`worker`預設之`converter`並非為`byte array converter` 時,需要在將以下參數使設定為`org.apache.kafka.connect.converters.ByteArrayConverter`使本工具順利執行。 + - key.converter - value.converter - header.converter 因為當`topic`內的資料為非 `worker converter`包裝的話,沒有設定`converter`會出現類似以下的錯誤資訊 + ``` org.apache.kafka.connect.errors.ConnectException: Tolerance exceeded in error handler at org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator.execAndHandleError(RetryWithToleranceOperator.java:223) @@ -46,6 +50,16 @@ Caused by: com.fasterxml.jackson.core.JsonParseException: Unrecognized token 'te ... ``` +###### offset from +此參數之設定需要小心,如果設定之 offset 超出 latest offset, exporter 會參照參數 `auto.reset.offset` 來重置 offset。 +此參數預設值為 `earliest`,即重置到最久之 record,如果需要變更此參數,請在創建 connector 時附上參數: +`"consumer.override.auto.reset.offset": "latest"` +如果將此參數變更為 `latest`,會導致所有備份之 partition 預設從最新資料開始備份。 + +> **Warning** +> 如果使用預設 auto.reset.offset = earliest,需注意各 topic partition 之設定,如果設定超出範圍會導致無限重複消費資訊(但不會備份), +> 需自行停止。 + #### 使用範例 @@ -54,6 +68,29 @@ Caused by: com.fasterxml.jackson.core.JsonParseException: Unrecognized token 'te # 在 worker 中創建 exporter 把 kafka 中的資料寫入到 ftp server 之中, # 每筆資料大小上限為 10MB,topic name 是 test, # 並且如果超過3秒沒有新資料流入,會結束所有正在寫入之檔案,等有新資料流入時會創建新的檔案來寫入 +# + +curl -X POST http://localhost:13575/connectors \ + -H "Content-Type: application/json" \ + -d '{"name": "FtpSink", + "config": { + "connector.class": "Exporter", + "tasks.max": "3", + "topics": "test", + "fs.schema": "ftp", + "fs.ftp.hostname": "localhost", + "fs.ftp.port": "21", + "fs.ftp.user": "user", + "fs.ftp.password": "password", + "size": "10MB", + "roll.duration": "3s" + "path": "/test/10MB", + } + }' +``` + +```shell +# 設定如上,並且新增條件: topic test 前 100 offset 不需備份 curl -X POST http://localhost:13575/connectors \ -H "Content-Type: application/json" \ @@ -70,6 +107,7 @@ curl -X POST http://localhost:13575/connectors \ "size": "10MB", "roll.duration": "3s" "path": "/test/10MB", + "test.offset.from": "101" } }' ``` \ No newline at end of file From f9d6ceda90c17fb5f92a8ed755ad01e45bff373b Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Fri, 9 Jun 2023 23:14:25 +0800 Subject: [PATCH 58/77] [COMMON] Remove open csv (#1807) --- app/build.gradle | 1 - common/build.gradle | 4 +- .../astraea/common/csv/CsvReaderBuilder.java | 40 ++-- .../astraea/common/csv/CsvWriterBuilder.java | 30 +-- .../org/astraea/common/csv/CsvBenchmark.java | 114 +++++++++ .../org/astraea/common/csv/CsvReaderTest.java | 226 ++++++++++++++++-- gradle/dependencies.gradle | 90 +++---- 7 files changed, 409 insertions(+), 96 deletions(-) create mode 100644 common/src/test/java/org/astraea/common/csv/CsvBenchmark.java diff --git a/app/build.gradle b/app/build.gradle index cf5c65bf73..43c14330e0 100644 --- a/app/build.gradle +++ b/app/build.gradle @@ -40,7 +40,6 @@ dependencies { implementation libs["jcommander"] // we don't use slf4j actually, and it is used by kafka so we swallow the log. implementation libs["slf4j-nop"] - implementation libs["opencsv"] implementation libs["commons-math3"] implementation libs["jackson-datatype-jdk8"] } diff --git a/common/build.gradle b/common/build.gradle index ea12107192..a772545153 100644 --- a/common/build.gradle +++ b/common/build.gradle @@ -34,6 +34,8 @@ dependencies { testImplementation libs["mockito-inline"] testImplementation libs["commons-io"] testImplementation libs["kafka-connect-runtime"] + testImplementation libs["jmh-core"] + testAnnotationProcessor libs["jmh-generator-annprocess"] testImplementation project(':it') @@ -41,7 +43,7 @@ dependencies { implementation libs["jackson-datatype-jdk8"] // we don't use slf4j actually, and it is used by kafka so we swallow the log. implementation libs["slf4j-nop"] - implementation libs["opencsv"] + implementation libs["commons-lang3"] implementation libs["commons-math3"] implementation 'com.google.protobuf:protobuf-java:3.22.2' } diff --git a/common/src/main/java/org/astraea/common/csv/CsvReaderBuilder.java b/common/src/main/java/org/astraea/common/csv/CsvReaderBuilder.java index 6cbfdb1969..69e4e0da41 100644 --- a/common/src/main/java/org/astraea/common/csv/CsvReaderBuilder.java +++ b/common/src/main/java/org/astraea/common/csv/CsvReaderBuilder.java @@ -16,27 +16,23 @@ */ package org.astraea.common.csv; -import com.opencsv.CSVReader; -import com.opencsv.CSVReaderBuilder; +import static java.util.Objects.requireNonNull; + +import java.io.BufferedReader; import java.io.Reader; import java.util.Arrays; import java.util.List; import java.util.NoSuchElementException; -import java.util.stream.Collectors; +import java.util.Optional; +import java.util.stream.IntStream; import org.astraea.common.Utils; -/** Construct CsvReaderBuilder so that we can use build pattern of opencsv. */ public class CsvReaderBuilder { - private final CSVReaderBuilder csvReaderBuilder; + private final Reader reader; private Boolean blankLine = false; CsvReaderBuilder(Reader source) { - this.csvReaderBuilder = new CSVReaderBuilder(source); - } - - public CsvReaderBuilder withKeepCarriageReturn(boolean keep) { - this.csvReaderBuilder.withKeepCarriageReturn(keep); - return this; + this.reader = source; } public CsvReaderBuilder blankLine(boolean allow) { @@ -45,25 +41,31 @@ public CsvReaderBuilder blankLine(boolean allow) { } public CsvReader build() { - return new CsvReaderImpl(csvReaderBuilder, blankLine); + return new CsvReaderImpl(reader, blankLine); } private static class CsvReaderImpl implements CsvReader { - private final CSVReader csvReader; + private static final String SEPARATOR = ","; + private final BufferedReader reader; private final boolean blankLine; private long currentLine = 0; private int genericLength = -1; private String[] nextLine; - private CsvReaderImpl(CSVReaderBuilder builder, Boolean blankLine) { + private CsvReaderImpl(Reader reader, boolean blankLine) { + this.reader = new BufferedReader(requireNonNull(reader, "reader is null")); this.blankLine = blankLine; - this.csvReader = builder.build(); } @Override public boolean hasNext() { if (nextLine == null) { - nextLine = Utils.packException(csvReader::readNext); + nextLine = + Utils.packException( + () -> + Optional.ofNullable(reader.readLine()) + .map(line -> line.split(SEPARATOR)) + .orElse(null)); if (nextLine != null) currentLine++; } return nextLine != null; @@ -103,7 +105,7 @@ public List rawNext() { throw new NoSuchElementException("There is no next line."); } try { - return Arrays.stream(nextLine).collect(Collectors.toUnmodifiableList()); + return Arrays.stream(nextLine).toList(); } finally { nextLine = null; } @@ -113,14 +115,14 @@ public List rawNext() { public void skip(int num) { if (num > 0) { currentLine = currentLine + num; - Utils.packException(() -> csvReader.skip(num)); + IntStream.range(0, num).forEach((ignored) -> Utils.packException(reader::readLine)); nextLine = null; } } @Override public void close() { - Utils.close(csvReader); + Utils.close(reader); } } } diff --git a/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java b/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java index 513a06ec53..a9c6ea6521 100644 --- a/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java @@ -16,24 +16,18 @@ */ package org.astraea.common.csv; -import com.opencsv.CSVWriterBuilder; -import com.opencsv.ICSVWriter; +import static java.util.Objects.requireNonNull; + import java.io.Writer; import java.util.List; import org.astraea.common.Utils; -/** Construct CSVWriterBuilder so that we can use its build pattern. */ public class CsvWriterBuilder { - private final CSVWriterBuilder csvWriterBuilder; + private final Writer writer; private boolean blankLine; CsvWriterBuilder(Writer sink) { - this.csvWriterBuilder = new CSVWriterBuilder(sink); - } - - public CsvWriterBuilder withLineEnd(String string) { - this.csvWriterBuilder.withLineEnd(string); - return this; + this.writer = sink; } public CsvWriterBuilder blankLine(boolean blankLine) { @@ -42,17 +36,18 @@ public CsvWriterBuilder blankLine(boolean blankLine) { } public CsvWriter build() { - return new CsvWriterImpl(csvWriterBuilder, blankLine); + return new CsvWriterImpl(writer, blankLine); } private static class CsvWriterImpl implements CsvWriter { - private final ICSVWriter csvWriter; + private static final String SEPARATOR = ","; + private final Writer writer; private final boolean blankLine; private int genericLength = -1; - private CsvWriterImpl(CSVWriterBuilder csvWriterBuilder, boolean blankLine) { + private CsvWriterImpl(Writer writer, boolean blankLine) { + this.writer = requireNonNull(writer); this.blankLine = blankLine; - this.csvWriter = csvWriterBuilder.build(); } @Override @@ -74,17 +69,18 @@ else if (genericLength != nextLine.size()) { @Override public void rawAppend(List nextLine) { - csvWriter.writeNext(nextLine.toArray(new String[0])); + Utils.packException(() -> writer.write(String.join(SEPARATOR, nextLine))); + Utils.packException(() -> writer.write('\n')); } @Override public void flush() { - Utils.packException(csvWriter::flush); + Utils.packException(writer::flush); } @Override public void close() { - Utils.close(csvWriter); + Utils.close(writer); } } } diff --git a/common/src/test/java/org/astraea/common/csv/CsvBenchmark.java b/common/src/test/java/org/astraea/common/csv/CsvBenchmark.java new file mode 100644 index 0000000000..8e6022ca5b --- /dev/null +++ b/common/src/test/java/org/astraea/common/csv/CsvBenchmark.java @@ -0,0 +1,114 @@ +/* + * 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.csv; + +import static java.lang.String.format; +import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; +import static java.util.concurrent.TimeUnit.SECONDS; + +import java.io.File; +import java.io.FileReader; +import java.io.FileWriter; +import java.nio.file.Files; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; +import org.astraea.common.Utils; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.VerboseMode; + +@State(Scope.Thread) +@OutputTimeUnit(SECONDS) +@Fork(1) +@Warmup(iterations = 10, time = 1, timeUnit = SECONDS) +@Measurement(iterations = 10, time = 1, timeUnit = SECONDS) +@BenchmarkMode(Mode.Throughput) +public class CsvBenchmark { + + @Benchmark + public void read(BenchmarkData benchmarkData, Blackhole blackhole) { + benchmarkData.csvReader.forEachRemaining(blackhole::consume); + } + + @Benchmark + public void write(BenchmarkData benchmarkData) { + benchmarkData.data.forEach(line -> benchmarkData.csvWriter.append(line)); + } + + @State(Scope.Thread) + public static class BenchmarkData { + private static final int ROW_COUNT = 15000; + private final List> data = new ArrayList<>(); + private CsvReader csvReader; + private CsvWriter csvWriter; + + @Setup + public void initialize() { + IntStream.range(0, ROW_COUNT) + .forEach((ignored) -> data.add(List.of("a", "b", "c", "d", "e"))); + csvReader = + CsvReader.builder(Utils.packException(() -> new FileReader(prepareCsv(data)))).build(); + csvWriter = CsvWriter.builder(FileWriter.nullWriter()).build(); + } + + private static File prepareCsv(List> csvData) { + File csvFile = + Utils.packException(() -> Files.createTempFile("csv_benchmark", ".csv").toFile()); + try (CsvWriter csvWriter = + CsvWriter.builder(Utils.packException(() -> new FileWriter(csvFile))).build()) { + csvData.forEach(csvWriter::append); + csvWriter.flush(); + return csvFile; + } + } + } + + public static void main(String[] args) throws RunnerException { + BenchmarkData data = new BenchmarkData(); + data.initialize(); + + Options options = + new OptionsBuilder() + .verbosity(VerboseMode.NORMAL) + .resultFormat(ResultFormatType.JSON) + .result( + format( + "%s/%s-result-%s.json", + System.getProperty("java.io.tmpdir"), + CsvBenchmark.class.getSimpleName(), + ISO_DATE_TIME.format(LocalDateTime.now()))) + .build(); + + new Runner(options).run(); + } +} diff --git a/common/src/test/java/org/astraea/common/csv/CsvReaderTest.java b/common/src/test/java/org/astraea/common/csv/CsvReaderTest.java index 23251d2bfd..2cd8b7f83c 100644 --- a/common/src/test/java/org/astraea/common/csv/CsvReaderTest.java +++ b/common/src/test/java/org/astraea/common/csv/CsvReaderTest.java @@ -90,8 +90,42 @@ void blankLineTest() { reader.next(); reader.skip(2); assertEquals( - mkString(reader.next()), - "\"2015-12-05 00:00:00\",\"333\",\"2015-12-07 23:59:00\",\"10.53\",\"0\",\"0.112\",\"9.638579\",\"20.8\",\"16.42\",\"0\",\"29.52\",\"2019-12-07 02:36:00\",\"16.69\",\"14.5\",\"2019-12-07 19:01:00\",\"15.6\",\"2019-12-07 04:59:00\",\"14.91368\",\"1024.1\",\"2019-12-07 22:44:00\",\"1019.6\",\"2019-12-07 01:22:00\",\"90\",\"65.3\",\"18.2\",\"15.7\",\"26\",\"2019-12-07 01:16:00\",\"-33.4\",\"2019-12-07 01:26:00\",\"61.1\",\"0\",\"0\",\"0\""); + List.of( + "2015-12-05 00:00:00", + "333", + "2015-12-07 23:59:00", + "10.53", + "0", + "0.112", + "9.638579", + "20.8", + "16.42", + "0", + "29.52", + "2019-12-07 02:36:00", + "16.69", + "14.5", + "2019-12-07 19:01:00", + "15.6", + "2019-12-07 04:59:00", + "14.91368", + "1024.1", + "2019-12-07 22:44:00", + "1019.6", + "2019-12-07 01:22:00", + "90", + "65.3", + "18.2", + "15.7", + "26", + "2019-12-07 01:16:00", + "-33.4", + "2019-12-07 01:26:00", + "61.1", + "0", + "0", + "0"), + reader.next()); assertThrows( RuntimeException.class, reader::next, @@ -106,8 +140,42 @@ void blankLineTest() { reader.next(); reader.skip(2); assertEquals( - mkString(reader.next()), - "\"2015-12-05 00:00:00\",\"333\",\"2015-12-07 23:59:00\",\"10.53\",\"0\",\"0.112\",\"9.638579\",\"20.8\",\"16.42\",\"0\",\"29.52\",\"2019-12-07 02:36:00\",\"16.69\",\"14.5\",\"2019-12-07 19:01:00\",\"15.6\",\"2019-12-07 04:59:00\",\"14.91368\",\"1024.1\",\"2019-12-07 22:44:00\",\"1019.6\",\"2019-12-07 01:22:00\",\"90\",\"65.3\",\"18.2\",\"15.7\",\"26\",\"2019-12-07 01:16:00\",\"-33.4\",\"2019-12-07 01:26:00\",\"61.1\",\"0\",\"0\",\"0\""); + List.of( + "2015-12-05 00:00:00", + "333", + "2015-12-07 23:59:00", + "10.53", + "0", + "0.112", + "9.638579", + "20.8", + "16.42", + "0", + "29.52", + "2019-12-07 02:36:00", + "16.69", + "14.5", + "2019-12-07 19:01:00", + "15.6", + "2019-12-07 04:59:00", + "14.91368", + "1024.1", + "2019-12-07 22:44:00", + "1019.6", + "2019-12-07 01:22:00", + "90", + "65.3", + "18.2", + "15.7", + "26", + "2019-12-07 01:16:00", + "-33.4", + "2019-12-07 01:26:00", + "61.1", + "0", + "0", + "0"), + reader.next()); reader.next(); } } @@ -135,8 +203,42 @@ void skipTest() { CsvReader.builder(Utils.packException(() -> Files.newBufferedReader(target))).build()) { reader.skip(4); assertEquals( - mkString(reader.next()), - "\"2015-12-05 00:00:00\",\"333\",\"2015-12-07 23:59:00\",\"10.53\",\"0\",\"0.112\",\"9.638579\",\"20.8\",\"16.42\",\"0\",\"29.52\",\"2019-12-07 02:36:00\",\"16.69\",\"14.5\",\"2019-12-07 19:01:00\",\"15.6\",\"2019-12-07 04:59:00\",\"14.91368\",\"1024.1\",\"2019-12-07 22:44:00\",\"1019.6\",\"2019-12-07 01:22:00\",\"90\",\"65.3\",\"18.2\",\"15.7\",\"26\",\"2019-12-07 01:16:00\",\"-33.4\",\"2019-12-07 01:26:00\",\"61.1\",\"2019-12-07 05:24:00\",\"-48\",\"013CAMPBELLCLIM50501VUE-500001112\""); + List.of( + "2015-12-05 00:00:00", + "333", + "2015-12-07 23:59:00", + "10.53", + "0", + "0.112", + "9.638579", + "20.8", + "16.42", + "0", + "29.52", + "2019-12-07 02:36:00", + "16.69", + "14.5", + "2019-12-07 19:01:00", + "15.6", + "2019-12-07 04:59:00", + "14.91368", + "1024.1", + "2019-12-07 22:44:00", + "1019.6", + "2019-12-07 01:22:00", + "90", + "65.3", + "18.2", + "15.7", + "26", + "2019-12-07 01:16:00", + "-33.4", + "2019-12-07 01:26:00", + "61.1", + "2019-12-07 05:24:00", + "-48", + "013CAMPBELLCLIM50501VUE-500001112"), + reader.next()); } } @@ -165,8 +267,42 @@ void nextTest() { reader.next(); reader.skip(2); assertEquals( - mkString(reader.next()), - "\"2015-12-05 00:00:00\",\"333\",\"2015-12-07 23:59:00\",\"10.53\",\"0\",\"0.112\",\"9.638579\",\"20.8\",\"16.42\",\"0\",\"29.52\",\"2019-12-07 02:36:00\",\"16.69\",\"14.5\",\"2019-12-07 19:01:00\",\"15.6\",\"2019-12-07 04:59:00\",\"14.91368\",\"1024.1\",\"2019-12-07 22:44:00\",\"1019.6\",\"2019-12-07 01:22:00\",\"90\",\"65.3\",\"18.2\",\"15.7\",\"26\",\"2019-12-07 01:16:00\",\"-33.4\",\"2019-12-07 01:26:00\",\"61.1\",\"0\",\"0\",\"0\""); + List.of( + "2015-12-05 00:00:00", + "333", + "2015-12-07 23:59:00", + "10.53", + "0", + "0.112", + "9.638579", + "20.8", + "16.42", + "0", + "29.52", + "2019-12-07 02:36:00", + "16.69", + "14.5", + "2019-12-07 19:01:00", + "15.6", + "2019-12-07 04:59:00", + "14.91368", + "1024.1", + "2019-12-07 22:44:00", + "1019.6", + "2019-12-07 01:22:00", + "90", + "65.3", + "18.2", + "15.7", + "26", + "2019-12-07 01:16:00", + "-33.4", + "2019-12-07 01:26:00", + "61.1", + "0", + "0", + "0"), + reader.next()); } } @@ -195,8 +331,39 @@ void rawNextTest() { reader.next(); reader.skip(2); assertEquals( - mkString(reader.rawNext()), - "\"2015-12-05 00:00:00\",\"333\",\"2015-12-07 23:59:00\",\"10.53\",\"0\",\"0.112\",\"9.638579\",\"20.8\",\"16.42\",\"0\",\"29.52\",\"2019-12-07 02:36:00\",\"16.69\",\"14.5\",\"2019-12-07 19:01:00\",\"15.6\",\"2019-12-07 04:59:00\",\"14.91368\",\"1024.1\",\"2019-12-07 22:44:00\",\"1019.6\",\"2019-12-07 01:22:00\",\"90\",\"65.3\",\"18.2\",\"15.7\",\"26\",\"2019-12-07 01:16:00\",\"-33.4\",\"2019-12-07 01:26:00\",\"61.1\""); + List.of( + "2015-12-05 00:00:00", + "333", + "2015-12-07 23:59:00", + "10.53", + "0", + "0.112", + "9.638579", + "20.8", + "16.42", + "0", + "29.52", + "2019-12-07 02:36:00", + "16.69", + "14.5", + "2019-12-07 19:01:00", + "15.6", + "2019-12-07 04:59:00", + "14.91368", + "1024.1", + "2019-12-07 22:44:00", + "1019.6", + "2019-12-07 01:22:00", + "90", + "65.3", + "18.2", + "15.7", + "26", + "2019-12-07 01:16:00", + "-33.4", + "2019-12-07 01:26:00", + "61.1"), + reader.rawNext()); } } @@ -226,8 +393,39 @@ void hasNextTest() { reader.skip(2); assertTrue(reader.hasNext()); assertEquals( - mkString(reader.rawNext()), - "\"2015-12-05 00:00:00\",\"333\",\"2015-12-07 23:59:00\",\"10.53\",\"0\",\"0.112\",\"9.638579\",\"20.8\",\"16.42\",\"0\",\"29.52\",\"2019-12-07 02:36:00\",\"16.69\",\"14.5\",\"2019-12-07 19:01:00\",\"15.6\",\"2019-12-07 04:59:00\",\"14.91368\",\"1024.1\",\"2019-12-07 22:44:00\",\"1019.6\",\"2019-12-07 01:22:00\",\"90\",\"65.3\",\"18.2\",\"15.7\",\"26\",\"2019-12-07 01:16:00\",\"-33.4\",\"2019-12-07 01:26:00\",\"61.1\""); + List.of( + "2015-12-05 00:00:00", + "333", + "2015-12-07 23:59:00", + "10.53", + "0", + "0.112", + "9.638579", + "20.8", + "16.42", + "0", + "29.52", + "2019-12-07 02:36:00", + "16.69", + "14.5", + "2019-12-07 19:01:00", + "15.6", + "2019-12-07 04:59:00", + "14.91368", + "1024.1", + "2019-12-07 22:44:00", + "1019.6", + "2019-12-07 01:22:00", + "90", + "65.3", + "18.2", + "15.7", + "26", + "2019-12-07 01:16:00", + "-33.4", + "2019-12-07 01:26:00", + "61.1"), + reader.rawNext()); assertFalse(reader.hasNext()); } } @@ -238,8 +436,4 @@ private void writeCSV(Path sink, List lists) { lists.forEach(line -> writer.rawAppend(Arrays.stream(line).collect(Collectors.toList()))); } } - - private String mkString(List arr) { - return "\"" + String.join("\",\"", arr) + "\""; - } } diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index e4aa608e24..374b3fd01e 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -20,49 +20,55 @@ ext { libs = [:] } def versions = [ - "commons-io" : project.properties['commons-io.version'] ?: "2.11.0", - "commons-math3" : project.properties['commons-math3.version'] ?: "3.6.1", - "commons-net" : project.properties['commons-net.version'] ?: "3.9.0", - "commons-logging" : project.properties['commons-logging.version'] ?: "1.2", - "ftp-server" : project.properties['ftp-server.version'] ?: "1.2.0", - jackson : project.properties['jackson.version'] ?: "2.14.2", - jcommander : project.properties['jcommander.version'] ?: "1.82", - junit : project.properties['junit.version'] ?: "5.9.2", - kafka : project.properties['kafka.version'] ?: "3.4.1", - mockito : project.properties['mockito.version'] ?: "5.2.0", - "opencsv" : project.properties['opencsv.version'] ?: "5.7.1", - scala : project.properties['scala.version'] ?: "2.13.10", - slf4j : project.properties['slf4j.version'] ?: "2.0.7", - spark : project.properties['spark.version'] ?: "3.3.2", - "hadoop" : project.properties["hadoop.version"] ?: "3.3.5", + "commons-io" : project.properties['commons-io.version'] ?: "2.11.0", + "commons-math3" : project.properties['commons-math3.version'] ?: "3.6.1", + "commons-net" : project.properties['commons-net.version'] ?: "3.9.0", + "commons-lang3" : project.properties['commons-lang3.version'] ?: "3.12.0", + "commons-logging" : project.properties['commons-logging.version'] ?: "1.2", + "ftp-server" : project.properties['ftp-server.version'] ?: "1.2.0", + jackson : project.properties['jackson.version'] ?: "2.14.2", + jcommander : project.properties['jcommander.version'] ?: "1.82", + "jmh-core" : project.properties['jmh-core.version'] ?: "1.36", + "jmh-generator-annprocess": project.properties['jmh-generator.version'] ?: "1.36", + junit : project.properties['junit.version'] ?: "5.9.2", + kafka : project.properties['kafka.version'] ?: "3.4.1", + mockito : project.properties['mockito.version'] ?: "5.2.0", + "opencsv" : project.properties['opencsv.version'] ?: "5.7.1", + scala : project.properties['scala.version'] ?: "2.13.10", + slf4j : project.properties['slf4j.version'] ?: "2.0.7", + spark : project.properties['spark.version'] ?: "3.3.2", + "hadoop" : project.properties["hadoop.version"] ?: "3.3.5", ] libs += [ - "commons-io" : "commons-io:commons-io:${versions["commons-io"]}", - "commons-math3" : "org.apache.commons:commons-math3:${versions["commons-math3"]}", - "commons-net" : "commons-net:commons-net:${versions["commons-net"]}", - "commons-logging" : "commons-logging:commons-logging:${versions["commons-logging"]}", - "ftp-server" : "org.apache.ftpserver:ftpserver-core:${versions["ftp-server"]}", - "jackson-databind" : "com.fasterxml.jackson.core:jackson-databind:${versions["jackson"]}", - "jackson-datatype-jdk8": "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:${versions["jackson"]}", - jcommander : "com.beust:jcommander:${versions["jcommander"]}", - junit : "org.junit.jupiter:junit-jupiter:${versions["junit"]}", - "kafka-client" : "org.apache.kafka:kafka-clients:${versions["kafka"]}", - "kafka-connect-api" : "org.apache.kafka:connect-api:${versions["kafka"]}", - "kafka-connect-json" : "org.apache.kafka:connect-json:${versions["kafka"]}", - "kafka-connect-runtime": "org.apache.kafka:connect-runtime:${versions["kafka"]}", - "kafka-core" : "org.apache.kafka:kafka_2.13:${versions["kafka"]}", - "kafka-server-common" : "org.apache.kafka:kafka-server-common:${versions["kafka"]}", - "mockito-core" : "org.mockito:mockito-core:${versions["mockito"]}", - "mockito-inline" : "org.mockito:mockito-inline:${versions["mockito"]}", - opencsv : "com.opencsv:opencsv:${versions["opencsv"]}", - scala : "org.scala-lang:scala-library:${versions["scala"]}", - "slf4j-nop" : "org.slf4j:slf4j-nop:${versions["slf4j"]}", - "spark-kafka" : "org.apache.spark:spark-sql-kafka-0-10_2.13:${versions["spark"]}", - "spark-sql" : "org.apache.spark:spark-sql_2.13:${versions["spark"]}", - "hadoop-common" : "org.apache.hadoop:hadoop-common:${versions["hadoop"]}", - "hadoop-client" : "org.apache.hadoop:hadoop-client:${versions["hadoop"]}", - "hadoop-minicluster" : "org.apache.hadoop:hadoop-minicluster:${versions["hadoop"]}", - "hadoop-client-api" : "org.apache.hadoop:hadoop-client-api:${versions["hadoop"]}", - "hadoop-client-runtime": "org.apache.hadoop:hadoop-client-runtime:${versions["hadoop"]}", + "commons-io" : "commons-io:commons-io:${versions["commons-io"]}", + "commons-math3" : "org.apache.commons:commons-math3:${versions["commons-math3"]}", + "commons-net" : "commons-net:commons-net:${versions["commons-net"]}", + "commons-lang3" : "org.apache.commons:commons-lang3:${versions["commons-lang3"]}", + "commons-logging" : "commons-logging:commons-logging:${versions["commons-logging"]}", + "ftp-server" : "org.apache.ftpserver:ftpserver-core:${versions["ftp-server"]}", + "jackson-databind" : "com.fasterxml.jackson.core:jackson-databind:${versions["jackson"]}", + "jackson-datatype-jdk8" : "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:${versions["jackson"]}", + jcommander : "com.beust:jcommander:${versions["jcommander"]}", + "jmh-core" : "org.openjdk.jmh:jmh-core:${versions["jmh-core"]}", + "jmh-generator-annprocess": "org.openjdk.jmh:jmh-generator-annprocess:${versions["jmh-generator-annprocess"]}", + junit : "org.junit.jupiter:junit-jupiter:${versions["junit"]}", + "kafka-client" : "org.apache.kafka:kafka-clients:${versions["kafka"]}", + "kafka-connect-api" : "org.apache.kafka:connect-api:${versions["kafka"]}", + "kafka-connect-json" : "org.apache.kafka:connect-json:${versions["kafka"]}", + "kafka-connect-runtime" : "org.apache.kafka:connect-runtime:${versions["kafka"]}", + "kafka-core" : "org.apache.kafka:kafka_2.13:${versions["kafka"]}", + "kafka-server-common" : "org.apache.kafka:kafka-server-common:${versions["kafka"]}", + "mockito-core" : "org.mockito:mockito-core:${versions["mockito"]}", + "mockito-inline" : "org.mockito:mockito-inline:${versions["mockito"]}", + opencsv : "com.opencsv:opencsv:${versions["opencsv"]}", + scala : "org.scala-lang:scala-library:${versions["scala"]}", + "slf4j-nop" : "org.slf4j:slf4j-nop:${versions["slf4j"]}", + "spark-kafka" : "org.apache.spark:spark-sql-kafka-0-10_2.13:${versions["spark"]}", + "spark-sql" : "org.apache.spark:spark-sql_2.13:${versions["spark"]}", + "hadoop-common" : "org.apache.hadoop:hadoop-common:${versions["hadoop"]}", + "hadoop-client" : "org.apache.hadoop:hadoop-client:${versions["hadoop"]}", + "hadoop-minicluster" : "org.apache.hadoop:hadoop-minicluster:${versions["hadoop"]}", + "hadoop-client-api" : "org.apache.hadoop:hadoop-client-api:${versions["hadoop"]}", + "hadoop-client-runtime" : "org.apache.hadoop:hadoop-client-runtime:${versions["hadoop"]}", ] From 985d4e3998c388af9191c7a291d34d3df877657d Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Mon, 12 Jun 2023 19:09:26 +0800 Subject: [PATCH 59/77] [FS] Using try-with-resources with Files.list() in local listFolders (#1813) --- .../org/astraea/fs/local/LocalFileSystem.java | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/fs/src/main/java/org/astraea/fs/local/LocalFileSystem.java b/fs/src/main/java/org/astraea/fs/local/LocalFileSystem.java index ceac7e1f6c..87552ace86 100644 --- a/fs/src/main/java/org/astraea/fs/local/LocalFileSystem.java +++ b/fs/src/main/java/org/astraea/fs/local/LocalFileSystem.java @@ -26,7 +26,6 @@ import java.nio.file.attribute.BasicFileAttributes; import java.util.List; import java.util.Optional; -import java.util.stream.Collectors; import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.fs.FileSystem; @@ -73,17 +72,23 @@ public synchronized List listFolders(String path) { private synchronized List listFolders(String path, boolean requireFile) { var folder = resolvePath(path); if (!Files.isDirectory(folder)) throw new IllegalArgumentException(path + " is not a folder"); - return Utils.packException(() -> Files.list(folder)) - .filter(f -> requireFile ? Files.isRegularFile(f) : Files.isDirectory(f)) - .map(Path::toAbsolutePath) - .map( - p -> - Path.of("/") - .resolve( - root.map(r -> p.subpath(Path.of(r).getNameCount(), p.getNameCount())) - .orElse(p)) - .toString()) - .collect(Collectors.toList()); + // We use this method within a try-with-resources statement to ensure that the stream's open + // directory is closed promptly after the stream's operations have completed. + // See + // https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/nio/file/Files.html#list(java.nio.file.Path) + try (var directories = Utils.packException(() -> Files.list(folder))) { + return directories + .filter(f -> requireFile ? Files.isRegularFile(f) : Files.isDirectory(f)) + .map(Path::toAbsolutePath) + .map( + p -> + Path.of("/") + .resolve( + root.map(r -> p.subpath(Path.of(r).getNameCount(), p.getNameCount())) + .orElse(p)) + .toString()) + .toList(); + } } @Override From a146af65324cb3be99c29eac4ef703e6d2443de0 Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Tue, 13 Jun 2023 01:02:36 +0800 Subject: [PATCH 60/77] [COMMON] Add ParameterizedTypeImpl and remove commons-lang3 (#1814) --- common/build.gradle | 1 - .../java/org/astraea/common/json/TypeRef.java | 72 +++++++++++++++++-- gradle/dependencies.gradle | 2 - 3 files changed, 68 insertions(+), 7 deletions(-) diff --git a/common/build.gradle b/common/build.gradle index a772545153..b01cccce64 100644 --- a/common/build.gradle +++ b/common/build.gradle @@ -43,7 +43,6 @@ dependencies { implementation libs["jackson-datatype-jdk8"] // we don't use slf4j actually, and it is used by kafka so we swallow the log. implementation libs["slf4j-nop"] - implementation libs["commons-lang3"] implementation libs["commons-math3"] implementation 'com.google.protobuf:protobuf-java:3.22.2' } diff --git a/common/src/main/java/org/astraea/common/json/TypeRef.java b/common/src/main/java/org/astraea/common/json/TypeRef.java index 3076fe03d6..21bb379963 100644 --- a/common/src/main/java/org/astraea/common/json/TypeRef.java +++ b/common/src/main/java/org/astraea/common/json/TypeRef.java @@ -16,13 +16,18 @@ */ package org.astraea.common.json; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + import java.lang.reflect.Array; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; +import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; -import org.apache.commons.lang3.reflect.TypeUtils; /** ParentType didn't erase , use reflection to get that type */ public abstract class TypeRef { @@ -41,7 +46,7 @@ public static TypeRef of(Class clz) { } public static TypeRef> array(Class clz) { - return of(TypeUtils.parameterize(List.class, clz)); + return of(new ParameterizedTypeImpl(List.class, new Type[] {clz})); } @SuppressWarnings("unchecked") @@ -50,11 +55,11 @@ public static TypeRef bytes() { } public static TypeRef> map(Class clz) { - return of(TypeUtils.parameterize(Map.class, String.class, clz)); + return of(new ParameterizedTypeImpl(Map.class, new Type[] {String.class, clz})); } public static TypeRef> set(Class clz) { - return of(TypeUtils.parameterize(Set.class, clz)); + return of(new ParameterizedTypeImpl(Set.class, new Type[] {clz})); } protected final Type type; @@ -83,4 +88,63 @@ public boolean equals(Object o) { return getType() != null ? getType().equals(typeRef.getType()) : typeRef.getType() == null; } + + private static class ParameterizedTypeImpl implements ParameterizedType { + private final Class raw; + private final Type useOwner; + private final Type[] typeArguments; + + ParameterizedTypeImpl(final Class rawClass, final Type[] typeArguments) { + this.raw = requireNonNull(rawClass); + this.useOwner = rawClass.getEnclosingClass(); + this.typeArguments = + Arrays.copyOf(requireNonNull(typeArguments), typeArguments.length, Type[].class); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) return true; + if (obj == null || getClass() != obj.getClass()) return false; + ParameterizedTypeImpl that = (ParameterizedTypeImpl) obj; + return Objects.equals(raw, that.raw) + && Objects.equals(useOwner, that.useOwner) + && Arrays.equals(typeArguments, that.typeArguments); + } + + @Override + public Type[] getActualTypeArguments() { + return typeArguments.clone(); + } + + @Override + public Type getOwnerType() { + return useOwner; + } + + @Override + public Type getRawType() { + return raw; + } + + @Override + public int hashCode() { + int result = Objects.hash(raw, useOwner); + return result * 31 + Arrays.hashCode(typeArguments); + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append( + useOwner == null ? raw.getName() : format("%s.%s", useOwner, raw.getTypeName())); + + if (typeArguments.length > 0) { + builder.append('<'); + builder.append(Arrays.stream(typeArguments).map(Type::getTypeName).collect(joining(", "))); + builder.append('>'); + } + + return builder.toString(); + } + } } diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 374b3fd01e..f046bd777b 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -23,7 +23,6 @@ def versions = [ "commons-io" : project.properties['commons-io.version'] ?: "2.11.0", "commons-math3" : project.properties['commons-math3.version'] ?: "3.6.1", "commons-net" : project.properties['commons-net.version'] ?: "3.9.0", - "commons-lang3" : project.properties['commons-lang3.version'] ?: "3.12.0", "commons-logging" : project.properties['commons-logging.version'] ?: "1.2", "ftp-server" : project.properties['ftp-server.version'] ?: "1.2.0", jackson : project.properties['jackson.version'] ?: "2.14.2", @@ -44,7 +43,6 @@ libs += [ "commons-io" : "commons-io:commons-io:${versions["commons-io"]}", "commons-math3" : "org.apache.commons:commons-math3:${versions["commons-math3"]}", "commons-net" : "commons-net:commons-net:${versions["commons-net"]}", - "commons-lang3" : "org.apache.commons:commons-lang3:${versions["commons-lang3"]}", "commons-logging" : "commons-logging:commons-logging:${versions["commons-logging"]}", "ftp-server" : "org.apache.ftpserver:ftpserver-core:${versions["ftp-server"]}", "jackson-databind" : "com.fasterxml.jackson.core:jackson-databind:${versions["jackson"]}", From d6a7164fb8741a571d1929a8e729659f67bf21ea Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Thu, 15 Jun 2023 11:01:52 +0800 Subject: [PATCH 61/77] [COMMON] Add duration string formatter (#1815) --- .../main/java/org/astraea/common/Utils.java | 40 +++++++++++++++++++ .../astraea/common/json/JsonConverter.java | 4 +- .../common/json/JsonConverterTest.java | 34 ++++++++++++++++ 3 files changed, 75 insertions(+), 3 deletions(-) diff --git a/common/src/main/java/org/astraea/common/Utils.java b/common/src/main/java/org/astraea/common/Utils.java index ea4eddf6d2..4b4b4a81a1 100644 --- a/common/src/main/java/org/astraea/common/Utils.java +++ b/common/src/main/java/org/astraea/common/Utils.java @@ -55,6 +55,12 @@ public final class Utils { public static final Pattern TIME_PATTERN = Pattern.compile("^(?[0-9]+)(?days|day|h|m|s|ms|us|ns|)$"); + private static final long DAY_IN_NANOS = Duration.ofDays(1).toNanos(); + private static final long HOUR_IN_NANOS = Duration.ofHours(1).toNanos(); + private static final long MINUTE_IN_NANOS = Duration.ofMinutes(1).toNanos(); + private static final long SECOND_IN_NANOS = 1000_000_000L; + private static final long MILLIS_IN_NANOS = 1000_000L; + private static final long MICROS_IN_NANOS = 1000L; /** * A converter for time unit. @@ -111,6 +117,40 @@ public static Duration toDuration(String input) { } } + /** + * A Duration string formatter + * + *

    This formatter is able to transform the following {@link Duration} into the corresponding + * string format (i.e. value + time unit) + * + *

      + *
    • {@code Duration.ofSeconds(30)} to {@code "30s"} + *
    • {@code Duration.ofMinutes(1)} to {@code "1m"} + *
    • {@code Duration.ofHours(24)} to {@code "1day"} + *
    • {@code Duration.ofDays(7)} to{@code "7day"} + *
    • {@code Duration.ofMillis(350)} to {@code "350ms"} + *
    • {@code Duration.ofNanos(123 * 1000)} to {@code "123us"} + *
    • {@code Duration.ofNanos(100)} to {@code "100ns"} + *
    + * + *

    If the value in the Duration is evenly divisible by nanoseconds per time unit (i.e. day, + * hour, minute, second, millisecond, microsecond, nanosecond), then this formatter will select + * the highest time unit as the time unit in the string format. + * + * @param input Duration + * @return duration in readable string format + */ + public static String toDurationString(Duration input) { + long inputInNanos = input.toNanos(); + if (inputInNanos % DAY_IN_NANOS == 0) return (inputInNanos / DAY_IN_NANOS) + "day"; + if (inputInNanos % HOUR_IN_NANOS == 0) return (inputInNanos / HOUR_IN_NANOS) + "h"; + if (inputInNanos % MINUTE_IN_NANOS == 0) return (inputInNanos / MINUTE_IN_NANOS) + "m"; + if (inputInNanos % SECOND_IN_NANOS == 0) return (inputInNanos / SECOND_IN_NANOS) + "s"; + if (inputInNanos % MILLIS_IN_NANOS == 0) return (inputInNanos / MILLIS_IN_NANOS) + "ms"; + if (inputInNanos % MICROS_IN_NANOS == 0) return (inputInNanos / MICROS_IN_NANOS) + "us"; + return inputInNanos + "ns"; + } + /** * Convert the exception thrown by getter to RuntimeException, except ExecutionException. * ExecutionException will be converted to ExecutionRuntimeException , in order to preserve the diff --git a/common/src/main/java/org/astraea/common/json/JsonConverter.java b/common/src/main/java/org/astraea/common/json/JsonConverter.java index d529b595c4..86a2b4bb2e 100644 --- a/common/src/main/java/org/astraea/common/json/JsonConverter.java +++ b/common/src/main/java/org/astraea/common/json/JsonConverter.java @@ -86,9 +86,7 @@ static JsonConverter jackson() { module.addSerializer(DataSize.class, serialzer(DataSize.class, DataSize::toString)); // Duration module.addDeserializer(Duration.class, deserializer(Duration.class, Utils::toDuration)); - // TODO: how to support ns? - // https://github.com/skiptests/astraea/issues/1430 - module.addSerializer(Duration.class, serialzer(Duration.class, d -> d.toMillis() + "ms")); + module.addSerializer(Duration.class, serialzer(Duration.class, Utils::toDurationString)); var objectMapper = JsonMapper.builder() .addModule(new Jdk8Module()) diff --git a/common/src/test/java/org/astraea/common/json/JsonConverterTest.java b/common/src/test/java/org/astraea/common/json/JsonConverterTest.java index 9a604e5af7..3e13c0a3bb 100644 --- a/common/src/test/java/org/astraea/common/json/JsonConverterTest.java +++ b/common/src/test/java/org/astraea/common/json/JsonConverterTest.java @@ -26,9 +26,13 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Stream; import org.astraea.common.DataSize; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; class JsonConverterTest { @@ -498,6 +502,36 @@ void testDuration() { () -> JsonConverter.defaultConverter().fromJson("{}", TypeRef.of(TestDuration.class))); } + @ParameterizedTest() + @MethodSource("provideDurationString") + void testDurationSerDe(String original, String expected) { + assertEquals( + expected, + JsonConverter.defaultConverter() + .toJson( + JsonConverter.defaultConverter() + .fromJson(original, TypeRef.of(TestDuration.class)))); + } + + private static Stream provideDurationString() { + return Stream.of( + Arguments.of("{\"value\":\"2day\"}", "{\"value\":\"2day\"}"), + Arguments.of("{\"value\":\"24h\"}", "{\"value\":\"1day\"}"), + Arguments.of("{\"value\":\"25h\"}", "{\"value\":\"25h\"}"), + Arguments.of("{\"value\":\"60m\"}", "{\"value\":\"1h\"}"), + Arguments.of("{\"value\":\"61m\"}", "{\"value\":\"61m\"}"), + Arguments.of("{\"value\":\"60s\"}", "{\"value\":\"1m\"}"), + Arguments.of("{\"value\":\"61s\"}", "{\"value\":\"61s\"}"), + Arguments.of("{\"value\":\"1000ms\"}", "{\"value\":\"1s\"}"), + Arguments.of("{\"value\":\"1001ms\"}", "{\"value\":\"1001ms\"}"), + Arguments.of("{\"value\":\"1000us\"}", "{\"value\":\"1ms\"}"), + Arguments.of("{\"value\":\"1001us\"}", "{\"value\":\"1001us\"}"), + Arguments.of("{\"value\":\"1000ns\"}", "{\"value\":\"1us\"}"), + Arguments.of("{\"value\":\"1001ns\"}", "{\"value\":\"1001ns\"}"), + Arguments.of("{\"value\":\"86400s\"}", "{\"value\":\"1day\"}"), + Arguments.of("{\"value\":\"123000000ns\"}", "{\"value\":\"123ms\"}")); + } + private static class TestDuration { Duration value; List list = List.of(); From b1c11a2095689a056893ad925d2923b3c689587a Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 16 Jun 2023 10:31:10 +0800 Subject: [PATCH 62/77] [COMMON] upgrade kafka from 3.4.1 to 3.5.0 (#1816) --- docker/start_broker.sh | 4 ++-- docker/start_controller.sh | 4 ++-- docker/start_worker.sh | 4 ++-- gradle/dependencies.gradle | 2 +- it/src/main/java/org/astraea/it/BrokerCluster.java | 4 +--- it/src/main/java/org/astraea/it/WorkerCluster.java | 7 ++++--- 6 files changed, 12 insertions(+), 13 deletions(-) diff --git a/docker/start_broker.sh b/docker/start_broker.sh index 42dd600702..810179922d 100755 --- a/docker/start_broker.sh +++ b/docker/start_broker.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.4.1}} +declare -r VERSION=${REVISION:-${VERSION:-3.5.0}} declare -r DOCKERFILE=$DOCKER_FOLDER/broker.dockerfile declare -r DATA_FOLDER_IN_CONTAINER_PREFIX="/tmp/log-folder" declare -r EXPORTER_VERSION="0.16.1" @@ -61,7 +61,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set broker JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.4.1 set version of kafka distribution" + echo " VERSION=3.5.0 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " DATA_FOLDERS=/tmp/folder1 set host folders used by broker" diff --git a/docker/start_controller.sh b/docker/start_controller.sh index 09e462fa56..3101cdb2be 100755 --- a/docker/start_controller.sh +++ b/docker/start_controller.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.4.1}} +declare -r VERSION=${REVISION:-${VERSION:-3.5.0}} declare -r DOCKERFILE=$DOCKER_FOLDER/controller.dockerfile declare -r EXPORTER_VERSION="0.16.1" declare -r CLUSTER_ID=${CLUSTER_ID:-"$(randomString)"} @@ -52,7 +52,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set controller JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.4.1 set version of kafka distribution" + echo " VERSION=3.5.0 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " META_FOLDER=/tmp/folder1 set host folder used by controller" diff --git a/docker/start_worker.sh b/docker/start_worker.sh index 201187adca..1fc2b88434 100755 --- a/docker/start_worker.sh +++ b/docker/start_worker.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.4.1}} +declare -r VERSION=${REVISION:-${VERSION:-3.5.0}} declare -r DOCKERFILE=$DOCKER_FOLDER/worker.dockerfile declare -r WORKER_PORT=${WORKER_PORT:-"$(getRandomPort)"} declare -r CONTAINER_NAME="worker-$WORKER_PORT" @@ -50,7 +50,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set worker JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.4.1 set version of kafka distribution" + echo " VERSION=3.5.0 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " WORKER_PLUGIN_PATH=/tmp/worker-plugins set plugin path to kafka worker" diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index f046bd777b..59d9235693 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -30,7 +30,7 @@ def versions = [ "jmh-core" : project.properties['jmh-core.version'] ?: "1.36", "jmh-generator-annprocess": project.properties['jmh-generator.version'] ?: "1.36", junit : project.properties['junit.version'] ?: "5.9.2", - kafka : project.properties['kafka.version'] ?: "3.4.1", + kafka : project.properties['kafka.version'] ?: "3.5.0", mockito : project.properties['mockito.version'] ?: "5.2.0", "opencsv" : project.properties['opencsv.version'] ?: "5.7.1", scala : project.properties['scala.version'] ?: "2.13.10", diff --git a/it/src/main/java/org/astraea/it/BrokerCluster.java b/it/src/main/java/org/astraea/it/BrokerCluster.java index 6d2a3b0268..8471bb5471 100644 --- a/it/src/main/java/org/astraea/it/BrokerCluster.java +++ b/it/src/main/java/org/astraea/it/BrokerCluster.java @@ -51,9 +51,7 @@ private static CompletableFuture> server( return CompletableFuture.supplyAsync( () -> { - var broker = - new KafkaRaftServer( - new KafkaConfig(configs), SystemTime.SYSTEM, scala.Option.empty()); + var broker = new KafkaRaftServer(new KafkaConfig(configs), SystemTime.SYSTEM); broker.startup(); return Map.entry(nodeId, broker); }); diff --git a/it/src/main/java/org/astraea/it/WorkerCluster.java b/it/src/main/java/org/astraea/it/WorkerCluster.java index dd9968a129..875fa9e430 100644 --- a/it/src/main/java/org/astraea/it/WorkerCluster.java +++ b/it/src/main/java/org/astraea/it/WorkerCluster.java @@ -29,6 +29,7 @@ import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.rest.RestServerConfig; public interface WorkerCluster extends AutoCloseable { @@ -71,12 +72,12 @@ static WorkerCluster of( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); configs.put( - WorkerConfig.LISTENERS_CONFIG, + RestServerConfig.LISTENERS_CONFIG, // the worker hostname is a part of information used by restful apis. // the 0.0.0.0 make all connector say that they are executed by 0.0.0.0 // and it does make sense in production. With a view to testing the // related codes in other modules, we have to define the "really" hostname - // in starting worker cluster. + // in starting worker cluster.j90 "http://" + Utils.hostname() + ":" + realPort); configs.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(500)); // enable us to override the connector configs @@ -86,7 +87,7 @@ static WorkerCluster of( configs.putAll(override); return new ConnectDistributed().startConnect(configs); }) - .collect(Collectors.toUnmodifiableList()); + .toList(); return new WorkerCluster() { @Override From cdde1b0cf4397b3482a2a1bb809e0d71803609a5 Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Sat, 17 Jun 2023 15:12:17 +0800 Subject: [PATCH 63/77] [COMMON] Use toList instead of Collectors.toUnmodifiableList (#1817) --- .../java/org/astraea/common/DataUnit.java | 3 +- .../java/org/astraea/common/admin/Admin.java | 2 +- .../org/astraea/common/admin/ClusterInfo.java | 30 +++++----------- .../common/admin/TransactionState.java | 3 +- .../common/assignor/SubscriptionInfo.java | 4 +-- .../common/backup/RecordWriterBuilder.java | 3 +- .../org/astraea/common/cost/NetworkCost.java | 2 +- .../org/astraea/common/cost/Normalizer.java | 11 +++--- .../astraea/common/metrics/JndiClient.java | 2 +- .../metrics/client/admin/AdminMetrics.java | 9 ++--- .../client/consumer/ConsumerMetrics.java | 10 +++--- .../client/producer/ProducerMetrics.java | 8 ++--- .../metrics/collector/BeanObjectClient.java | 2 +- .../metrics/connector/ConnectorMetrics.java | 34 ++++++------------- .../org/astraea/common/producer/Builder.java | 7 ++-- .../common/producer/RecordGenerator.java | 6 ++-- .../astraea/common/DistributionTypeTest.java | 5 +-- .../astraea/common/admin/ClusterBeanTest.java | 10 +++--- .../common/balancer/BalancerConsoleTest.java | 6 +--- .../executor/StraightPlanExecutorTest.java | 2 +- .../astraea/common/consumer/ConsumerTest.java | 15 ++++---- .../common/cost/NetworkIngressCostTest.java | 15 +++----- .../common/cost/NodeLatencyCostTest.java | 3 +- .../astraea/common/cost/NormalizerTest.java | 7 ++-- .../common/cost/WeightProviderTest.java | 23 ++++--------- .../cost/utils/ClusterInfoSensorTest.java | 13 +++---- .../common/metrics/broker/LogMetricsTest.java | 4 +-- .../client/consumer/ConsumerMetricsTest.java | 3 +- .../client/producer/ProducerMetricsTest.java | 3 +- .../metrics/collector/MetricFetcherTest.java | 14 ++------ .../StrictCostPartitionerTest.java | 6 +--- .../astraea/common/producer/ProducerTest.java | 3 +- .../org/astraea/connector/SourceDataTest.java | 3 +- .../org/astraea/gui/button/SelectBox.java | 4 +-- .../org/astraea/gui/table/TableViewer.java | 2 +- .../java/org/astraea/it/BrokerCluster.java | 2 +- .../main/java/org/astraea/it/FtpServer.java | 5 ++- 37 files changed, 94 insertions(+), 190 deletions(-) diff --git a/common/src/main/java/org/astraea/common/DataUnit.java b/common/src/main/java/org/astraea/common/DataUnit.java index ccf24386cf..6a2d0de555 100644 --- a/common/src/main/java/org/astraea/common/DataUnit.java +++ b/common/src/main/java/org/astraea/common/DataUnit.java @@ -20,7 +20,6 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; -import java.util.stream.Collectors; /** * Utility related to data and unit, this class dedicated to easing the pain of converting and @@ -131,5 +130,5 @@ public DataSize of(long measurement) { Arrays.stream(DataUnit.values()) .filter(x -> x.candidateUnitForToString) .sorted(Comparator.comparing(x -> x.bits)) - .collect(Collectors.toUnmodifiableList()); + .toList(); } diff --git a/common/src/main/java/org/astraea/common/admin/Admin.java b/common/src/main/java/org/astraea/common/admin/Admin.java index 71b160c313..291ae248f3 100644 --- a/common/src/main/java/org/astraea/common/admin/Admin.java +++ b/common/src/main/java/org/astraea/common/admin/Admin.java @@ -80,7 +80,7 @@ default CompletionStage> topicNames(List checkers) { FutureUtils.sequence( checkers.stream() .map(checker -> checker.test(this, topicNames).toCompletableFuture()) - .collect(Collectors.toUnmodifiableList())) + .toList()) .thenApply( all -> all.stream() 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 54743a1f8f..132b23d872 100644 --- a/common/src/main/java/org/astraea/common/admin/ClusterInfo.java +++ b/common/src/main/java/org/astraea/common/admin/ClusterInfo.java @@ -169,10 +169,7 @@ static Map leaderSize(ClusterInfo clusterInfo) { * @return A list of {@link Replica}. */ default List replicaLeaders() { - return replicaStream() - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .collect(Collectors.toUnmodifiableList()); + return replicaStream().filter(Replica::isLeader).filter(Replica::isOnline).toList(); } /** @@ -182,10 +179,7 @@ default List replicaLeaders() { * @return A list of {@link Replica}. */ default List replicaLeaders(String topic) { - return replicaStream(topic) - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .collect(Collectors.toUnmodifiableList()); + return replicaStream(topic).filter(Replica::isLeader).filter(Replica::isOnline).toList(); } /** @@ -195,10 +189,7 @@ default List replicaLeaders(String topic) { * @return A list of {@link Replica}. */ default List replicaLeaders(int broker) { - return replicaStream(broker) - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .collect(Collectors.toUnmodifiableList()); + return replicaStream(broker).filter(Replica::isLeader).filter(Replica::isOnline).toList(); } /** @@ -207,10 +198,7 @@ default List replicaLeaders(int broker) { * @return A list of {@link Replica}. */ default List replicaLeaders(BrokerTopic brokerTopic) { - return replicaStream(brokerTopic) - .filter(Replica::isLeader) - .filter(Replica::isOnline) - .collect(Collectors.toUnmodifiableList()); + return replicaStream(brokerTopic).filter(Replica::isLeader).filter(Replica::isOnline).toList(); } /** @@ -235,7 +223,7 @@ default Optional replicaLeader(TopicPartition topicPartition) { * @return A list of {@link Replica}. */ default List availableReplicas(String topic) { - return replicaStream(topic).filter(Replica::isOnline).collect(Collectors.toUnmodifiableList()); + return replicaStream(topic).filter(Replica::isOnline).toList(); } // ---------------------[for replicas]---------------------// @@ -244,7 +232,7 @@ default List availableReplicas(String topic) { * @return all replicas cached by this cluster info. */ default List replicas() { - return replicaStream().collect(Collectors.toUnmodifiableList()); + return replicaStream().toList(); } /** @@ -254,7 +242,7 @@ default List replicas() { * @return A list of {@link Replica}. */ default List replicas(String topic) { - return replicaStream(topic).collect(Collectors.toUnmodifiableList()); + return replicaStream(topic).toList(); } /** @@ -265,7 +253,7 @@ default List replicas(String topic) { * @return A list of {@link Replica}. */ default List replicas(TopicPartition topicPartition) { - return replicaStream(topicPartition).collect(Collectors.toUnmodifiableList()); + return replicaStream(topicPartition).toList(); } /** @@ -273,7 +261,7 @@ default List replicas(TopicPartition topicPartition) { * @return the replica matched to input replica */ default List replicas(TopicPartitionReplica replica) { - return replicaStream(replica).collect(Collectors.toUnmodifiableList()); + return replicaStream(replica).toList(); } // ---------------------[others]---------------------// diff --git a/common/src/main/java/org/astraea/common/admin/TransactionState.java b/common/src/main/java/org/astraea/common/admin/TransactionState.java index ef4d582ae4..f90b550c88 100644 --- a/common/src/main/java/org/astraea/common/admin/TransactionState.java +++ b/common/src/main/java/org/astraea/common/admin/TransactionState.java @@ -18,7 +18,6 @@ import java.util.Arrays; import java.util.Collection; -import java.util.stream.Collectors; import org.astraea.common.EnumInfo; public enum TransactionState implements EnumInfo { @@ -63,6 +62,6 @@ public static TransactionState of(String value) { } public static Collection all() { - return Arrays.stream(TransactionState.values()).collect(Collectors.toUnmodifiableList()); + return Arrays.stream(TransactionState.values()).toList(); } } diff --git a/common/src/main/java/org/astraea/common/assignor/SubscriptionInfo.java b/common/src/main/java/org/astraea/common/assignor/SubscriptionInfo.java index 7ce60a653e..fccee719c8 100644 --- a/common/src/main/java/org/astraea/common/assignor/SubscriptionInfo.java +++ b/common/src/main/java/org/astraea/common/assignor/SubscriptionInfo.java @@ -72,9 +72,7 @@ public static SubscriptionInfo from(ConsumerPartitionAssignor.Subscription subsc var ownPartitions = subscription.ownedPartitions() == null ? List.of() - : subscription.ownedPartitions().stream() - .map(TopicPartition::from) - .collect(Collectors.toUnmodifiableList()); + : subscription.ownedPartitions().stream().map(TopicPartition::from).toList(); var kafkaUserData = subscription.userData(); // convert ByteBuffer into Map diff --git a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java index 52ca446357..d9d3b34801 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.zip.GZIPOutputStream; import org.astraea.common.ByteUtils; import org.astraea.common.DataSize; @@ -70,7 +69,7 @@ public void append(Record record) { ? ByteString.EMPTY : ByteString.copyFrom(header.value())) .build()) - .collect(Collectors.toUnmodifiableList())) + .toList()) .build() .writeDelimitedTo(outputStream)); count.incrementAndGet(); diff --git a/common/src/main/java/org/astraea/common/cost/NetworkCost.java b/common/src/main/java/org/astraea/common/cost/NetworkCost.java index 28d150eccc..5a1e7b2df9 100644 --- a/common/src/main/java/org/astraea/common/cost/NetworkCost.java +++ b/common/src/main/java/org/astraea/common/cost/NetworkCost.java @@ -92,7 +92,7 @@ void noMetricCheck(ClusterBean clusterBean) { clusterBean.all().entrySet().stream() .filter(e -> e.getValue().size() == 0) .map(Map.Entry::getKey) - .collect(Collectors.toUnmodifiableList()); + .toList(); if (!noMetricBrokers.isEmpty()) throw new NoSufficientMetricsException( this, diff --git a/common/src/main/java/org/astraea/common/cost/Normalizer.java b/common/src/main/java/org/astraea/common/cost/Normalizer.java index 2e73d3a797..bbc4d64b38 100644 --- a/common/src/main/java/org/astraea/common/cost/Normalizer.java +++ b/common/src/main/java/org/astraea/common/cost/Normalizer.java @@ -56,13 +56,10 @@ static Normalizer minMax(boolean positive) { double max = values.stream().max(comparator).orElse(0.0); double min = values.stream().min(comparator).orElse(0.0); // there is nothing to rescale, so we just all same values - if (max == min) - return IntStream.range(0, values.size()) - .mapToObj(ignored -> 1.0) - .collect(Collectors.toUnmodifiableList()); + if (max == min) return IntStream.range(0, values.size()).mapToObj(ignored -> 1.0).toList(); return values.stream() .map(value -> (positive ? value - min : max - value) / (max - min)) - .collect(Collectors.toUnmodifiableList()); + .toList(); }; } @@ -76,7 +73,7 @@ static Normalizer minMax(boolean positive) { static Normalizer proportion() { return values -> { var sum = values.stream().mapToDouble(i -> i).sum(); - return values.stream().map(v -> v / sum).collect(Collectors.toUnmodifiableList()); + return values.stream().map(v -> v / sum).toList(); }; } @@ -105,7 +102,7 @@ static Normalizer TScore() { } return Math.round(score * 100) / 100.0; }) - .collect(Collectors.toUnmodifiableList()); + .toList(); }; } diff --git a/common/src/main/java/org/astraea/common/metrics/JndiClient.java b/common/src/main/java/org/astraea/common/metrics/JndiClient.java index fe471eac06..1f74bf3d72 100644 --- a/common/src/main/java/org/astraea/common/metrics/JndiClient.java +++ b/common/src/main/java/org/astraea/common/metrics/JndiClient.java @@ -166,7 +166,7 @@ public Collection beans( return Stream.empty(); } }) - .collect(Collectors.toUnmodifiableList())); + .toList()); } /** diff --git a/common/src/main/java/org/astraea/common/metrics/client/admin/AdminMetrics.java b/common/src/main/java/org/astraea/common/metrics/client/admin/AdminMetrics.java index ee389a00cc..70a21e02d6 100644 --- a/common/src/main/java/org/astraea/common/metrics/client/admin/AdminMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/client/admin/AdminMetrics.java @@ -17,7 +17,6 @@ package org.astraea.common.metrics.client.admin; import java.util.Collection; -import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.metrics.BeanQuery; import org.astraea.common.metrics.MBeanClient; @@ -51,14 +50,10 @@ public class AdminMetrics { * @return key is broker id, and value is associated to broker metrics recorded by all consumers */ public static Collection node(MBeanClient mBeanClient) { - return mBeanClient.beans(NODE_QUERY).stream() - .map(b -> (HasNodeMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + return mBeanClient.beans(NODE_QUERY).stream().map(b -> (HasNodeMetrics) () -> b).toList(); } public static Collection admin(MBeanClient mBeanClient) { - return mBeanClient.beans(ADMIN_QUERY).stream() - .map(b -> (HasSelectorMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + return mBeanClient.beans(ADMIN_QUERY).stream().map(b -> (HasSelectorMetrics) () -> b).toList(); } } diff --git a/common/src/main/java/org/astraea/common/metrics/client/consumer/ConsumerMetrics.java b/common/src/main/java/org/astraea/common/metrics/client/consumer/ConsumerMetrics.java index ff13d159ba..f6ab94db12 100644 --- a/common/src/main/java/org/astraea/common/metrics/client/consumer/ConsumerMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/client/consumer/ConsumerMetrics.java @@ -77,26 +77,24 @@ public static List appInfo(MBeanClient client) { * @return key is broker id, and value is associated to broker metrics recorded by all consumers */ public static Collection node(MBeanClient mBeanClient) { - return mBeanClient.beans(NODE_QUERY).stream() - .map(b -> (HasNodeMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + return mBeanClient.beans(NODE_QUERY).stream().map(b -> (HasNodeMetrics) () -> b).toList(); } public static Collection coordinator(MBeanClient mBeanClient) { return mBeanClient.beans(COORDINATOR_QUERY).stream() .map(b -> (HasConsumerCoordinatorMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static Collection fetch(MBeanClient mBeanClient) { return mBeanClient.beans(FETCH_QUERY).stream() .map(b -> (HasConsumerFetchMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static Collection consumer(MBeanClient mBeanClient) { return mBeanClient.beans(CONSUMER_QUERY).stream() .map(b -> (HasConsumerMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } } diff --git a/common/src/main/java/org/astraea/common/metrics/client/producer/ProducerMetrics.java b/common/src/main/java/org/astraea/common/metrics/client/producer/ProducerMetrics.java index 7c0409deba..d86b50d574 100644 --- a/common/src/main/java/org/astraea/common/metrics/client/producer/ProducerMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/client/producer/ProducerMetrics.java @@ -73,9 +73,7 @@ public static List appInfo(MBeanClient client) { * @return key is broker id, and value is associated to broker metrics recorded by all producers */ public static Collection node(MBeanClient mBeanClient) { - return mBeanClient.beans(NODE_QUERY).stream() - .map(b -> (HasNodeMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + return mBeanClient.beans(NODE_QUERY).stream().map(b -> (HasNodeMetrics) () -> b).toList(); } /** @@ -87,13 +85,13 @@ public static Collection node(MBeanClient mBeanClient) { public static Collection topic(MBeanClient mBeanClient) { return mBeanClient.beans(TOPIC_QUERY).stream() .map(b -> (HasProducerTopicMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static Collection producer(MBeanClient mBeanClient) { return mBeanClient.beans(PRODUCER_QUERY).stream() .map(b -> (HasProducerMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } private ProducerMetrics() {} diff --git a/common/src/main/java/org/astraea/common/metrics/collector/BeanObjectClient.java b/common/src/main/java/org/astraea/common/metrics/collector/BeanObjectClient.java index b39863a9dc..8d27355d33 100644 --- a/common/src/main/java/org/astraea/common/metrics/collector/BeanObjectClient.java +++ b/common/src/main/java/org/astraea/common/metrics/collector/BeanObjectClient.java @@ -87,7 +87,7 @@ public Collection beans( && e.getValue() .matcher(storedEntry.properties().get(e.getKey())) .matches())) - .collect(Collectors.toUnmodifiableList()); + .toList(); } }; } diff --git a/common/src/main/java/org/astraea/common/metrics/connector/ConnectorMetrics.java b/common/src/main/java/org/astraea/common/metrics/connector/ConnectorMetrics.java index b706bf1998..8134fc1981 100644 --- a/common/src/main/java/org/astraea/common/metrics/connector/ConnectorMetrics.java +++ b/common/src/main/java/org/astraea/common/metrics/connector/ConnectorMetrics.java @@ -120,25 +120,21 @@ public class ConnectorMetrics { public static List sourceTaskInfo(MBeanClient client) { return client.beans(SOURCE_TASK_INFO_QUERY).stream() .map(b -> (SourceTaskInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static List sinkTaskInfo(MBeanClient client) { - return client.beans(SINK_TASK_INFO_QUERY).stream() - .map(b -> (SinkTaskInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + return client.beans(SINK_TASK_INFO_QUERY).stream().map(b -> (SinkTaskInfo) () -> b).toList(); } public static List taskError(MBeanClient client) { - return client.beans(TASK_ERROR_QUERY).stream() - .map(b -> (TaskError) () -> b) - .collect(Collectors.toUnmodifiableList()); + return client.beans(TASK_ERROR_QUERY).stream().map(b -> (TaskError) () -> b).toList(); } public static List connectorTaskInfo(MBeanClient client) { return client.beans(CONNECTOR_TASK_INFO_QUERY).stream() .map(b -> (ConnectorTaskInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static List appInfo(MBeanClient client) { @@ -150,42 +146,34 @@ public static List appInfo(MBeanClient client) { public static List coordinatorInfo(MBeanClient client) { return client.beans(COORDINATOR_INFO_QUERY).stream() .map(b -> (ConnectCoordinatorInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static List connectorInfo(MBeanClient client) { - return client.beans(CONNECTOR_INFO_QUERY).stream() - .map(b -> (ConnectorInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + return client.beans(CONNECTOR_INFO_QUERY).stream().map(b -> (ConnectorInfo) () -> b).toList(); } public static List workerRebalanceInfo(MBeanClient client) { return client.beans(WORKER_REBALANCE_INFO_QUERY).stream() .map(b -> (ConnectWorkerRebalanceInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static List nodeInfo(MBeanClient client) { - return client.beans(NODE_INFO_QUERY).stream() - .map(b -> (HasNodeMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + return client.beans(NODE_INFO_QUERY).stream().map(b -> (HasNodeMetrics) () -> b).toList(); } public static List workerInfo(MBeanClient client) { - return client.beans(WORKER_INFO_QUERY).stream() - .map(b -> (ConnectWorkerInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + return client.beans(WORKER_INFO_QUERY).stream().map(b -> (ConnectWorkerInfo) () -> b).toList(); } public static List workerConnectorInfo(MBeanClient client) { return client.beans(WORKER_CONNECTOR_INFO_QUERY).stream() .map(b -> (ConnectWorkerConnectorInfo) () -> b) - .collect(Collectors.toUnmodifiableList()); + .toList(); } public static List connector(MBeanClient client) { - return client.beans(CONNECTOR_QUERY).stream() - .map(b -> (HasSelectorMetrics) () -> b) - .collect(Collectors.toUnmodifiableList()); + return client.beans(CONNECTOR_QUERY).stream().map(b -> (HasSelectorMetrics) () -> b).toList(); } } diff --git a/common/src/main/java/org/astraea/common/producer/Builder.java b/common/src/main/java/org/astraea/common/producer/Builder.java index be30c6ebd4..7b5eb72570 100644 --- a/common/src/main/java/org/astraea/common/producer/Builder.java +++ b/common/src/main/java/org/astraea/common/producer/Builder.java @@ -184,7 +184,7 @@ public CompletionStage send(Record record) { @Override public Collection> send(Collection> records) { - return records.stream().map(this::send).collect(Collectors.toUnmodifiableList()); + return records.stream().map(this::send).toList(); } @Override @@ -212,10 +212,7 @@ public CompletionStage send(Record record) { public Collection> send(Collection> records) { try { kafkaProducer.beginTransaction(); - var futures = - records.stream() - .map(r -> doSend(kafkaProducer, r)) - .collect(Collectors.toUnmodifiableList()); + var futures = records.stream().map(r -> doSend(kafkaProducer, r)).toList(); kafkaProducer.commitTransaction(); return futures; } catch (ProducerFencedException | OutOfOrderSequenceException | AuthorizationException e) { diff --git a/common/src/main/java/org/astraea/common/producer/RecordGenerator.java b/common/src/main/java/org/astraea/common/producer/RecordGenerator.java index 123c23df62..545276b023 100644 --- a/common/src/main/java/org/astraea/common/producer/RecordGenerator.java +++ b/common/src/main/java/org/astraea/common/producer/RecordGenerator.java @@ -44,13 +44,11 @@ class Builder { int batchSize = 1; long keyTableSeed = ThreadLocalRandom.current().nextLong(); long valueTableSeed = ThreadLocalRandom.current().nextLong(); - List keyRange = - LongStream.rangeClosed(0, 10000).boxed().collect(Collectors.toUnmodifiableList()); + List keyRange = LongStream.rangeClosed(0, 10000).boxed().toList(); Supplier keyDistribution = DistributionType.UNIFORM.create(10000, Configuration.EMPTY); Supplier keySizeDistribution = DistributionType.UNIFORM.create(10000, Configuration.EMPTY); - List valueRange = - LongStream.rangeClosed(0, 10000).boxed().collect(Collectors.toUnmodifiableList()); + List valueRange = LongStream.rangeClosed(0, 10000).boxed().toList(); Supplier valueDistribution = DistributionType.UNIFORM.create(10000, Configuration.EMPTY); Supplier valueSizeDistribution = DistributionType.UNIFORM.create(10000, Configuration.EMPTY); diff --git a/common/src/test/java/org/astraea/common/DistributionTypeTest.java b/common/src/test/java/org/astraea/common/DistributionTypeTest.java index a7c18b5472..ba9619665c 100644 --- a/common/src/test/java/org/astraea/common/DistributionTypeTest.java +++ b/common/src/test/java/org/astraea/common/DistributionTypeTest.java @@ -91,10 +91,7 @@ void testZipfianConfig() { 10000, new Configuration(Map.of(DistributionType.ZIPFIAN_SEED, "100"))); Assertions.assertEquals( List.of(11, 18, 0, 1126, 12), - IntStream.range(0, 5) - .map(i -> zip100.get().intValue()) - .boxed() - .collect(Collectors.toUnmodifiableList()), + IntStream.range(0, 5).map(i -> zip100.get().intValue()).boxed().toList(), "Random sequence fixed by specific seed"); // random fixed seed diff --git a/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java b/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java index 9d421096c3..090a7b7232 100644 --- a/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java +++ b/common/src/test/java/org/astraea/common/admin/ClusterBeanTest.java @@ -108,9 +108,7 @@ void testBeans() { } static List fakeTopics = - IntStream.range(0, 10) - .mapToObj(i -> Utils.randomString()) - .collect(Collectors.toUnmodifiableList()); + IntStream.range(0, 10).mapToObj(i -> Utils.randomString()).toList(); Stream random(int seed) { var random = new Random(seed); @@ -165,9 +163,9 @@ Stream random(int seed) { ClusterBean cb = ClusterBean.of( Map.of( - 1, random(0x0ae10).limit(3000).collect(Collectors.toUnmodifiableList()), - 2, random(0x0f0c1).limit(3000).collect(Collectors.toUnmodifiableList()), - 3, random(0x4040f).limit(3000).collect(Collectors.toUnmodifiableList()))); + 1, random(0x0ae10).limit(3000).toList(), + 2, random(0x0f0c1).limit(3000).toList(), + 3, random(0x4040f).limit(3000).toList())); Set allMetrics = cb.all().values().stream() .flatMap(Collection::stream) diff --git a/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java b/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java index 67216abb30..c6ba59d178 100644 --- a/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java +++ b/common/src/test/java/org/astraea/common/balancer/BalancerConsoleTest.java @@ -25,7 +25,6 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; 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; @@ -288,10 +287,7 @@ void testCheckNoOngoingMigration() { cluster.clusterId(), cluster.brokers(), cluster.topics(), - cluster - .replicaStream() - .map(mapper) - .collect(Collectors.toUnmodifiableList())))) + cluster.replicaStream().map(mapper).toList()))) .when(spy) .clusterInfo(Mockito.anySet()); Assertions.assertThrows( diff --git a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java index e0687ab5c0..6ceb02fa70 100644 --- a/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java +++ b/common/src/test/java/org/astraea/common/balancer/executor/StraightPlanExecutorTest.java @@ -107,7 +107,7 @@ void testAsyncRun() { .values() .stream() .flatMap(Collection::stream) - .collect(Collectors.toUnmodifiableList()); + .toList(); final var expectedAllocation = ClusterInfoTest.of(allocation); final var expectedTopicPartition = expectedAllocation.topicPartitions(); diff --git a/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java b/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java index cb945b82ca..6fdc15c7b5 100644 --- a/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java +++ b/common/src/test/java/org/astraea/common/consumer/ConsumerTest.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -496,10 +495,12 @@ void testCreateConsumersConcurrent() { .seek(SEEK_TO, 0) .consumerRebalanceListener(ps -> log.put(index, ps.size())) .build()) { - while (!closed.get()) consumer.poll(Duration.ofSeconds(2)); + while (!closed.get()) { + consumer.poll(Duration.ofSeconds(2)); + } } })) - .collect(Collectors.toUnmodifiableList())); + .toList()); Utils.waitFor(() -> log.size() == consumers, Duration.ofSeconds(15)); Utils.waitFor( () -> log.values().stream().filter(ps -> ps == 0).count() == 1, Duration.ofSeconds(15)); @@ -686,10 +687,12 @@ void testRandomAssignorWithTwoTopicsAndMultipleConsumers() { latches.countDown(); }) .build()) { - while (!closed.get()) consumer.poll(Duration.ofSeconds(2)); + while (!closed.get()) { + consumer.poll(Duration.ofSeconds(2)); + } } })) - .collect(Collectors.toUnmodifiableList())); + .toList()); Utils.waitFor(() -> latches.getCount() == 0, Duration.ofSeconds(10)); assignments.values().forEach(v -> totalPartitions.set(totalPartitions.get() + v)); @@ -711,7 +714,7 @@ void testCreateMultiConsumersWithSameGroup() { .config(ConsumerConfigs.GROUP_ID_CONFIG, groupId) .seek(SeekStrategy.DISTANCE_FROM_BEGINNING, 0) .build()) - .collect(Collectors.toUnmodifiableList()); + .toList(); consumers.forEach(consumer -> consumer.poll(Duration.ofSeconds(1))); consumers.forEach(Consumer::close); } diff --git a/common/src/test/java/org/astraea/common/cost/NetworkIngressCostTest.java b/common/src/test/java/org/astraea/common/cost/NetworkIngressCostTest.java index b0687fe911..4da32ec6f7 100644 --- a/common/src/test/java/org/astraea/common/cost/NetworkIngressCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NetworkIngressCostTest.java @@ -21,7 +21,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; -import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.Configuration; import org.astraea.common.Utils; @@ -48,10 +47,7 @@ void testIncompatibility() { "1Byte", NetworkIngressCost.NETWORK_COST_ESTIMATION_METHOD, "BROKER_TOPIC_FIFTEEN_MINUTE_RATE"))); - var topics = - IntStream.range(0, 10) - .mapToObj(i -> Utils.randomString(6)) - .collect(Collectors.toUnmodifiableList()); + var topics = IntStream.range(0, 10).mapToObj(i -> Utils.randomString(6)).toList(); var clusterInfo = ClusterInfo.builder() .addNode(Set.of(1)) @@ -74,7 +70,7 @@ void testIncompatibility() { ServerMetrics.Topic.BYTES_IN_PER_SEC, topic, index.getAndIncrement())) - .collect(Collectors.toUnmodifiableList()))); + .toList())); var partitionCost = networkCost.partitionCost(clusterInfo, clusterBean); var incompatible = partitionCost.incompatibility(); @@ -92,10 +88,7 @@ void testEmptyIncompatibility() { "1Byte", NetworkIngressCost.NETWORK_COST_ESTIMATION_METHOD, "BROKER_TOPIC_FIFTEEN_MINUTE_RATE"))); - var topics = - IntStream.range(0, 10) - .mapToObj(i -> Utils.randomString(6)) - .collect(Collectors.toUnmodifiableList()); + var topics = IntStream.range(0, 10).mapToObj(i -> Utils.randomString(6)).toList(); var clusterInfo = ClusterInfo.builder() .addNode(Set.of(1)) @@ -112,7 +105,7 @@ void testEmptyIncompatibility() { 1, topics.stream() .map(topic -> bandwidth(ServerMetrics.Topic.BYTES_IN_PER_SEC, topic, 1.0)) - .collect(Collectors.toUnmodifiableList()))); + .toList())); var partitionCost = networkCost.partitionCost(clusterInfo, clusterBean); var incompatible = partitionCost.incompatibility(); diff --git a/common/src/test/java/org/astraea/common/cost/NodeLatencyCostTest.java b/common/src/test/java/org/astraea/common/cost/NodeLatencyCostTest.java index 4a7aa6fc28..4fe09474a9 100644 --- a/common/src/test/java/org/astraea/common/cost/NodeLatencyCostTest.java +++ b/common/src/test/java/org/astraea/common/cost/NodeLatencyCostTest.java @@ -19,7 +19,6 @@ import java.time.Duration; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.admin.ClusterInfo; @@ -93,7 +92,7 @@ void testCost() { -1, ProducerMetrics.node(JndiClient.local()).stream() .map(b -> (HasBeanObject) b) - .collect(Collectors.toUnmodifiableList())))) + .toList()))) .value() .size() >= 1); diff --git a/common/src/test/java/org/astraea/common/cost/NormalizerTest.java b/common/src/test/java/org/astraea/common/cost/NormalizerTest.java index 54f7286e96..97fa06028f 100644 --- a/common/src/test/java/org/astraea/common/cost/NormalizerTest.java +++ b/common/src/test/java/org/astraea/common/cost/NormalizerTest.java @@ -34,10 +34,7 @@ void testRandomValues(Normalizer normalizer) { index -> { // generate random data var data = - IntStream.range(0, 100) - .boxed() - .map(i -> Math.random() * i * 10000) - .collect(Collectors.toUnmodifiableList()); + IntStream.range(0, 100).boxed().map(i -> Math.random() * i * 10000).toList(); var result = normalizer.normalize(data); Assertions.assertNotEquals(0, result.size()); // make sure there is no NaN @@ -57,7 +54,7 @@ void testSmallValues(Normalizer normalizer) { var data = IntStream.range(0, 100) .mapToObj(i -> Math.max(0.3, Math.min(0.7, Math.random()))) - .collect(Collectors.toUnmodifiableList()); + .toList(); var result = normalizer.normalize(data); Assertions.assertNotEquals(0, result.size()); // make sure there is no NaN diff --git a/common/src/test/java/org/astraea/common/cost/WeightProviderTest.java b/common/src/test/java/org/astraea/common/cost/WeightProviderTest.java index 5247f9a63d..55943c053d 100644 --- a/common/src/test/java/org/astraea/common/cost/WeightProviderTest.java +++ b/common/src/test/java/org/astraea/common/cost/WeightProviderTest.java @@ -36,11 +36,9 @@ void testPositiveEntropy() { var confusion = Map.of( "0", - IntStream.range(0, 100) - .mapToObj(i -> Math.random() * i) - .collect(Collectors.toUnmodifiableList()), + IntStream.range(0, 100).mapToObj(i -> Math.random() * i).toList(), "1", - IntStream.range(0, 100).mapToObj(i -> 1.0).collect(Collectors.toUnmodifiableList())); + IntStream.range(0, 100).mapToObj(i -> 1.0).toList()); var entropy = weightProvider.weight(confusion); // "0" range of 0 to 100 numbers * Math.random(). @@ -76,17 +74,12 @@ void testPositiveEntropy() { .collect( Collectors.toMap( String::valueOf, - ignored -> - IntStream.range(0, 100) - .mapToObj(i -> 1.0 + i % 10 * 0.1) - .collect(Collectors.toUnmodifiableList()))); + ignored -> IntStream.range(0, 100).mapToObj(i -> 1.0 + i % 10 * 0.1).toList())); // Since entropy represents the degree of uncertainty, uniform distribution does not represent // certainty in the concept of information entropy. On the contrary, because the uniform // distribution means that it is possible at every point, the degree of uncertainty is greater // and the weight given by the entropy method is also greater. - uniformDistribution.put( - "1", - IntStream.range(0, 100).mapToObj(i -> 0.0 + i).collect(Collectors.toUnmodifiableList())); + uniformDistribution.put("1", IntStream.range(0, 100).mapToObj(i -> 0.0 + i).toList()); var uniformEntropy = weightProvider.weight(uniformDistribution); Assertions.assertTrue(uniformEntropy.get("0") < uniformEntropy.get("1")); @@ -105,21 +98,19 @@ void testEntropy(Normalizer normalizer) { Collectors.toMap( String::valueOf, ignored -> - IntStream.range(0, numberOfObjects) - .mapToObj(i -> Math.random()) - .collect(Collectors.toUnmodifiableList()))); + IntStream.range(0, numberOfObjects).mapToObj(i -> Math.random()).toList())); // The smaller entropy means larger weight, so we sort the entropies/weights by different order // to check the metrics name later var entropies = weightProvider.entropies(raw).entrySet().stream() .sorted(Map.Entry.comparingByValue()) - .collect(Collectors.toUnmodifiableList()); + .toList(); entropies.forEach(e -> Assertions.assertTrue(0 <= e.getValue() && e.getValue() <= 1)); var weights = weightProvider.weight(raw).entrySet().stream() .sorted(Map.Entry.comparingByValue(Comparator.reverseOrder())) - .collect(Collectors.toUnmodifiableList()); + .toList(); Assertions.assertEquals(entropies.size(), weights.size()); IntStream.range(0, entropies.size()) .forEach(i -> Assertions.assertEquals(entropies.get(i).getKey(), weights.get(i).getKey())); diff --git a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java index bf3aa187a7..0fc7ca7bb2 100644 --- a/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java +++ b/common/src/test/java/org/astraea/common/cost/utils/ClusterInfoSensorTest.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.Utils; @@ -71,7 +70,7 @@ void testClusterInfoSensor() { .send( IntStream.range(0, 100) .mapToObj(x -> Record.builder().topic(topic).value(new byte[x]).build()) - .collect(Collectors.toUnmodifiableList())) + .toList()) .forEach(i -> i.toCompletableFuture().join()); } @@ -190,17 +189,13 @@ Stream partition(int partition, int replica) { 1, Stream.of(topic.partition(0, 1), topic.partition(1, 2), topic.partition(2, 3)) .flatMap(x -> x) - .collect(Collectors.toUnmodifiableList())), + .toList()), Map.entry( 2, Stream.of(topic.partition(1, 0), topic.partition(2, 0)) .flatMap(x -> x) - .collect(Collectors.toUnmodifiableList())), - Map.entry( - 3, - Stream.of(topic.partition(2, 0)) - .flatMap(x -> x) - .collect(Collectors.toUnmodifiableList())))); + .toList()), + Map.entry(3, Stream.of(topic.partition(2, 0)).flatMap(x -> x).toList()))); var info = ClusterInfoSensor.metricViewCluster(cb); Assertions.assertEquals(Set.of("topic"), info.topicNames()); diff --git a/common/src/test/java/org/astraea/common/metrics/broker/LogMetricsTest.java b/common/src/test/java/org/astraea/common/metrics/broker/LogMetricsTest.java index 452764d624..d8abe7b510 100644 --- a/common/src/test/java/org/astraea/common/metrics/broker/LogMetricsTest.java +++ b/common/src/test/java/org/astraea/common/metrics/broker/LogMetricsTest.java @@ -79,9 +79,7 @@ void testMetrics(LogMetrics.Log log) { admin.creator().topic(topicName).numberOfPartitions(2).run().toCompletableFuture().join(); Utils.sleep(Duration.ofSeconds(2)); var beans = - log.fetch(JndiClient.local()).stream() - .filter(m -> m.topic().equals(topicName)) - .collect(Collectors.toUnmodifiableList()); + log.fetch(JndiClient.local()).stream().filter(m -> m.topic().equals(topicName)).toList(); Assertions.assertEquals(2, beans.size()); Assertions.assertEquals( 2, diff --git a/common/src/test/java/org/astraea/common/metrics/client/consumer/ConsumerMetricsTest.java b/common/src/test/java/org/astraea/common/metrics/client/consumer/ConsumerMetricsTest.java index 59deccff13..126fefc1e3 100644 --- a/common/src/test/java/org/astraea/common/metrics/client/consumer/ConsumerMetricsTest.java +++ b/common/src/test/java/org/astraea/common/metrics/client/consumer/ConsumerMetricsTest.java @@ -18,7 +18,6 @@ import java.time.Duration; import java.util.Set; -import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.consumer.Consumer; @@ -70,7 +69,7 @@ void testMultiBrokers() { Assertions.assertTrue( metrics.stream() .map(HasNodeMetrics::brokerId) - .collect(Collectors.toUnmodifiableList()) + .toList() .containsAll(SERVICE.dataFolders().keySet())); metrics.forEach(ConsumerMetricsTest::check); } diff --git a/common/src/test/java/org/astraea/common/metrics/client/producer/ProducerMetricsTest.java b/common/src/test/java/org/astraea/common/metrics/client/producer/ProducerMetricsTest.java index 114c7f2d55..307e3f0d7b 100644 --- a/common/src/test/java/org/astraea/common/metrics/client/producer/ProducerMetricsTest.java +++ b/common/src/test/java/org/astraea/common/metrics/client/producer/ProducerMetricsTest.java @@ -17,7 +17,6 @@ package org.astraea.common.metrics.client.producer; import java.time.Duration; -import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.metrics.JndiClient; @@ -178,7 +177,7 @@ void testNodeMetrics() { Assertions.assertTrue( metrics.stream() .map(HasNodeMetrics::brokerId) - .collect(Collectors.toUnmodifiableList()) + .toList() .containsAll(SERVICE.dataFolders().keySet())); metrics.forEach(ProducerMetricsTest::check); } diff --git a/common/src/test/java/org/astraea/common/metrics/collector/MetricFetcherTest.java b/common/src/test/java/org/astraea/common/metrics/collector/MetricFetcherTest.java index af04f030ad..b8013ac162 100644 --- a/common/src/test/java/org/astraea/common/metrics/collector/MetricFetcherTest.java +++ b/common/src/test/java/org/astraea/common/metrics/collector/MetricFetcherTest.java @@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.function.Supplier; -import java.util.stream.Collectors; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.consumer.Consumer; @@ -73,19 +72,13 @@ void testPublishAndClose() { Utils.sleep(Duration.ofSeconds(3)); Assertions.assertEquals(Set.of(-1000), fetcher.identities()); Assertions.assertNotEquals(0, queue.size()); - queue.forEach( - (id, es) -> - Assertions.assertEquals( - beans, es.stream().distinct().collect(Collectors.toUnmodifiableList()))); + queue.forEach((id, es) -> Assertions.assertEquals(beans, es.stream().distinct().toList())); var latest = fetcher.latest(); Assertions.assertEquals(1, latest.size()); latest .values() - .forEach( - bs -> - Assertions.assertEquals( - beans, bs.stream().distinct().collect(Collectors.toUnmodifiableList()))); + .forEach(bs -> Assertions.assertEquals(beans, bs.stream().distinct().toList())); } // make sure client get closed Mockito.verify(client, Mockito.times(1)).close(); @@ -161,8 +154,7 @@ void testTopic() throws InterruptedException, ExecutionException { .valueDeserializer(Deserializer.BEAN_OBJECT) .seek(SeekStrategy.DISTANCE_FROM_BEGINNING, 0) .build()) { - var records = - consumer.poll(Duration.ofSeconds(5)).stream().collect(Collectors.toUnmodifiableList()); + var records = consumer.poll(Duration.ofSeconds(5)).stream().toList(); Assertions.assertEquals(1, records.size()); var getBean = records.get(0).value(); Assertions.assertEquals(testBean.domainName(), getBean.domainName()); diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 87a00930d1..1bf8443317 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -22,7 +22,6 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import java.util.stream.IntStream; import org.astraea.common.Configuration; import org.astraea.common.Utils; @@ -243,10 +242,7 @@ void testRoundRobinLease() { partitioner.roundRobinKeeper.tryToUpdate(ClusterInfo.empty(), Map::of); var t = partitioner.roundRobinKeeper.lastUpdated.get(); - var rr = - Arrays.stream(partitioner.roundRobinKeeper.roundRobin) - .boxed() - .collect(Collectors.toUnmodifiableList()); + var rr = Arrays.stream(partitioner.roundRobinKeeper.roundRobin).boxed().toList(); Assertions.assertEquals(StrictCostPartitioner.ROUND_ROBIN_LENGTH, rr.size()); // the rr is not updated yet partitioner.roundRobinKeeper.tryToUpdate(ClusterInfo.empty(), Map::of); diff --git a/common/src/test/java/org/astraea/common/producer/ProducerTest.java b/common/src/test/java/org/astraea/common/producer/ProducerTest.java index 8ee723a5ad..ee68aa5a6e 100644 --- a/common/src/test/java/org/astraea/common/producer/ProducerTest.java +++ b/common/src/test/java/org/astraea/common/producer/ProducerTest.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.Header; @@ -171,7 +170,7 @@ void testMultiplesSend(Producer producer) throws InterruptedExce .send( IntStream.range(0, count) .mapToObj(i -> Record.builder().topic(topic).value(new byte[10]).build()) - .collect(Collectors.toUnmodifiableList())) + .toList()) .forEach(f -> f.whenComplete((m, e) -> latch.countDown())); latch.await(); diff --git a/connector/src/test/java/org/astraea/connector/SourceDataTest.java b/connector/src/test/java/org/astraea/connector/SourceDataTest.java index bec16a752a..92d34f7016 100644 --- a/connector/src/test/java/org/astraea/connector/SourceDataTest.java +++ b/connector/src/test/java/org/astraea/connector/SourceDataTest.java @@ -97,8 +97,7 @@ void testConsumeDataFromSource() { ConsumerConfigs.AUTO_OFFSET_RESET_CONFIG, ConsumerConfigs.AUTO_OFFSET_RESET_EARLIEST) .build()) { - var records = - consumer.poll(Duration.ofSeconds(10)).stream().collect(Collectors.toUnmodifiableList()); + var records = consumer.poll(Duration.ofSeconds(10)).stream().toList(); Assertions.assertEquals(1, records.size()); Assertions.assertArrayEquals(KEY, records.get(0).key()); Assertions.assertArrayEquals(VALUE, records.get(0).value()); diff --git a/gui/src/main/java/org/astraea/gui/button/SelectBox.java b/gui/src/main/java/org/astraea/gui/button/SelectBox.java index ba27de86cc..11491d1daf 100644 --- a/gui/src/main/java/org/astraea/gui/button/SelectBox.java +++ b/gui/src/main/java/org/astraea/gui/button/SelectBox.java @@ -49,7 +49,7 @@ static SelectBox single(List keys, int sizeOfColumns) { }); return box; }) - .collect(Collectors.toUnmodifiableList()); + .toList(); items.get(0).setSelected(true); var node = grid(items.stream().map(m -> (Node) m).collect(Collectors.toList()), sizeOfColumns); @@ -71,7 +71,7 @@ static SelectBox multi(List keys, int sizeOfColumns) { }); return box; }) - .collect(Collectors.toUnmodifiableList()); + .toList(); var node = grid(items.stream().map(m -> (Node) m).collect(Collectors.toList()), sizeOfColumns); return of(() -> selectedKeys, node); } diff --git a/gui/src/main/java/org/astraea/gui/table/TableViewer.java b/gui/src/main/java/org/astraea/gui/table/TableViewer.java index 40db14b0b3..04caddfe38 100644 --- a/gui/src/main/java/org/astraea/gui/table/TableViewer.java +++ b/gui/src/main/java/org/astraea/gui/table/TableViewer.java @@ -228,7 +228,7 @@ private void refresh() { }); return col; }) - .collect(Collectors.toUnmodifiableList()); + .toList(); Runnable updater = () -> { table.getColumns().setAll(columns); diff --git a/it/src/main/java/org/astraea/it/BrokerCluster.java b/it/src/main/java/org/astraea/it/BrokerCluster.java index 8471bb5471..ea3c4abf56 100644 --- a/it/src/main/java/org/astraea/it/BrokerCluster.java +++ b/it/src/main/java/org/astraea/it/BrokerCluster.java @@ -77,7 +77,7 @@ static BrokerCluster of(int numberOfBrokers, Map override) { .map(ignored -> Utils.availablePort()) .distinct() .limit(numberOfBrokers) - .collect(Collectors.toUnmodifiableList()); + .toList(); if (ports.size() != numberOfBrokers) throw new RuntimeException("failed to get enough available ports."); diff --git a/it/src/main/java/org/astraea/it/FtpServer.java b/it/src/main/java/org/astraea/it/FtpServer.java index bb3233e028..d82b8505ed 100644 --- a/it/src/main/java/org/astraea/it/FtpServer.java +++ b/it/src/main/java/org/astraea/it/FtpServer.java @@ -140,8 +140,7 @@ public FtpServer build() { listenerFactory.setPort(controlPort); var connectionConfig = new DataConnectionConfigurationFactory(); - List availableDataPorts = - dataPorts.stream().map(Utils::resolvePort).collect(Collectors.toUnmodifiableList()); + List availableDataPorts = dataPorts.stream().map(Utils::resolvePort).toList(); connectionConfig.setActiveEnabled(false); connectionConfig.setPassiveExternalAddress(Utils.hostname()); @@ -196,7 +195,7 @@ public String password() { public List dataPorts() { return Stream.of(connectionConfig.getPassivePorts().split(",")) .map(Integer::valueOf) - .collect(Collectors.toUnmodifiableList()); + .toList(); } @Override From e5721e1c7bf1837ca56cea9127740d0b57d90926 Mon Sep 17 00:00:00 2001 From: Haser Date: Wed, 21 Jun 2023 16:35:58 +0800 Subject: [PATCH 64/77] [EXPORTER] Close fs in Task.close() (#1819) * close fs in Task.close() * using Utils.close() --- .../src/main/java/org/astraea/connector/backup/Exporter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 9b537911c6..28980e2c72 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -408,6 +408,7 @@ private void checkSeekOffset(Record r) { protected void close() { this.closed.set(true); Utils.packException(() -> writerFuture.toCompletableFuture().get(10, TimeUnit.SECONDS)); + Utils.close(this.fs); } boolean isWriterDone() { From 8ab86c9db0ae84a622744a01f5ec1be7341e9edd Mon Sep 17 00:00:00 2001 From: Haser Date: Sat, 24 Jun 2023 10:15:32 +0800 Subject: [PATCH 65/77] [EXPORTER] Fix the size in RecordWriter never be updated (#1821) * fix the size in RecordWriter never be updated, also add a test for this. * add test for size in recordWriter --- .../common/backup/RecordWriterBuilder.java | 59 ++++++++++--------- .../connector/backup/ExporterTest.java | 22 ++++++- 2 files changed, 51 insertions(+), 30 deletions(-) diff --git a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java index d9d3b34801..c084e7d1c2 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java @@ -44,34 +44,37 @@ public class RecordWriterBuilder { @Override public void append(Record record) { Utils.packException( - () -> - RecordOuterClass.Record.newBuilder() - .setTopic(record.topic()) - .setPartition(record.partition()) - .setOffset(record.offset()) - .setTimestamp(record.timestamp()) - .setKey( - record.key() == null - ? ByteString.EMPTY - : ByteString.copyFrom(record.key())) - .setValue( - record.value() == null - ? ByteString.EMPTY - : ByteString.copyFrom(record.value())) - .addAllHeaders( - record.headers().stream() - .map( - header -> - RecordOuterClass.Record.Header.newBuilder() - .setKey(header.key()) - .setValue( - header.value() == null - ? ByteString.EMPTY - : ByteString.copyFrom(header.value())) - .build()) - .toList()) - .build() - .writeDelimitedTo(outputStream)); + () -> { + var recordBuilder = + RecordOuterClass.Record.newBuilder() + .setTopic(record.topic()) + .setPartition(record.partition()) + .setOffset(record.offset()) + .setTimestamp(record.timestamp()) + .setKey( + record.key() == null + ? ByteString.EMPTY + : ByteString.copyFrom(record.key())) + .setValue( + record.value() == null + ? ByteString.EMPTY + : ByteString.copyFrom(record.value())) + .addAllHeaders( + record.headers().stream() + .map( + header -> + RecordOuterClass.Record.Header.newBuilder() + .setKey(header.key()) + .setValue( + header.value() == null + ? ByteString.EMPTY + : ByteString.copyFrom(header.value())) + .build()) + .toList()) + .build(); + recordBuilder.writeDelimitedTo(outputStream); + this.size.add(recordBuilder.getSerializedSize()); + }); count.incrementAndGet(); this.latestAppendTimestamp.set(System.currentTimeMillis()); } diff --git a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java index 0f88ab4435..6a27990054 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java @@ -465,15 +465,25 @@ void testHdfsSinkTask() { Record.builder() .topic(topicName) .key("test".getBytes()) - .value("test0".getBytes()) + .value(Utils.randomString(1024).getBytes()) .partition(0) + .offset(0) .timestamp(System.currentTimeMillis()) .build(), Record.builder() .topic(topicName) .key("test".getBytes()) - .value("test1".getBytes()) + .value("test".getBytes()) + .partition(0) + .offset(1) + .timestamp(System.currentTimeMillis()) + .build(), + Record.builder() + .topic(topicName) + .key("test".getBytes()) + .value("test".getBytes()) .partition(1) + .offset(0) .timestamp(System.currentTimeMillis()) .build()); @@ -489,6 +499,10 @@ void testHdfsSinkTask() { Assertions.assertEquals( 2, fs.listFolders("/" + String.join("/", fileSize, topicName)).size()); + Assertions.assertEquals( + 2, fs.listFiles("/" + String.join("/", fileSize, topicName, "0")).size()); + Assertions.assertEquals( + 1, fs.listFiles("/" + String.join("/", fileSize, topicName, "1")).size()); records.forEach( sinkRecord -> { @@ -770,6 +784,8 @@ void testCreateRecordWriter() { writers.put(tp, recordWriter); + Assertions.assertEquals(DataSize.ZERO, recordWriter.size()); + recordWriter.append( Record.builder() .topic(topicName) @@ -780,6 +796,8 @@ void testCreateRecordWriter() { .timestamp(System.currentTimeMillis()) .build()); + Assertions.assertNotEquals(DataSize.ZERO, recordWriter.size()); + task.removeOldWriters(writers); // the writer should not be closed before sleep. From 9db90e26fdc102dc1c15abdec878293b42595653 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 26 Jun 2023 19:07:32 +0800 Subject: [PATCH 66/77] [COMMON] upgrade depedencies (#1824) --- gradle/dependencies.gradle | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 59d9235693..5e0acfc4fe 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -20,20 +20,21 @@ ext { libs = [:] } def versions = [ - "commons-io" : project.properties['commons-io.version'] ?: "2.11.0", + "commons-io" : project.properties['commons-io.version'] ?: "2.13.0", "commons-math3" : project.properties['commons-math3.version'] ?: "3.6.1", "commons-net" : project.properties['commons-net.version'] ?: "3.9.0", "commons-logging" : project.properties['commons-logging.version'] ?: "1.2", "ftp-server" : project.properties['ftp-server.version'] ?: "1.2.0", - jackson : project.properties['jackson.version'] ?: "2.14.2", + jackson : project.properties['jackson.version'] ?: "2.15.2", jcommander : project.properties['jcommander.version'] ?: "1.82", "jmh-core" : project.properties['jmh-core.version'] ?: "1.36", "jmh-generator-annprocess": project.properties['jmh-generator.version'] ?: "1.36", - junit : project.properties['junit.version'] ?: "5.9.2", + junit : project.properties['junit.version'] ?: "5.9.3", kafka : project.properties['kafka.version'] ?: "3.5.0", - mockito : project.properties['mockito.version'] ?: "5.2.0", + mockito : project.properties['mockito.version'] ?: "5.4.0", + "mockito-inline" : project.properties['mockito.version'] ?: "5.2.0", "opencsv" : project.properties['opencsv.version'] ?: "5.7.1", - scala : project.properties['scala.version'] ?: "2.13.10", + scala : project.properties['scala.version'] ?: "2.13.11", slf4j : project.properties['slf4j.version'] ?: "2.0.7", spark : project.properties['spark.version'] ?: "3.3.2", "hadoop" : project.properties["hadoop.version"] ?: "3.3.5", @@ -58,7 +59,7 @@ libs += [ "kafka-core" : "org.apache.kafka:kafka_2.13:${versions["kafka"]}", "kafka-server-common" : "org.apache.kafka:kafka-server-common:${versions["kafka"]}", "mockito-core" : "org.mockito:mockito-core:${versions["mockito"]}", - "mockito-inline" : "org.mockito:mockito-inline:${versions["mockito"]}", + "mockito-inline" : "org.mockito:mockito-inline:${versions["mockito-inline"]}", opencsv : "com.opencsv:opencsv:${versions["opencsv"]}", scala : "org.scala-lang:scala-library:${versions["scala"]}", "slf4j-nop" : "org.slf4j:slf4j-nop:${versions["slf4j"]}", From 0d81aa2b51d856d902faf298945970b60f9ae407 Mon Sep 17 00:00:00 2001 From: Kuan-Po Tseng Date: Tue, 27 Jun 2023 12:27:48 +0800 Subject: [PATCH 67/77] [COMMON] Fix null elements in CsvWriter and remove opencsv (#1828) --- .../astraea/common/csv/CsvWriterBuilder.java | 8 ++++- .../org/astraea/common/csv/CsvWriterTest.java | 18 +++++++++++ etl/build.gradle | 1 - .../scala/org/astraea/etl/FileCreator.scala | 31 ++++++++++--------- gradle/dependencies.gradle | 2 -- 5 files changed, 42 insertions(+), 18 deletions(-) diff --git a/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java b/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java index a9c6ea6521..e615cd9d77 100644 --- a/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/csv/CsvWriterBuilder.java @@ -20,6 +20,7 @@ import java.io.Writer; import java.util.List; +import java.util.stream.Collectors; import org.astraea.common.Utils; public class CsvWriterBuilder { @@ -69,7 +70,12 @@ else if (genericLength != nextLine.size()) { @Override public void rawAppend(List nextLine) { - Utils.packException(() -> writer.write(String.join(SEPARATOR, nextLine))); + Utils.packException( + () -> + writer.write( + nextLine.stream() + .map(e -> e == null ? "" : e) + .collect(Collectors.joining(SEPARATOR)))); Utils.packException(() -> writer.write('\n')); } diff --git a/common/src/test/java/org/astraea/common/csv/CsvWriterTest.java b/common/src/test/java/org/astraea/common/csv/CsvWriterTest.java index 5dcd700102..76451f88cd 100644 --- a/common/src/test/java/org/astraea/common/csv/CsvWriterTest.java +++ b/common/src/test/java/org/astraea/common/csv/CsvWriterTest.java @@ -18,10 +18,12 @@ import static org.astraea.it.Utils.createTempDirectory; import static org.astraea.it.Utils.mkdir; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; import java.util.List; import org.astraea.common.Utils; import org.junit.jupiter.api.Test; @@ -101,4 +103,20 @@ void nullErrorTest() { assertThrows(RuntimeException.class, () -> writer.append(null)); } } + + @Test + void nullElementsTest() { + var local_csv = createTempDirectory("local_CSV"); + var sink = mkdir(local_csv + "/sink"); + var target = Path.of(sink + "/" + DATA_MAME); + try (var writer = + CsvWriter.builder(Utils.packException(() -> Files.newBufferedWriter(target))).build()) { + List nullElements = new ArrayList<>(); + nullElements.add(null); + nullElements.add(null); + nullElements.add(null); + writer.append(nullElements); + } + assertEquals(",,\n", Utils.packException(() -> Files.readString(target))); + } } diff --git a/etl/build.gradle b/etl/build.gradle index eed8dca287..c99e2b5df7 100644 --- a/etl/build.gradle +++ b/etl/build.gradle @@ -42,7 +42,6 @@ dependencies { testImplementation libs["junit"] - testImplementation libs["opencsv"] // This will be removed when spark is updated to 3.3.2. https://issues.apache.org/jira/browse/SPARK-40886 testImplementation enforcedPlatform('com.fasterxml.jackson:jackson-bom:2.13.4.20221013') // there are unit tests requiring spark, so we add them back for test scope diff --git a/etl/src/test/scala/org/astraea/etl/FileCreator.scala b/etl/src/test/scala/org/astraea/etl/FileCreator.scala index 1d5d1d4fff..2cdecc08db 100644 --- a/etl/src/test/scala/org/astraea/etl/FileCreator.scala +++ b/etl/src/test/scala/org/astraea/etl/FileCreator.scala @@ -16,7 +16,7 @@ */ package org.astraea.etl -import com.opencsv.CSVWriter +import org.astraea.common.csv.CsvWriter import java.io.{BufferedWriter, FileWriter} import java.nio.file.{Files, Path} @@ -56,19 +56,22 @@ object FileCreator { path: String, rows: List[List[String]] ): Try[Unit] = - Try(new CSVWriter(new BufferedWriter(new FileWriter(path)))).flatMap( - (csvWriter: CSVWriter) => - Try { - csvWriter.writeAll(rows.map(_.toArray).asJava) - csvWriter.close() - } match { - case f @ Failure(_) => - Try(csvWriter.close()).recoverWith { case _ => - f - } - case success => - success - } + Try( + CsvWriter + .builder(new BufferedWriter(new FileWriter(path))) + .build() + ).flatMap((csvWriter: CsvWriter) => + Try { + rows.map(_.asJava).foreach(csvWriter.append) + csvWriter.close() + } match { + case f @ Failure(_) => + Try(csvWriter.close()).recoverWith { case _ => + f + } + case success => + success + } ) def getCSVFile(file: Path): Seq[Path] = { diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 5e0acfc4fe..3a53a93efc 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -33,7 +33,6 @@ def versions = [ kafka : project.properties['kafka.version'] ?: "3.5.0", mockito : project.properties['mockito.version'] ?: "5.4.0", "mockito-inline" : project.properties['mockito.version'] ?: "5.2.0", - "opencsv" : project.properties['opencsv.version'] ?: "5.7.1", scala : project.properties['scala.version'] ?: "2.13.11", slf4j : project.properties['slf4j.version'] ?: "2.0.7", spark : project.properties['spark.version'] ?: "3.3.2", @@ -60,7 +59,6 @@ libs += [ "kafka-server-common" : "org.apache.kafka:kafka-server-common:${versions["kafka"]}", "mockito-core" : "org.mockito:mockito-core:${versions["mockito"]}", "mockito-inline" : "org.mockito:mockito-inline:${versions["mockito-inline"]}", - opencsv : "com.opencsv:opencsv:${versions["opencsv"]}", scala : "org.scala-lang:scala-library:${versions["scala"]}", "slf4j-nop" : "org.slf4j:slf4j-nop:${versions["slf4j"]}", "spark-kafka" : "org.apache.spark:spark-sql-kafka-0-10_2.13:${versions["spark"]}", From 0ffd7621f323692f989cc4736d5efd3d5d5a8d72 Mon Sep 17 00:00:00 2001 From: Chao-Heng Lee Date: Thu, 29 Jun 2023 15:40:49 +0800 Subject: [PATCH 68/77] [WEB] Add BackupHandler into WebService (#1820) --- app/build.gradle | 2 + .../astraea/app/argument/URLListField.java | 48 ++ .../org/astraea/app/web/BackupHandler.java | 241 ++++++++ .../java/org/astraea/app/web/WebService.java | 18 +- .../org/astraea/app/argument/URLListTest.java | 55 ++ .../astraea/app/web/BackupHandlerTest.java | 553 ++++++++++++++++++ .../org/astraea/app/web/TopicHandlerTest.java | 3 +- .../org/astraea/app/web/WebServiceTest.java | 25 +- docs/web_server/README.md | 3 +- docs/web_server/web_api_backups_chinese.md | 219 +++++++ 10 files changed, 1157 insertions(+), 10 deletions(-) create mode 100644 app/src/main/java/org/astraea/app/argument/URLListField.java create mode 100644 app/src/main/java/org/astraea/app/web/BackupHandler.java create mode 100644 app/src/test/java/org/astraea/app/argument/URLListTest.java create mode 100644 app/src/test/java/org/astraea/app/web/BackupHandlerTest.java create mode 100644 docs/web_server/web_api_backups_chinese.md diff --git a/app/build.gradle b/app/build.gradle index 43c14330e0..d43c8014f9 100644 --- a/app/build.gradle +++ b/app/build.gradle @@ -36,12 +36,14 @@ dependencies { implementation project(':common') implementation project(':fs') + implementation project(':connector') implementation libs["kafka-client"] implementation libs["jcommander"] // we don't use slf4j actually, and it is used by kafka so we swallow the log. implementation libs["slf4j-nop"] implementation libs["commons-math3"] implementation libs["jackson-datatype-jdk8"] + implementation libs["kafka-connect-api"] } application { diff --git a/app/src/main/java/org/astraea/app/argument/URLListField.java b/app/src/main/java/org/astraea/app/argument/URLListField.java new file mode 100644 index 0000000000..a61dbca7dd --- /dev/null +++ b/app/src/main/java/org/astraea/app/argument/URLListField.java @@ -0,0 +1,48 @@ +/* + * 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.app.argument; + +import com.beust.jcommander.ParameterException; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.astraea.common.Utils; + +public class URLListField extends ListField { + @Override + public List convert(String value) { + return Stream.of(value.split(SEPARATOR)) + .map(string -> Utils.packException(() -> new URL(string))) + .collect(Collectors.toList()); + } + + @Override + protected void check(String name, String value) { + super.check(name, value); + Stream.of(value.split(SEPARATOR)) + .forEach( + string -> { + try { + new URL(string); + } catch (MalformedURLException e) { + throw new ParameterException(string + " should be URL"); + } + }); + } +} diff --git a/app/src/main/java/org/astraea/app/web/BackupHandler.java b/app/src/main/java/org/astraea/app/web/BackupHandler.java new file mode 100644 index 0000000000..7e5e312741 --- /dev/null +++ b/app/src/main/java/org/astraea/app/web/BackupHandler.java @@ -0,0 +1,241 @@ +/* + * 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.app.web; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.astraea.common.FutureUtils; +import org.astraea.common.connector.ConnectorClient; +import org.astraea.common.connector.ConnectorConfigs; +import org.astraea.common.connector.ConnectorInfo; +import org.astraea.common.connector.ConnectorStatus; +import org.astraea.common.connector.TaskInfo; +import org.astraea.common.connector.TaskStatus; +import org.astraea.common.json.TypeRef; +import org.astraea.connector.backup.Exporter; +import org.astraea.connector.backup.Importer; + +public class BackupHandler implements Handler { + + private final ConnectorClient connectorClient; + + BackupHandler(ConnectorClient connectorClient) { + this.connectorClient = connectorClient; + } + + @Override + public CompletionStage get(Channel channel) { + return channel + .target() + .map(name -> CompletableFuture.completedStage(Set.of(name))) + .orElseGet(connectorClient::connectorNames) + .thenCompose( + names -> + FutureUtils.sequence( + names.stream() + .map(name -> connectorClient.connectorStatus(name).toCompletableFuture()) + .toList())) + .thenApply(this::statusResponse); + } + + @Override + public CompletionStage post(Channel channel) { + var postRequest = channel.request(TypeRef.of(BackupRequest.class)); + return FutureUtils.sequence( + nameConfigEntry(postRequest) + .map( + entry -> + connectorClient + .createConnector(entry.getKey(), entry.getValue()) + .toCompletableFuture()) + .toList()) + .thenApply(this::infoResponse); + } + + @Override + public CompletionStage delete(Channel channel) { + return channel + .target() + .map(name -> connectorClient.deleteConnector(name).thenApply(ignore -> Response.OK)) + .orElse(CompletableFuture.completedStage(Response.NOT_FOUND)); + } + + @Override + public CompletionStage put(Channel channel) { + var putRequest = channel.request(TypeRef.of(BackupRequest.class)); + return FutureUtils.sequence( + nameConfigEntry(putRequest) + .map( + entry -> + connectorClient + .updateConnector(entry.getKey(), entry.getValue()) + .toCompletableFuture()) + .toList()) + .thenApply(this::infoResponse); + } + + private Stream>> nameConfigEntry(BackupRequest request) { + return Stream.concat( + request.importer.stream() + .map( + importer -> { + var config = connectorConfigMap(importer); + config.put(ConnectorConfigs.CONNECTOR_CLASS_KEY, Importer.class.getName()); + importer.cleanSourcePolicy.ifPresent( + policy -> config.put("clean.source", policy)); + importer.archiveDir.ifPresent(dir -> config.put("archive.dir", dir)); + return Map.entry(importer.name, config); + }), + request.exporter.stream() + .map( + exporter -> { + var config = connectorConfigMap(exporter); + config.put(ConnectorConfigs.CONNECTOR_CLASS_KEY, Exporter.class.getName()); + exporter.size.ifPresent(size -> config.put("size", size)); + exporter.rollDuration.ifPresent( + duration -> config.put("roll.duration", duration)); + exporter.writerBufferSize.ifPresent( + size -> config.put("writer.buffer.size", size)); + exporter.offsetFrom.ifPresent(config::putAll); + return Map.entry(exporter.name, config); + })); + } + + private Map connectorConfigMap(ConnectorConfig config) { + var configMap = new HashMap(); + configMap.put(ConnectorConfigs.TOPICS_KEY, config.topics); + configMap.put(ConnectorConfigs.TASK_MAX_KEY, config.tasksMax); + configMap.put(ConnectorConfigs.KEY_CONVERTER_KEY, config.keyConverter); + configMap.put(ConnectorConfigs.VALUE_CONVERTER_KEY, config.valueConverter); + configMap.put(ConnectorConfigs.HEADER_CONVERTER_KEY, config.headerConverter); + configMap.put("path", config.path); + configMap.put("fs.schema", config.fsSchema); + config.hostname.ifPresent( + hostname -> configMap.put("fs." + config.fsSchema + ".hostname", hostname)); + config.port.ifPresent(port -> configMap.put("fs." + config.fsSchema + ".port", port)); + config.user.ifPresent(user -> configMap.put("fs." + config.fsSchema + ".user", user)); + config.password.ifPresent( + password -> configMap.put("fs." + config.fsSchema + ".password", password)); + return configMap; + } + + private ConnectorInfoResponse infoResponse(List connectorInfos) { + var groups = + connectorInfos.stream() + .map( + connectorInfo -> + new ConnectorInfoClass( + connectorInfo.name(), + connectorInfo.config(), + connectorInfo.tasks().stream().map(TaskInfo::id).toList())) + .collect( + Collectors.groupingBy( + connectorInfoClass -> + connectorInfoClass.config.get(ConnectorConfigs.CONNECTOR_CLASS_KEY))); + return new ConnectorInfoResponse( + groups.get(Importer.class.getName()), groups.get(Exporter.class.getName())); + } + + private ConnectorStatusResponse statusResponse(List connectorStatuses) { + var groups = + connectorStatuses.stream() + .map( + connectorStatus -> + new ConnectorStatusClass( + connectorStatus.name(), + connectorStatus.state(), + connectorStatus.configs(), + connectorStatus.tasks().stream() + .collect(Collectors.toMap(TaskStatus::id, TaskStatus::state)))) + .collect( + Collectors.groupingBy( + connectorStatusClass -> + connectorStatusClass.configs.get(ConnectorConfigs.CONNECTOR_CLASS_KEY))); + return new ConnectorStatusResponse( + groups.get(Importer.class.getName()), groups.get(Exporter.class.getName())); + } + + record ConnectorStatusClass( + String name, String state, Map configs, Map tasks) + implements Response { + private ConnectorStatusClass() { + this("", "", Map.of(), Map.of()); + } + } + + record ConnectorInfoClass(String name, Map config, List tasks) + implements Response { + private ConnectorInfoClass() { + this("", Map.of(), List.of()); + } + } + + record ConnectorStatusResponse( + List importers, List exporters) + implements Response { + private ConnectorStatusResponse() { + this(List.of(), List.of()); + } + } + + record ConnectorInfoResponse( + List importers, List exporters) implements Response { + + private ConnectorInfoResponse() { + this(List.of(), List.of()); + } + } + + static class ConnectorConfig implements Request { + String name; + String topics; + String tasksMax; + String path; + String keyConverter = ConnectorConfigs.BYTE_ARRAY_CONVERTER_CLASS; + String valueConverter = ConnectorConfigs.BYTE_ARRAY_CONVERTER_CLASS; + String headerConverter = ConnectorConfigs.BYTE_ARRAY_CONVERTER_CLASS; + String fsSchema; + Optional hostname = Optional.empty(); + Optional port = Optional.empty(); + Optional user = Optional.empty(); + Optional password = Optional.empty(); + } + + static class ImporterConfig extends ConnectorConfig implements Request { + Optional cleanSourcePolicy = Optional.empty(); + Optional archiveDir = Optional.empty(); + } + + static class ExporterConfig extends ConnectorConfig implements Request { + Optional size = Optional.empty(); + Optional rollDuration = Optional.empty(); + Optional writerBufferSize = Optional.empty(); + Optional> offsetFrom = Optional.empty(); + } + + static class BackupRequest implements Request { + List importer = List.of(); + List exporter = List.of(); + } +} diff --git a/app/src/main/java/org/astraea/app/web/WebService.java b/app/src/main/java/org/astraea/app/web/WebService.java index e4e9eab6b4..93d02342d8 100644 --- a/app/src/main/java/org/astraea/app/web/WebService.java +++ b/app/src/main/java/org/astraea/app/web/WebService.java @@ -20,6 +20,7 @@ import com.sun.net.httpserver.HttpHandler; import com.sun.net.httpserver.HttpServer; import java.net.InetSocketAddress; +import java.net.URL; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -34,10 +35,12 @@ import org.astraea.app.argument.DurationField; import org.astraea.app.argument.IntegerMapField; import org.astraea.app.argument.NonNegativeIntegerField; +import org.astraea.app.argument.URLListField; import org.astraea.common.Configuration; import org.astraea.common.Utils; import org.astraea.common.admin.Admin; import org.astraea.common.admin.Broker; +import org.astraea.common.connector.ConnectorClient; import org.astraea.common.metrics.JndiClient; import org.astraea.common.metrics.MBeanClient; import org.astraea.common.metrics.collector.MetricSensor; @@ -58,7 +61,8 @@ public WebService( int port, Function brokerIdToJmxPort, Duration beanExpiration, - Configuration config) { + Configuration config, + List workerUrls) { this.admin = admin; Supplier>> sensorsSupplier = () -> @@ -112,6 +116,9 @@ public WebService( server.createContext("/reassignments", to(new ReassignmentHandler(admin))); server.createContext("/balancer", to(new BalancerHandler(admin, metricStore))); server.createContext("/throttles", to(new ThrottleHandler(admin))); + if (!workerUrls.isEmpty()) + server.createContext( + "/backups", to(new BackupHandler(ConnectorClient.builder().urls(workerUrls).build()))); server.start(); } @@ -135,7 +142,8 @@ public static void main(String[] args) throws Exception { arg.port, arg::jmxPortMapping, arg.beanExpiration, - new Configuration(arg.configs()))) { + new Configuration(arg.configs()), + arg.workerUrls)) { if (arg.ttl == null) { System.out.println("enter ctrl + c to terminate web service"); TimeUnit.MILLISECONDS.sleep(Long.MAX_VALUE); @@ -196,6 +204,12 @@ int jmxPortMapping(int brokerId) { validateWith = DurationField.class, converter = DurationField.class) Duration beanExpiration = Duration.ofHours(1); + + @Parameter( + names = {"--worker.urls"}, + description = "List: the worker's urls", + validateWith = URLListField.class) + List workerUrls = List.of(); } static class Sensors { diff --git a/app/src/test/java/org/astraea/app/argument/URLListTest.java b/app/src/test/java/org/astraea/app/argument/URLListTest.java new file mode 100644 index 0000000000..1ea00f9a2e --- /dev/null +++ b/app/src/test/java/org/astraea/app/argument/URLListTest.java @@ -0,0 +1,55 @@ +/* + * 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.app.argument; + +import com.beust.jcommander.Parameter; +import java.net.URL; +import java.util.List; +import org.astraea.common.Utils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class URLListTest { + private static class FakeParameter { + @Parameter( + names = {"--field"}, + listConverter = URLListField.class, + validateWith = URLListField.class, + variableArity = true) + List value; + } + + @Test + public void testURLListTest() { + var param = + Argument.parse( + new FakeParameter(), + new String[] { + "--field", + "http://192.168.103.44:12345,http://192.168.103.44:12346,http://192.168.103.44:12347" + }); + + Assertions.assertEquals( + Utils.packException( + () -> + List.of( + new URL("http://192.168.103.44:12345"), + new URL("http://192.168.103.44:12346"), + new URL("http://192.168.103.44:12347"))), + param.value); + } +} diff --git a/app/src/test/java/org/astraea/app/web/BackupHandlerTest.java b/app/src/test/java/org/astraea/app/web/BackupHandlerTest.java new file mode 100644 index 0000000000..6ed0a309d2 --- /dev/null +++ b/app/src/test/java/org/astraea/app/web/BackupHandlerTest.java @@ -0,0 +1,553 @@ +/* + * 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.app.web; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import org.astraea.common.Configuration; +import org.astraea.common.Utils; +import org.astraea.common.admin.Admin; +import org.astraea.common.connector.ConnectorClient; +import org.astraea.common.connector.ConnectorConfigs; +import org.astraea.common.http.HttpExecutor; +import org.astraea.common.json.TypeRef; +import org.astraea.connector.backup.Exporter; +import org.astraea.connector.backup.Importer; +import org.astraea.it.Service; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class BackupHandlerTest { + private static final Service SERVICE = + Service.builder().numberOfBrokers(3).numberOfWorkers(3).build(); + + @AfterAll + static void closeService() { + SERVICE.close(); + } + + @Test + void testWithWebService() { + try (var admin = Admin.of(SERVICE.bootstrapServers())) { + var exporterName = Utils.randomString(10); + var exporterPath = Utils.randomString(10); + var topicName = Utils.randomString(10); + var request = new BackupHandler.BackupRequest(); + var exporterConfig = new BackupHandler.ExporterConfig(); + exporterConfig.name = exporterName; + exporterConfig.fsSchema = "local"; + exporterConfig.tasksMax = "3"; + exporterConfig.path = "/tmp/" + exporterPath; + exporterConfig.topics = topicName; + request.exporter = List.of(exporterConfig); + request.importer = List.of(); + + admin + .creator() + .topic(topicName) + .numberOfPartitions(3) + .numberOfReplicas((short) 1) + .run() + .toCompletableFuture() + .join(); + Utils.sleep(Duration.ofSeconds(3)); + + try (var service = + new WebService( + Admin.of(SERVICE.bootstrapServers()), + 0, + id -> SERVICE.jmxServiceURL().getPort(), + Duration.ofMillis(5), + Configuration.EMPTY, + List.copyOf(SERVICE.workerUrls()))) { + var response = + HttpExecutor.builder() + .build() + .post( + "http://localhost:" + service.port() + "/backups", + request, + TypeRef.of(BackupHandler.ConnectorInfoResponse.class)) + .toCompletableFuture() + .join(); + + var responseExporter = response.body().exporters().get(0); + Assertions.assertEquals(200, response.statusCode()); + Assertions.assertEquals(exporterConfig.name, responseExporter.name()); + Assertions.assertEquals( + exporterConfig.topics, responseExporter.config().get(ConnectorConfigs.TOPICS_KEY)); + Assertions.assertEquals( + exporterConfig.tasksMax, responseExporter.config().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals( + exporterConfig.fsSchema, responseExporter.config().get("fs.schema")); + Assertions.assertEquals(exporterConfig.path, responseExporter.config().get("path")); + Assertions.assertEquals(0, response.body().importers().size()); + + HttpExecutor.builder() + .build() + .delete("http://localhost:" + service.port() + "/backups/" + exporterName) + .toCompletableFuture() + .join(); + } + } + } + + @Test + void testGetWithoutQuery() { + try (var admin = Admin.of(SERVICE.bootstrapServers())) { + var topicName = Utils.randomString(10); + var importerName = Utils.randomString(10); + var exporterName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + + admin + .creator() + .topic(topicName) + .numberOfPartitions(3) + .numberOfReplicas((short) 1) + .run() + .toCompletableFuture() + .join(); + Utils.sleep(Duration.ofSeconds(3)); + + connectorClient + .createConnector( + importerName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)) + .toCompletableFuture() + .join(); + connectorClient + .createConnector( + exporterName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Exporter.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax, + ConnectorConfigs.TOPICS_KEY, + topicName, + ConnectorConfigs.KEY_CONVERTER_KEY, + ConnectorConfigs.BYTE_ARRAY_CONVERTER_CLASS, + ConnectorConfigs.VALUE_CONVERTER_KEY, + ConnectorConfigs.BYTE_ARRAY_CONVERTER_CLASS, + ConnectorConfigs.HEADER_CONVERTER_KEY, + ConnectorConfigs.BYTE_ARRAY_CONVERTER_CLASS)) + .toCompletableFuture() + .join(); + + var handler = new BackupHandler(connectorClient); + var response = + Assertions.assertInstanceOf( + BackupHandler.ConnectorStatusResponse.class, + handler.get(Channel.EMPTY).toCompletableFuture().join()); + var importer = response.importers().get(0); + var exporter = response.exporters().get(0); + + Assertions.assertEquals(1, response.importers().size()); + Assertions.assertEquals(importerName, importer.name()); + Assertions.assertEquals(tasksMax, importer.configs().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(fsSchema, importer.configs().get("fs.schema")); + Assertions.assertEquals(pathName, importer.configs().get("path")); + Assertions.assertEquals(cleanSourcePolicy, importer.configs().get("clean.source")); + Assertions.assertEquals(1, response.exporters().size()); + Assertions.assertEquals(exporterName, response.exporters().get(0).name()); + Assertions.assertEquals(topicName, exporter.configs().get(ConnectorConfigs.TOPICS_KEY)); + Assertions.assertEquals(tasksMax, exporter.configs().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(fsSchema, exporter.configs().get("fs.schema")); + Assertions.assertEquals(pathName, exporter.configs().get("path")); + + connectorClient.deleteConnector(importerName).toCompletableFuture().join(); + connectorClient.deleteConnector(exporterName).toCompletableFuture().join(); + } + } + + @Test + void testGetWithQuery() { + var firstImporterName = Utils.randomString(10); + var secondImporterName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + + connectorClient + .createConnector( + firstImporterName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)) + .toCompletableFuture() + .join(); + connectorClient + .createConnector( + secondImporterName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)) + .toCompletableFuture() + .join(); + + var handler = new BackupHandler(connectorClient); + var response = + Assertions.assertInstanceOf( + BackupHandler.ConnectorStatusResponse.class, + handler.get(Channel.ofTarget(firstImporterName)).toCompletableFuture().join()); + var importer = response.importers().get(0); + + Assertions.assertEquals( + 2, connectorClient.connectorNames().toCompletableFuture().join().size()); + Assertions.assertEquals(1, response.importers().size()); + Assertions.assertEquals(firstImporterName, importer.name()); + Assertions.assertEquals(tasksMax, importer.configs().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(fsSchema, importer.configs().get("fs.schema")); + Assertions.assertEquals(pathName, importer.configs().get("path")); + Assertions.assertEquals(cleanSourcePolicy, importer.configs().get("clean.source")); + + connectorClient.deleteConnector(firstImporterName).toCompletableFuture().join(); + connectorClient.deleteConnector(secondImporterName).toCompletableFuture().join(); + } + + @Test + void testPostSingleBackupOperation() { + var importerName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + var beforePostSize = connectorClient.connectorNames().toCompletableFuture().join().size(); + var handler = new BackupHandler(connectorClient); + var request = + Channel.ofRequest( + String.format( + "{\"importer\":" + + "[{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\"," + + "\"path\":\"%s\",\"cleanSourcePolicy\":\"%s\"}]}", + importerName, fsSchema, tasksMax, pathName, cleanSourcePolicy)); + var response = + Assertions.assertInstanceOf( + BackupHandler.ConnectorInfoResponse.class, + handler.post(request).toCompletableFuture().join()); + var importer = response.importers().get(0); + var afterPostSize = connectorClient.connectorNames().toCompletableFuture().join().size(); + Assertions.assertEquals(0, beforePostSize); + Assertions.assertEquals(1, afterPostSize); + Assertions.assertEquals(importerName, importer.name()); + Assertions.assertEquals(pathName, importer.config().get("path")); + Assertions.assertEquals(fsSchema, importer.config().get("fs.schema")); + Assertions.assertEquals(tasksMax, importer.config().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(cleanSourcePolicy, importer.config().get("clean.source")); + + connectorClient.deleteConnector(importerName).toCompletableFuture().join(); + } + + @Test + void testPostBackupOperations() { + var firstImporterName = Utils.randomString(10); + var secondImporterName = Utils.randomString(10); + var firstExporterName = Utils.randomString(10); + var secondExporterName = Utils.randomString(10); + var topicName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + var handler = new BackupHandler(connectorClient); + var request = + Channel.ofRequest( + String.format( + "{\"importer\":" + + "[{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\",\"path\":\"%s\"," + + "\"cleanSourcePolicy\":\"%s\"}," + + "{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\",\"path\":\"%s\"," + + "\"cleanSourcePolicy\":\"%s\"}]," + + "\"exporter\":" + + "[{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\",\"path\":\"%s\"," + + "\"topics\":\"%s\"}," + + "{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\",\"path\":\"%s\"," + + "\"topics\":\"%s\"}]}", + firstImporterName, + fsSchema, + tasksMax, + pathName, + cleanSourcePolicy, + secondImporterName, + fsSchema, + tasksMax, + pathName, + cleanSourcePolicy, + firstExporterName, + fsSchema, + tasksMax, + pathName, + topicName, + secondExporterName, + fsSchema, + tasksMax, + pathName, + topicName)); + var response = + Assertions.assertInstanceOf( + BackupHandler.ConnectorInfoResponse.class, + handler.post(request).toCompletableFuture().join()); + Assertions.assertEquals(2, response.importers().size()); + Assertions.assertEquals(2, response.exporters().size()); + Assertions.assertTrue( + response.importers().stream() + .map(BackupHandler.ConnectorInfoClass::name) + .anyMatch(name -> List.of(firstImporterName, secondImporterName).contains(name))); + response + .importers() + .forEach( + importer -> { + Assertions.assertEquals(pathName, importer.config().get("path")); + Assertions.assertEquals(fsSchema, importer.config().get("fs.schema")); + Assertions.assertEquals( + tasksMax, importer.config().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(cleanSourcePolicy, importer.config().get("clean.source")); + }); + Assertions.assertTrue( + response.exporters().stream() + .map(BackupHandler.ConnectorInfoClass::name) + .anyMatch(name -> List.of(firstExporterName, secondExporterName).contains(name))); + response + .exporters() + .forEach( + exporter -> { + Assertions.assertEquals( + topicName, exporter.config().get(ConnectorConfigs.TOPICS_KEY)); + Assertions.assertEquals( + tasksMax, exporter.config().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(fsSchema, exporter.config().get("fs.schema")); + Assertions.assertEquals(pathName, exporter.config().get("path")); + }); + + connectorClient.deleteConnector(firstImporterName).toCompletableFuture().join(); + connectorClient.deleteConnector(secondImporterName).toCompletableFuture().join(); + connectorClient.deleteConnector(firstExporterName).toCompletableFuture().join(); + connectorClient.deleteConnector(secondExporterName).toCompletableFuture().join(); + } + + @Test + void testDelete() { + var importerName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + + connectorClient + .createConnector( + importerName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)) + .toCompletableFuture() + .join(); + + var beforeDeleteSize = connectorClient.connectorNames().toCompletableFuture().join().size(); + var handler = new BackupHandler(connectorClient); + handler.delete(Channel.ofTarget(importerName)).toCompletableFuture().join(); + var afterDeleteSize = connectorClient.connectorNames().toCompletableFuture().join().size(); + + Assertions.assertEquals(1, beforeDeleteSize); + Assertions.assertEquals(0, afterDeleteSize); + } + + @Test + void testPutSingleBackupOperation() { + var importerName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + + connectorClient + .createConnector( + importerName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)) + .toCompletableFuture() + .join(); + + var newTasksMax = "5"; + var newPathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var newCleanSourcePolicy = "delete"; + var request = + Channel.ofRequest( + String.format( + "{\"importer\":" + + "[{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\"," + + "\"path\":\"%s\",\"cleanSourcePolicy\":\"%s\"}]}", + importerName, fsSchema, newTasksMax, newPathName, newCleanSourcePolicy)); + var handler = new BackupHandler(connectorClient); + var response = + Assertions.assertInstanceOf( + BackupHandler.ConnectorInfoResponse.class, + handler.put(request).toCompletableFuture().join()); + var importer = response.importers().get(0); + + Assertions.assertEquals( + 1, connectorClient.connectorNames().toCompletableFuture().join().size()); + Assertions.assertEquals(importerName, importer.name()); + Assertions.assertEquals(newPathName, importer.config().get("path")); + Assertions.assertEquals(fsSchema, importer.config().get("fs.schema")); + Assertions.assertEquals(newTasksMax, importer.config().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(newCleanSourcePolicy, importer.config().get("clean.source")); + + connectorClient.deleteConnector(importerName).toCompletableFuture().join(); + } + + @Test + void testPutBackupOperations() { + var firstImporterName = Utils.randomString(10); + var secondImporterName = Utils.randomString(10); + var pathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var fsSchema = "local"; + var tasksMax = "3"; + var cleanSourcePolicy = "off"; + var connectorClient = ConnectorClient.builder().urls(SERVICE.workerUrls()).build(); + + connectorClient.createConnector( + firstImporterName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)); + connectorClient.createConnector( + secondImporterName, + Map.of( + "fs.schema", + fsSchema, + "path", + pathName, + "clean.source", + cleanSourcePolicy, + ConnectorConfigs.CONNECTOR_CLASS_KEY, + Importer.class.getName(), + ConnectorConfigs.TASK_MAX_KEY, + tasksMax)); + + var newTasksMax = "5"; + var newPathName = org.astraea.it.Utils.createTempDirectory(Utils.randomString(10)).toString(); + var newCleanSourcePolicy = "delete"; + var handler = new BackupHandler(connectorClient); + var request = + Channel.ofRequest( + String.format( + "{\"importer\":" + + "[{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\",\"path\":\"%s\"," + + "\"cleanSourcePolicy\":\"%s\"}," + + "{\"name\":\"%s\",\"fsSchema\":\"%s\",\"tasksMax\":\"%s\",\"path\":\"%s\"," + + "\"cleanSourcePolicy\":\"%s\"}]}", + firstImporterName, + fsSchema, + newTasksMax, + newPathName, + newCleanSourcePolicy, + secondImporterName, + fsSchema, + newTasksMax, + newPathName, + newCleanSourcePolicy)); + var response = + Assertions.assertInstanceOf( + BackupHandler.ConnectorInfoResponse.class, + handler.put(request).toCompletableFuture().join()); + Assertions.assertEquals(2, response.importers().size()); + Assertions.assertTrue( + response.importers().stream() + .map(BackupHandler.ConnectorInfoClass::name) + .anyMatch(name -> List.of(firstImporterName, secondImporterName).contains(name))); + response + .importers() + .forEach( + importer -> { + Assertions.assertEquals(newPathName, importer.config().get("path")); + Assertions.assertEquals(fsSchema, importer.config().get("fs.schema")); + Assertions.assertEquals( + newTasksMax, importer.config().get(ConnectorConfigs.TASK_MAX_KEY)); + Assertions.assertEquals(newCleanSourcePolicy, importer.config().get("clean.source")); + }); + + connectorClient.deleteConnector(firstImporterName).toCompletableFuture().join(); + connectorClient.deleteConnector(secondImporterName).toCompletableFuture().join(); + } +} diff --git a/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java b/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java index ce8569ee66..aefa425b57 100644 --- a/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java +++ b/app/src/test/java/org/astraea/app/web/TopicHandlerTest.java @@ -71,7 +71,8 @@ void testWithWebService() { 0, id -> SERVICE.jmxServiceURL().getPort(), Duration.ofMillis(5), - Configuration.EMPTY)) { + Configuration.EMPTY, + List.of())) { Response response = HttpExecutor.builder() .build() diff --git a/app/src/test/java/org/astraea/app/web/WebServiceTest.java b/app/src/test/java/org/astraea/app/web/WebServiceTest.java index 5ba689465a..cc6659f8b2 100644 --- a/app/src/test/java/org/astraea/app/web/WebServiceTest.java +++ b/app/src/test/java/org/astraea/app/web/WebServiceTest.java @@ -17,6 +17,7 @@ package org.astraea.app.web; import java.time.Duration; +import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; @@ -47,7 +48,12 @@ void testArgument() { void testClose() { var web = new WebService( - Mockito.mock(Admin.class), 0, id -> -1, Duration.ofMillis(5), Configuration.EMPTY); + Mockito.mock(Admin.class), + 0, + id -> -1, + Duration.ofMillis(5), + Configuration.EMPTY, + List.of()); web.close(); } @@ -116,7 +122,12 @@ void testMetricStoreConfiguration() { // Test default metric store configuration try (var web = new WebService( - Mockito.mock(Admin.class), 0, id -> -1, Duration.ofMillis(5), Configuration.EMPTY)) { + Mockito.mock(Admin.class), + 0, + id -> -1, + Duration.ofMillis(5), + Configuration.EMPTY, + List.of())) { Assertions.assertEquals(1, localReceiverCount.get()); Assertions.assertEquals(0, topicReceiverCount.get()); @@ -130,8 +141,8 @@ void testMetricStoreConfiguration() { 0, id -> -1, Duration.ofMillis(5), - new Configuration( - Map.of(WebService.METRIC_STORE_KEY, WebService.METRIC_STORE_LOCAL)))) { + new Configuration(Map.of(WebService.METRIC_STORE_KEY, WebService.METRIC_STORE_LOCAL)), + List.of())) { Assertions.assertEquals(1, localReceiverCount.get()); Assertions.assertEquals(0, topicReceiverCount.get()); @@ -150,7 +161,8 @@ void testMetricStoreConfiguration() { WebService.METRIC_STORE_KEY, WebService.METRIC_STORE_TOPIC, WebService.BOOTSTRAP_SERVERS_KEY, - "ignore")))) { + "ignore")), + List.of())) { // topic collector may create local receiver to receive local jmx metric Assertions.assertEquals(1, topicReceiverCount.get()); @@ -165,7 +177,8 @@ void testMetricStoreConfiguration() { 0, id -> -1, Duration.ofMillis(5), - new Configuration(Map.of(WebService.METRIC_STORE_KEY, "unknown")))); + new Configuration(Map.of(WebService.METRIC_STORE_KEY, "unknown")), + List.of())); } } } diff --git a/docs/web_server/README.md b/docs/web_server/README.md index d311cc75f4..3ef7c2c547 100644 --- a/docs/web_server/README.md +++ b/docs/web_server/README.md @@ -25,4 +25,5 @@ Astraea 建立了一套 Web Server 服務,使用者可以透過簡易好上手 - [/reassignments](./web_api_reassignments_chinese.md) - [/records](./web_api_records_chinese.md) - [/balancer](./web_api_balancer_chinese.md) -- [/throttles](./web_api_throttles_chinese.md) \ No newline at end of file +- [/throttles](./web_api_throttles_chinese.md) +- [/backups](./web_api_backups_chinese.md) \ No newline at end of file diff --git a/docs/web_server/web_api_backups_chinese.md b/docs/web_server/web_api_backups_chinese.md new file mode 100644 index 0000000000..d42c66e04e --- /dev/null +++ b/docs/web_server/web_api_backups_chinese.md @@ -0,0 +1,219 @@ +/backups +=== + +此 api 提供建立、查詢、刪除及修改以下備份工具: +1. `exporter`:將資料備份至指定檔案系統。 +2. `importer`:從檔案系統中還原備份資料。 + + +- [建立 exporter](#建立-exporter) +- [建立 importer](#建立-importer) +- [查詢所有 exporters 及 importers](#查詢所有-exporters-及-importers) +- [查詢指定名稱的 exporter 或 importer](#查詢指定名稱的-exporter-或-importer) +- [更改已存在的 exporter 及 importer 參數](#更改已存在的-exporter-及-importer-參數) +- [刪除 exporter 或 importer](#刪除-exporter-或-importer) + +## 建立 exporter +```shell +POST /backups +``` + +#### exporter 參數 +| 參數名稱 | 說明 | 預設值 | +|:-------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------| +| name | (必填) exporter 名稱 | 無 | +| fsSchema | (必填) 決定儲存目標為何種檔案系統,例如: `local`, `ftp`等 | 無 | +| hostname | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之 `hostname` | 無 | +| port | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之 `port` | 無 | +| user | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之登入 `user` | 無 | +| password | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之登入 `password` | 無 | +| path | (必填) 填入目標檔案系統要儲存的資料夾目錄之目標位置 | 無 | +| topics | (必填) 填入目標 `topics` | 無 | +| tasksMax | (選填) 設定 task 數量上限 | 1 | +| size | (選填) 寫入檔案目標超過此設定之大小上限時會創建新檔案,並且寫入目標改為新創建之檔案。
    檔案大小單位: `Bit`, `Kb`, `KiB`, `Mb`, etc. | 100MB | +| rollDuration | (選填) 如果在超過此時間沒有任何資料流入,會把當下所有已創建之檔案關閉,並在之後有新資料時會創建新檔案並寫入。
    時間單位: `s`, `m`, `h`, `day`, etc. | 3s | +| offsetFrom | (選填)Map 格式:"`topic`.`partition`.offset.from": "`offset`"
    針對想要的 topic 或是 topicPartition 指定備份 offset 的起點,如果要針對整個 topic 指定需將 partition 留空。ex: {"topicName.offset.from": "101"} | 無 | + +cURL 範例 1 + +建立一個 exporter,將 `topic` : "chia01" 的資料從 `offset` : "101" 開始備份至 local 檔案系統的 `/backup` 目錄下。 +當檔案超過 500MB 創建新檔案,並會在 5m 內沒有資料流入時關閉當下檔案。 +```shell +curl -X POST http://localhost:8001/backups \ + -H "Content-Type: application/json" \ + -d '{"exporter":[{ + "name": "export_local", + "topics": "chia01", + "fsSchema": "local", + "tasksMax": "3", + "path": "/backup", + "size": "500MB", + "rollDuration": "5m", + "offsetFrom": { + "chia01.offset.from": "101" + }}]}' +``` + +cURL 範例 2 + +建立一個 exporter,將 +1. `topic` : "chia01" `partition` : "0" 的資料從 `offset` : "101" 開始 +2. `topic` : "chia01" `partition` : "1" 的資料從 `offset` : "201" 開始 + +備份至 local 檔案系統的 `/backup` 目錄下。 +```shell +curl -X POST http://localhost:8001/backups \ + -H "Content-Type: application/json" \ + -d '{"exporter":[{ + "name": "export_local", + "topics": "chia01", + "fsSchema": "local", + "tasksMax": "3", + "path": "/backup", + "offsetFrom": { + "chia01.0.offset.from": "101", + "chia01.1.offset.from": "201" + }}]}' +``` + +## 建立 importer +```shell +POST /backups +``` +#### importer 參數 + +| 參數名稱 | 說明 | 預設值 | +|:------------------|--------------------------------------------------------------------------------------------------------------|-----| +| name | (必填) importer 名稱 | 無 | +| fsSchema | (必填) 決定儲存目標為何種檔案系統,例如: `local`, `ftp`, `hdfs`等 | 無 | +| hostname | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之 `hostname` | 無 | +| port | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之 `port` | 無 | +| user | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之登入 `user` | 無 | +| password | (`fsSchema` 為 `ftp`, `hdfs`) 填入目標檔案系統之登入 `password` | 無 | +| path | (必填) 填入目標檔案系統要讀取的檔案目錄位置 | 無 | +| tasksMax | (選填) 設定 task 數量上限 | 1 | +| cleanSourcePolicy | (選填) 選擇已讀入之檔案的處理方式
    `off`:不做處理
    `delete`:將檔案移除
    `archive`: 將檔案移至`archive.dir`(須填入 `archive.dir` 參數) | off | +| archiveDir | (`cleanSourcePolicy` 為 `archive`) 填入封存已經處理好的檔案目錄位置 | 無 | + +cURL 範例 1 + +建立一個 importer,從 local 檔案系統 `/backup` 目錄下讀取已備份的檔案,在讀取後將檔案移動至 `/finished` 目錄中。 +```shell +curl -X POST http://localhost:8001/backups \ + -H "Content-Type: application/json" \ + -d '{"importer":[{ + "name": "import_local", + "fsSchema": "local", + "tasksMax": "3", + "path": "/backup", + "cleanSourcePolicy": "archive", + "archiveDir": "/finished" + }]}' +``` + +cURL 範例 2 + +建立一個 importer,從 ftp 檔案系統 `/backup` 目錄下讀取已備份的檔案,在讀取後將檔案移除。 +```shell +curl -X POST http://localhost:8001/backups \ + -H "Content-Type: application/json" \ + -d '{"importer":[{ + "name": "import_local", + "fsSchema": "ftp", + "hostname": "localhost", + "port": "21", + "user": "astraea", + "password": "astraea", + "tasksMax": "3", + "path": "/backup", + "cleanSourcePolicy": "delete" + }]}' +``` +## 查詢所有 exporters 及 importers +```shell +Get /backups +``` + +cURL 範例 + +```shell +curl -X GET http://localhost:8001/backups +``` + +JSON Response 範例 +- `name`: importer / exporter 名稱 +- `configs`: importer / exporter 參數 +- `state`: importer / exporter 狀態 +- `tasks`: 各個 task 狀態 + +```json +{ + "importers": [ + { + "name": "import_local", + "configs": { + "connector.class": "org.astraea.connector.backup.Importer", + "fs.schema": "local", + "name": "import_local", + "path": "/backup", + "tasks.max": "3", + "clean.source": "archive", + "archive.dir": "/finished", + "header.converter": "org.apache.kafka.connect.converters.ByteArrayConverter", + "key.converter": "org.apache.kafka.connect.converters.ByteArrayConverter", + "value.converter": "org.apache.kafka.connect.converters.ByteArrayConverter" + }, + "state": "RUNNING", + "tasks": { + "0": "RUNNING", + "1": "RUNNING", + "2": "RUNNING" + } + } + ] +} +``` + +## 查詢指定名稱的 exporter 或 importer +```shell +Get /backups/{name} +``` + +cURL 範例 + +```shell +curl -X GET http://localhost:8001/backups/import_local +``` + +## 更改已存在的 exporter 及 importer 參數 +```shell +PUT /backups +``` + +參數設置方式與[建立時的參數](#importer-參數)相同 + +cURL 範例 + +更改一個存在的 exporter,將 "chia02" `topic` 的資料備份至 local 檔案系統的 `/newBackup` 目錄下。 +```shell +curl -X PUT http://localhost:8001/backups \ + -H "Content-Type: application/json" \ + -d '{"exporter":[{ + "name": "export_local", + "topics": "chia02", + "fsSchema": "local", + "tasksMax": "3", + "path": "/newBackup" + }]}' +``` + +## 刪除 exporter 或 importer +```shell +DELETE /backups/{name} +``` + +cURL 範例 + +```shell +curl -X DELETE http://localhost:8001/backups/export_local +``` \ No newline at end of file From 37412c65a02eb5aad9eff5de3b0e7c40bdb42c00 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 30 Jun 2023 18:32:55 +0800 Subject: [PATCH 69/77] [DOCKER] fix the broken GRADLE url (#1829) --- docker/build_deps_image.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/build_deps_image.sh b/docker/build_deps_image.sh index 915fce81aa..3c9f1b829c 100755 --- a/docker/build_deps_image.sh +++ b/docker/build_deps_image.sh @@ -35,7 +35,7 @@ RUN apt-get update && apt-get install -y \ # download gradle 5 for previous kafka having no built-in gradlew WORKDIR /tmp -RUN wget https://downloads.gradle-dn.com/distributions/gradle-5.6.4-bin.zip +RUN wget https://downloads.gradle.org/distributions/gradle-5.6.4-bin.zip RUN unzip gradle-5.6.4-bin.zip # build code and download dependencies From 55c83d0755c11d71923b27aa2fc7d02d05cb925e Mon Sep 17 00:00:00 2001 From: Haser Date: Sun, 9 Jul 2023 13:48:21 +0800 Subject: [PATCH 70/77] [EXPORTER] Add support for gzip compression (#1826) * add support for gzip compression * add test for checking the file is compressed in gzip format. * fix test failed caused by not setting the compression type. * make the field compressionType in Task not case-sensitive --- .../common/backup/RecordWriterBuilder.java | 14 +++- .../astraea/connector/backup/Exporter.java | 19 +++++- .../connector/backup/ExporterTest.java | 65 +++++++++++++++++++ 3 files changed, 94 insertions(+), 4 deletions(-) diff --git a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java index c084e7d1c2..7e5070c0e9 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java @@ -18,7 +18,6 @@ import com.google.protobuf.ByteString; import java.io.BufferedOutputStream; -import java.io.IOException; import java.io.OutputStream; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -119,8 +118,17 @@ public void close() { this.fs = outputStream; } - public RecordWriterBuilder compression() throws IOException { - this.fs = new GZIPOutputStream(this.fs); + public RecordWriterBuilder compression(String type) { + switch (type) { + case "gzip": + this.fs = Utils.packException(() -> new GZIPOutputStream(this.fs)); + break; + case "none": + // do nothing. + break; + default: + throw new IllegalArgumentException("unsupported compression type: " + type); + } return this; } diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 28980e2c72..855320ec1a 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -134,6 +134,15 @@ public class Exporter extends SinkConnector { + "backups of a particular topic or topic partition. it can be used in 2 ways: " + "'.offset.from' or '..offset.from'.") .build(); + + static String COMPRESSION_TYPE_DEFAULT = "none"; + static Definition COMPRESSION_TYPE_KEY = + Definition.builder() + .name("compression.type") + .type(Definition.Type.STRING) + .documentation("a value that can specify the compression type.") + .defaultValue(COMPRESSION_TYPE_DEFAULT) + .build(); private Configuration configs; @Override @@ -162,7 +171,8 @@ protected List definitions() { PATH_KEY, SIZE_KEY, OVERRIDE_KEY, - BUFFER_SIZE_KEY); + BUFFER_SIZE_KEY, + COMPRESSION_TYPE_KEY); } public static class Task extends SinkTask { @@ -183,6 +193,7 @@ public static class Task extends SinkTask { String path; DataSize size; long interval; + String compressionType; // a map of > private final Map> offsetForTopicPartition = new HashMap<>(); @@ -199,6 +210,7 @@ RecordWriter createRecordWriter(TopicPartition tp, long offset) { return RecordWriter.builder( fs.write( String.join("/", path, tp.topic(), String.valueOf(tp.partition()), fileName))) + .compression(this.compressionType) .build(); } @@ -299,6 +311,11 @@ protected void init(Configuration configuration, SinkTaskContext context) { .orElse(BUFFER_SIZE_DEFAULT) .bytes(); this.taskContext = context; + this.compressionType = + configuration + .string(COMPRESSION_TYPE_KEY.name()) + .orElse(COMPRESSION_TYPE_DEFAULT) + .toLowerCase(); // fetches key-value pairs from the configuration's variable matching the regular expression // '.*offset.from', updates the values of 'offsetForTopic' or 'offsetForTopicPartition' based diff --git a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java index 6a27990054..28e8d14de3 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java @@ -777,6 +777,7 @@ void testCreateRecordWriter() { var task = new Exporter.Task(); task.fs = FileSystem.of("hdfs", new Configuration(configs)); task.interval = 1000; + task.compressionType = "none"; RecordWriter recordWriter = task.createRecordWriter(tp, offset); @@ -841,6 +842,7 @@ void testWriteRecords() { var task = new Exporter.Task(); task.fs = FileSystem.of("hdfs", new Configuration(configs)); + task.compressionType = "none"; task.size = DataSize.of("100MB"); task.bufferSize.reset(); task.recordsQueue.add( @@ -950,4 +952,67 @@ void testIsValid() { Assertions.assertEquals(1, task.seekOffset.size()); } } + + @Test + void testCompression() { + try (var server = HdfsServer.local()) { + var fileSize = "500Byte"; + var topicName = Utils.randomString(10); + + var task = new Exporter.Task(); + var configs = + Map.of( + "fs.schema", + "hdfs", + "topics", + topicName, + "fs.hdfs.hostname", + String.valueOf(server.hostname()), + "fs.hdfs.port", + String.valueOf(server.port()), + "fs.hdfs.user", + String.valueOf(server.user()), + "path", + "/" + fileSize, + "size", + fileSize, + "fs.hdfs.override.dfs.client.use.datanode.hostname", + "true", + "compression.type", + "gzip"); + + task.start(configs); + + var records = + List.of( + Record.builder() + .topic(topicName) + .key("test".getBytes()) + .value("test value".getBytes()) + .partition(0) + .offset(0) + .timestamp(System.currentTimeMillis()) + .build(), + Record.builder() + .topic(topicName) + .key("test".getBytes()) + .value("test value".getBytes()) + .partition(0) + .offset(1) + .timestamp(System.currentTimeMillis()) + .build()); + + task.put(records); + + Utils.sleep(Duration.ofMillis(1000)); + + task.close(); + var fs = FileSystem.of("hdfs", new Configuration(configs)); + + var input = fs.read("/" + String.join("/", fileSize, topicName, "0/0")); + + Assertions.assertArrayEquals( + new byte[] {(byte) 0x1f, (byte) 0x8b}, Utils.packException(() -> input.readNBytes(2))); + } + } } From d647b4ccfc46d9a5f9dd3dd067b06389179e7326 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 13 Jul 2023 22:10:36 +0800 Subject: [PATCH 71/77] [EXPORTER] fix typo: targeOffset -> targetOffset (#1831) --- .../java/org/astraea/connector/backup/Exporter.java | 4 ++-- .../org/astraea/connector/backup/ExporterTest.java | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 855320ec1a..2080d8dc29 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -373,7 +373,7 @@ protected void put(List> records) { } protected boolean isValid(Record r) { - var targetOffset = targeOffset(r); + var targetOffset = targetOffset(r); // If the target offset exists and the record's offset is less than the target offset, // set the seek offset to the target offset and return false. @@ -394,7 +394,7 @@ protected boolean isValid(Record r) { * is not found. */ // visible for test - protected Optional targeOffset(Record r) { + protected Optional targetOffset(Record r) { var topicMap = this.offsetForTopicPartition.get(r.topic()); // If we are unable to obtain the target offset from the 'offsetForTopicPartition' map, diff --git a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java index 28e8d14de3..6f03beba1f 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java @@ -937,11 +937,11 @@ void testIsValid() { .timestamp(System.currentTimeMillis()) .build(); - Assertions.assertFalse(task.targeOffset(record1).isPresent()); - Assertions.assertEquals(100, task.targeOffset(record2).orElse(null)); - Assertions.assertEquals(200, task.targeOffset(record3).orElse(null)); - Assertions.assertEquals(200, task.targeOffset(record4).orElse(null)); - Assertions.assertFalse(task.targeOffset(record5).isPresent()); + Assertions.assertFalse(task.targetOffset(record1).isPresent()); + Assertions.assertEquals(100, task.targetOffset(record2).orElse(null)); + Assertions.assertEquals(200, task.targetOffset(record3).orElse(null)); + Assertions.assertEquals(200, task.targetOffset(record4).orElse(null)); + Assertions.assertFalse(task.targetOffset(record5).isPresent()); Assertions.assertTrue(task.isValid(record1)); Assertions.assertFalse(task.isValid(record2)); From 27e6a87e0fb49d13cef876bfacd903b0b71d58b3 Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Fri, 21 Jul 2023 10:39:30 +0800 Subject: [PATCH 72/77] [METRICS] Not block on BeanObject receiving (#1833) --- .../org/astraea/common/metrics/collector/MetricStore.java | 7 ++++++- .../common/partitioner/StrictCostPartitionerTest.java | 3 ++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java index a3d9f79ce2..616109f693 100644 --- a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java +++ b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java @@ -261,7 +261,12 @@ private MetricStoreImpl( while (!closed.get()) { try { receivers.stream() - .map(r -> r.receive(Duration.ofSeconds(3))) + // TODO: Busy waiting on metric receiving. + // issue: https://github.com/skiptests/astraea/issues/1834 + // To prevent specific receiver block other receivers' job, we set receive + // timeout to zero. But if all receivers return empty immediately, it may cause + // this thread busy waiting on doing `receiver.receive`. + .map(r -> r.receive(Duration.ZERO)) .forEach( allBeans -> { beanReceivedSensor.record( diff --git a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java index 1bf8443317..3535206515 100644 --- a/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java +++ b/common/src/test/java/org/astraea/common/partitioner/StrictCostPartitionerTest.java @@ -149,7 +149,8 @@ void testCostFunctionWithoutSensor() { new byte[0], new byte[0], ClusterInfoTest.of(List.of(replicaInfo0, replicaInfo1))); - Assertions.assertEquals(0, partitioner.metricStore.sensors().size()); + Utils.sleep(Duration.ofSeconds(1)); + Assertions.assertEquals(1, partitioner.metricStore.sensors().size()); } } From e48ef42bb673eac12a7332d44d172531683da752 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sat, 22 Jul 2023 19:40:40 +0800 Subject: [PATCH 73/77] [COMMON] upgrade gradle from 8.1.1 to 8.2.1 (#1835) --- gradle/wrapper/gradle-wrapper.properties | 2 +- gradlew | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 4ac1109a16..a014da1379 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists # please update the gradle version in gradlew too -distributionUrl=https\://services.gradle.org/distributions/gradle-8.1.1-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.2.1-bin.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew index 1f6cf91961..39acf636e6 100755 --- a/gradlew +++ b/gradlew @@ -116,7 +116,7 @@ esac # gradle-wrapper.jar is not in git repo anymore, so we have to download it now. if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then - if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v8.1.1/gradle/wrapper/gradle-wrapper.jar"; then + if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v8.2.1/gradle/wrapper/gradle-wrapper.jar"; then rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" echo "failed to download gradle-wrapper.jar" exit 1 From 4140b68ca6950206d65c6cc71be0203f1fa1ce74 Mon Sep 17 00:00:00 2001 From: Haser Date: Wed, 26 Jul 2023 22:19:34 +0800 Subject: [PATCH 74/77] [EXPORTER] Add metadata in record writer (#1832) * recordWriter will append metadata before closing. During the creation of a new recordWriter, the exporter task will transmit both the configuration and the first record for writing metadata. A new method has been added to the SinkContext for retrieving the connector name to be used within the metadata. * a new version of recordWriter v1, which contains metadata at the end of a file. The reason for creating a new version instead of replacing version 0 is because the importer is not yet able to handle files with metadata information. * Fix RecordReader to not error out when receiving V1 files While it can accept V1 files, it was only processing the initial data and not the following metadata * Remove topic/partition/1st record info metadata Removing these data from metadata to give writers more flexibility. This allows writers to write data without restrictions, allowing the exporter to decide how they want to handle the data. --- .../common/backup/RecordReaderBuilder.java | 7 +- .../astraea/common/backup/RecordWriter.java | 7 +- .../common/backup/RecordWriterBuilder.java | 182 ++++++++++++++---- .../astraea/connector/SinkTaskContext.java | 13 ++ .../astraea/connector/backup/Exporter.java | 33 +++- .../connector/backup/ExporterTest.java | 49 ++++- .../org/astraea/fs/ftp/FtpFileSystem.java | 5 + 7 files changed, 237 insertions(+), 59 deletions(-) diff --git a/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java index d6f154bafd..7c2c63df5d 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordReaderBuilder.java @@ -16,6 +16,7 @@ */ package org.astraea.common.backup; +import com.google.protobuf.InvalidProtocolBufferException; import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; @@ -77,8 +78,10 @@ private static Record readRecord(InputStream inputStream) { .serializedValueSize(outerRecord.getValue().size()) .build(); } catch (IOException e) { - throw new SerializationException(e); + // swallow the exception until the importer can read metadata at the end of the file. + if (!(e instanceof InvalidProtocolBufferException)) throw new SerializationException(e); } + return null; } private InputStream fs; @@ -104,7 +107,7 @@ public RecordReaderBuilder buffered(int size) { public RecordReader build() { var version = ByteUtils.readShort(fs); - if (version == 0) return V0.apply(fs); + if (version == 0 || version == 1) return V0.apply(fs); throw new IllegalArgumentException("unsupported version: " + version); } diff --git a/common/src/main/java/org/astraea/common/backup/RecordWriter.java b/common/src/main/java/org/astraea/common/backup/RecordWriter.java index ea8597bd14..ee56c11b37 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordWriter.java +++ b/common/src/main/java/org/astraea/common/backup/RecordWriter.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.FileOutputStream; import java.io.OutputStream; +import org.astraea.common.Configuration; import org.astraea.common.DataSize; import org.astraea.common.Utils; import org.astraea.common.consumer.Record; @@ -52,6 +53,10 @@ static RecordWriterBuilder builder(File file) { } static RecordWriterBuilder builder(OutputStream outputStream) { - return new RecordWriterBuilder(RecordWriterBuilder.LATEST_VERSION, outputStream); + return new RecordWriterBuilder((short) 0, outputStream); + } + + static RecordWriterBuilder builder(OutputStream outputStream, Configuration configuration) { + return new RecordWriterBuilder(RecordWriterBuilder.LATEST_VERSION, outputStream, configuration); } } diff --git a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java index 7e5070c0e9..6401d6f52f 100644 --- a/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java +++ b/common/src/main/java/org/astraea/common/backup/RecordWriterBuilder.java @@ -19,12 +19,15 @@ import com.google.protobuf.ByteString; import java.io.BufferedOutputStream; import java.io.OutputStream; +import java.time.Duration; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.zip.GZIPOutputStream; import org.astraea.common.ByteUtils; +import org.astraea.common.Configuration; import org.astraea.common.DataSize; import org.astraea.common.Utils; import org.astraea.common.consumer.Record; @@ -32,6 +35,32 @@ public class RecordWriterBuilder { + static RecordOuterClass.Record record2Builder(Record record) { + return Utils.packException( + () -> + RecordOuterClass.Record.newBuilder() + .setTopic(record.topic()) + .setPartition(record.partition()) + .setOffset(record.offset()) + .setTimestamp(record.timestamp()) + .setKey(record.key() == null ? ByteString.EMPTY : ByteString.copyFrom(record.key())) + .setValue( + record.value() == null ? ByteString.EMPTY : ByteString.copyFrom(record.value())) + .addAllHeaders( + record.headers().stream() + .map( + header -> + RecordOuterClass.Record.Header.newBuilder() + .setKey(header.key()) + .setValue( + header.value() == null + ? ByteString.EMPTY + : ByteString.copyFrom(header.value())) + .build()) + .toList()) + .build()); + } + private static final Function V0 = outputStream -> new RecordWriter() { @@ -44,33 +73,7 @@ public class RecordWriterBuilder { public void append(Record record) { Utils.packException( () -> { - var recordBuilder = - RecordOuterClass.Record.newBuilder() - .setTopic(record.topic()) - .setPartition(record.partition()) - .setOffset(record.offset()) - .setTimestamp(record.timestamp()) - .setKey( - record.key() == null - ? ByteString.EMPTY - : ByteString.copyFrom(record.key())) - .setValue( - record.value() == null - ? ByteString.EMPTY - : ByteString.copyFrom(record.value())) - .addAllHeaders( - record.headers().stream() - .map( - header -> - RecordOuterClass.Record.Header.newBuilder() - .setKey(header.key()) - .setValue( - header.value() == null - ? ByteString.EMPTY - : ByteString.copyFrom(header.value())) - .build()) - .toList()) - .build(); + var recordBuilder = record2Builder(record); recordBuilder.writeDelimitedTo(outputStream); this.size.add(recordBuilder.getSerializedSize()); }); @@ -108,28 +111,124 @@ public void close() { } }; - public static final short LATEST_VERSION = (short) 0; + private static final BiFunction V1 = + (configuration, outputStream) -> + new RecordWriter() { + private final AtomicInteger count = new AtomicInteger(); + private final LongAdder size = new LongAdder(); + private final AtomicLong latestAppendTimestamp = new AtomicLong(); + private final String connectorName; + private final Long interval; + private final String compressionType; + private OutputStream targetOutputStream; + + // instance initializer block + { + this.connectorName = configuration.requireString("connector.name"); + this.interval = + configuration + .string("roll.duration") + .map(Utils::toDuration) + .orElse(Duration.ofSeconds(3)) + .toMillis(); + this.compressionType = configuration.string("compression.type").orElse("none"); + + switch (this.compressionType) { + case "gzip" -> Utils.packException( + () -> this.targetOutputStream = new GZIPOutputStream(outputStream)); + case "none" -> this.targetOutputStream = outputStream; + default -> throw new IllegalArgumentException( + String.format("compression type '%s' is not supported", this.compressionType)); + } + } + + byte[] extendString(String input, int length) { + byte[] original = input.getBytes(); + byte[] result = new byte[length]; + System.arraycopy(original, 0, result, 0, original.length); + for (int i = original.length; i < result.length; i++) { + result[i] = (byte) ' '; + } + return result; + } + + void appendMetadata() { + Utils.packException( + () -> { + if (this.compressionType.equals("gzip")) { + ((GZIPOutputStream) targetOutputStream).finish(); + } + + // 552 Bytes total for whole metadata. + outputStream.write( + this.extendString(this.connectorName, 255)); // 255 Bytes for this connector + outputStream.write(ByteUtils.toBytes(this.count())); // 4 Bytes for count + outputStream.write( + ByteUtils.toBytes(this.interval)); // 8 Bytes for mills of roll.duration + outputStream.write( + this.extendString( + this.compressionType, 10)); // 10 Bytes for compression type name. + }); + } + + @Override + public void append(Record record) { + Utils.packException( + () -> { + var recordBuilder = record2Builder(record); + recordBuilder.writeDelimitedTo(this.targetOutputStream); + this.size.add(recordBuilder.getSerializedSize()); + }); + count.incrementAndGet(); + this.latestAppendTimestamp.set(System.currentTimeMillis()); + } + + @Override + public DataSize size() { + return DataSize.Byte.of(size.sum()); + } + + @Override + public int count() { + return count.get(); + } + + @Override + public void flush() { + Utils.packException(outputStream::flush); + } + + @Override + public long latestAppendTimestamp() { + return this.latestAppendTimestamp.get(); + } + + @Override + public void close() { + Utils.packException( + () -> { + appendMetadata(); + outputStream.flush(); + outputStream.close(); + }); + } + }; + + public static final short LATEST_VERSION = (short) 1; private final short version; private OutputStream fs; + private Configuration configuration; RecordWriterBuilder(short version, OutputStream outputStream) { this.version = version; this.fs = outputStream; } - public RecordWriterBuilder compression(String type) { - switch (type) { - case "gzip": - this.fs = Utils.packException(() -> new GZIPOutputStream(this.fs)); - break; - case "none": - // do nothing. - break; - default: - throw new IllegalArgumentException("unsupported compression type: " + type); - } - return this; + RecordWriterBuilder(short version, OutputStream outputStream, Configuration configuration) { + this.version = version; + this.fs = outputStream; + this.configuration = configuration; } public RecordWriterBuilder buffered() { @@ -148,6 +247,9 @@ public RecordWriter build() { if (version == 0) { fs.write(ByteUtils.toBytes(version)); return V0.apply(fs); + } else if (version == 1) { + fs.write(ByteUtils.toBytes(version)); + return V1.apply(this.configuration, fs); } throw new IllegalArgumentException("unsupported version: " + version); }); diff --git a/connector/src/main/java/org/astraea/connector/SinkTaskContext.java b/connector/src/main/java/org/astraea/connector/SinkTaskContext.java index fc2c069c57..363ad3c66b 100644 --- a/connector/src/main/java/org/astraea/connector/SinkTaskContext.java +++ b/connector/src/main/java/org/astraea/connector/SinkTaskContext.java @@ -37,6 +37,11 @@ public void pause(Collection partitions) {} @Override public void requestCommit() {} + + @Override + public Map configs() { + return null; + } }; /** @@ -67,6 +72,8 @@ public void requestCommit() {} */ void requestCommit(); + Map configs(); + static SinkTaskContext of(org.apache.kafka.connect.sink.SinkTaskContext context) { return new SinkTaskContext() { @Override @@ -94,6 +101,12 @@ public void pause(Collection partitions) { public void requestCommit() { context.requestCommit(); } + + @Override + public Map configs() { + return context.configs(); + } + ; }; } } diff --git a/connector/src/main/java/org/astraea/connector/backup/Exporter.java b/connector/src/main/java/org/astraea/connector/backup/Exporter.java index 2080d8dc29..d6258aecfd 100644 --- a/connector/src/main/java/org/astraea/connector/backup/Exporter.java +++ b/connector/src/main/java/org/astraea/connector/backup/Exporter.java @@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.LongAdder; +import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.astraea.common.Configuration; @@ -195,6 +196,8 @@ public static class Task extends SinkTask { long interval; String compressionType; + Configuration configuration; + // a map of > private final Map> offsetForTopicPartition = new HashMap<>(); @@ -205,12 +208,24 @@ public static class Task extends SinkTask { private SinkTaskContext taskContext; - RecordWriter createRecordWriter(TopicPartition tp, long offset) { - var fileName = String.valueOf(offset); + // create for test + RecordWriter createRecordWriter(Record record, Configuration configuration) { + var fileName = String.valueOf(record.offset()); return RecordWriter.builder( fs.write( - String.join("/", path, tp.topic(), String.valueOf(tp.partition()), fileName))) - .compression(this.compressionType) + String.join( + "/", path, record.topic(), String.valueOf(record.partition()), fileName)), + configuration) + .build(); + } + + RecordWriter createRecordWriter(Record record) { + var fileName = String.valueOf(record.offset()); + return RecordWriter.builder( + fs.write( + String.join( + "/", path, record.topic(), String.valueOf(record.partition()), fileName)), + this.configuration) .build(); } @@ -249,8 +264,7 @@ void writeRecords(HashMap writers) { records.forEach( record -> { var writer = - writers.computeIfAbsent( - record.topicPartition(), tp -> createRecordWriter(tp, record.offset())); + writers.computeIfAbsent(record.topicPartition(), tp -> createRecordWriter(record)); writer.append(record); if (writer.size().greaterThan(size)) { writers.remove(record.topicPartition()).close(); @@ -317,6 +331,13 @@ protected void init(Configuration configuration, SinkTaskContext context) { .orElse(COMPRESSION_TYPE_DEFAULT) .toLowerCase(); + var originalConfiguration = + configuration.raw().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + originalConfiguration.computeIfAbsent( + "connector.name", k -> this.taskContext.configs().get("name")); + this.configuration = new Configuration(originalConfiguration); + // fetches key-value pairs from the configuration's variable matching the regular expression // '.*offset.from', updates the values of 'offsetForTopic' or 'offsetForTopicPartition' based // on the diff --git a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java index 6f03beba1f..5f1fd179ab 100644 --- a/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java +++ b/connector/src/test/java/org/astraea/connector/backup/ExporterTest.java @@ -20,6 +20,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.time.Duration; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,6 +40,7 @@ import org.astraea.common.connector.Value; import org.astraea.common.consumer.Record; import org.astraea.common.producer.Producer; +import org.astraea.connector.SinkTaskContext; import org.astraea.fs.FileSystem; import org.astraea.it.FtpServer; import org.astraea.it.HdfsServer; @@ -52,6 +54,26 @@ public class ExporterTest { private static final Service SERVICE = Service.builder().numberOfWorkers(1).numberOfBrokers(1).build(); + private static final SinkTaskContext context = + new SinkTaskContext() { + @Override + public void offset(Map offsets) {} + + @Override + public void offset(TopicPartition topicPartition, long offset) {} + + @Override + public void pause(Collection partitions) {} + + @Override + public void requestCommit() {} + + @Override + public Map configs() { + return Map.of("name", "test"); + } + }; + @AfterAll static void closeService() { SERVICE.close(); @@ -191,7 +213,7 @@ void testFtpSinkTask() { var fs = FileSystem.of("ftp", new Configuration(configs)); - task.start(configs); + task.init(new Configuration(configs), context); var records = List.of( @@ -278,7 +300,7 @@ void testFtpSinkTaskIntervalWith1File() { var fs = FileSystem.of("ftp", new Configuration(configs)); - task.start(configs); + task.init(new Configuration(configs), context); var records1 = Record.builder() @@ -353,7 +375,7 @@ void testFtpSinkTaskIntervalWith2Writers() { var fs = FileSystem.of("ftp", new Configuration(configs)); - task.start(configs); + task.init(new Configuration(configs), context); var record1 = Record.builder() @@ -458,7 +480,7 @@ void testHdfsSinkTask() { "fs.hdfs.override.dfs.client.use.datanode.hostname", "true"); - task.start(configs); + task.init(new Configuration(configs), context); var records = List.of( @@ -557,7 +579,7 @@ void testHdfsSinkTaskIntervalWith1File() { "roll.duration", "300ms"); - task.start(configs); + task.init(new Configuration(configs), context); var records1 = Record.builder() @@ -630,7 +652,7 @@ void testHdfsSinkTaskIntervalWith2Writers() { "roll.duration", "100ms"); - task.start(configs); + task.init(new Configuration(configs), context); var record1 = Record.builder() @@ -751,8 +773,11 @@ void testCreateRecordWriter() { var topicName = Utils.randomString(10); var tp = TopicPartition.of(topicName, 0); long offset = 123; + var testRecord = Record.builder().topic(topicName).topicPartition(tp).offset(offset).build(); var configs = Map.of( + "connector.name", + "test", "fs.schema", "hdfs", "topics", @@ -779,7 +804,7 @@ void testCreateRecordWriter() { task.interval = 1000; task.compressionType = "none"; - RecordWriter recordWriter = task.createRecordWriter(tp, offset); + RecordWriter recordWriter = task.createRecordWriter(testRecord, new Configuration(configs)); Assertions.assertNotNull(recordWriter); @@ -819,6 +844,8 @@ void testWriteRecords() { var path = "/test"; var configs = Map.of( + "connector.name", + "test", "fs.schema", "hdfs", "topics", @@ -843,6 +870,7 @@ void testWriteRecords() { var task = new Exporter.Task(); task.fs = FileSystem.of("hdfs", new Configuration(configs)); task.compressionType = "none"; + task.configuration = new Configuration(configs); task.size = DataSize.of("100MB"); task.bufferSize.reset(); task.recordsQueue.add( @@ -889,7 +917,7 @@ void testIsValid() { var task = new Exporter.Task(); - task.start(configs); + task.init(new Configuration(configs), context); var record1 = Record.builder() @@ -981,7 +1009,7 @@ void testCompression() { "compression.type", "gzip"); - task.start(configs); + task.init(new Configuration(configs), context); var records = List.of( @@ -1012,7 +1040,8 @@ void testCompression() { var input = fs.read("/" + String.join("/", fileSize, topicName, "0/0")); Assertions.assertArrayEquals( - new byte[] {(byte) 0x1f, (byte) 0x8b}, Utils.packException(() -> input.readNBytes(2))); + new byte[] {(byte) 0x0, (byte) 0x1, (byte) 0x1f, (byte) 0x8b}, + Utils.packException(() -> input.readNBytes(4))); } } } diff --git a/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java b/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java index e27e4bec67..ce172c1476 100644 --- a/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java +++ b/fs/src/main/java/org/astraea/fs/ftp/FtpFileSystem.java @@ -207,6 +207,11 @@ public void write(int b) throws IOException { outputStream.write(b); } + @Override + public void write(byte b[]) throws IOException { + outputStream.write(b); + } + @Override public void write(byte b[], int off, int len) throws IOException { outputStream.write(b, off, len); From d882fb7bdccb6c072d6442c6a8188da65400b60f Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 9 Aug 2023 00:18:52 +0800 Subject: [PATCH 75/77] [COMMON] upgrade kafka from 3.5.0 to 3.5.1 (#1838) --- docker/start_broker.sh | 4 ++-- docker/start_controller.sh | 4 ++-- docker/start_worker.sh | 4 ++-- gradle/dependencies.gradle | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/start_broker.sh b/docker/start_broker.sh index 810179922d..60bf7eaac4 100755 --- a/docker/start_broker.sh +++ b/docker/start_broker.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.5.0}} +declare -r VERSION=${REVISION:-${VERSION:-3.5.1}} declare -r DOCKERFILE=$DOCKER_FOLDER/broker.dockerfile declare -r DATA_FOLDER_IN_CONTAINER_PREFIX="/tmp/log-folder" declare -r EXPORTER_VERSION="0.16.1" @@ -61,7 +61,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set broker JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.5.0 set version of kafka distribution" + echo " VERSION=3.5.1 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " DATA_FOLDERS=/tmp/folder1 set host folders used by broker" diff --git a/docker/start_controller.sh b/docker/start_controller.sh index 3101cdb2be..22305d8a15 100755 --- a/docker/start_controller.sh +++ b/docker/start_controller.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.5.0}} +declare -r VERSION=${REVISION:-${VERSION:-3.5.1}} declare -r DOCKERFILE=$DOCKER_FOLDER/controller.dockerfile declare -r EXPORTER_VERSION="0.16.1" declare -r CLUSTER_ID=${CLUSTER_ID:-"$(randomString)"} @@ -52,7 +52,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set controller JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.5.0 set version of kafka distribution" + echo " VERSION=3.5.1 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " META_FOLDER=/tmp/folder1 set host folder used by controller" diff --git a/docker/start_worker.sh b/docker/start_worker.sh index 1fc2b88434..395b90d74b 100755 --- a/docker/start_worker.sh +++ b/docker/start_worker.sh @@ -20,7 +20,7 @@ source $DOCKER_FOLDER/docker_build_common.sh # ===============================[global variables]=============================== declare -r ACCOUNT=${ACCOUNT:-skiptests} declare -r KAFKA_ACCOUNT=${KAFKA_ACCOUNT:-apache} -declare -r VERSION=${REVISION:-${VERSION:-3.5.0}} +declare -r VERSION=${REVISION:-${VERSION:-3.5.1}} declare -r DOCKERFILE=$DOCKER_FOLDER/worker.dockerfile declare -r WORKER_PORT=${WORKER_PORT:-"$(getRandomPort)"} declare -r CONTAINER_NAME="worker-$WORKER_PORT" @@ -50,7 +50,7 @@ function showHelp() { echo " ACCOUNT=skiptests set the github account for astraea repo" echo " HEAP_OPTS=\"-Xmx2G -Xms2G\" set worker JVM memory" echo " REVISION=trunk set revision of kafka source code to build container" - echo " VERSION=3.5.0 set version of kafka distribution" + echo " VERSION=3.5.1 set version of kafka distribution" echo " BUILD=false set true if you want to build image locally" echo " RUN=false set false if you want to build/pull image only" echo " WORKER_PLUGIN_PATH=/tmp/worker-plugins set plugin path to kafka worker" diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 3a53a93efc..53db1cb544 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -30,7 +30,7 @@ def versions = [ "jmh-core" : project.properties['jmh-core.version'] ?: "1.36", "jmh-generator-annprocess": project.properties['jmh-generator.version'] ?: "1.36", junit : project.properties['junit.version'] ?: "5.9.3", - kafka : project.properties['kafka.version'] ?: "3.5.0", + kafka : project.properties['kafka.version'] ?: "3.5.1", mockito : project.properties['mockito.version'] ?: "5.4.0", "mockito-inline" : project.properties['mockito.version'] ?: "5.2.0", scala : project.properties['scala.version'] ?: "2.13.11", From 53aed8776496f0188c0ae941e04e63abfd79597f Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Tue, 15 Aug 2023 15:35:41 +0800 Subject: [PATCH 76/77] [BALANCER] Expose number of plans generated (#1840) --- .../org/astraea/common/balancer/algorithms/GreedyBalancer.java | 3 +++ .../astraea/common/balancer/algorithms/GreedyBalancerTest.java | 1 + 2 files changed, 4 insertions(+) diff --git a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java index 3516b36a25..98c33e3ca3 100644 --- a/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java +++ b/common/src/main/java/org/astraea/common/balancer/algorithms/GreedyBalancer.java @@ -194,6 +194,7 @@ public Optional offer(AlgorithmConfig config) { final var loop = new AtomicInteger(iteration); final var start = System.currentTimeMillis(); final var executionTime = config.timeout().toMillis(); + final var plans = new LongAdder(); Supplier moreRoom = () -> System.currentTimeMillis() - start < executionTime && loop.getAndDecrement() > 0; BiFunction> next = @@ -201,6 +202,7 @@ public Optional offer(AlgorithmConfig config) { allocationTweaker .generate(currentAllocation) .takeWhile(ignored -> moreRoom.get()) + .peek(ignore -> plans.increment()) .filter( newAllocation -> !moveCostFunction @@ -231,6 +233,7 @@ public Optional offer(AlgorithmConfig config) { .property("run", Integer.toString(run.getAndIncrement())) .attribute("Iteration", Long.class, currentIteration::sum) .attribute("MinCost", Double.class, currentMinCost::get) + .attribute("Plans", Long.class, plans::longValue) .register(); while (true) { diff --git a/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java b/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java index fe28ddde1c..a2688d681e 100644 --- a/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java +++ b/common/src/test/java/org/astraea/common/balancer/algorithms/GreedyBalancerTest.java @@ -93,6 +93,7 @@ void testJmx() { Assertions.assertEquals("astraea.balancer", bean.domainName()); Assertions.assertTrue(0 < (long) bean.attributes().get("Iteration")); Assertions.assertTrue(1.0 > (double) bean.attributes().get("MinCost")); + Assertions.assertTrue(0 < (long) bean.attributes().get("Plans")); }); } } From d4c3d4bf64f83da063c03c04e95518b03128be94 Mon Sep 17 00:00:00 2001 From: Ching-Hong Fang Date: Thu, 17 Aug 2023 16:43:48 +0800 Subject: [PATCH 77/77] [METRICS] New thread for waiting list checking (#1836) --- .../common/metrics/collector/MetricStore.java | 61 +++++++++---------- 1 file changed, 28 insertions(+), 33 deletions(-) diff --git a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java index 616109f693..c323a6d1a9 100644 --- a/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java +++ b/common/src/main/java/org/astraea/common/metrics/collector/MetricStore.java @@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -222,8 +221,8 @@ class MetricStoreImpl implements MetricStore { private final Set identities = new ConcurrentSkipListSet<>(); private volatile Map> lastSensors = Map.of(); - private final Map> waitingList = - new ConcurrentHashMap<>(); + // Monitor for detecting cluster bean changing. + private final Object beanUpdateMonitor = new Object(); // For mbean register. To distinguish mbeans of different metricStore. private final String uid = Utils.randomString(); private final Sensor beanReceivedSensor = @@ -292,7 +291,10 @@ private MetricStoreImpl( if (!allBeans.isEmpty()) { // generate new cluster bean updateClusterBean(); - checkWaitingList(this.waitingList, clusterBean()); + // Tell waiting threads that cluster bean has been changed + synchronized (beanUpdateMonitor) { + beanUpdateMonitor.notifyAll(); + } } }); } catch (Exception e) { @@ -349,23 +351,31 @@ public void close() { receivers.forEach(Receiver::close); } - /** User thread will "wait" until being awakened by the metric store or being timeout. */ + /** + * User thread will wait until checker pass or timeout. When cluster bean has changed, the + * waiting threads will be notified. + */ @Override - public void wait(Predicate checker, Duration timeout) { - var latch = new CountDownLatch(1); - try { - waitingList.put(latch, checker); - // Check the newly added checker immediately - checkWaitingList(Map.of(latch, checker), clusterBean()); - // Wait until being awake or timeout - if (!latch.await(timeout.toMillis(), TimeUnit.MILLISECONDS)) { - throw new IllegalStateException("Timeout waiting for the checker"); + public void wait(Predicate checker, Duration duration) { + var endTime = System.currentTimeMillis() + duration.toMillis(); + var timeout = duration.toMillis(); + if (checker.test(clusterBean())) return; + + while (timeout > 0) { + try { + synchronized (beanUpdateMonitor) { + // Release the lock and wait for clusterBean being updated + this.beanUpdateMonitor.wait(timeout); + } + if (checker.test(clusterBean())) return; + } catch (NoSufficientMetricsException e) { + // Check failed. Try again next time. + } catch (InterruptedException ie) { + throw new IllegalStateException("Interrupted while waiting for the checker"); } - } catch (InterruptedException ie) { - throw new IllegalStateException("Interrupted while waiting for the checker"); - } finally { - waitingList.remove(latch); + timeout = endTime - System.currentTimeMillis(); } + throw new IllegalStateException("Timeout waiting for the checker"); } private void updateClusterBean() { @@ -377,20 +387,5 @@ private void updateClusterBean() { Collectors.toUnmodifiableMap( Map.Entry::getKey, e -> List.copyOf(e.getValue())))); } - - /** - * Check the checkers in the waiting list. If the checker returns true, count down the latch. - */ - private static void checkWaitingList( - Map> waitingList, ClusterBean clusterBean) { - waitingList.forEach( - (latch, checker) -> { - try { - if (checker.test(clusterBean)) latch.countDown(); - } catch (NoSufficientMetricsException e) { - // Check failed. Try again next time. - } - }); - } } }