Skip to content

Commit

Permalink
[hotfix][kafka] Throw FlinkKafkaProducer011Exception with error codes…
Browse files Browse the repository at this point in the history
… instead of generic Exception
  • Loading branch information
pnowojski authored and aljoscha committed Nov 23, 2017
1 parent ccf917d commit 2ac32c5
Show file tree
Hide file tree
Showing 3 changed files with 81 additions and 9 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* 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
*
* https://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,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.connectors.kafka;

/**
* Error codes used in {@link FlinkKafka011Exception}.
*/
public enum FlinkKafka011ErrorCode {
PRODUCERS_POOL_EMPTY,
EXTERNAL_ERROR
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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
*
* https://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,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.streaming.connectors.kafka;

import org.apache.flink.util.FlinkException;

/**
* Exception used by {@link FlinkKafkaProducer011} and {@link FlinkKafkaConsumer011}.
*/
public class FlinkKafka011Exception extends FlinkException {

private final FlinkKafka011ErrorCode errorCode;

public FlinkKafka011Exception(FlinkKafka011ErrorCode errorCode, String message) {
super(message);
this.errorCode = errorCode;
}

public FlinkKafka011Exception(FlinkKafka011ErrorCode errorCode, String message, Throwable cause) {
super(message, cause);
this.errorCode = errorCode;
}

public FlinkKafka011ErrorCode getErrorCode() {
return errorCode;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -551,7 +551,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) {
}

@Override
public void invoke(KafkaTransactionState transaction, IN next, Context context) throws Exception {
public void invoke(KafkaTransactionState transaction, IN next, Context context) throws FlinkKafka011Exception {
checkErroneous();

byte[] serializedKey = schema.serializeKey(next);
Expand Down Expand Up @@ -587,7 +587,7 @@ record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, seria
}

@Override
public void close() throws Exception {
public void close() throws FlinkKafka011Exception {
final KafkaTransactionState currentTransaction = currentTransaction();
if (currentTransaction != null) {
// to avoid exceptions on aborting transactions with some pending records
Expand All @@ -612,7 +612,7 @@ public void close() throws Exception {
// ------------------- Logic for handling checkpoint flushing -------------------------- //

@Override
protected KafkaTransactionState beginTransaction() throws Exception {
protected KafkaTransactionState beginTransaction() throws FlinkKafka011Exception {
switch (semantic) {
case EXACTLY_ONCE:
FlinkKafkaProducer<byte[], byte[]> producer = createOrGetProducerFromPool();
Expand All @@ -631,12 +631,13 @@ protected KafkaTransactionState beginTransaction() throws Exception {
}
}

private FlinkKafkaProducer<byte[], byte[]> createOrGetProducerFromPool() throws Exception {
private FlinkKafkaProducer<byte[], byte[]> createOrGetProducerFromPool() throws FlinkKafka011Exception {
FlinkKafkaProducer<byte[], byte[]> producer = getProducersPool().poll();
if (producer == null) {
String transactionalId = availableTransactionalIds.poll();
if (transactionalId == null) {
throw new Exception(
throw new FlinkKafka011Exception(
FlinkKafka011ErrorCode.PRODUCERS_POOL_EMPTY,
"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checkpoints.");
}
producer = initTransactionalProducer(transactionalId, true);
Expand All @@ -646,7 +647,7 @@ private FlinkKafkaProducer<byte[], byte[]> createOrGetProducerFromPool() throws
}

@Override
protected void preCommit(KafkaTransactionState transaction) throws Exception {
protected void preCommit(KafkaTransactionState transaction) throws FlinkKafka011Exception {
switch (semantic) {
case EXACTLY_ONCE:
case AT_LEAST_ONCE:
Expand Down Expand Up @@ -740,7 +741,7 @@ private void acknowledgeMessage() {
* Flush pending records.
* @param transaction
*/
private void flush(KafkaTransactionState transaction) throws Exception {
private void flush(KafkaTransactionState transaction) throws FlinkKafka011Exception {
if (transaction.producer != null) {
transaction.producer.flush();
}
Expand Down Expand Up @@ -936,12 +937,15 @@ private FlinkKafkaProducer<byte[], byte[]> initProducer(boolean registerMetrics)
return producer;
}

private void checkErroneous() throws Exception {
private void checkErroneous() throws FlinkKafka011Exception {
Exception e = asyncException;
if (e != null) {
// prevent double throwing
asyncException = null;
throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
throw new FlinkKafka011Exception(
FlinkKafka011ErrorCode.EXTERNAL_ERROR,
"Failed to send data to Kafka: " + e.getMessage(),
e);
}
}

Expand Down

0 comments on commit 2ac32c5

Please sign in to comment.