@ -22,10 +22,10 @@ import org.apache.kafka.clients.producer.ProducerRecord;
@@ -22,10 +22,10 @@ import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata ;
import org.apache.kafka.clients.producer.internals.DefaultPartitioner ;
import org.apache.kafka.common.Cluster ;
import org.apache.kafka.common.KafkaException ;
import org.apache.kafka.common.Node ;
import org.apache.kafka.common.PartitionInfo ;
import org.apache.kafka.common.TopicPartition ;
import org.apache.kafka.common.errors.TimeoutException ;
import org.apache.kafka.common.serialization.ByteArraySerializer ;
import org.apache.kafka.common.serialization.StringSerializer ;
import org.apache.kafka.common.utils.LogContext ;
@ -38,9 +38,9 @@ import java.util.Collections;
@@ -38,9 +38,9 @@ import java.util.Collections;
import java.util.List ;
import java.util.Map ;
import java.util.concurrent.Future ;
import java.util.concurrent.atomic.AtomicInteger ;
import static org.junit.Assert.assertEquals ;
import static org.junit.Assert.fail ;
public class RecordCollectorTest {
@ -124,47 +124,24 @@ public class RecordCollectorTest {
@@ -124,47 +124,24 @@ public class RecordCollectorTest {
assertEquals ( ( Long ) 0L , offsets . get ( new TopicPartition ( "topic1" , 2 ) ) ) ;
}
@SuppressWarnings ( "unchecked" )
@Test
public void shouldRetryWhenTimeoutExceptionOccursOnSend ( ) {
final AtomicInteger attempt = new AtomicInteger ( 0 ) ;
final RecordCollectorImpl collector = new RecordCollectorImpl (
new MockProducer ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord record , final Callback callback ) {
if ( attempt . getAndIncrement ( ) = = 0 ) {
throw new TimeoutException ( ) ;
}
return super . send ( record , callback ) ;
}
} ,
"test" ,
logContext ) ;
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
final Long offset = collector . offsets ( ) . get ( new TopicPartition ( "topic1" , 0 ) ) ;
assertEquals ( Long . valueOf ( 0L ) , offset ) ;
}
@SuppressWarnings ( "unchecked" )
@Test ( expected = StreamsException . class )
public void shouldThrowStreamsExceptionAfterMaxAttempts ( ) {
public void shouldThrowStreamsExceptionOnAnyExceptionButProducerFencedException ( ) {
final RecordCollector collector = new RecordCollectorImpl (
new MockProducer ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@Override
public synchronized Future < RecordMetadata > send ( final ProducerRecord record , final Callback callback ) {
throw new Timeout Exception( ) ;
throw new KafkaException ( ) ;
}
} ,
"test" ,
logContext ) ;
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
}
@SuppressWarnings ( "unchecked" )
@Test ( expected = StreamsException . class )
@Test
public void shouldThrowStreamsExceptionOnSubsequentCallIfASendFails ( ) {
final RecordCollector collector = new RecordCollectorImpl (
new MockProducer ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@ -177,11 +154,15 @@ public class RecordCollectorTest {
@@ -177,11 +154,15 @@ public class RecordCollectorTest {
"test" ,
logContext ) ;
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
try {
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
fail ( "Should have thrown StreamsException" ) ;
} catch ( final StreamsException expected ) { /* ok */ }
}
@SuppressWarnings ( "unchecked" )
@Test ( expected = StreamsException . class )
@Test
public void shouldThrowStreamsExceptionOnFlushIfASendFailed ( ) {
final RecordCollector collector = new RecordCollectorImpl (
new MockProducer ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@ -194,11 +175,15 @@ public class RecordCollectorTest {
@@ -194,11 +175,15 @@ public class RecordCollectorTest {
"test" ,
logContext ) ;
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
collector . flush ( ) ;
try {
collector . flush ( ) ;
fail ( "Should have thrown StreamsException" ) ;
} catch ( final StreamsException expected ) { /* ok */ }
}
@SuppressWarnings ( "unchecked" )
@Test ( expected = StreamsException . class )
@Test
public void shouldThrowStreamsExceptionOnCloseIfASendFailed ( ) {
final RecordCollector collector = new RecordCollectorImpl (
new MockProducer ( cluster , true , new DefaultPartitioner ( ) , byteArraySerializer , byteArraySerializer ) {
@ -211,7 +196,11 @@ public class RecordCollectorTest {
@@ -211,7 +196,11 @@ public class RecordCollectorTest {
"test" ,
logContext ) ;
collector . send ( "topic1" , "3" , "0" , null , stringSerializer , stringSerializer , streamPartitioner ) ;
collector . close ( ) ;
try {
collector . close ( ) ;
fail ( "Should have thrown StreamsException" ) ;
} catch ( final StreamsException expected ) { /* ok */ }
}
@SuppressWarnings ( "unchecked" )