|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.kafka.streams.integration; |
| 19 | + |
| 20 | +import org.apache.kafka.common.TopicPartition; |
| 21 | +import org.apache.kafka.common.serialization.Serdes; |
| 22 | +import org.apache.kafka.common.serialization.StringSerializer; |
| 23 | +import org.apache.kafka.streams.KafkaStreams; |
| 24 | +import org.apache.kafka.streams.KeyValue; |
| 25 | +import org.apache.kafka.streams.StreamsBuilder; |
| 26 | +import org.apache.kafka.streams.StreamsConfig; |
| 27 | +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; |
| 28 | +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; |
| 29 | +import org.apache.kafka.streams.kstream.KStream; |
| 30 | +import org.apache.kafka.streams.processor.AbstractProcessor; |
| 31 | +import org.apache.kafka.streams.processor.TaskMetadata; |
| 32 | +import org.apache.kafka.test.IntegrationTest; |
| 33 | +import org.apache.kafka.test.TestUtils; |
| 34 | +import org.junit.After; |
| 35 | +import org.junit.AfterClass; |
| 36 | +import org.junit.Before; |
| 37 | +import org.junit.BeforeClass; |
| 38 | +import org.junit.Rule; |
| 39 | +import org.junit.Test; |
| 40 | +import org.junit.experimental.categories.Category; |
| 41 | +import org.junit.rules.TestName; |
| 42 | + |
| 43 | +import java.io.IOException; |
| 44 | +import java.time.Duration; |
| 45 | +import java.util.Collections; |
| 46 | +import java.util.List; |
| 47 | +import java.util.Properties; |
| 48 | +import java.util.concurrent.atomic.AtomicBoolean; |
| 49 | +import java.util.stream.Collectors; |
| 50 | + |
| 51 | +import static org.apache.kafka.common.utils.Utils.mkEntry; |
| 52 | +import static org.apache.kafka.common.utils.Utils.mkMap; |
| 53 | +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; |
| 54 | +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; |
| 55 | +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; |
| 56 | +import static org.hamcrest.CoreMatchers.equalTo; |
| 57 | +import static org.hamcrest.MatcherAssert.assertThat; |
| 58 | + |
| 59 | +@Category(IntegrationTest.class) |
| 60 | +public class TaskMetadataIntegrationTest { |
| 61 | + |
| 62 | + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1, new Properties(), 0L, 0L); |
| 63 | + |
| 64 | + @BeforeClass |
| 65 | + public static void startCluster() throws IOException { |
| 66 | + CLUSTER.start(); |
| 67 | + } |
| 68 | + |
| 69 | + @AfterClass |
| 70 | + public static void closeCluster() { |
| 71 | + CLUSTER.stop(); |
| 72 | + } |
| 73 | + public static final Duration DEFAULT_DURATION = Duration.ofSeconds(30); |
| 74 | + |
| 75 | + @Rule |
| 76 | + public TestName testName = new TestName(); |
| 77 | + |
| 78 | + private String inputTopic; |
| 79 | + private static StreamsBuilder builder; |
| 80 | + private static Properties properties; |
| 81 | + private static String appId = "TaskMetadataTest_"; |
| 82 | + private AtomicBoolean process; |
| 83 | + private AtomicBoolean commit; |
| 84 | + |
| 85 | + @Before |
| 86 | + public void setup() { |
| 87 | + final String testId = safeUniqueTestName(getClass(), testName); |
| 88 | + appId = appId + testId; |
| 89 | + inputTopic = "input" + testId; |
| 90 | + IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic); |
| 91 | + |
| 92 | + builder = new StreamsBuilder(); |
| 93 | + |
| 94 | + process = new AtomicBoolean(true); |
| 95 | + commit = new AtomicBoolean(true); |
| 96 | + |
| 97 | + final KStream<String, String> stream = builder.stream(inputTopic); |
| 98 | + stream.process(PauseProcessor::new); |
| 99 | + |
| 100 | + properties = mkObjectProperties( |
| 101 | + mkMap( |
| 102 | + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), |
| 103 | + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), |
| 104 | + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()), |
| 105 | + mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2), |
| 106 | + mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), |
| 107 | + mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), |
| 108 | + mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1L) |
| 109 | + ) |
| 110 | + ); |
| 111 | + } |
| 112 | + |
| 113 | + @Test |
| 114 | + public void shouldReportCorrectCommittedOffsetInformation() { |
| 115 | + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { |
| 116 | + IntegrationTestUtils.startApplicationAndWaitUntilRunning(Collections.singletonList(kafkaStreams), DEFAULT_DURATION); |
| 117 | + final TaskMetadata taskMetadata = getTaskMetadata(kafkaStreams); |
| 118 | + assertThat(taskMetadata.committedOffsets().size(), equalTo(1)); |
| 119 | + final TopicPartition topicPartition = new TopicPartition(inputTopic, 0); |
| 120 | + |
| 121 | + produceMessages(0L, inputTopic, "test"); |
| 122 | + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); |
| 123 | + TestUtils.waitForCondition(() -> taskMetadata.committedOffsets().get(topicPartition) == 1L, "the record was processed"); |
| 124 | + process.set(true); |
| 125 | + |
| 126 | + produceMessages(0L, inputTopic, "test1"); |
| 127 | + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); |
| 128 | + TestUtils.waitForCondition(() -> taskMetadata.committedOffsets().get(topicPartition) == 2L, "the record was processed"); |
| 129 | + process.set(true); |
| 130 | + |
| 131 | + produceMessages(0L, inputTopic, "test1"); |
| 132 | + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); |
| 133 | + TestUtils.waitForCondition(() -> taskMetadata.committedOffsets().get(topicPartition) == 3L, "the record was processed"); |
| 134 | + } catch (final Exception e) { |
| 135 | + e.printStackTrace(); |
| 136 | + } |
| 137 | + } |
| 138 | + |
| 139 | + @Test |
| 140 | + public void shouldReportCorrectEndOffsetInformation() { |
| 141 | + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { |
| 142 | + IntegrationTestUtils.startApplicationAndWaitUntilRunning(Collections.singletonList(kafkaStreams), DEFAULT_DURATION); |
| 143 | + final TaskMetadata taskMetadata = getTaskMetadata(kafkaStreams); |
| 144 | + assertThat(taskMetadata.endOffsets().size(), equalTo(1)); |
| 145 | + final TopicPartition topicPartition = new TopicPartition(inputTopic, 0); |
| 146 | + commit.set(false); |
| 147 | + |
| 148 | + for (int i = 0; i < 10; i++) { |
| 149 | + produceMessages(0L, inputTopic, "test"); |
| 150 | + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); |
| 151 | + process.set(true); |
| 152 | + } |
| 153 | + assertThat(taskMetadata.endOffsets().get(topicPartition), equalTo(9L)); |
| 154 | + |
| 155 | + } catch (final Exception e) { |
| 156 | + e.printStackTrace(); |
| 157 | + } |
| 158 | + } |
| 159 | + |
| 160 | + private TaskMetadata getTaskMetadata(final KafkaStreams kafkaStreams) { |
| 161 | + final List<TaskMetadata> taskMetadataList = kafkaStreams.localThreadsMetadata().stream().flatMap(t -> t.activeTasks().stream()).collect(Collectors.toList()); |
| 162 | + assertThat("only one task", taskMetadataList.size() == 1); |
| 163 | + return taskMetadataList.get(0); |
| 164 | + } |
| 165 | + |
| 166 | + @After |
| 167 | + public void teardown() throws IOException { |
| 168 | + purgeLocalStreamsState(properties); |
| 169 | + } |
| 170 | + |
| 171 | + private void produceMessages(final long timestamp, final String streamOneInput, final String msg) { |
| 172 | + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( |
| 173 | + streamOneInput, |
| 174 | + Collections.singletonList(new KeyValue<>("1", msg)), |
| 175 | + TestUtils.producerConfig( |
| 176 | + CLUSTER.bootstrapServers(), |
| 177 | + StringSerializer.class, |
| 178 | + StringSerializer.class, |
| 179 | + new Properties()), |
| 180 | + timestamp); |
| 181 | + } |
| 182 | + |
| 183 | + private class PauseProcessor extends AbstractProcessor<String, String> { |
| 184 | + @Override |
| 185 | + public void process(final String key, final String value) { |
| 186 | + while (!process.get()) { |
| 187 | + try { |
| 188 | + wait(100); |
| 189 | + } catch (final InterruptedException e) { |
| 190 | + |
| 191 | + } |
| 192 | + } |
| 193 | + context().forward(key, value); |
| 194 | + if (commit.get()) { |
| 195 | + context().commit(); |
| 196 | + } |
| 197 | + process.set(false); |
| 198 | + } |
| 199 | + } |
| 200 | +} |
0 commit comments