This is an automated email from the ASF dual-hosted git repository.
arvid pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/flink-connector-kafka.git
The following commit(s) were added to refs/heads/main by this push:
new ea0b9ab3 [FLINK-38451] Stop logging interrupts as ERROR
ea0b9ab3 is described below
commit ea0b9ab3f88126331bb9b058c5aa6072dfafb4d1
Author: Arvid Heise <[email protected]>
AuthorDate: Mon Sep 29 10:43:29 2025 +0200
[FLINK-38451] Stop logging interrupts as ERROR
When cancelling a job, we may interrupt the KafkaCommitter. This currently
leads to ERROR log "Transaction ... encountered error and data has been
potentially lost."
However, that exception is expected and not leading to any data loss beyond
the normal inconsistencies because of cancellation. In many cases, the commit
already succeeded. Further, a job restart will lead to data being eventually
committed.
---
.../kafka/sink/internal/KafkaCommitter.java | 27 ++++++--
.../kafka/sink/internal/KafkaCommitterTest.java | 78 ++++++++++++++++++++--
2 files changed, 91 insertions(+), 14 deletions(-)
diff --git
a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitter.java
b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitter.java
index ff95b1e9..36299b9a 100644
---
a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitter.java
+++
b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitter.java
@@ -1,12 +1,13 @@
/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
@@ -26,6 +27,7 @@ import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.IOUtils;
import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.InvalidTxnStateException;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RetriableException;
@@ -125,6 +127,17 @@ public class KafkaCommitter implements
Committer<KafkaCommittable>, Closeable {
e);
handleFailedTransaction(producer);
request.signalFailedWithKnownReason(e);
+ } catch (InterruptException e) {
+ // note that we do not attempt to recover from this exception;
producer is likely
+ // left in an inconsistent state
+ LOG.info(
+ "Committing transaction ({}) was interrupted. This
most likely happens because the task is being cancelled.",
+ request,
+ e);
+ // reset the interrupt flag that is set when
InterruptException is created
+ Thread.interrupted();
+ // propagate interruption through
java.lang.InterruptedException instead
+ throw new InterruptedException(e.getMessage());
} catch (Exception e) {
LOG.error(
"Transaction ({}) encountered error and data has been
potentially lost.",
diff --git
a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitterTest.java
b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitterTest.java
index 1a969b53..e16a0f0f 100644
---
a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitterTest.java
+++
b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/KafkaCommitterTest.java
@@ -1,12 +1,13 @@
/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
@@ -19,6 +20,7 @@ package org.apache.flink.connector.kafka.sink.internal;
import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest;
import org.apache.flink.connector.kafka.sink.KafkaCommittable;
+import org.apache.flink.testutils.logging.LoggerAuditingExtension;
import org.apache.flink.util.TestLoggerExtension;
import org.apache.kafka.clients.CommonClientConfigs;
@@ -29,17 +31,22 @@ import org.assertj.core.api.Condition;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException;
+import java.net.ServerSocket;
import java.util.Collections;
import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiFunction;
import static
org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.slf4j.event.Level.ERROR;
/** Tests for {@link KafkaCommitter}. */
@ExtendWith({TestLoggerExtension.class})
@@ -53,6 +60,10 @@ class KafkaCommitterTest {
private static final BiFunction<Properties, String,
FlinkKafkaInternalProducer<?, ?>>
MOCK_FACTORY = (properties, transactionalId) -> new
MockProducer(properties, null);
+ @RegisterExtension
+ public final LoggerAuditingExtension errorLogger =
+ new LoggerAuditingExtension(KafkaCommitter.class, ERROR);
+
@AfterEach
public void check() {
checkProducerLeak();
@@ -158,6 +169,59 @@ class KafkaCommitterTest {
}
}
+ @Test
+ public void testInterrupt() throws IOException {
+ ServerSocket serverSocket = new ServerSocket(0);
+ Properties properties = getProperties();
+ properties.put(
+ CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
+ "http://localhost:" + serverSocket.getLocalPort());
+ properties.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, "1000");
+ properties.put(ProducerConfig.MAX_BLOCK_MS_CONFIG,
String.valueOf(Long.MAX_VALUE));
+ try (final KafkaCommitter committer =
+ new KafkaCommitter(
+ properties, TRANS_ID, SUB_ID, ATTEMPT, false,
MOCK_FACTORY);
+ FlinkKafkaInternalProducer<Object, Object> producer =
+ new FlinkKafkaInternalProducer<>(properties, TRANS_ID);
+ ReadableBackchannel<TransactionFinished> backchannel =
+ BackchannelFactory.getInstance()
+ .getReadableBackchannel(SUB_ID, ATTEMPT,
TRANS_ID)) {
+ final MockCommitRequest<KafkaCommittable> request =
+ new MockCommitRequest<>(KafkaCommittable.of(producer));
+
+ producer.resumeTransaction(PRODUCER_ID, EPOCH);
+
+ AtomicBoolean interrupting =
interruptOnMessage(Thread.currentThread(), serverSocket);
+ assertThatThrownBy(() ->
committer.commit(Collections.singletonList(request)))
+ .isInstanceOf(InterruptedException.class);
+
+ // verify that the interrupt happened only after committing started
+ assertThat(interrupting).isTrue();
+
+ // no errors are logged
+ assertThat(errorLogger.getMessages()).isEmpty();
+
+ assertThat(backchannel).doesNotHave(transactionFinished(true));
+ }
+ }
+
+ private AtomicBoolean interruptOnMessage(Thread mainThread, ServerSocket
serverSocket) {
+ final AtomicBoolean interrupting = new AtomicBoolean();
+ new Thread(
+ () -> {
+ try {
+ serverSocket.accept().getInputStream().read();
+ interrupting.set(true);
+ mainThread.interrupt();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ },
+ "canceller")
+ .start();
+ return interrupting;
+ }
+
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testKafkaCommitterRecyclesTransactionalId(boolean hasProducer)