|
|
|
@ -20,6 +20,7 @@ import org.apache.kafka.clients.admin.MockAdminClient;
@@ -20,6 +20,7 @@ import org.apache.kafka.clients.admin.MockAdminClient;
|
|
|
|
|
import org.apache.kafka.clients.consumer.Consumer; |
|
|
|
|
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; |
|
|
|
|
import org.apache.kafka.clients.consumer.ConsumerRecord; |
|
|
|
|
import org.apache.kafka.clients.consumer.ConsumerRecords; |
|
|
|
|
import org.apache.kafka.clients.consumer.InvalidOffsetException; |
|
|
|
|
import org.apache.kafka.clients.consumer.MockConsumer; |
|
|
|
|
import org.apache.kafka.clients.consumer.OffsetResetStrategy; |
|
|
|
@ -78,6 +79,7 @@ import org.slf4j.Logger;
@@ -78,6 +79,7 @@ import org.slf4j.Logger;
|
|
|
|
|
import java.io.File; |
|
|
|
|
import java.time.Duration; |
|
|
|
|
import java.util.ArrayList; |
|
|
|
|
import java.util.Collection; |
|
|
|
|
import java.util.Collections; |
|
|
|
|
import java.util.HashMap; |
|
|
|
|
import java.util.HashSet; |
|
|
|
@ -701,6 +703,85 @@ public class StreamThreadTest {
@@ -701,6 +703,85 @@ public class StreamThreadTest {
|
|
|
|
|
EasyMock.verify(taskManager); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
public void shouldNotThrowWhenPendingShutdownInRunOnce() { |
|
|
|
|
mockRunOnce(true); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
public void shouldNotThrowWithoutPendingShutdownInRunOnce() { |
|
|
|
|
// A reference test to verify that without intermediate shutdown the runOnce should pass
|
|
|
|
|
// without any exception.
|
|
|
|
|
mockRunOnce(false); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private void mockRunOnce(final boolean shutdownOnPoll) { |
|
|
|
|
final Collection<TopicPartition> assignedPartitions = Collections.singletonList(t1p1); |
|
|
|
|
class MockStreamThreadConsumer<K, V> extends MockConsumer<K, V> { |
|
|
|
|
|
|
|
|
|
private StreamThread streamThread; |
|
|
|
|
|
|
|
|
|
private MockStreamThreadConsumer(final OffsetResetStrategy offsetResetStrategy) { |
|
|
|
|
super(offsetResetStrategy); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Override |
|
|
|
|
public synchronized ConsumerRecords<K, V> poll(final Duration timeout) { |
|
|
|
|
assertNotNull(streamThread); |
|
|
|
|
if (shutdownOnPoll) { |
|
|
|
|
streamThread.shutdown(); |
|
|
|
|
} |
|
|
|
|
streamThread.rebalanceListener.onPartitionsAssigned(assignedPartitions); |
|
|
|
|
return super.poll(timeout); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private void setStreamThread(final StreamThread streamThread) { |
|
|
|
|
this.streamThread = streamThread; |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
final MockStreamThreadConsumer<byte[], byte[]> mockStreamThreadConsumer = |
|
|
|
|
new MockStreamThreadConsumer<>(OffsetResetStrategy.EARLIEST); |
|
|
|
|
|
|
|
|
|
final TaskManager taskManager = new TaskManager(new MockChangelogReader(), |
|
|
|
|
processId, |
|
|
|
|
"log-prefix", |
|
|
|
|
mockStreamThreadConsumer, |
|
|
|
|
streamsMetadataState, |
|
|
|
|
null, |
|
|
|
|
null, |
|
|
|
|
null, |
|
|
|
|
new AssignedStreamsTasks(new LogContext()), |
|
|
|
|
new AssignedStandbyTasks(new LogContext())); |
|
|
|
|
taskManager.setConsumer(mockStreamThreadConsumer); |
|
|
|
|
taskManager.setAssignmentMetadata(Collections.emptyMap(), Collections.emptyMap()); |
|
|
|
|
|
|
|
|
|
final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, clientId); |
|
|
|
|
final StreamThread thread = new StreamThread( |
|
|
|
|
mockTime, |
|
|
|
|
config, |
|
|
|
|
null, |
|
|
|
|
mockStreamThreadConsumer, |
|
|
|
|
mockStreamThreadConsumer, |
|
|
|
|
null, |
|
|
|
|
taskManager, |
|
|
|
|
streamsMetrics, |
|
|
|
|
internalTopologyBuilder, |
|
|
|
|
clientId, |
|
|
|
|
new LogContext(""), |
|
|
|
|
new AtomicInteger() |
|
|
|
|
).updateThreadMetadata(getSharedAdminClientId(clientId)); |
|
|
|
|
|
|
|
|
|
mockStreamThreadConsumer.setStreamThread(thread); |
|
|
|
|
mockStreamThreadConsumer.assign(assignedPartitions); |
|
|
|
|
mockStreamThreadConsumer.updateBeginningOffsets(Collections.singletonMap(t1p1, 0L)); |
|
|
|
|
|
|
|
|
|
addRecord(mockStreamThreadConsumer, 1L, 0L); |
|
|
|
|
thread.setState(StreamThread.State.STARTING); |
|
|
|
|
thread.setState(StreamThread.State.PARTITIONS_REVOKED); |
|
|
|
|
thread.runOnce(); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
public void shouldOnlyShutdownOnce() { |
|
|
|
|
final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class); |
|
|
|
|