mirror of https://github.com/apache/kafka.git
KAFKA-19042 Move PlaintextConsumerCallbackTest to client-integration-tests module (#19298)
Use Java to rewrite `PlaintextConsumerCallbackTest` by new test infra and move it to client-integration-tests module. Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
73afcc9b69
commit
ae608c1cb2
|
@ -0,0 +1,352 @@
|
||||||
|
/*
|
||||||
|
* 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.apache.kafka.clients.consumer;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.producer.Producer;
|
||||||
|
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.record.TimestampType;
|
||||||
|
import org.apache.kafka.common.test.ClusterInstance;
|
||||||
|
import org.apache.kafka.common.test.TestUtils;
|
||||||
|
import org.apache.kafka.common.test.api.ClusterTest;
|
||||||
|
import org.apache.kafka.common.test.api.ClusterTestDefaults;
|
||||||
|
import org.apache.kafka.common.test.api.Type;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Locale;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.function.BiConsumer;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
|
||||||
|
import static org.apache.kafka.clients.consumer.GroupProtocol.CLASSIC;
|
||||||
|
import static org.apache.kafka.clients.consumer.GroupProtocol.CONSUMER;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
|
@ClusterTestDefaults(
|
||||||
|
types = {Type.KRAFT},
|
||||||
|
brokers = 3
|
||||||
|
)
|
||||||
|
public class PlaintextConsumerCallbackTest {
|
||||||
|
|
||||||
|
private final ClusterInstance cluster;
|
||||||
|
private final String topic = "topic";
|
||||||
|
private final TopicPartition tp = new TopicPartition(topic, 0);
|
||||||
|
|
||||||
|
public PlaintextConsumerCallbackTest(ClusterInstance clusterInstance) {
|
||||||
|
this.cluster = clusterInstance;
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerRebalanceListenerAssignOnPartitionsAssigned() throws InterruptedException {
|
||||||
|
testRebalanceListenerAssignOnPartitionsAssigned(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerRebalanceListenerAssignOnPartitionsAssigned() throws InterruptedException {
|
||||||
|
testRebalanceListenerAssignOnPartitionsAssigned(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testRebalanceListenerAssignOnPartitionsAssigned(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
try (var consumer = createConsumer(groupProtocol)) {
|
||||||
|
triggerOnPartitionsAssigned(tp, consumer, (executeConsumer, partitions) -> {
|
||||||
|
var e = assertThrows(IllegalStateException.class, () -> executeConsumer.assign(List.of(tp)));
|
||||||
|
assertEquals("Subscription to topics, partitions and pattern are mutually exclusive", e.getMessage());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerRebalanceListenerAssignmentOnPartitionsAssigned() throws InterruptedException {
|
||||||
|
testRebalanceListenerAssignmentOnPartitionsAssigned(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerRebalanceListenerAssignmentOnPartitionsAssigned() throws InterruptedException {
|
||||||
|
testRebalanceListenerAssignmentOnPartitionsAssigned(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testRebalanceListenerAssignmentOnPartitionsAssigned(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
try (var consumer = createConsumer(groupProtocol)) {
|
||||||
|
triggerOnPartitionsAssigned(tp, consumer,
|
||||||
|
(executeConsumer, partitions) -> assertTrue(executeConsumer.assignment().contains(tp))
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerRebalanceListenerBeginningOffsetsOnPartitionsAssigned() throws InterruptedException {
|
||||||
|
testRebalanceListenerBeginningOffsetsOnPartitionsAssigned(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerRebalanceListenerBeginningOffsetsOnPartitionsAssigned() throws InterruptedException {
|
||||||
|
testRebalanceListenerBeginningOffsetsOnPartitionsAssigned(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testRebalanceListenerBeginningOffsetsOnPartitionsAssigned(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
try (var consumer = createConsumer(groupProtocol)) {
|
||||||
|
triggerOnPartitionsAssigned(tp, consumer, (executeConsumer, partitions) -> {
|
||||||
|
var map = executeConsumer.beginningOffsets(List.of(tp));
|
||||||
|
assertTrue(map.containsKey(tp));
|
||||||
|
assertEquals(0L, map.get(tp));
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerRebalanceListenerAssignOnPartitionsRevoked() throws InterruptedException {
|
||||||
|
testRebalanceListenerAssignOnPartitionsRevoked(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerRebalanceListenerAssignOnPartitionsRevoked() throws InterruptedException {
|
||||||
|
testRebalanceListenerAssignOnPartitionsRevoked(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testRebalanceListenerAssignOnPartitionsRevoked(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
triggerOnPartitionsRevoked(tp, groupProtocol, (consumer, partitions) -> {
|
||||||
|
var e = assertThrows(IllegalStateException.class, () -> consumer.assign(List.of(tp)));
|
||||||
|
assertEquals("Subscription to topics, partitions and pattern are mutually exclusive", e.getMessage());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerRebalanceListenerAssignmentOnPartitionsRevoked() throws InterruptedException {
|
||||||
|
triggerOnPartitionsRevoked(tp, CLASSIC,
|
||||||
|
(consumer, partitions) -> assertTrue(consumer.assignment().contains(tp))
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerRebalanceListenerAssignmentOnPartitionsRevoked() throws InterruptedException {
|
||||||
|
triggerOnPartitionsRevoked(tp, CONSUMER,
|
||||||
|
(consumer, partitions) -> assertTrue(consumer.assignment().contains(tp))
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerRebalanceListenerBeginningOffsetsOnPartitionsRevoked() throws InterruptedException {
|
||||||
|
testRebalanceListenerBeginningOffsetsOnPartitionsRevoked(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerRebalanceListenerBeginningOffsetsOnPartitionsRevoked() throws InterruptedException {
|
||||||
|
testRebalanceListenerBeginningOffsetsOnPartitionsRevoked(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testRebalanceListenerBeginningOffsetsOnPartitionsRevoked(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
triggerOnPartitionsRevoked(tp, groupProtocol, (consumer, partitions) -> {
|
||||||
|
var map = consumer.beginningOffsets(List.of(tp));
|
||||||
|
assertTrue(map.containsKey(tp));
|
||||||
|
assertEquals(0L, map.get(tp));
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
|
||||||
|
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
|
||||||
|
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
try (var consumer = createConsumer(groupProtocol)) {
|
||||||
|
triggerOnPartitionsAssigned(tp, consumer,
|
||||||
|
(executeConsumer, partitions) -> assertDoesNotThrow(() -> executeConsumer.position(tp))
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testClassicConsumerSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
|
||||||
|
testSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback(CLASSIC);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ClusterTest
|
||||||
|
public void testAsyncConsumerSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
|
||||||
|
testSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
|
try (var consumer = createConsumer(groupProtocol)) {
|
||||||
|
var startingOffset = 100L;
|
||||||
|
var totalRecords = 120;
|
||||||
|
var startingTimestamp = 0L;
|
||||||
|
|
||||||
|
sendRecords(totalRecords, startingTimestamp);
|
||||||
|
|
||||||
|
triggerOnPartitionsAssigned(tp, consumer, (executeConsumer, partitions) -> {
|
||||||
|
executeConsumer.seek(tp, startingOffset);
|
||||||
|
executeConsumer.pause(List.of(tp));
|
||||||
|
});
|
||||||
|
|
||||||
|
assertTrue(consumer.paused().contains(tp));
|
||||||
|
consumer.resume(List.of(tp));
|
||||||
|
consumeAndVerifyRecords(
|
||||||
|
consumer,
|
||||||
|
(int) (totalRecords - startingOffset),
|
||||||
|
(int) startingOffset,
|
||||||
|
(int) startingOffset,
|
||||||
|
startingOffset
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void triggerOnPartitionsAssigned(
|
||||||
|
TopicPartition tp,
|
||||||
|
Consumer<byte[], byte[]> consumer,
|
||||||
|
BiConsumer<Consumer<byte[], byte[]>, Collection<TopicPartition>> execute
|
||||||
|
) throws InterruptedException {
|
||||||
|
var partitionsAssigned = new AtomicBoolean(false);
|
||||||
|
consumer.subscribe(List.of(topic), new ConsumerRebalanceListener() {
|
||||||
|
@Override
|
||||||
|
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||||
|
// Make sure the partition used in the test is actually assigned before continuing.
|
||||||
|
if (partitions.contains(tp)) {
|
||||||
|
execute.accept(consumer, partitions);
|
||||||
|
partitionsAssigned.set(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||||
|
// noop
|
||||||
|
}
|
||||||
|
});
|
||||||
|
TestUtils.waitForCondition(
|
||||||
|
() -> {
|
||||||
|
consumer.poll(Duration.ofMillis(100));
|
||||||
|
return partitionsAssigned.get();
|
||||||
|
},
|
||||||
|
"Timed out before expected rebalance completed"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void triggerOnPartitionsRevoked(
|
||||||
|
TopicPartition tp,
|
||||||
|
GroupProtocol protocol,
|
||||||
|
BiConsumer<Consumer<byte[], byte[]>, Collection<TopicPartition>> execute
|
||||||
|
) throws InterruptedException {
|
||||||
|
var partitionsAssigned = new AtomicBoolean(false);
|
||||||
|
var partitionsRevoked = new AtomicBoolean(false);
|
||||||
|
try (var consumer = createConsumer(protocol)) {
|
||||||
|
consumer.subscribe(List.of(topic), new ConsumerRebalanceListener() {
|
||||||
|
@Override
|
||||||
|
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||||
|
// Make sure the partition used in the test is actually assigned before continuing.
|
||||||
|
if (partitions.contains(tp)) {
|
||||||
|
partitionsAssigned.set(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||||
|
// Make sure the partition used in the test is actually revoked before continuing.
|
||||||
|
if (partitions.contains(tp)) {
|
||||||
|
execute.accept(consumer, partitions);
|
||||||
|
partitionsRevoked.set(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
TestUtils.waitForCondition(
|
||||||
|
() -> {
|
||||||
|
consumer.poll(Duration.ofMillis(100));
|
||||||
|
return partitionsAssigned.get();
|
||||||
|
},
|
||||||
|
"Timed out before expected rebalance completed"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
assertTrue(partitionsRevoked.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
private Consumer<byte[], byte[]> createConsumer(GroupProtocol protocol) {
|
||||||
|
return cluster.consumer(Map.of(
|
||||||
|
GROUP_PROTOCOL_CONFIG, protocol.name().toLowerCase(Locale.ROOT),
|
||||||
|
ENABLE_AUTO_COMMIT_CONFIG, "false"
|
||||||
|
));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void sendRecords(int numRecords, long startingTimestamp) {
|
||||||
|
try (Producer<byte[], byte[]> producer = cluster.producer()) {
|
||||||
|
for (var i = 0; i < numRecords; i++) {
|
||||||
|
var timestamp = startingTimestamp + i;
|
||||||
|
var record = new ProducerRecord<>(
|
||||||
|
tp.topic(),
|
||||||
|
tp.partition(),
|
||||||
|
timestamp,
|
||||||
|
("key " + i).getBytes(),
|
||||||
|
("value " + i).getBytes()
|
||||||
|
);
|
||||||
|
producer.send(record);
|
||||||
|
}
|
||||||
|
producer.flush();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void consumeAndVerifyRecords(
|
||||||
|
Consumer<byte[], byte[]> consumer,
|
||||||
|
int numRecords,
|
||||||
|
int startingOffset,
|
||||||
|
int startingKeyAndValueIndex,
|
||||||
|
long startingTimestamp
|
||||||
|
) throws InterruptedException {
|
||||||
|
var records = consumeRecords(consumer, numRecords);
|
||||||
|
for (var i = 0; i < numRecords; i++) {
|
||||||
|
var record = records.get(i);
|
||||||
|
var offset = startingOffset + i;
|
||||||
|
|
||||||
|
assertEquals(tp.topic(), record.topic());
|
||||||
|
assertEquals(tp.partition(), record.partition());
|
||||||
|
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, record.timestampType());
|
||||||
|
var timestamp = startingTimestamp + i;
|
||||||
|
assertEquals(timestamp, record.timestamp());
|
||||||
|
|
||||||
|
assertEquals(offset, record.offset());
|
||||||
|
var keyAndValueIndex = startingKeyAndValueIndex + i;
|
||||||
|
assertEquals("key " + keyAndValueIndex, new String(record.key()));
|
||||||
|
assertEquals("value " + keyAndValueIndex, new String(record.value()));
|
||||||
|
// this is true only because K and V are byte arrays
|
||||||
|
assertEquals(("key " + keyAndValueIndex).length(), record.serializedKeySize());
|
||||||
|
assertEquals(("value " + keyAndValueIndex).length(), record.serializedValueSize());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected <K, V> List<ConsumerRecord<K, V>> consumeRecords(
|
||||||
|
Consumer<K, V> consumer,
|
||||||
|
int numRecords
|
||||||
|
) throws InterruptedException {
|
||||||
|
List<ConsumerRecord<K, V>> records = new ArrayList<>();
|
||||||
|
TestUtils.waitForCondition(() -> {
|
||||||
|
consumer.poll(Duration.ofMillis(100)).forEach(records::add);
|
||||||
|
return records.size() >= numRecords;
|
||||||
|
}, 60000, "Timed out before consuming expected " + numRecords + " records.");
|
||||||
|
|
||||||
|
return records;
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,175 +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 integration.kafka.api
|
|
||||||
|
|
||||||
import kafka.api.AbstractConsumerTest
|
|
||||||
import kafka.utils.{TestInfoUtils, TestUtils}
|
|
||||||
import org.apache.kafka.clients.consumer.{Consumer, ConsumerRebalanceListener}
|
|
||||||
import org.apache.kafka.common.TopicPartition
|
|
||||||
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertThrows, assertTrue}
|
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.MethodSource
|
|
||||||
|
|
||||||
import java.util
|
|
||||||
import java.util.Arrays.asList
|
|
||||||
import java.util.Collections
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Integration tests for the consumer that cover interaction with the consumer from within callbacks
|
|
||||||
* and listeners.
|
|
||||||
*/
|
|
||||||
class PlaintextConsumerCallbackTest extends AbstractConsumerTest {
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testConsumerRebalanceListenerAssignOnPartitionsAssigned(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsAssigned(tp, { (consumer, _) =>
|
|
||||||
val e: Exception = assertThrows(classOf[IllegalStateException], () => consumer.assign(Collections.singletonList(tp)))
|
|
||||||
assertEquals(e.getMessage, "Subscription to topics, partitions and pattern are mutually exclusive")
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testConsumerRebalanceListenerAssignmentOnPartitionsAssigned(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsAssigned(tp, { (consumer, _) =>
|
|
||||||
assertTrue(consumer.assignment().contains(tp));
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testConsumerRebalanceListenerBeginningOffsetsOnPartitionsAssigned(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsAssigned(tp, { (consumer, _) =>
|
|
||||||
val map = consumer.beginningOffsets(Collections.singletonList(tp))
|
|
||||||
assertTrue(map.containsKey(tp))
|
|
||||||
assertEquals(0, map.get(tp))
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testConsumerRebalanceListenerAssignOnPartitionsRevoked(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsRevoked(tp, { (consumer, _) =>
|
|
||||||
val e: Exception = assertThrows(classOf[IllegalStateException], () => consumer.assign(Collections.singletonList(tp)))
|
|
||||||
assertEquals(e.getMessage, "Subscription to topics, partitions and pattern are mutually exclusive")
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testConsumerRebalanceListenerAssignmentOnPartitionsRevoked(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsRevoked(tp, { (consumer, _) =>
|
|
||||||
assertTrue(consumer.assignment().contains(tp))
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testConsumerRebalanceListenerBeginningOffsetsOnPartitionsRevoked(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsRevoked(tp, { (consumer, _) =>
|
|
||||||
val map = consumer.beginningOffsets(Collections.singletonList(tp))
|
|
||||||
assertTrue(map.containsKey(tp))
|
|
||||||
assertEquals(0, map.get(tp))
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(groupProtocol: String): Unit = {
|
|
||||||
val tp = new TopicPartition(topic, 0)
|
|
||||||
triggerOnPartitionsAssigned(tp, { (consumer, _) => assertDoesNotThrow(() => consumer.position(tp)) })
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames)
|
|
||||||
@MethodSource(Array("getTestGroupProtocolParametersAll"))
|
|
||||||
def testSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback(groupProtocol: String): Unit = {
|
|
||||||
val consumer = createConsumer()
|
|
||||||
val startingOffset = 100L
|
|
||||||
val totalRecords = 120L
|
|
||||||
|
|
||||||
val producer = createProducer()
|
|
||||||
val startingTimestamp = 0
|
|
||||||
sendRecords(producer, totalRecords.toInt, tp, startingTimestamp)
|
|
||||||
|
|
||||||
triggerOnPartitionsAssigned(tp, consumer, { (consumer, _) =>
|
|
||||||
consumer.seek(tp, startingOffset)
|
|
||||||
consumer.pause(asList(tp))
|
|
||||||
})
|
|
||||||
|
|
||||||
assertTrue(consumer.paused().contains(tp))
|
|
||||||
consumer.resume(asList(tp))
|
|
||||||
consumeAndVerifyRecords(consumer, numRecords = (totalRecords - startingOffset).toInt,
|
|
||||||
startingOffset = startingOffset.toInt, startingKeyAndValueIndex = startingOffset.toInt,
|
|
||||||
startingTimestamp = startingOffset)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def triggerOnPartitionsAssigned(tp: TopicPartition,
|
|
||||||
execute: (Consumer[Array[Byte], Array[Byte]], util.Collection[TopicPartition]) => Unit): Unit = {
|
|
||||||
val consumer = createConsumer()
|
|
||||||
triggerOnPartitionsAssigned(tp, consumer, execute)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def triggerOnPartitionsAssigned(tp: TopicPartition,
|
|
||||||
consumer: Consumer[Array[Byte], Array[Byte]],
|
|
||||||
execute: (Consumer[Array[Byte], Array[Byte]], util.Collection[TopicPartition]) => Unit): Unit = {
|
|
||||||
val partitionsAssigned = new AtomicBoolean(false)
|
|
||||||
consumer.subscribe(asList(topic), new ConsumerRebalanceListener {
|
|
||||||
override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = {
|
|
||||||
// Make sure the partition used in the test is actually assigned before continuing.
|
|
||||||
if (partitions.contains(tp)) {
|
|
||||||
execute(consumer, partitions)
|
|
||||||
partitionsAssigned.set(true)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {
|
|
||||||
// noop
|
|
||||||
}
|
|
||||||
})
|
|
||||||
TestUtils.pollUntilTrue(consumer, () => partitionsAssigned.get(), "Timed out before expected rebalance completed")
|
|
||||||
}
|
|
||||||
|
|
||||||
private def triggerOnPartitionsRevoked(tp: TopicPartition,
|
|
||||||
execute: (Consumer[Array[Byte], Array[Byte]], util.Collection[TopicPartition]) => Unit): Unit = {
|
|
||||||
val consumer = createConsumer()
|
|
||||||
val partitionsAssigned = new AtomicBoolean(false)
|
|
||||||
val partitionsRevoked = new AtomicBoolean(false)
|
|
||||||
consumer.subscribe(asList(topic), new ConsumerRebalanceListener {
|
|
||||||
override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]): Unit = {
|
|
||||||
// Make sure the partition used in the test is actually assigned before continuing.
|
|
||||||
if (partitions.contains(tp)) {
|
|
||||||
partitionsAssigned.set(true)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]): Unit = {
|
|
||||||
// Make sure the partition used in the test is actually revoked before continuing.
|
|
||||||
if (partitions.contains(tp)) {
|
|
||||||
execute(consumer, partitions)
|
|
||||||
partitionsRevoked.set(true)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
TestUtils.pollUntilTrue(consumer, () => partitionsAssigned.get(), "Timed out before expected rebalance completed")
|
|
||||||
consumer.close()
|
|
||||||
assertTrue(partitionsRevoked.get())
|
|
||||||
}
|
|
||||||
}
|
|
Loading…
Reference in New Issue