mirror of https://github.com/apache/kafka.git
KAFKA-19705: Enable streams rebalance protocol in IQv2 integration test (#20541)
Update IQv2 Integration tests for streams group protocol Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
This commit is contained in:
parent
e647bdcee5
commit
04b4a8f571
|
@ -57,11 +57,12 @@ import org.apache.kafka.test.TestUtils;
|
||||||
import org.junit.jupiter.api.AfterAll;
|
import org.junit.jupiter.api.AfterAll;
|
||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.BeforeAll;
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
|
||||||
import org.junit.jupiter.api.Tag;
|
import org.junit.jupiter.api.Tag;
|
||||||
import org.junit.jupiter.api.Test;
|
|
||||||
import org.junit.jupiter.api.TestInfo;
|
import org.junit.jupiter.api.TestInfo;
|
||||||
import org.junit.jupiter.api.Timeout;
|
import org.junit.jupiter.api.Timeout;
|
||||||
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
|
import org.junit.jupiter.params.provider.Arguments;
|
||||||
|
import org.junit.jupiter.params.provider.MethodSource;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.lang.reflect.Field;
|
import java.lang.reflect.Field;
|
||||||
|
@ -76,6 +77,7 @@ import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static java.util.Collections.singleton;
|
import static java.util.Collections.singleton;
|
||||||
import static org.apache.kafka.streams.query.StateQueryRequest.inStore;
|
import static org.apache.kafka.streams.query.StateQueryRequest.inStore;
|
||||||
|
@ -99,6 +101,7 @@ public class IQv2IntegrationTest {
|
||||||
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
|
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
|
||||||
|
|
||||||
private KafkaStreams kafkaStreams;
|
private KafkaStreams kafkaStreams;
|
||||||
|
private String groupProtocol;
|
||||||
|
|
||||||
@BeforeAll
|
@BeforeAll
|
||||||
public static void before()
|
public static void before()
|
||||||
|
@ -149,8 +152,8 @@ public class IQv2IntegrationTest {
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeEach
|
private void setup(final String groupProtocol, final TestInfo testInfo) {
|
||||||
public void beforeTest(final TestInfo testInfo) {
|
this.groupProtocol = groupProtocol;
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
final StreamsBuilder builder = new StreamsBuilder();
|
||||||
|
|
||||||
builder.table(
|
builder.table(
|
||||||
|
@ -159,7 +162,6 @@ public class IQv2IntegrationTest {
|
||||||
Materialized.as(STORE_NAME)
|
Materialized.as(STORE_NAME)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
final String safeTestName = safeUniqueTestName(testInfo);
|
final String safeTestName = safeUniqueTestName(testInfo);
|
||||||
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
|
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
|
||||||
kafkaStreams.cleanUp();
|
kafkaStreams.cleanUp();
|
||||||
|
@ -167,17 +169,21 @@ public class IQv2IntegrationTest {
|
||||||
|
|
||||||
@AfterEach
|
@AfterEach
|
||||||
public void afterTest() {
|
public void afterTest() {
|
||||||
|
if (kafkaStreams != null) {
|
||||||
kafkaStreams.close(Duration.ofSeconds(60));
|
kafkaStreams.close(Duration.ofSeconds(60));
|
||||||
kafkaStreams.cleanUp();
|
kafkaStreams.cleanUp();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@AfterAll
|
@AfterAll
|
||||||
public static void after() {
|
public static void after() {
|
||||||
CLUSTER.stop();
|
CLUSTER.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldFailUnknownStore() {
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldFailUnknownStore(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||||
inStore("unknown-store").withQuery(query);
|
inStore("unknown-store").withQuery(query);
|
||||||
|
@ -185,8 +191,10 @@ public class IQv2IntegrationTest {
|
||||||
assertThrows(UnknownStateStoreException.class, () -> kafkaStreams.query(request));
|
assertThrows(UnknownStateStoreException.class, () -> kafkaStreams.query(request));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldFailNotStarted() {
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldFailNotStarted(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||||
inStore(STORE_NAME).withQuery(query);
|
inStore(STORE_NAME).withQuery(query);
|
||||||
|
@ -194,8 +202,10 @@ public class IQv2IntegrationTest {
|
||||||
assertThrows(StreamsNotStartedException.class, () -> kafkaStreams.query(request));
|
assertThrows(StreamsNotStartedException.class, () -> kafkaStreams.query(request));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldFailStopped() {
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldFailStopped(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||||
inStore(STORE_NAME).withQuery(query);
|
inStore(STORE_NAME).withQuery(query);
|
||||||
|
@ -205,9 +215,11 @@ public class IQv2IntegrationTest {
|
||||||
assertThrows(StreamsStoppedException.class, () -> kafkaStreams.query(request));
|
assertThrows(StreamsStoppedException.class, () -> kafkaStreams.query(request));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldRejectNonRunningActive()
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldRejectNonRunningActive(final String groupProtocol, final String testName, final TestInfo testInfo)
|
||||||
throws NoSuchFieldException, IllegalAccessException {
|
throws NoSuchFieldException, IllegalAccessException {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||||
inStore(STORE_NAME).withQuery(query).requireActive();
|
inStore(STORE_NAME).withQuery(query).requireActive();
|
||||||
|
@ -261,8 +273,10 @@ public class IQv2IntegrationTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldFetchFromPartition() {
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldFetchFromPartition(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final int partition = 1;
|
final int partition = 1;
|
||||||
final Set<Integer> partitions = singleton(partition);
|
final Set<Integer> partitions = singleton(partition);
|
||||||
|
@ -276,8 +290,10 @@ public class IQv2IntegrationTest {
|
||||||
assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
|
assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldFetchExplicitlyFromAllPartitions() {
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldFetchExplicitlyFromAllPartitions(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final Set<Integer> partitions = Set.of(0, 1);
|
final Set<Integer> partitions = Set.of(0, 1);
|
||||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||||
|
@ -290,8 +306,10 @@ public class IQv2IntegrationTest {
|
||||||
assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
|
assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@ParameterizedTest(name = "{1}")
|
||||||
public void shouldNotRequireQueryHandler(final TestInfo testInfo) {
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void shouldNotRequireQueryHandler(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||||
final int partition = 1;
|
final int partition = 1;
|
||||||
final Set<Integer> partitions = singleton(partition);
|
final Set<Integer> partitions = singleton(partition);
|
||||||
|
@ -423,8 +441,11 @@ public class IQv2IntegrationTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
// Discard the basic streams and replace with test-specific topology
|
// Discard the basic streams and replace with test-specific topology
|
||||||
|
if (kafkaStreams != null) {
|
||||||
kafkaStreams.close();
|
kafkaStreams.close();
|
||||||
|
}
|
||||||
final String safeTestName = safeUniqueTestName(testInfo);
|
final String safeTestName = safeUniqueTestName(testInfo);
|
||||||
|
this.groupProtocol = groupProtocol;
|
||||||
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
|
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
|
||||||
kafkaStreams.cleanUp();
|
kafkaStreams.cleanUp();
|
||||||
|
|
||||||
|
@ -446,7 +467,7 @@ public class IQv2IntegrationTest {
|
||||||
private Properties streamsConfiguration(final String safeTestName) {
|
private Properties streamsConfiguration(final String safeTestName) {
|
||||||
final Properties config = new Properties();
|
final Properties config = new Properties();
|
||||||
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
||||||
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName + "-" + groupProtocol);
|
||||||
config.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + (++port));
|
config.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + (++port));
|
||||||
config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||||
|
@ -458,6 +479,14 @@ public class IQv2IntegrationTest {
|
||||||
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||||
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
|
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
|
||||||
|
config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static Stream<Arguments> groupProtocolParameters() {
|
||||||
|
return Stream.of(
|
||||||
|
Arguments.of("classic", "CLASSIC protocol"),
|
||||||
|
Arguments.of("streams", "STREAMS protocol")
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -360,7 +360,9 @@ public class IQv2StoreIntegrationTest {
|
||||||
for (final boolean logEnabled : Arrays.asList(true, false)) {
|
for (final boolean logEnabled : Arrays.asList(true, false)) {
|
||||||
for (final StoresToTest toTest : StoresToTest.values()) {
|
for (final StoresToTest toTest : StoresToTest.values()) {
|
||||||
for (final String kind : Arrays.asList("DSL", "PAPI")) {
|
for (final String kind : Arrays.asList("DSL", "PAPI")) {
|
||||||
values.add(Arguments.of(cacheEnabled, logEnabled, toTest.name(), kind));
|
for (final String groupProtocol : Arrays.asList("classic", "streams")) {
|
||||||
|
values.add(Arguments.of(cacheEnabled, logEnabled, toTest.name(), kind, groupProtocol));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -426,13 +428,14 @@ public class IQv2StoreIntegrationTest {
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setup(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) {
|
public void setup(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) {
|
||||||
final StoreSupplier<?> supplier = storeToTest.supplier();
|
final StoreSupplier<?> supplier = storeToTest.supplier();
|
||||||
final Properties streamsConfig = streamsConfiguration(
|
final Properties streamsConfig = streamsConfiguration(
|
||||||
cache,
|
cache,
|
||||||
log,
|
log,
|
||||||
storeToTest.name(),
|
storeToTest.name(),
|
||||||
kind
|
kind,
|
||||||
|
groupProtocol
|
||||||
);
|
);
|
||||||
|
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
final StreamsBuilder builder = new StreamsBuilder();
|
||||||
|
@ -765,8 +768,8 @@ public class IQv2StoreIntegrationTest {
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("data")
|
@MethodSource("data")
|
||||||
public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) {
|
public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) {
|
||||||
setup(cache, log, storeToTest, kind);
|
setup(cache, log, storeToTest, kind, groupProtocol);
|
||||||
try {
|
try {
|
||||||
if (storeToTest.global()) {
|
if (storeToTest.global()) {
|
||||||
// See KAFKA-13523
|
// See KAFKA-13523
|
||||||
|
@ -2030,10 +2033,10 @@ public class IQv2StoreIntegrationTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Properties streamsConfiguration(final boolean cache, final boolean log,
|
private static Properties streamsConfiguration(final boolean cache, final boolean log,
|
||||||
final String supplier, final String kind) {
|
final String supplier, final String kind, final String groupProtocol) {
|
||||||
final String safeTestName =
|
final String safeTestName =
|
||||||
IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier
|
IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier
|
||||||
+ "-" + kind + "-" + RANDOM.nextInt();
|
+ "-" + kind + "-" + groupProtocol + "-" + RANDOM.nextInt();
|
||||||
final Properties config = new Properties();
|
final Properties config = new Properties();
|
||||||
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
||||||
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||||
|
@ -2048,6 +2051,7 @@ public class IQv2StoreIntegrationTest {
|
||||||
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||||
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||||
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
|
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
|
||||||
|
config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -47,7 +47,10 @@ import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.BeforeAll;
|
import org.junit.jupiter.api.BeforeAll;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Tag;
|
import org.junit.jupiter.api.Tag;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.TestInfo;
|
||||||
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
|
import org.junit.jupiter.params.provider.Arguments;
|
||||||
|
import org.junit.jupiter.params.provider.MethodSource;
|
||||||
|
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import java.time.Instant;
|
import java.time.Instant;
|
||||||
|
@ -57,7 +60,9 @@ import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName;
|
||||||
import static org.hamcrest.CoreMatchers.nullValue;
|
import static org.hamcrest.CoreMatchers.nullValue;
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
import static org.hamcrest.MatcherAssert.assertThat;
|
||||||
import static org.hamcrest.Matchers.empty;
|
import static org.hamcrest.Matchers.empty;
|
||||||
|
@ -83,16 +88,25 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
private static final Long[] RECORD_TIMESTAMPS = {BASE_TIMESTAMP_LONG, BASE_TIMESTAMP_LONG + 10, BASE_TIMESTAMP_LONG + 20, BASE_TIMESTAMP_LONG + 30};
|
private static final Long[] RECORD_TIMESTAMPS = {BASE_TIMESTAMP_LONG, BASE_TIMESTAMP_LONG + 10, BASE_TIMESTAMP_LONG + 20, BASE_TIMESTAMP_LONG + 30};
|
||||||
private static final int RECORD_NUMBER = RECORD_VALUES.length;
|
private static final int RECORD_NUMBER = RECORD_VALUES.length;
|
||||||
private static final int LAST_INDEX = RECORD_NUMBER - 1;
|
private static final int LAST_INDEX = RECORD_NUMBER - 1;
|
||||||
private static final Position INPUT_POSITION = Position.emptyPosition();
|
private Position inputPosition;
|
||||||
|
|
||||||
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, Utils.mkProperties(Collections.singletonMap("auto.create.topics.enable", "true")));
|
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, Utils.mkProperties(Collections.singletonMap("auto.create.topics.enable", "true")));
|
||||||
|
|
||||||
private KafkaStreams kafkaStreams;
|
private KafkaStreams kafkaStreams;
|
||||||
|
private String groupProtocol;
|
||||||
|
|
||||||
@BeforeAll
|
@BeforeAll
|
||||||
public static void before() throws Exception {
|
public static void beforeAll() throws Exception {
|
||||||
CLUSTER.start();
|
CLUSTER.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void beforeEach() throws Exception {
|
||||||
|
// Delete and recreate the topic to ensure clean state for each test
|
||||||
|
CLUSTER.deleteTopic(INPUT_TOPIC_NAME);
|
||||||
|
CLUSTER.createTopic(INPUT_TOPIC_NAME, 1, 1);
|
||||||
|
|
||||||
|
// Set up fresh test data
|
||||||
final Properties producerProps = new Properties();
|
final Properties producerProps = new Properties();
|
||||||
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
|
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
|
||||||
|
@ -103,19 +117,21 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[2], RECORD_KEY, RECORD_VALUES[2])).get();
|
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[2], RECORD_KEY, RECORD_VALUES[2])).get();
|
||||||
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[3], RECORD_KEY, RECORD_VALUES[3])).get();
|
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[3], RECORD_KEY, RECORD_VALUES[3])).get();
|
||||||
}
|
}
|
||||||
INPUT_POSITION.withComponent(INPUT_TOPIC_NAME, 0, 3);
|
inputPosition = Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 3);
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeEach
|
private void setup(final String groupProtocol, final TestInfo testInfo) {
|
||||||
public void beforeTest() {
|
this.groupProtocol = groupProtocol;
|
||||||
final StreamsBuilder builder = new StreamsBuilder();
|
final StreamsBuilder builder = new StreamsBuilder();
|
||||||
builder.table(INPUT_TOPIC_NAME,
|
builder.table(INPUT_TOPIC_NAME,
|
||||||
Materialized.as(Stores.persistentVersionedKeyValueStore(STORE_NAME, HISTORY_RETENTION, SEGMENT_INTERVAL)));
|
Materialized.as(Stores.persistentVersionedKeyValueStore(STORE_NAME, HISTORY_RETENTION, SEGMENT_INTERVAL)));
|
||||||
final Properties configs = new Properties();
|
final Properties configs = new Properties();
|
||||||
configs.put(StreamsConfig.APPLICATION_ID_CONFIG, "app");
|
final String safeTestName = safeUniqueTestName(testInfo);
|
||||||
|
configs.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||||
configs.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
configs.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
configs.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
|
configs.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
|
||||||
configs.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
|
configs.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
|
||||||
|
configs.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||||
kafkaStreams = IntegrationTestUtils.getStartedStreams(configs, builder, true);
|
kafkaStreams = IntegrationTestUtils.getStartedStreams(configs, builder, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -132,8 +148,19 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
CLUSTER.stop();
|
CLUSTER.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
private static Stream<Arguments> groupProtocolParameters() {
|
||||||
public void verifyStore() {
|
return Stream.of(
|
||||||
|
Arguments.of("classic", "CLASSIC protocol"),
|
||||||
|
Arguments.of("streams", "STREAMS protocol")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest(name = "{1}")
|
||||||
|
@MethodSource("groupProtocolParameters")
|
||||||
|
public void verifyStore(final String groupProtocol, final String testName, final TestInfo testInfo) throws Exception {
|
||||||
|
// Set up streams
|
||||||
|
setup(groupProtocol, testInfo);
|
||||||
|
|
||||||
/* Test Versioned Key Queries */
|
/* Test Versioned Key Queries */
|
||||||
// retrieve the latest value
|
// retrieve the latest value
|
||||||
shouldHandleVersionedKeyQuery(Optional.empty(), RECORD_VALUES[3], RECORD_TIMESTAMPS[3], Optional.empty());
|
shouldHandleVersionedKeyQuery(Optional.empty(), RECORD_VALUES[3], RECORD_TIMESTAMPS[3], Optional.empty());
|
||||||
|
@ -255,7 +282,10 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
private void shouldHandleRaceCondition() {
|
private void shouldHandleRaceCondition() {
|
||||||
final MultiVersionedKeyQuery<Integer, Integer> query = defineQuery(RECORD_KEY, Optional.empty(), Optional.empty(), ResultOrder.ANY);
|
final MultiVersionedKeyQuery<Integer, Integer> query = defineQuery(RECORD_KEY, Optional.empty(), Optional.empty(), ResultOrder.ANY);
|
||||||
|
|
||||||
final Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResults = sendRequestAndReceiveResults(query, kafkaStreams);
|
// For race condition test, we don't use position bounds since we're testing concurrent updates
|
||||||
|
final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query);
|
||||||
|
final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||||
|
final Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResults = result.getPartitionResults();
|
||||||
|
|
||||||
// verify results in two steps
|
// verify results in two steps
|
||||||
for (final Entry<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResultsEntry : partitionResults.entrySet()) {
|
for (final Entry<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResultsEntry : partitionResults.entrySet()) {
|
||||||
|
@ -327,14 +357,14 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
return query;
|
return query;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
private Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
||||||
final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(INPUT_POSITION));
|
final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition));
|
||||||
final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||||
return result.getPartitionResults();
|
return result.getPartitionResults();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static QueryResult<VersionedRecord<Integer>> sendRequestAndReceiveResults(final VersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
private QueryResult<VersionedRecord<Integer>> sendRequestAndReceiveResults(final VersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
||||||
final StateQueryRequest<VersionedRecord<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(INPUT_POSITION));
|
final StateQueryRequest<VersionedRecord<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition));
|
||||||
final StateQueryResult<VersionedRecord<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
final StateQueryResult<VersionedRecord<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||||
return result.getOnlyPartitionResult();
|
return result.getOnlyPartitionResult();
|
||||||
}
|
}
|
||||||
|
@ -352,7 +382,7 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
/**
|
/**
|
||||||
* This method inserts a new value (999999) for the key in the oldest timestamp (RECORD_TIMESTAMPS[0]).
|
* This method inserts a new value (999999) for the key in the oldest timestamp (RECORD_TIMESTAMPS[0]).
|
||||||
*/
|
*/
|
||||||
private static void updateRecordValue() {
|
private void updateRecordValue() {
|
||||||
// update the record value at RECORD_TIMESTAMPS[0]
|
// update the record value at RECORD_TIMESTAMPS[0]
|
||||||
final Properties producerProps = new Properties();
|
final Properties producerProps = new Properties();
|
||||||
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||||
|
@ -361,8 +391,9 @@ public class IQv2VersionedStoreIntegrationTest {
|
||||||
try (final KafkaProducer<Integer, Integer> producer = new KafkaProducer<>(producerProps)) {
|
try (final KafkaProducer<Integer, Integer> producer = new KafkaProducer<>(producerProps)) {
|
||||||
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[0], RECORD_KEY, 999999));
|
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[0], RECORD_KEY, 999999));
|
||||||
}
|
}
|
||||||
INPUT_POSITION.withComponent(INPUT_TOPIC_NAME, 0, 4);
|
|
||||||
assertThat(INPUT_POSITION, equalTo(Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 4)));
|
inputPosition = inputPosition.withComponent(INPUT_TOPIC_NAME, 0, 4);
|
||||||
|
assertThat(inputPosition, equalTo(Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 4)));
|
||||||
|
|
||||||
// make sure that the new value is picked up by the store
|
// make sure that the new value is picked up by the store
|
||||||
final Properties consumerProps = new Properties();
|
final Properties consumerProps = new Properties();
|
||||||
|
|
Loading…
Reference in New Issue