diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java index 23272a2cb5d..c561fc23608 100644 --- a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java +++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java @@ -43,6 +43,7 @@ import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.MAX_BLOCK_MS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.RETRIES_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG; import static org.apache.kafka.common.config.SaslConfigs.SASL_JAAS_CONFIG; import static org.apache.kafka.common.config.SaslConfigs.SASL_KERBEROS_SERVICE_NAME; import static org.apache.kafka.common.config.SaslConfigs.SASL_MECHANISM; @@ -290,6 +291,9 @@ public class KafkaLog4jAppender extends AppenderSkeleton { props.put(DELIVERY_TIMEOUT_MS_CONFIG, deliveryTimeoutMs); props.put(LINGER_MS_CONFIG, lingerMs); props.put(BATCH_SIZE_CONFIG, batchSize); + // Disable idempotence to avoid deadlock when the producer network thread writes a log line while interacting + // with the TransactionManager, see KAFKA-13761 for more information. + props.put(ENABLE_IDEMPOTENCE_CONFIG, false); if (securityProtocol != null) { props.put(SECURITY_PROTOCOL_CONFIG, securityProtocol);