1. Delay the initialization (producer.initTxn) from construction to maybeInitialize; if it times out we just swallow and retry in the next iteration.
2. If completeRestoration (consumer.committed) times out, just swallow and retry in the next iteration.
3. For other calls (producer.partitionsFor, producer.commitTxn, consumer.commit), treat the timeout exception as fatal.
Reviewers: Matthias J. Sax <matthias@confluent.io>
@ -70,6 +71,7 @@ public class RecordCollectorImpl implements RecordCollector {
@@ -70,6 +71,7 @@ public class RecordCollectorImpl implements RecordCollector {
// used when eosEnabled is true only
privatebooleantransactionInFlight=false;
privatebooleantransactionInitialized=false;
privateProducer<byte[],byte[]>producer;
privatevolatileKafkaExceptionsendException;
@ -95,24 +97,30 @@ public class RecordCollectorImpl implements RecordCollector {
@@ -95,24 +97,30 @@ public class RecordCollectorImpl implements RecordCollector {
// 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
try{
producer.initTransactions();
transactionInitialized=true;
}catch(finalTimeoutExceptionexception){
finalStringerrorMessage="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, "+
"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
thrownewStreamsException(errorMessage,exception);
throwexception;
}catch(finalKafkaExceptionexception){
thrownewStreamsException("Error encountered while initializing transactions for task "+taskId,exception);
}
@ -163,7 +171,7 @@ public class RecordCollectorImpl implements RecordCollector {
@@ -163,7 +171,7 @@ public class RecordCollectorImpl implements RecordCollector {
}catch(finalProducerFencedExceptionerror){
thrownewTaskMigratedException(taskId,"Producer get fenced trying to commit a transaction",error);
}catch(finalTimeoutExceptionerror){
// 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
thrownewStreamsException("Timed out while committing transaction via producer for task "+taskId,error);
}catch(finalKafkaExceptionerror){
thrownewStreamsException("Error encountered sending offsets and committing transaction "+
@ -176,7 +184,7 @@ public class RecordCollectorImpl implements RecordCollector {
@@ -176,7 +184,7 @@ public class RecordCollectorImpl implements RecordCollector {
"indicating the corresponding thread is no longer part of the group.",error);
}catch(finalTimeoutExceptionerror){
// 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
thrownewStreamsException("Timed out while committing offsets via consumer for task "+taskId,error);
}catch(finalKafkaExceptionerror){
thrownewStreamsException("Error encountered committing offsets via consumer for task "+taskId,error);
@ -244,9 +252,16 @@ public class RecordCollectorImpl implements RecordCollector {
@@ -244,9 +252,16 @@ public class RecordCollectorImpl implements RecordCollector {
@ -76,6 +76,9 @@ public class MockRecordCollector implements RecordCollector {
@@ -76,6 +76,9 @@ public class MockRecordCollector implements RecordCollector {