You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/11/16 16:57:44 UTC

[GitHub] pnowojski closed pull request #7108: [FLINK-10455][Kafka Tx] Close transactional producers in case of failure and termination

pnowojski closed pull request #7108: [FLINK-10455][Kafka Tx] Close transactional producers in case of failure and termination
URL: https://github.com/apache/flink/pull/7108
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
index adecab16d8c..d332cd0ca6a 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
@@ -45,6 +45,7 @@
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
 import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.NetUtils;
 
 import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
@@ -671,6 +672,9 @@ public void close() throws FlinkKafka011Exception {
 		}
 		// make sure we propagate pending errors
 		checkErroneous();
+		pendingTransactions().forEach(transaction ->
+			IOUtils.closeQuietly(transaction.getValue().producer)
+		);
 	}
 
 	// ------------------- Logic for handling checkpoint flushing -------------------------- //
@@ -713,8 +717,11 @@ protected void preCommit(KafkaTransactionState transaction) throws FlinkKafka011
 	@Override
 	protected void commit(KafkaTransactionState transaction) {
 		if (transaction.isTransactional()) {
-			transaction.producer.commitTransaction();
-			recycleTransactionalProducer(transaction.producer);
+			try {
+				transaction.producer.commitTransaction();
+			} finally {
+				recycleTransactionalProducer(transaction.producer);
+			}
 		}
 	}
 
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
index 8398aa8cdbf..f2725242b2c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
@@ -38,20 +38,24 @@
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
 import java.io.IOException;
 import java.time.Clock;
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.stream.Stream;
 
 import static java.util.Objects.requireNonNull;
 import static org.apache.flink.util.Preconditions.checkArgument;
@@ -149,6 +153,12 @@ protected TXN currentTransaction() {
 		return currentTransactionHolder == null ? null : currentTransactionHolder.handle;
 	}
 
+	@Nonnull
+	protected Stream<Map.Entry<Long, TXN>> pendingTransactions() {
+		return pendingCommitTransactions.entrySet().stream()
+			.map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().handle));
+	}
+
 	// ------ methods that should be implemented in child class to support two phase commit algorithm ------
 
 	/**
@@ -256,6 +266,7 @@ public final void notifyCheckpointComplete(long checkpointId) throws Exception {
 
 		Iterator<Map.Entry<Long, TransactionHolder<TXN>>> pendingTransactionIterator = pendingCommitTransactions.entrySet().iterator();
 		checkState(pendingTransactionIterator.hasNext(), "checkpoint completed, but no transaction pending");
+		Throwable firstError = null;
 
 		while (pendingTransactionIterator.hasNext()) {
 			Map.Entry<Long, TransactionHolder<TXN>> entry = pendingTransactionIterator.next();
@@ -269,12 +280,23 @@ public final void notifyCheckpointComplete(long checkpointId) throws Exception {
 				name(), checkpointId, pendingTransaction, pendingTransactionCheckpointId);
 
 			logWarningIfTimeoutAlmostReached(pendingTransaction);
-			commit(pendingTransaction.handle);
+			try {
+				commit(pendingTransaction.handle);
+			} catch (Throwable t) {
+				if (firstError == null) {
+					firstError = t;
+				}
+			}
 
 			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
 
 			pendingTransactionIterator.remove();
 		}
+
+		if (firstError != null) {
+			throw new FlinkRuntimeException("Committing one of transactions failed, logging first encountered failure",
+				firstError);
+		}
 	}
 
 	@Override


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services