Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[PERF] Enhance performance partition change rate #958

29 changes: 25 additions & 4 deletions app/src/main/java/org/astraea/app/performance/ConsumerThread.java
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,10 @@

public interface ConsumerThread extends AbstractThread {

ConcurrentMap<String, Set<TopicPartition>> CLIENT_ID_PARTITIONS = new ConcurrentHashMap<>();
ConcurrentMap<String, Set<TopicPartition>> CLIENT_ID_ASSIGNED_PARTITIONS =
new ConcurrentHashMap<>();
ConcurrentMap<String, Set<TopicPartition>> CLIENT_ID_REVOKED_PARTITIONS =
new ConcurrentHashMap<>();

static List<ConsumerThread> create(
int consumers,
Expand All @@ -64,8 +67,7 @@ static List<ConsumerThread> create(
index -> {
@SuppressWarnings("resource")
var clientId = Utils.randomString();
var consumer =
consumerSupplier.apply(clientId, ps -> CLIENT_ID_PARTITIONS.put(clientId, ps));
var consumer = consumerSupplier.apply(clientId, new PartitionRatioListener(clientId));
var closed = new AtomicBoolean(false);
var closeLatch = closeLatches.get(index);
var subscribed = new AtomicBoolean(true);
Expand All @@ -87,7 +89,8 @@ static List<ConsumerThread> create(
Utils.swallowException(consumer::close);
closeLatch.countDown();
closed.set(true);
CLIENT_ID_PARTITIONS.remove(clientId);
CLIENT_ID_ASSIGNED_PARTITIONS.remove(clientId);
CLIENT_ID_REVOKED_PARTITIONS.remove(clientId);
}
});
return new ConsumerThread() {
Expand Down Expand Up @@ -125,4 +128,22 @@ public void close() {
void resubscribe();

void unsubscribe();

class PartitionRatioListener implements ConsumerRebalanceListener {
private final String clientId;

PartitionRatioListener(String clientId) {
this.clientId = clientId;
}

@Override
public void onPartitionAssigned(Set<TopicPartition> partitions) {
CLIENT_ID_ASSIGNED_PARTITIONS.put(clientId, partitions);
}

@Override
public void onPartitionsRevoked(Set<TopicPartition> partitions) {
CLIENT_ID_REVOKED_PARTITIONS.put(clientId, partitions);
}
}
}
12 changes: 12 additions & 0 deletions app/src/main/java/org/astraea/app/performance/ReportFormat.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Set;
import java.util.function.Supplier;
import java.util.stream.IntStream;
import org.astraea.common.EnumInfo;
Expand Down Expand Up @@ -207,6 +208,17 @@ private static List<CSVContentElement> latencyAndIO() {
CSVContentElement.create(
"Consumer[" + i + "] average publish latency (ms)",
() -> Double.toString(consumerReports.get(i).avgLatency())));
elements.add(
CSVContentElement.create(
"Consumer[" + i + "] partition difference",
() ->
Integer.toString(
(ConsumerThread.CLIENT_ID_ASSIGNED_PARTITIONS
.getOrDefault(consumerReports.get(i).clientId(), Set.of())
.size()
- ConsumerThread.CLIENT_ID_REVOKED_PARTITIONS
.getOrDefault(consumerReports.get(i).clientId(), Set.of())
.size()))));
});
return elements;
}
Expand Down
17 changes: 16 additions & 1 deletion app/src/main/java/org/astraea/app/performance/TrackerThread.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,13 @@
import java.time.Duration;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Supplier;
import java.util.function.ToDoubleFunction;
import java.util.stream.Collectors;
import org.astraea.common.DataSize;
import org.astraea.common.Utils;
import org.astraea.common.metrics.HasBeanObject;
Expand Down Expand Up @@ -140,10 +142,23 @@ boolean tryToPrint(Duration duration) {
.ifPresent(i -> System.out.printf(" rebalance average latency: %.3f ms%n", i));
for (var i = 0; i < reports.size(); ++i) {
var report = reports.get(i);
var clientId = report.clientId();
var ms = metrics.stream().filter(m -> m.clientId().equals(report.clientId())).findFirst();
var assignedPartitions =
ConsumerThread.CLIENT_ID_ASSIGNED_PARTITIONS.getOrDefault(clientId, Set.of());
var revokedPartitions =
ConsumerThread.CLIENT_ID_REVOKED_PARTITIONS.getOrDefault(clientId, Set.of());
var nonStickyPartitions =
harryteng9527 marked this conversation as resolved.
Show resolved Hide resolved
assignedPartitions.stream()
.filter(tp -> !revokedPartitions.contains(tp))
.collect(Collectors.toSet());
if (ms.isPresent()) {
System.out.printf(
" consumer[%d] has %d partitions%n", i, (int) ms.get().assignedPartitions());
" consumer[%d] has %d partitions. Among them, there are %d non-sticky partitions and was assigned %d more partitions than before re-balancing%n",
i,
(int) ms.get().assignedPartitions(),
nonStickyPartitions.size(),
(assignedPartitions.size() - revokedPartitions.size()));
}
System.out.printf(
" consumed[%d] average throughput: %s%n",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@ void testConsumerPrinter() {
var report = Mockito.mock(Report.class);
var records = new AtomicLong(0);
Mockito.when(report.records()).thenAnswer(a -> records.get());
Mockito.when(report.clientId()).thenReturn("forTest");
var printer = new TrackerThread.ConsumerPrinter(() -> List.of(report));
Assertions.assertFalse(printer.tryToPrint(Duration.ofSeconds(1)));
records.set(100);
Expand Down