@ -141,7 +141,7 @@ public class RecordCollectorTest {
@@ -141,7 +141,7 @@ public class RecordCollectorTest {
@After
public void cleanup ( ) {
collector . close ( ) ;
collector . closeClean ( ) ;
}
@Test
@ -299,7 +299,25 @@ public class RecordCollectorTest {
@@ -299,7 +299,25 @@ public class RecordCollectorTest {
}
@Test
public void shouldAbortTxIfEosEnabled ( ) {
public void shouldNotAbortTxOnCloseCleanIfEosEnabled ( ) {
final StreamsProducer streamsProducer = mock ( StreamsProducer . class ) ;
expect ( streamsProducer . eosEnabled ( ) ) . andReturn ( true ) ;
replay ( streamsProducer ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
streamsProducer ,
productionExceptionHandler ,
streamsMetrics ) ;
collector . closeClean ( ) ;
verify ( streamsProducer ) ;
}
@Test
public void shouldAbortTxOnCloseDirtyIfEosEnabled ( ) {
final StreamsProducer streamsProducer = mock ( StreamsProducer . class ) ;
expect ( streamsProducer . eosEnabled ( ) ) . andReturn ( true ) ;
streamsProducer . abortTransaction ( ) ;
@ -312,7 +330,7 @@ public class RecordCollectorTest {
@@ -312,7 +330,7 @@ public class RecordCollectorTest {
productionExceptionHandler ,
streamsMetrics ) ;
collector . close ( ) ;
collector . closeDirty ( ) ;
verify ( streamsProducer ) ;
}
@ -430,30 +448,12 @@ public class RecordCollectorTest {
@@ -430,30 +448,12 @@ public class RecordCollectorTest {
}
@Test
public void shouldThrowTaskMigratedExceptionOnSubsequentCall WhenProducerFencedInCallback ( ) {
public void shouldThrowTaskMigratedExceptionOnSubsequentSend WhenProducerFencedInCallback ( ) {
final KafkaException exception = new ProducerFencedException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
new StreamsProducer (
eosConfig ,
"threadId" ,
new MockClientSupplier ( ) {
@Override
public Producer < byte [ ] , byte [ ] > getProducer ( final Map < String , Object > config ) {
return new MockProducer < byte [ ] , byte [ ] > ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord < byte [ ] , byte [ ] > record , final Callback callback ) {
callback . onCompletion ( null , exception ) ;
return null ;
}
} ;
}
} ,
taskId ,
null ,
logContext
) ,
getExceptionalStreamsProducer ( exception ) ,
productionExceptionHandler ,
streamsMetrics
) ;
@ -461,72 +461,84 @@ public class RecordCollectorTest {
@@ -461,72 +461,84 @@ public class RecordCollectorTest {
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
TaskMigratedException thrown = assertThrows (
final TaskMigratedException thrown = assertThrows (
TaskMigratedException . class , ( ) - >
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner )
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner )
) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.ProducerFencedException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced," +
" indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated." )
"\norg.apache.kafka.common.errors.ProducerFencedException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced," +
" indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated." )
) ;
}
@Test
public void shouldThrowTaskMigratedExceptionOnSubsequentFlushWhenProducerFencedInCallback ( ) {
final KafkaException exception = new ProducerFencedException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
productionExceptionHandler ,
streamsMetrics
) ;
collector . initialize ( ) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
thrown = assertThrows ( TaskMigratedException . class , collector : : flush ) ;
final TaskMigratedException thrown = assertThrows ( TaskMigratedException . class , collector : : flush ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.ProducerFencedException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced," +
" indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated." )
"\norg.apache.kafka.common.errors.ProducerFencedException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced," +
" indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated." )
) ;
}
@Test
public void shouldThrowTaskMigratedExceptionOnSubsequentCloseWhenProducerFencedInCallback ( ) {
final KafkaException exception = new ProducerFencedException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
productionExceptionHandler ,
streamsMetrics
) ;
collector . initialize ( ) ;
thrown = assertThrows ( TaskMigratedException . class , collector : : close ) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final TaskMigratedException thrown = assertThrows ( TaskMigratedException . class , collector : : closeClean ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.ProducerFencedException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced," +
" indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated." )
"\norg.apache.kafka.common.errors.ProducerFencedException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced," +
" indicating the task may be migrated out; it means all tasks belonging to this thread should be migrated." )
) ;
}
@Test
public void shouldThrowStreamsExceptionOnSubsequentCall IfASendFailsWithDefaultExceptionHandler ( ) {
public void shouldThrowStreamsExceptionOnSubsequentSend IfASendFailsWithDefaultExceptionHandler ( ) {
final KafkaException exception = new KafkaException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
new StreamsProducer (
config ,
"threadId" ,
new MockClientSupplier ( ) {
@Override
public Producer < byte [ ] , byte [ ] > getProducer ( final Map < String , Object > config ) {
return new MockProducer < byte [ ] , byte [ ] > ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord < byte [ ] , byte [ ] > record , final Callback callback ) {
callback . onCompletion ( null , exception ) ;
return null ;
}
} ;
}
} ,
null ,
null ,
logContext
) ,
getExceptionalStreamsProducer ( exception ) ,
productionExceptionHandler ,
streamsMetrics
) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
StreamsException thrown = assertThrows (
final StreamsException thrown = assertThrows (
StreamsException . class ,
( ) - > collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner )
) ;
@ -537,23 +549,123 @@ public class RecordCollectorTest {
@@ -537,23 +549,123 @@ public class RecordCollectorTest {
"\norg.apache.kafka.common.KafkaException: KABOOM!" +
"\nException handler choose to FAIL the processing, no more records would be sent." )
) ;
}
@Test
public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsWithDefaultExceptionHandler ( ) {
final KafkaException exception = new KafkaException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
productionExceptionHandler ,
streamsMetrics
) ;
thrown = assertThrows ( StreamsException . class , collector : : flush ) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final StreamsException thrown = assertThrows ( StreamsException . class , collector : : flush ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.KafkaException: KABOOM!" +
"\nException handler choose to FAIL the processing, no more records would be sent." )
"\norg.apache.kafka.common.KafkaException: KABOOM!" +
"\nException handler choose to FAIL the processing, no more records would be sent." )
) ;
}
thrown = assertThrows ( StreamsException . class , collector : : close ) ;
@Test
public void shouldThrowStreamsExceptionOnSubsequentCloseIfASendFailsWithDefaultExceptionHandler ( ) {
final KafkaException exception = new KafkaException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
productionExceptionHandler ,
streamsMetrics
) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final StreamsException thrown = assertThrows ( StreamsException . class , collector : : closeClean ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.KafkaException: KABOOM!" +
"\nException handler choose to FAIL the processing, no more records would be sent." )
"\norg.apache.kafka.common.KafkaException: KABOOM!" +
"\nException handler choose to FAIL the processing, no more records would be sent." )
) ;
}
@Test
public void shouldThrowStreamsExceptionOnSubsequentSendIfFatalEvenWithContinueExceptionHandler ( ) {
final KafkaException exception = new AuthenticationException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
new AlwaysContinueProductionExceptionHandler ( ) ,
streamsMetrics
) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final StreamsException thrown = assertThrows (
StreamsException . class ,
( ) - > collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner )
) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.AuthenticationException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error." )
) ;
}
@Test
public void shouldThrowStreamsExceptionOnSubsequentFlushIfFatalEvenWithContinueExceptionHandler ( ) {
final KafkaException exception = new AuthenticationException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
new AlwaysContinueProductionExceptionHandler ( ) ,
streamsMetrics
) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final StreamsException thrown = assertThrows ( StreamsException . class , collector : : flush ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.AuthenticationException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error." )
) ;
}
@Test
public void shouldThrowStreamsExceptionOnSubsequentCloseIfFatalEvenWithContinueExceptionHandler ( ) {
final KafkaException exception = new AuthenticationException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
getExceptionalStreamsProducer ( exception ) ,
new AlwaysContinueProductionExceptionHandler ( ) ,
streamsMetrics
) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final StreamsException thrown = assertThrows ( StreamsException . class , collector : : closeClean ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.AuthenticationException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error." )
) ;
}
@ -562,25 +674,7 @@ public class RecordCollectorTest {
@@ -562,25 +674,7 @@ public class RecordCollectorTest {
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
new StreamsProducer (
config ,
"threadId" ,
new MockClientSupplier ( ) {
@Override
public Producer < byte [ ] , byte [ ] > getProducer ( final Map < String , Object > config ) {
return new MockProducer < byte [ ] , byte [ ] > ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord < byte [ ] , byte [ ] > record , final Callback callback ) {
callback . onCompletion ( null , new Exception ( ) ) ;
return null ;
}
} ;
}
} ,
null ,
null ,
logContext
) ,
getExceptionalStreamsProducer ( new Exception ( ) ) ,
new AlwaysContinueProductionExceptionHandler ( ) ,
streamsMetrics
) ;
@ -616,73 +710,11 @@ public class RecordCollectorTest {
@@ -616,73 +710,11 @@ public class RecordCollectorTest {
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
collector . flush ( ) ;
collector . close ( ) ;
}
@Test
public void shouldThrowStreamsExceptionOnSubsequentCallIfFatalEvenWithContinueExceptionHandler ( ) {
final KafkaException exception = new AuthenticationException ( "KABOOM!" ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
taskId ,
new StreamsProducer (
config ,
"threadId" ,
new MockClientSupplier ( ) {
@Override
public Producer < byte [ ] , byte [ ] > getProducer ( final Map < String , Object > config ) {
return new MockProducer < byte [ ] , byte [ ] > ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord < byte [ ] , byte [ ] > record , final Callback callback ) {
callback . onCompletion ( null , exception ) ;
return null ;
}
} ;
}
} ,
null ,
null ,
logContext
) ,
new AlwaysContinueProductionExceptionHandler ( ) ,
streamsMetrics
) ;
collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner ) ;
StreamsException thrown = assertThrows (
StreamsException . class ,
( ) - > collector . send ( topic , "3" , "0" , null , null , stringSerializer , stringSerializer , streamPartitioner )
) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.AuthenticationException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error." )
) ;
thrown = assertThrows ( StreamsException . class , collector : : flush ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.AuthenticationException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error." )
) ;
thrown = assertThrows ( StreamsException . class , collector : : close ) ;
assertEquals ( exception , thrown . getCause ( ) ) ;
assertThat (
thrown . getMessage ( ) ,
equalTo ( "Error encountered sending record to topic topic for task 0_0 due to:" +
"\norg.apache.kafka.common.errors.AuthenticationException: KABOOM!" +
"\nWritten offsets would not be recorded and no more records would be sent since this is a fatal error." )
) ;
collector . closeClean ( ) ;
}
@Test
public void shouldNotAbortTxnOnEOSCloseIfNothingSent ( ) {
public void shouldNotAbortTxnOnEOSCloseDirtyIfNothingSent ( ) {
final AtomicBoolean functionCalled = new AtomicBoolean ( false ) ;
final RecordCollector collector = new RecordCollectorImpl (
logContext ,
@ -709,7 +741,7 @@ public class RecordCollectorTest {
@@ -709,7 +741,7 @@ public class RecordCollectorTest {
streamsMetrics
) ;
collector . close ( ) ;
collector . closeDirty ( ) ;
assertFalse ( functionCalled . get ( ) ) ;
}
@ -774,7 +806,7 @@ public class RecordCollectorTest {
@@ -774,7 +806,7 @@ public class RecordCollectorTest {
streamsMetrics
) ;
collector . close ( ) ;
collector . closeClean ( ) ;
// Flush should not throw as producer is still alive.
streamsProducer . flush ( ) ;
@ -782,12 +814,34 @@ public class RecordCollectorTest {
@@ -782,12 +814,34 @@ public class RecordCollectorTest {
@Test
public void shouldNotCloseInternalProducerForNonEOS ( ) {
collector . close ( ) ;
collector . closeClean ( ) ;
// Flush should not throw as producer is still alive.
streamsProducer . flush ( ) ;
}
private StreamsProducer getExceptionalStreamsProducer ( final Exception exception ) {
return new StreamsProducer (
config ,
"threadId" ,
new MockClientSupplier ( ) {
@Override
public Producer < byte [ ] , byte [ ] > getProducer ( final Map < String , Object > config ) {
return new MockProducer < byte [ ] , byte [ ] > ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord < byte [ ] , byte [ ] > record , final Callback callback ) {
callback . onCompletion ( null , exception ) ;
return null ;
}
} ;
}
} ,
null ,
null ,
logContext
) ;
}
private static class CustomStringSerializer extends StringSerializer {
private boolean isKey ;