@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.CommitFailedException;
import org.apache.kafka.clients.consumer.Consumer ;
import org.apache.kafka.clients.consumer.Consumer ;
import org.apache.kafka.clients.consumer.OffsetAndMetadata ;
import org.apache.kafka.clients.consumer.OffsetAndMetadata ;
import org.apache.kafka.clients.producer.Producer ;
import org.apache.kafka.clients.producer.Producer ;
import org.apache.kafka.clients.producer.ProducerConfig ;
import org.apache.kafka.clients.producer.ProducerRecord ;
import org.apache.kafka.clients.producer.ProducerRecord ;
import org.apache.kafka.common.KafkaException ;
import org.apache.kafka.common.KafkaException ;
import org.apache.kafka.common.PartitionInfo ;
import org.apache.kafka.common.PartitionInfo ;
@ -70,6 +71,7 @@ public class RecordCollectorImpl implements RecordCollector {
// used when eosEnabled is true only
// used when eosEnabled is true only
private boolean transactionInFlight = false ;
private boolean transactionInFlight = false ;
private boolean transactionInitialized = false ;
private Producer < byte [ ] , byte [ ] > producer ;
private Producer < byte [ ] , byte [ ] > producer ;
private volatile KafkaException sendException ;
private volatile KafkaException sendException ;
@ -95,24 +97,30 @@ public class RecordCollectorImpl implements RecordCollector {
this . droppedRecordsSensor = TaskMetrics . droppedRecordsSensorOrSkippedRecordsSensor ( threadId , taskId . toString ( ) , streamsMetrics ) ;
this . droppedRecordsSensor = TaskMetrics . droppedRecordsSensorOrSkippedRecordsSensor ( threadId , taskId . toString ( ) , streamsMetrics ) ;
producer = producerSupplier . get ( taskId ) ;
producer = producerSupplier . get ( taskId ) ;
}
@Override
public void initialize ( ) {
maybeInitTxns ( ) ;
maybeInitTxns ( ) ;
}
}
private void maybeInitTxns ( ) {
private void maybeInitTxns ( ) {
if ( eosEnabled ) {
if ( eosEnabled & & ! transactionInitialized ) {
// initialize transactions if eos is turned on, which will block if the previous transaction has not
// initialize transactions if eos is turned on, which will block if the previous transaction has not
// completed yet; do not start the first transaction until the topology has been initialized later
// completed yet; do not start the first transaction until the topology has been initialized later
try {
try {
producer . initTransactions ( ) ;
producer . initTransactions ( ) ;
transactionInitialized = true ;
} catch ( final TimeoutException exception ) {
} catch ( final TimeoutException exception ) {
final String errorMessage = "Timeout exception caught when initializing transactions for task " + taskId + ". " +
log . warn ( "Timeout exception caught when initializing transactions for task {} . " +
"\nThe broker is either slow or in bad state (like not having enough replicas) in responding to the request, " +
"\nThe broker is either slow or in bad state (like not having enough replicas) in responding to the request, " +
"or the connection to broker was interrupted sending the request or receiving the response. " +
"or the connection to broker was interrupted sending the request or receiving the response. " +
"\n Consider overwriting `max.block.ms` to a larger value to avoid timeout errors" ;
"Would retry initializing the task in the next loop." +
"\nConsider overwriting producer config {} to a larger value to avoid timeout errors" ,
ProducerConfig . MAX_BLOCK_MS_CONFIG , taskId ) ;
// TODO K9113: we do NOT try to handle timeout exception here but throw it as a fatal error
throw exception ;
throw new StreamsException ( errorMessage , exception ) ;
} catch ( final KafkaException exception ) {
} catch ( final KafkaException exception ) {
throw new StreamsException ( "Error encountered while initializing transactions for task " + taskId , exception ) ;
throw new StreamsException ( "Error encountered while initializing transactions for task " + taskId , exception ) ;
}
}
@ -163,7 +171,7 @@ public class RecordCollectorImpl implements RecordCollector {
} catch ( final ProducerFencedException error ) {
} catch ( final ProducerFencedException error ) {
throw new TaskMigratedException ( taskId , "Producer get fenced trying to commit a transaction" , error ) ;
throw new TaskMigratedException ( taskId , "Producer get fenced trying to commit a transaction" , error ) ;
} catch ( final TimeoutException error ) {
} catch ( final TimeoutException error ) {
// TODO K9113: currently handle timeout exception as a fatal error, should discuss whether we want to handle it
// TODO KIP-447: we can consider treating it as non-fatal and retry on the thread level
throw new StreamsException ( "Timed out while committing transaction via producer for task " + taskId , error ) ;
throw new StreamsException ( "Timed out while committing transaction via producer for task " + taskId , error ) ;
} catch ( final KafkaException error ) {
} catch ( final KafkaException error ) {
throw new StreamsException ( "Error encountered sending offsets and committing transaction " +
throw new StreamsException ( "Error encountered sending offsets and committing transaction " +
@ -176,7 +184,7 @@ public class RecordCollectorImpl implements RecordCollector {
throw new TaskMigratedException ( taskId , "Consumer committing offsets failed, " +
throw new TaskMigratedException ( taskId , "Consumer committing offsets failed, " +
"indicating the corresponding thread is no longer part of the group." , error ) ;
"indicating the corresponding thread is no longer part of the group." , error ) ;
} catch ( final TimeoutException error ) {
} catch ( final TimeoutException error ) {
// TODO K9113: currently handle timeout exception as a fatal error
// TODO KIP-447: we can consider treating it as non-fatal and retry on the thread level
throw new StreamsException ( "Timed out while committing offsets via consumer for task " + taskId , error ) ;
throw new StreamsException ( "Timed out while committing offsets via consumer for task " + taskId , error ) ;
} catch ( final KafkaException error ) {
} catch ( final KafkaException error ) {
throw new StreamsException ( "Error encountered committing offsets via consumer for task " + taskId , error ) ;
throw new StreamsException ( "Error encountered committing offsets via consumer for task " + taskId , error ) ;
@ -244,9 +252,16 @@ public class RecordCollectorImpl implements RecordCollector {
final StreamPartitioner < ? super K , ? super V > partitioner ) {
final StreamPartitioner < ? super K , ? super V > partitioner ) {
final Integer partition ;
final Integer partition ;
// TODO K9113: we need to decide how to handle exceptions from partitionsFor
if ( partitioner ! = null ) {
if ( partitioner ! = null ) {
final List < PartitionInfo > partitions = producer . partitionsFor ( topic ) ;
final List < PartitionInfo > partitions ;
try {
partitions = producer . partitionsFor ( topic ) ;
} catch ( final KafkaException e ) {
// here we cannot drop the message on the floor even if it is a transient timeout exception,
// so we treat everything the same as a fatal exception
throw new StreamsException ( "Could not determine the number of partitions for topic '" + topic +
"' for task " + taskId + " due to " + e . toString ( ) ) ;
}
if ( partitions . size ( ) > 0 ) {
if ( partitions . size ( ) > 0 ) {
partition = partitioner . partition ( topic , key , value , partitions . size ( ) ) ;
partition = partitioner . partition ( topic , key , value , partitions . size ( ) ) ;
} else {
} else {