|
23 | 23 | import org.apache.kafka.common.serialization.Serdes;
|
24 | 24 | import org.apache.kafka.common.serialization.StringDeserializer;
|
25 | 25 | import org.apache.kafka.common.serialization.StringSerializer;
|
| 26 | +import org.apache.kafka.common.utils.LogCaptureAppender; |
| 27 | +import org.apache.kafka.common.utils.MockTime; |
26 | 28 | import org.apache.kafka.streams.KafkaStreams;
|
27 | 29 | import org.apache.kafka.streams.KeyValue;
|
28 | 30 | import org.apache.kafka.streams.StreamsBuilder;
|
|
39 | 41 | import org.apache.kafka.streams.processor.api.Processor;
|
40 | 42 | import org.apache.kafka.streams.processor.api.ProcessorContext;
|
41 | 43 | import org.apache.kafka.streams.processor.api.Record;
|
| 44 | +import org.apache.kafka.streams.processor.internals.StreamThread; |
42 | 45 | import org.apache.kafka.streams.state.Stores;
|
43 | 46 | import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
|
44 | 47 | import org.apache.kafka.test.TestUtils;
|
45 | 48 |
|
| 49 | +import org.apache.logging.log4j.Level; |
46 | 50 | import org.junit.jupiter.api.AfterAll;
|
47 | 51 | import org.junit.jupiter.api.AfterEach;
|
48 | 52 | import org.junit.jupiter.api.BeforeAll;
|
@@ -331,18 +335,23 @@ private void testShutdownApplication(final int numThreads) throws Exception {
|
331 | 335 | properties.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, numThreads);
|
332 | 336 |
|
333 | 337 | final Topology topology = builder.build();
|
334 |
| - |
335 |
| - try (final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, properties); |
336 |
| - final KafkaStreams kafkaStreams2 = new KafkaStreams(topology, properties)) { |
| 338 | + final MockTime time = new MockTime(0L); |
| 339 | + |
| 340 | + try (final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, properties, time); |
| 341 | + final KafkaStreams kafkaStreams2 = new KafkaStreams(topology, properties, time); |
| 342 | + final LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister()) { |
337 | 343 | kafkaStreams1.setUncaughtExceptionHandler(exception -> SHUTDOWN_APPLICATION);
|
338 | 344 | kafkaStreams2.setUncaughtExceptionHandler(exception -> SHUTDOWN_APPLICATION);
|
| 345 | + logCaptureAppender.setClassLogger(StreamThread.class, Level.WARN); |
339 | 346 |
|
340 | 347 | startApplicationAndWaitUntilRunning(asList(kafkaStreams1, kafkaStreams2));
|
341 | 348 |
|
342 | 349 | produceMessages(NOW, inputTopic, "A");
|
343 | 350 | waitForApplicationState(asList(kafkaStreams1, kafkaStreams2), KafkaStreams.State.ERROR, DEFAULT_DURATION);
|
344 | 351 |
|
345 | 352 | assertThat(processorValueCollector.size(), equalTo(1));
|
| 353 | + assertThat("Shutdown warning log message should be exported exactly once", |
| 354 | + logCaptureAppender.getMessages("WARN").stream().filter(msg -> msg.contains("Detected that shutdown was requested")).count(), equalTo(1L)); |
346 | 355 | }
|
347 | 356 | }
|
348 | 357 |
|
|
0 commit comments