@ -44,6 +44,7 @@ import org.apache.kafka.streams.StreamsConfig;
@@ -44,6 +44,7 @@ import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.LockException ;
import org.apache.kafka.streams.errors.ProcessorStateException ;
import org.apache.kafka.streams.errors.StreamsException ;
import org.apache.kafka.streams.errors.TaskMigratedException ;
import org.apache.kafka.streams.processor.PunctuationType ;
import org.apache.kafka.streams.processor.Punctuator ;
import org.apache.kafka.streams.processor.StateStore ;
@ -1752,7 +1753,7 @@ public class StreamTaskTest {
@@ -1752,7 +1753,7 @@ public class StreamTaskTest {
}
@Test
public void shouldUnregisterMetricsInCloseAndRecycle ( ) {
public void shouldUnregisterMetricsInCloseClean AndRecycleStat e ( ) {
EasyMock . expect ( stateManager . changelogPartitions ( ) ) . andReturn ( Collections . emptySet ( ) ) . anyTimes ( ) ;
EasyMock . expect ( recordCollector . offsets ( ) ) . andReturn ( Collections . emptyMap ( ) ) . anyTimes ( ) ;
EasyMock . replay ( stateManager , recordCollector ) ;
@ -1761,7 +1762,7 @@ public class StreamTaskTest {
@@ -1761,7 +1762,7 @@ public class StreamTaskTest {
task . suspend ( ) ;
assertThat ( getTaskMetrics ( ) , not ( empty ( ) ) ) ;
task . closeAndRecycleState ( ) ;
task . closeClean AndRecycleState ( ) ;
assertThat ( getTaskMetrics ( ) , empty ( ) ) ;
}
@ -1798,6 +1799,32 @@ public class StreamTaskTest {
@@ -1798,6 +1799,32 @@ public class StreamTaskTest {
assertThat ( task . inputPartitions ( ) , equalTo ( newPartitions ) ) ;
}
@Test
public void shouldThrowIfCleanClosingDirtyTask ( ) {
task = createStatelessTask ( createConfig ( false , "0" ) , StreamsConfig . METRICS_LATEST ) ;
task . initializeIfNeeded ( ) ;
task . completeRestoration ( ) ;
task . addRecords ( partition1 , singletonList ( getConsumerRecord ( partition1 , 0 ) ) ) ;
task . process ( 0L ) ;
assertTrue ( task . commitNeeded ( ) ) ;
assertThrows ( TaskMigratedException . class , ( ) - > task . closeClean ( ) ) ;
}
@Test
public void shouldThrowIfRecyclingDirtyTask ( ) {
task = createStatelessTask ( createConfig ( false , "0" ) , StreamsConfig . METRICS_LATEST ) ;
task . initializeIfNeeded ( ) ;
task . completeRestoration ( ) ;
task . addRecords ( partition1 , singletonList ( getConsumerRecord ( partition1 , 0 ) ) ) ;
task . process ( 0L ) ;
assertTrue ( task . commitNeeded ( ) ) ;
assertThrows ( TaskMigratedException . class , ( ) - > task . closeCleanAndRecycleState ( ) ) ;
}
@Test
public void shouldOnlyRecycleSuspendedTasks ( ) {
stateManager . recycle ( ) ;
@ -1805,16 +1832,16 @@ public class StreamTaskTest {
@@ -1805,16 +1832,16 @@ public class StreamTaskTest {
EasyMock . replay ( stateManager , recordCollector ) ;
task = createStatefulTask ( createConfig ( false , "100" ) , true ) ;
assertThrows ( IllegalStateException . class , ( ) - > task . closeAndRecycleState ( ) ) ; // CREATED
assertThrows ( IllegalStateException . class , ( ) - > task . closeClean AndRecycleState ( ) ) ; // CREATED
task . initializeIfNeeded ( ) ;
assertThrows ( IllegalStateException . class , ( ) - > task . closeAndRecycleState ( ) ) ; // RESTORING
assertThrows ( IllegalStateException . class , ( ) - > task . closeClean AndRecycleState ( ) ) ; // RESTORING
task . completeRestoration ( ) ;
assertThrows ( IllegalStateException . class , ( ) - > task . closeAndRecycleState ( ) ) ; // RUNNING
assertThrows ( IllegalStateException . class , ( ) - > task . closeClean AndRecycleState ( ) ) ; // RUNNING
task . suspend ( ) ;
task . closeAndRecycleState ( ) ; // SUSPENDED
task . closeClean AndRecycleState ( ) ; // SUSPENDED
EasyMock . verify ( stateManager , recordCollector ) ;
}