Skip to content
Merged
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 @@ -29,17 +29,14 @@
import org.apache.kafka.streams.TestInputTopic;
import org.apache.kafka.streams.TestOutputTopic;
import org.apache.kafka.streams.TopologyTestDriver;
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.kstream.ValueJoiner;
import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.apache.kafka.streams.test.TestRecord;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
Expand All @@ -66,9 +63,6 @@
@Category({IntegrationTest.class})
@RunWith(value = Parameterized.class)
public abstract class AbstractJoinIntegrationTest {
@ClassRule
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

@Rule
public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory());

Expand Down Expand Up @@ -122,26 +116,20 @@ public static Collection<Object[]> data() {
@BeforeClass
public static void setupConfigsAndUtils() {

STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "topology_driver:0000");
STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
Copy link
Copy Markdown
Member

@bbejeck bbejeck Aug 12, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@albert02lowis I've confirmed locally that the test failures are related. The TopologyTestDriver still needs a bootstrap servers config value.
Adding something like STREAMS_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "test:0000"); into the AbstractJoinIntegrationTest.setupConfigsAndUtils() gets all tests passing.

I'm not sure how the JDK 14 tests passed as locally I got the same errors as the JDK 8 and 11 build locally.

Copy link
Copy Markdown
Contributor Author

@albert02lowis albert02lowis Aug 13, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, let me add back this BOOTSTRAP_SERVERS_CONFIG inside setupConfigsAndUtils then 👍🏻

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @bbejeck I have done the required change in a new commit. I have also extracted out redundant calls to set BOOTSTRAP_SERVERS_CONFIG in the subclasses' test methods.

STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL);
}

void prepareEnvironment() throws InterruptedException {
CLUSTER.createTopics(INPUT_TOPIC_LEFT, INPUT_TOPIC_RIGHT, OUTPUT_TOPIC);

if (!cacheEnabled) {
STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
}

STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());
}
@After
public void cleanup() throws InterruptedException {
CLUSTER.deleteAllTopicsAndWait(120000);
}

void runTestWithDriver(final List<List<TestRecord<Long, String>>> expectedResult) {
runTestWithDriver(expectedResult, null);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/*
* 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.streams.integration;

import java.util.Properties;
import java.util.concurrent.CountDownLatch;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StoreQueryParameters;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.JoinWindows;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.StreamJoined;
import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;

import static java.time.Duration.ofMillis;
import static org.junit.Assert.assertThrows;

@Category({IntegrationTest.class})
public class JoinStoreIntegrationTest {
@ClassRule
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

@Rule
public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory());

private static final String APP_ID = "join-store-integration-test";
private static final Long COMMIT_INTERVAL = 100L;
static final Properties STREAMS_CONFIG = new Properties();
static final String INPUT_TOPIC_RIGHT = "inputTopicRight";
static final String INPUT_TOPIC_LEFT = "inputTopicLeft";
static final String OUTPUT_TOPIC = "outputTopic";

StreamsBuilder builder;

@BeforeClass
public static void setupConfigsAndUtils() {
STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL);
}

@Before
public void prepareTopology() throws InterruptedException {
CLUSTER.createTopics(INPUT_TOPIC_LEFT, INPUT_TOPIC_RIGHT, OUTPUT_TOPIC);
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());

builder = new StreamsBuilder();
}

@After
public void cleanup() throws InterruptedException {
CLUSTER.deleteAllTopicsAndWait(120000);
}

@Test
public void shouldNotAccessJoinStoresWhenGivingName() throws InterruptedException {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-no-store-access");
final StreamsBuilder builder = new StreamsBuilder();

final KStream<String, Integer> left = builder.stream(INPUT_TOPIC_LEFT, Consumed.with(Serdes.String(), Serdes.Integer()));
final KStream<String, Integer> right = builder.stream(INPUT_TOPIC_RIGHT, Consumed.with(Serdes.String(), Serdes.Integer()));
final CountDownLatch latch = new CountDownLatch(1);

left.join(
right,
(value1, value2) -> value1 + value2,
JoinWindows.of(ofMillis(100)),
StreamJoined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer()).withStoreName("join-store"));

try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), STREAMS_CONFIG)) {
kafkaStreams.setStateListener((newState, oldState) -> {
if (newState == KafkaStreams.State.RUNNING) {
latch.countDown();
}
});

kafkaStreams.start();
latch.await();
assertThrows(InvalidStateStoreException.class, () -> kafkaStreams.store(StoreQueryParameters.fromNameAndType("join-store", QueryableStoreTypes.keyValueStore())));
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.streams.integration;

import java.util.Properties;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreamsWrapper;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.ValueJoiner;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;

import static org.junit.Assert.assertTrue;

@Category({IntegrationTest.class})
public class JoinWithIncompleteMetadataIntegrationTest {
@ClassRule
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

@Rule
public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory());

private static final String APP_ID = "join-incomplete-metadata-integration-test";
private static final Long COMMIT_INTERVAL = 100L;
static final Properties STREAMS_CONFIG = new Properties();
static final String INPUT_TOPIC_RIGHT = "inputTopicRight";
static final String NON_EXISTENT_INPUT_TOPIC_LEFT = "inputTopicLeft-not-exist";
static final String OUTPUT_TOPIC = "outputTopic";

StreamsBuilder builder;
final ValueJoiner<String, String, String> valueJoiner = (value1, value2) -> value1 + "-" + value2;
private KTable<Long, String> rightTable;

@BeforeClass
public static void setupConfigsAndUtils() {
STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL);
}

@Before
public void prepareTopology() throws InterruptedException {
CLUSTER.createTopics(INPUT_TOPIC_RIGHT, OUTPUT_TOPIC);
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath());

builder = new StreamsBuilder();
rightTable = builder.table(INPUT_TOPIC_RIGHT);
}

@After
public void cleanup() throws InterruptedException {
CLUSTER.deleteAllTopicsAndWait(120000);
}

@Test
public void testShouldAutoShutdownOnJoinWithIncompleteMetadata() throws InterruptedException {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID);
STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());

final KStream<Long, String> notExistStream = builder.stream(NON_EXISTENT_INPUT_TOPIC_LEFT);

final KTable<Long, String> aggregatedTable = notExistStream.leftJoin(rightTable, valueJoiner)
.groupBy((key, value) -> key)
.reduce((value1, value2) -> value1 + value2);

// Write the (continuously updating) results to the output topic.
aggregatedTable.toStream().to(OUTPUT_TOPIC);

final KafkaStreamsWrapper streams = new KafkaStreamsWrapper(builder.build(), STREAMS_CONFIG);
final IntegrationTestUtils.StateListenerStub listener = new IntegrationTestUtils.StateListenerStub();
streams.setStreamThreadStateListener(listener);
streams.start();

TestUtils.waitForCondition(listener::transitToPendingShutdownSeen, "Did not seen thread state transited to PENDING_SHUTDOWN");

streams.close();
assertTrue(listener.transitToPendingShutdownSeen());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,17 +16,10 @@
*/
package org.apache.kafka.streams.integration;

import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StoreQueryParameters;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.JoinWindows;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.StreamJoined;
import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.test.TestRecord;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockMapper;
Expand All @@ -40,11 +33,8 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;

import static java.time.Duration.ofMillis;
import static java.time.Duration.ofSeconds;
import static org.junit.Assert.assertThrows;

/**
* Tests all available joins of Kafka Streams DSL.
Expand All @@ -70,34 +60,6 @@ public void prepareTopology() throws InterruptedException {
rightStream = builder.stream(INPUT_TOPIC_RIGHT);
}

@Test
public void shouldNotAccessJoinStoresWhenGivingName() throws InterruptedException {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-no-store-access");
final StreamsBuilder builder = new StreamsBuilder();

final KStream<String, Integer> left = builder.stream(INPUT_TOPIC_LEFT, Consumed.with(Serdes.String(), Serdes.Integer()));
final KStream<String, Integer> right = builder.stream(INPUT_TOPIC_RIGHT, Consumed.with(Serdes.String(), Serdes.Integer()));
final CountDownLatch latch = new CountDownLatch(1);

left.join(
right,
(value1, value2) -> value1 + value2,
JoinWindows.of(ofMillis(100)),
StreamJoined.with(Serdes.String(), Serdes.Integer(), Serdes.Integer()).withStoreName("join-store"));

try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), STREAMS_CONFIG)) {
kafkaStreams.setStateListener((newState, oldState) -> {
if (newState == KafkaStreams.State.RUNNING) {
latch.countDown();
}
});

kafkaStreams.start();
latch.await();
assertThrows(InvalidStateStoreException.class, () -> kafkaStreams.store(StoreQueryParameters.fromNameAndType("join-store", QueryableStoreTypes.keyValueStore())));
}
}

@Test
public void testInner() {
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner");
Expand Down
Loading