mirror of https://github.com/apache/kafka.git
MINOR: refine DeleteOffsetsConsumerGroupCommandIntegrationTest#produceRecord (#15802)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
d88c15fc3e
commit
c287ad5dbe
|
@ -38,7 +38,6 @@ import org.apache.kafka.common.TopicPartition;
|
|||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
|
||||
import org.apache.kafka.common.serialization.ByteArraySerializer;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
|
@ -51,9 +50,9 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
||||
|
@ -65,8 +64,8 @@ import static org.junit.jupiter.api.Assertions.assertNull;
|
|||
})
|
||||
@ExtendWith(ClusterTestExtensions.class)
|
||||
public class DeleteOffsetsConsumerGroupCommandIntegrationTest {
|
||||
public static final String TOPIC = "foo";
|
||||
public static final String GROUP = "test.group";
|
||||
public static final String TOPIC_PREFIX = "foo.";
|
||||
public static final String GROUP_PREFIX = "test.group.";
|
||||
private final ClusterInstance clusterInstance;
|
||||
|
||||
private final Iterable<Map<String, Object>> consumerConfigs;
|
||||
|
@ -91,65 +90,89 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest {
|
|||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfStableConsumerGroupWithTopicPartition() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
createTopic(TOPIC);
|
||||
testWithConsumerGroup(TOPIC, 0, 0, Errors.GROUP_SUBSCRIBED_TO_TOPIC, true, consumerConfig);
|
||||
removeTopic(TOPIC);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
createTopic(topic);
|
||||
Runnable validateRunnable = getValidateRunnable(topic, group, 0, 0, Errors.GROUP_SUBSCRIBED_TO_TOPIC);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, true, validateRunnable);
|
||||
removeTopic(topic);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfStableConsumerGroupWithTopicOnly() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
createTopic(TOPIC);
|
||||
testWithConsumerGroup(TOPIC, -1, 0, Errors.GROUP_SUBSCRIBED_TO_TOPIC, true, consumerConfig);
|
||||
removeTopic(TOPIC);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
createTopic(topic);
|
||||
Runnable validateRunnable = getValidateRunnable(topic, group, -1, 0, Errors.GROUP_SUBSCRIBED_TO_TOPIC);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, true, validateRunnable);
|
||||
removeTopic(topic);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfStableConsumerGroupWithUnknownTopicPartition() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
testWithConsumerGroup("foobar", 0, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION, true, consumerConfig);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
Runnable validateRunnable = getValidateRunnable("foobar", group, 0, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, true, validateRunnable);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfStableConsumerGroupWithUnknownTopicOnly() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
testWithConsumerGroup("foobar", -1, -1, Errors.UNKNOWN_TOPIC_OR_PARTITION, true, consumerConfig);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
Runnable validateRunnable = getValidateRunnable("foobar", group, -1, -1, Errors.UNKNOWN_TOPIC_OR_PARTITION);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, true, validateRunnable);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfEmptyConsumerGroupWithTopicPartition() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
createTopic(TOPIC);
|
||||
testWithConsumerGroup(TOPIC, 0, 0, Errors.NONE, false, consumerConfig);
|
||||
removeTopic(TOPIC);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
createTopic(topic);
|
||||
Runnable validateRunnable = getValidateRunnable(topic, group, 0, 0, Errors.NONE);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, false, validateRunnable);
|
||||
removeTopic(topic);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfEmptyConsumerGroupWithTopicOnly() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
createTopic(TOPIC);
|
||||
testWithConsumerGroup(TOPIC, -1, 0, Errors.NONE, false, consumerConfig);
|
||||
removeTopic(TOPIC);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
createTopic(topic);
|
||||
Runnable validateRunnable = getValidateRunnable(topic, group, -1, 0, Errors.NONE);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, false, validateRunnable);
|
||||
removeTopic(topic);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfEmptyConsumerGroupWithUnknownTopicPartition() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
testWithConsumerGroup("foobar", 0, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION, false, consumerConfig);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
Runnable validateRunnable = getValidateRunnable("foobar", group, 0, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, false, validateRunnable);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testDeleteOffsetsOfEmptyConsumerGroupWithUnknownTopicOnly() {
|
||||
for (Map<String, Object> consumerConfig : consumerConfigs) {
|
||||
testWithConsumerGroup("foobar", -1, -1, Errors.UNKNOWN_TOPIC_OR_PARTITION, false, consumerConfig);
|
||||
for (Map<String, Object> consumerConfig: consumerConfigs) {
|
||||
String topic = TOPIC_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
String group = GROUP_PREFIX + consumerConfig.getOrDefault(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name());
|
||||
Runnable validateRunnable = getValidateRunnable("foobar", group, -1, -1, Errors.UNKNOWN_TOPIC_OR_PARTITION);
|
||||
testWithConsumerGroup(topic, group, consumerConfig, false, validateRunnable);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -169,17 +192,15 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest {
|
|||
);
|
||||
}
|
||||
|
||||
private void testWithConsumerGroup(String inputTopic,
|
||||
int inputPartition,
|
||||
int expectedPartition,
|
||||
Errors expectedError,
|
||||
boolean isStable,
|
||||
Map<String, Object> consumerConfig) {
|
||||
produceRecord();
|
||||
this.withConsumerGroup(() -> {
|
||||
private Runnable getValidateRunnable(String inputTopic,
|
||||
String inputGroup,
|
||||
int inputPartition,
|
||||
int expectedPartition,
|
||||
Errors expectedError) {
|
||||
return () -> {
|
||||
String topic = inputPartition >= 0 ? inputTopic + ":" + inputPartition : inputTopic;
|
||||
try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = consumerGroupService(getArgs(GROUP, topic))) {
|
||||
Entry<Errors, Map<TopicPartition, Throwable>> res = consumerGroupService.deleteOffsets(GROUP, Collections.singletonList(topic));
|
||||
try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = consumerGroupService(getArgs(inputGroup, topic))) {
|
||||
Entry<Errors, Map<TopicPartition, Throwable>> res = consumerGroupService.deleteOffsets(inputGroup, Collections.singletonList(topic));
|
||||
Errors topLevelError = res.getKey();
|
||||
Map<TopicPartition, Throwable> partitions = res.getValue();
|
||||
TopicPartition tp = new TopicPartition(inputTopic, expectedPartition);
|
||||
|
@ -192,32 +213,31 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest {
|
|||
else
|
||||
assertEquals(expectedError.exception(), partitions.get(tp).getCause());
|
||||
}
|
||||
}, isStable, consumerConfig);
|
||||
};
|
||||
}
|
||||
|
||||
private void produceRecord() {
|
||||
KafkaProducer<byte[], byte[]> producer = createProducer();
|
||||
try {
|
||||
producer.send(new ProducerRecord<>(TOPIC, 0, null, null)).get();
|
||||
} catch (ExecutionException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
Utils.closeQuietly(producer, "producer");
|
||||
}
|
||||
}
|
||||
|
||||
private void withConsumerGroup(Runnable body, boolean isStable, Map<String, Object> consumerConfig) {
|
||||
try (Consumer<byte[], byte[]> consumer = createConsumer(consumerConfig)) {
|
||||
consumer.subscribe(Collections.singletonList(TOPIC));
|
||||
private void testWithConsumerGroup(String inputTopic,
|
||||
String inputGroup,
|
||||
Map<String, Object> consumerConfig,
|
||||
boolean isStable,
|
||||
Runnable validateRunnable) {
|
||||
produceRecord(inputTopic);
|
||||
try (Consumer<byte[], byte[]> consumer = createConsumer(inputGroup, consumerConfig)) {
|
||||
consumer.subscribe(Collections.singletonList(inputTopic));
|
||||
ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(DEFAULT_MAX_WAIT_MS));
|
||||
Assertions.assertNotEquals(0, records.count());
|
||||
consumer.commitSync();
|
||||
if (isStable) {
|
||||
body.run();
|
||||
validateRunnable.run();
|
||||
}
|
||||
}
|
||||
if (!isStable) {
|
||||
body.run();
|
||||
validateRunnable.run();
|
||||
}
|
||||
}
|
||||
|
||||
private void produceRecord(String topic) {
|
||||
try (KafkaProducer<byte[], byte[]> producer = createProducer()) {
|
||||
assertDoesNotThrow(() -> producer.send(new ProducerRecord<>(topic, 0, null, null)).get());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -231,11 +251,11 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest {
|
|||
return new KafkaProducer<>(config);
|
||||
}
|
||||
|
||||
private Consumer<byte[], byte[]> createConsumer(Map<String, Object> config) {
|
||||
private Consumer<byte[], byte[]> createConsumer(String group, Map<String, Object> config) {
|
||||
Map<String, Object> consumerConfig = new HashMap<>(config);
|
||||
consumerConfig.putIfAbsent(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers());
|
||||
consumerConfig.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
|
||||
consumerConfig.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, GROUP);
|
||||
consumerConfig.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, group);
|
||||
consumerConfig.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
|
||||
consumerConfig.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
|
||||
// Increase timeouts to avoid having a rebalance during the test
|
||||
|
|
Loading…
Reference in New Issue