Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -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,
Expand All @@ -26,6 +27,7 @@
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;
Expand Down Expand Up @@ -125,6 +127,17 @@ public void commit(Collection<CommitRequest<KafkaCommittable>> requests)
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.",
Expand Down
Original file line number Diff line number Diff line change
@@ -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,
Expand All @@ -19,6 +20,7 @@

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;
Expand All @@ -29,17 +31,22 @@
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})
Expand All @@ -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();
Expand Down Expand Up @@ -158,6 +169,59 @@ public void testCommitterProducerClosedOnError() throws IOException, Interrupted
}
}

@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();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't this also trigger by some kind of metadata request that the internal kafkaProducer does? I want to avoid interrupting before we actually call commit.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We are only listening right before commitTransaction. So any other producer initialization would be done already. However, since we use resumeTransaction with reflection hacks, we actually don't communicate with the broker at all (it's not available for this test by design).

Note that commitTransaction actually issues two requests (one is a metadata, one the actual commit). I haven't found a way or a reason to only wait on the second without relying on internals. WDYT?

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)
Expand Down
Loading