mirror of https://github.com/apache/kafka.git
MINOR: Improve ListConsumerGroupTest.testListGroupCommand (#15382)
While reviewing https://github.com/apache/kafka/pull/15150, I found that our tests verifying the console output are really hard to read. Here is my proposal to make it better. Reviewers: Ritika Reddy <rreddy@confluent.io>, Justine Olshan <jolshan@confluent.io>
This commit is contained in:
parent
98fb3bd304
commit
e247bd03af
|
@ -26,10 +26,15 @@ import org.junit.jupiter.params.provider.ValueSource;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
|
@ -123,42 +128,88 @@ public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
|
|||
String simpleGroup = "simple-group";
|
||||
addSimpleGroupExecutor(simpleGroup);
|
||||
addConsumerGroupExecutor(1);
|
||||
|
||||
validateListOutput(
|
||||
Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list"),
|
||||
Collections.emptyList(),
|
||||
mkSet(
|
||||
Collections.singletonList(GROUP),
|
||||
Collections.singletonList(simpleGroup)
|
||||
)
|
||||
);
|
||||
|
||||
validateListOutput(
|
||||
Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"),
|
||||
Arrays.asList("GROUP", "STATE"),
|
||||
mkSet(
|
||||
Arrays.asList(GROUP, "Stable"),
|
||||
Arrays.asList(simpleGroup, "Empty")
|
||||
)
|
||||
);
|
||||
|
||||
validateListOutput(
|
||||
Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "Stable"),
|
||||
Arrays.asList("GROUP", "STATE"),
|
||||
mkSet(
|
||||
Arrays.asList(GROUP, "Stable")
|
||||
)
|
||||
);
|
||||
|
||||
validateListOutput(
|
||||
Arrays.asList("--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "stable"),
|
||||
Arrays.asList("GROUP", "STATE"),
|
||||
mkSet(
|
||||
Arrays.asList(GROUP, "Stable")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates that the output of the list command corresponds to the expected values.
|
||||
*
|
||||
* @param args The arguments for the command line tool.
|
||||
* @param expectedHeader The expected header as a list of strings; or an empty list
|
||||
* if a header is not expected.
|
||||
* @param expectedRows The expected rows as a set of list of columns.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private static void validateListOutput(
|
||||
List<String> args,
|
||||
List<String> expectedHeader,
|
||||
Set<List<String>> expectedRows
|
||||
) throws InterruptedException {
|
||||
final AtomicReference<String> out = new AtomicReference<>("");
|
||||
|
||||
String[] cgcArgs1 = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
|
||||
TestUtils.waitForCondition(() -> {
|
||||
out.set(kafka.utils.TestUtils.grabConsoleOutput(() -> {
|
||||
ConsumerGroupCommand.main(cgcArgs1);
|
||||
return null;
|
||||
}));
|
||||
return !out.get().contains("STATE") && out.get().contains(simpleGroup) && out.get().contains(GROUP);
|
||||
}, "Expected to find " + simpleGroup + ", " + GROUP + " and no header, but found " + out.get());
|
||||
String output = runAndGrabConsoleOutput(args);
|
||||
out.set(output);
|
||||
|
||||
String[] cgcArgs2 = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state"};
|
||||
TestUtils.waitForCondition(() -> {
|
||||
out.set(kafka.utils.TestUtils.grabConsoleOutput(() -> {
|
||||
ConsumerGroupCommand.main(cgcArgs2);
|
||||
return null;
|
||||
}));
|
||||
return out.get().contains("STATE") && out.get().contains(simpleGroup) && out.get().contains(GROUP);
|
||||
}, "Expected to find " + simpleGroup + ", " + GROUP + " and the header, but found " + out.get());
|
||||
int index = 0;
|
||||
String[] lines = output.split("\n");
|
||||
|
||||
String[] cgcArgs3 = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "Stable"};
|
||||
TestUtils.waitForCondition(() -> {
|
||||
out.set(kafka.utils.TestUtils.grabConsoleOutput(() -> {
|
||||
ConsumerGroupCommand.main(cgcArgs3);
|
||||
return null;
|
||||
}));
|
||||
return out.get().contains("STATE") && out.get().contains(GROUP) && out.get().contains("Stable");
|
||||
}, "Expected to find " + GROUP + " in state Stable and the header, but found " + out.get());
|
||||
// Parse the header if one is expected.
|
||||
if (!expectedHeader.isEmpty()) {
|
||||
if (lines.length == 0) return false;
|
||||
List<String> header = Arrays.stream(lines[index++].split("\\s+")).collect(Collectors.toList());
|
||||
if (!expectedHeader.equals(header)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
String[] cgcArgs4 = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--list", "--state", "stable"};
|
||||
TestUtils.waitForCondition(() -> {
|
||||
out.set(kafka.utils.TestUtils.grabConsoleOutput(() -> {
|
||||
ConsumerGroupCommand.main(cgcArgs4);
|
||||
return null;
|
||||
}));
|
||||
return out.get().contains("STATE") && out.get().contains(GROUP) && out.get().contains("Stable");
|
||||
}, "Expected to find " + GROUP + " in state Stable and the header, but found " + out.get());
|
||||
// Parse the groups.
|
||||
Set<List<String>> groups = new HashSet<>();
|
||||
for (; index < lines.length; index++) {
|
||||
groups.add(Arrays.stream(lines[index].split("\\s+")).collect(Collectors.toList()));
|
||||
}
|
||||
return expectedRows.equals(groups);
|
||||
}, () -> String.format("Expected header=%s and groups=%s, but found:%n%s", expectedHeader, expectedRows, out.get()));
|
||||
}
|
||||
|
||||
private static String runAndGrabConsoleOutput(
|
||||
List<String> args
|
||||
) {
|
||||
return kafka.utils.TestUtils.grabConsoleOutput(() -> {
|
||||
ConsumerGroupCommand.main(args.toArray(new String[0]));
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue