Skip to content

Deserialization failure handling #15

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Closed
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
15 changes: 15 additions & 0 deletions flink-jar-runner/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,12 @@
<inceptionYear>2024</inceptionYear>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
Expand Down Expand Up @@ -251,6 +257,15 @@
</excludes>
</artifactSet>
<filters>
<filter>
<artifact>org.apache.flink:flink-connector-kafka</artifact>
<excludes>
<exclude>org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema**</exclude>
<exclude>org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource**</exclude>
<exclude>org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory**</exclude>
<exclude>org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory**</exclude>
</excludes>
</filter>
<filter>
<!-- Do not copy the signatures in the META-INF folder.
Otherwise, this might cause SecurityExceptions when using the JAR. -->
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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
*
* 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.table;

import org.apache.flink.configuration.ReadableConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;
import java.io.IOException;
import java.io.Serializable;
import java.util.Properties;

import static org.apache.flink.streaming.connectors.kafka.table.DeserFailureHandlerOptions.SCAN_DESER_FAILURE_HANDLER;
import static org.apache.flink.streaming.connectors.kafka.table.DeserFailureHandlerOptions.SCAN_DESER_FAILURE_TOPIC;

public class DeserFailureHandler {

private static final Logger LOG = LoggerFactory.getLogger(DeserFailureHandler.class);

private final DeserFailureHandlerType handlerType;
private final @Nullable DeserFailureProducer producer;

DeserFailureHandler(DeserFailureHandlerType handlerType, @Nullable DeserFailureProducer producer) {
this.handlerType = handlerType;
this.producer = producer;
}

static DeserFailureHandler of(ReadableConfig tableOptions, Properties consumerProps) {
DeserFailureHandlerType handlerType = tableOptions.get(SCAN_DESER_FAILURE_HANDLER);

DeserFailureProducer producer =
handlerType == DeserFailureHandlerType.KAFKA
? new DeserFailureProducer(tableOptions.get(SCAN_DESER_FAILURE_TOPIC), consumerProps)
: null;

return new DeserFailureHandler(handlerType, producer);
}

void deserWithFailureHandling(ConsumerRecord<byte[], byte[]> record, DeserializationCaller deser)
throws IOException {

try {
deser.call();
} catch (IOException e) {
if (DeserFailureHandlerType.NONE == handlerType) {
throw e;

} else if (DeserFailureHandlerType.LOG == handlerType) {
LOG.info(
"Deserialization failure occurred for record. Topic: {}, Partition: {}, Offset: {}",
record.topic(),
record.partition(),
record.offset());

} else if (DeserFailureHandlerType.KAFKA == handlerType) {
LOG.info(
"Deserialization failure occurred for record, sending it to the configured topic ({}). Topic: {}, Partition: {}, Offset: {}",
producer.getTopic(),
record.topic(),
record.partition(),
record.offset());
producer.send(record);
}
}
}

interface DeserializationCaller extends Serializable {
void call() throws IOException;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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
*
* 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.table;

import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.util.StringUtils;

public class DeserFailureHandlerOptions {

public static final ConfigOption<DeserFailureHandlerType> SCAN_DESER_FAILURE_HANDLER =
ConfigOptions.key("scan.deser-failure.handler")
.enumType(DeserFailureHandlerType.class)
.defaultValue(DeserFailureHandlerType.NONE);

public static final ConfigOption<String> SCAN_DESER_FAILURE_TOPIC =
ConfigOptions.key("scan.deser-failure.topic")
.stringType()
.noDefaultValue();

public static void validateDeserFailureHandlerOptions(ReadableConfig tableOptions) {
var handler = tableOptions.get(SCAN_DESER_FAILURE_HANDLER);
var topic = tableOptions.get(SCAN_DESER_FAILURE_TOPIC);

if (handler == DeserFailureHandlerType.KAFKA && StringUtils.isNullOrWhitespaceOnly(topic)) {
throw new ValidationException(
String.format(
"'%s' is set to '%s', but '%s' is not specified.",
SCAN_DESER_FAILURE_HANDLER.key(),
DeserFailureHandlerType.KAFKA,
SCAN_DESER_FAILURE_TOPIC.key()));
}

if (handler != DeserFailureHandlerType.KAFKA && !StringUtils.isNullOrWhitespaceOnly(topic)) {
throw new ValidationException(
String.format(
"'%s' is not set to '%s', but '%s' is specified.",
SCAN_DESER_FAILURE_HANDLER.key(),
DeserFailureHandlerType.KAFKA,
SCAN_DESER_FAILURE_TOPIC.key()));
}
}

private DeserFailureHandlerOptions() {}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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
*
* 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.table;

public enum DeserFailureHandlerType {
/**
* No deserialization failure handling is applied.
* In case of a problematic record, the application will fail.
* This is the default setting.
*/
NONE,

/**
* In case of a problematic record, helpful information will be logged.
* The application continues the execution.
*/
LOG,

/**
* In case of a problematic record, helpful information will be logged,
* and the record will be sent to a configured Kafka topic as well.
* The application continues the execution.
*/
KAFKA
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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
*
* 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.table;

import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Serializable;
import java.util.Properties;

import static org.apache.flink.util.Preconditions.checkNotNull;

class DeserFailureProducer implements Serializable {

private static final Logger LOG = LoggerFactory.getLogger(DeserFailureProducer.class);

private final String topic;
private final Properties producerProps;

private transient KafkaProducer<byte[], byte[]> kafkaProducer;

DeserFailureProducer(String topic, Properties consumerProps) {
this.topic = checkNotNull(topic);

producerProps = new Properties();
producerProps.putAll(consumerProps);
producerProps.setProperty(
ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
producerProps.setProperty(
ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
}

private void init() {
if (kafkaProducer == null) {
LOG.debug("Initializing deserialization failure producer.");
kafkaProducer = new KafkaProducer<>(producerProps);
}
}

void send(ConsumerRecord<byte[], byte[]> record) {
init();

if (record == null) {
LOG.info("Unable to send deserialization failed record: Record was null.");
} else if (kafkaProducer == null) {
LOG.warn("Unable to send deserialization failed record: Kafka producer is not initialized.");
} else {
kafkaProducer.send(
new ProducerRecord<>(
topic,
null,
null,
record.key(),
record.value(),
record.headers()));
}
}

public String getTopic() {
return topic;
}
}
Loading