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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.text.SimpleDateFormat;
import java.time.Duration;
import java.util.Collection;
import java.util.HashSet;
import java.util.Map;
import java.util.Properties;
import java.util.Random;
Expand Down Expand Up @@ -222,24 +223,32 @@ private static void printExtendedProgress(long bytesRead,
public static class ConsumerPerfRebListener implements ConsumerRebalanceListener {
private final AtomicLong joinTimeMs;
private final AtomicLong joinTimeMsInSingleRound;
private final Collection<TopicPartition> assignedPartitions;
private long joinStartMs;

public ConsumerPerfRebListener(AtomicLong joinTimeMs, long joinStartMs, AtomicLong joinTimeMsInSingleRound) {
this.joinTimeMs = joinTimeMs;
this.joinStartMs = joinStartMs;
this.joinTimeMsInSingleRound = joinTimeMsInSingleRound;
this.assignedPartitions = new HashSet<>();
}

@Override
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
joinStartMs = System.currentTimeMillis();
assignedPartitions.removeAll(partitions);
if (assignedPartitions.isEmpty()) {
joinStartMs = System.currentTimeMillis();
}
}

@Override
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
long elapsedMs = System.currentTimeMillis() - joinStartMs;
joinTimeMs.addAndGet(elapsedMs);
joinTimeMsInSingleRound.addAndGet(elapsedMs);
if (assignedPartitions.isEmpty()) {
long elapsedMs = System.currentTimeMillis() - joinStartMs;
joinTimeMs.addAndGet(elapsedMs);
joinTimeMsInSingleRound.addAndGet(elapsedMs);
}
assignedPartitions.addAll(partitions);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.kafka.tools;

import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.Exit;

import org.junit.jupiter.api.AfterEach;
Expand All @@ -30,8 +31,11 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.text.SimpleDateFormat;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class ConsumerPerformanceTest {
Expand Down Expand Up @@ -126,6 +130,61 @@ public void testDefaultClientId() throws IOException {
assertEquals("perf-consumer-client", config.props().getProperty(ConsumerConfig.CLIENT_ID_CONFIG));
}

@Test
public void testConsumerListenerWithAllPartitionRevokedAndAssigned() throws InterruptedException {
String topicName = "topic";
TopicPartition tp0 = new TopicPartition(topicName, 0);
TopicPartition tp1 = new TopicPartition(topicName, 1);
AtomicLong joinTimeMs = new AtomicLong(0);
AtomicLong joinTimeMsInSingleRound = new AtomicLong(0);
ConsumerPerformance.ConsumerPerfRebListener listener = new ConsumerPerformance.ConsumerPerfRebListener(joinTimeMs, 0, joinTimeMsInSingleRound);
listener.onPartitionsAssigned(Set.of(tp0));
long lastJoinTimeMs = joinTimeMs.get();

// All assigned partitions have been revoked.
listener.onPartitionsRevoked(Set.of(tp0));
Thread.sleep(100);
listener.onPartitionsAssigned(Set.of(tp1));

assertNotEquals(lastJoinTimeMs, joinTimeMs.get());
}

@Test
public void testConsumerListenerWithPartialPartitionRevokedAndAssigned() throws InterruptedException {
String topicName = "topic";
TopicPartition tp0 = new TopicPartition(topicName, 0);
TopicPartition tp1 = new TopicPartition(topicName, 1);
AtomicLong joinTimeMs = new AtomicLong(0);
AtomicLong joinTimeMsInSingleRound = new AtomicLong(0);
ConsumerPerformance.ConsumerPerfRebListener listener = new ConsumerPerformance.ConsumerPerfRebListener(joinTimeMs, 0, joinTimeMsInSingleRound);
listener.onPartitionsAssigned(Set.of(tp0, tp1));
long lastJoinTimeMs = joinTimeMs.get();

// The assigned partitions were partially revoked.
listener.onPartitionsRevoked(Set.of(tp0));
Thread.sleep(100);
listener.onPartitionsAssigned(Set.of(tp0));

assertEquals(lastJoinTimeMs, joinTimeMs.get());
}

@Test
public void testConsumerListenerWithoutPartitionRevoked() throws InterruptedException {
String topicName = "topic";
TopicPartition tp0 = new TopicPartition(topicName, 0);
TopicPartition tp1 = new TopicPartition(topicName, 1);
AtomicLong joinTimeMs = new AtomicLong(0);
AtomicLong joinTimeMsInSingleRound = new AtomicLong(0);
ConsumerPerformance.ConsumerPerfRebListener listener = new ConsumerPerformance.ConsumerPerfRebListener(joinTimeMs, 0, joinTimeMsInSingleRound);
listener.onPartitionsAssigned(Set.of(tp0));
long lastJoinTimeMs = joinTimeMs.get();

Thread.sleep(100);
listener.onPartitionsAssigned(Set.of(tp1));

assertEquals(lastJoinTimeMs, joinTimeMs.get());
}

private void testHeaderMatchContent(boolean detailed, int expectedOutputLineCount, Runnable runnable) {
String out = ToolsTestUtils.captureStandardOut(() -> {
ConsumerPerformance.printHeader(detailed);
Expand Down