Skip to content

feat: support of errors.tolerance and dead letter queue #143

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

Merged
merged 6 commits into from
Jun 23, 2025
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
2 changes: 1 addition & 1 deletion .github/ISSUE_TEMPLATE/BUG-REPORT.yml
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ body:
description: What version of our software are you running?
options:
- latest
- 2.3.0
- 2.4.0 (Default)
- 1.3.5
- older (<1.3.5)
validations:
Expand Down
114 changes: 67 additions & 47 deletions README.md

Large diffs are not rendered by default.

6 changes: 6 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,12 @@
<version>3.7.1</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>connect-runtime</artifactId>
<version>3.7.1</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>javax.jms</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ protected JMSContext getJmsContext() throws Exception {
return jmsContext;
}

protected Map<String, String> getDefaultConnectorProperties() {
public static Map<String, String> getDefaultConnectorProperties() {
final Map<String, String> props = new HashMap<>();
props.put("mq.queue.manager", QMGR_NAME);
props.put("mq.connection.mode", "client");
Expand All @@ -106,7 +106,7 @@ protected Map<String, String> getDefaultConnectorProperties() {
return props;
}

public AbstractConfig getPropertiesConfig(Map<String, String> props) {
public AbstractConfig getPropertiesConfig(final Map<String, String> props) {
final AbstractConfig connectorConfig = new AbstractConfig(MQSourceConnector.CONFIGDEF, props);
return connectorConfig;
}
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,15 @@
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;

import java.util.HashMap;
import java.util.List;
import java.util.Map;

import javax.jms.BytesMessage;
import javax.jms.MapMessage;
import javax.jms.TextMessage;

import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.Test;

Expand Down Expand Up @@ -102,4 +104,50 @@ public void buildFromJmsMapMessage() throws Exception {
// verify the exception
assertEquals("Unsupported JMS message type", exc.getMessage());
}

@Test
public void buildFromJmsTestJsonError() throws Exception {
// create MQ message
final TextMessage message = getJmsContext().createTextMessage("Not a valid json string");

// use the builder to convert it to a Kafka record
final JsonRecordBuilder builder = new JsonRecordBuilder();
final DataException exec = assertThrows(DataException.class, () -> builder.toSourceRecord(getJmsContext(), topic, isJMS, message));
assertEquals("Converting byte[] to Kafka Connect data failed due to serialization error: ", exec.getMessage());
}

@Test
public void buildFromJmsTestErrorTolerance() throws Exception {
// create MQ message
final TextMessage message = getJmsContext().createTextMessage("Not a valid json string");

// use the builder to convert it to a Kafka record
final JsonRecordBuilder builder = new JsonRecordBuilder();
final Map<String, String> config = AbstractJMSContextIT.getDefaultConnectorProperties();
config.put("errors.tolerance", "all");
config.put("mq.message.body.jms", "true");
config.put("mq.record.builder", "com.ibm.eventstreams.connect.mqsource.builders.JsonRecordBuilder");

builder.configure(config);
final SourceRecord record = builder.toSourceRecord(getJmsContext(), topic, isJMS, message);
assertNull(record);
}

@Test
public void buildFromJmsTestErrorToleranceNone() throws Exception {
// create MQ message
final TextMessage message = getJmsContext().createTextMessage("Not a valid json string");

// use the builder to convert it to a Kafka record
final JsonRecordBuilder builder = new JsonRecordBuilder();
final HashMap<String, String> config = new HashMap<String, String>();
config.put("errors.tolerance", "none");
config.put("mq.message.body.jms", "true");
config.put("mq.record.builder", "com.ibm.eventstreams.connect.mqsource.builders.JsonRecordBuilder");

builder.configure(config);
assertThrows(DataException.class, () -> {
builder.toSourceRecord(getJmsContext(), topic, isJMS, message);
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,22 @@ public class MQSourceConnector extends SourceConnector {
public static final long CONFIG_RECONNECT_DELAY_MAX_DEFAULT = 8192L;
public static final long CONFIG_RECONNECT_DELAY_MAX_MINIMUM = 10L;

public static final String DLQ_PREFIX = "errors.deadletterqueue.";

public static final String DLQ_TOPIC_NAME_CONFIG = DLQ_PREFIX + "topic.name";
public static final String DLQ_TOPIC_NAME_DOC = "The name of the topic to be used as the dead letter queue (DLQ) for messages that " +
"result in an error when processed by this source connector, or its transformations or converters. The topic name is blank by default, " +
"which means that no messages are to be recorded in the DLQ.";
public static final String DLQ_TOPIC_DEFAULT = "";
private static final String DLQ_TOPIC_DISPLAY = "Dead Letter Queue Topic Name";

public static final String DLQ_CONTEXT_HEADERS_ENABLE_CONFIG = DLQ_PREFIX + "context.headers.enable";
public static final boolean DLQ_CONTEXT_HEADERS_ENABLE_DEFAULT = false;
public static final String DLQ_CONTEXT_HEADERS_ENABLE_DOC = "If true, add headers containing error context to the messages " +
"written to the dead letter queue. To avoid clashing with headers from the original record, all error context header " +
"keys, all error context header keys will start with <code>__connect.errors.</code>";
private static final String DLQ_CONTEXT_HEADERS_ENABLE_DISPLAY = "Enable Error Context Headers";

// Define valid reconnect options
public static final String[] CONFIG_VALUE_MQ_VALID_RECONNECT_OPTIONS = {
CONFIG_VALUE_MQ_CLIENT_RECONNECT_OPTION_ASDEF,
Expand Down Expand Up @@ -616,6 +632,22 @@ null, new ReadableFile(),
CONFIG_GROUP_MQ, 29,
Width.MEDIUM,
CONFIG_DISPLAY_RECONNECT_DELAY_MAX);
CONFIGDEF.define(DLQ_TOPIC_NAME_CONFIG,
Type.STRING,
DLQ_TOPIC_DEFAULT,
Importance.MEDIUM,
DLQ_TOPIC_NAME_DOC,
CONFIG_GROUP_MQ, 30,
Width.MEDIUM,
DLQ_TOPIC_DISPLAY);
CONFIGDEF.define(DLQ_CONTEXT_HEADERS_ENABLE_CONFIG,
Type.BOOLEAN,
DLQ_CONTEXT_HEADERS_ENABLE_DEFAULT,
Importance.MEDIUM,
DLQ_CONTEXT_HEADERS_ENABLE_DOC,
CONFIG_GROUP_MQ, 31,
Width.MEDIUM,
DLQ_CONTEXT_HEADERS_ENABLE_DISPLAY);

CONFIGDEF.define(CONFIG_NAME_TOPIC,
Type.STRING,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@ public void start(final Map<String, String> props) {
final JMSWorker reader = new JMSWorker();
JMSWorker dedicated = null;
SequenceStateClient client = null;

if (MQSourceConnector.configSupportsExactlyOnce(props)) {
dedicated = new JMSWorker();
client = new SequenceStateClient(props.get(CONFIG_NAME_MQ_EXACTLY_ONCE_STATE_QUEUE), reader, dedicated);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,45 +15,62 @@
*/
package com.ibm.eventstreams.connect.mqsource.builders;

import com.ibm.eventstreams.connect.mqsource.MQSourceConnector;
import com.ibm.eventstreams.connect.mqsource.processor.JmsToKafkaHeaderConverter;
import java.util.Map;
import java.util.Optional;

import javax.jms.JMSContext;
import javax.jms.JMSException;
import javax.jms.Message;

import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.source.SourceRecord;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.jms.JMSContext;
import javax.jms.JMSException;
import javax.jms.Message;
import java.util.Map;
import java.util.Optional;
import com.ibm.eventstreams.connect.mqsource.MQSourceConnector;
import com.ibm.eventstreams.connect.mqsource.processor.JmsToKafkaHeaderConverter;
import com.ibm.eventstreams.connect.mqsource.util.ErrorHandler;

/**
* Builds Kafka Connect SourceRecords from messages.
*/
public abstract class BaseRecordBuilder implements RecordBuilder {
private static final Logger log = LoggerFactory.getLogger(BaseRecordBuilder.class);

public enum KeyHeader { NONE, MESSAGE_ID, CORRELATION_ID, CORRELATION_ID_AS_BYTES, DESTINATION };
protected KeyHeader keyheader = KeyHeader.NONE;
public enum KeyHeader {
NONE, MESSAGE_ID, CORRELATION_ID, CORRELATION_ID_AS_BYTES, DESTINATION
};

protected KeyHeader keyheader = KeyHeader.NONE;

private boolean copyJmsPropertiesFlag = Boolean.FALSE;
private JmsToKafkaHeaderConverter jmsToKafkaHeaderConverter;
private ErrorHandler errorHandler = new ErrorHandler();

/**
* Configure this class.
*
* @param props initial configuration
*
* @throws RecordBuilderException Operation failed and connector should stop.
* @throws RecordBuilderException Operation failed and connector should stop.
*/
@Override public void configure(final Map<String, String> props) {
@Override
public void configure(final Map<String, String> props) {
log.trace("[{}] Entry {}.configure, props={}", Thread.currentThread().getId(), this.getClass().getName(),
props);

configureKeyHeader(props);
configureJmsProperties(props);
configureErrorHandler(props);

log.trace("[{}] Exit {}.configure", Thread.currentThread().getId(), this.getClass().getName());
}

/**
* Configure key header settings.
*/
private void configureKeyHeader(final Map<String, String> props) {
final String kh = props.get(MQSourceConnector.CONFIG_NAME_MQ_RECORD_BUILDER_KEY_HEADER);
if (kh != null) {
if (kh.equals(MQSourceConnector.CONFIG_VALUE_MQ_RECORD_BUILDER_KEY_HEADER_JMSMESSAGEID)) {
Expand All @@ -73,12 +90,22 @@ public enum KeyHeader { NONE, MESSAGE_ID, CORRELATION_ID, CORRELATION_ID_AS_BYTE
throw new RecordBuilderException("Unsupported MQ record builder key header value");
}
}
}

/**
* Configure JMS properties settings.
*/
private void configureJmsProperties(final Map<String, String> props) {
final String str = props.get(MQSourceConnector.CONFIG_NAME_MQ_JMS_PROPERTY_COPY_TO_KAFKA_HEADER);
copyJmsPropertiesFlag = Boolean.parseBoolean(Optional.ofNullable(str).orElse("false"));
jmsToKafkaHeaderConverter = new JmsToKafkaHeaderConverter();
}

log.trace("[{}] Exit {}.configure", Thread.currentThread().getId(), this.getClass().getName());
/**
* Configure error handler.
*/
public void configureErrorHandler(final Map<String, String> props) {
errorHandler.configure(props, copyJmsPropertiesFlag, jmsToKafkaHeaderConverter);
}

/**
Expand Down Expand Up @@ -160,38 +187,58 @@ public abstract SchemaAndValue getValue(JMSContext context, String topic, boolea
* @throws JMSException Message could not be converted
*/
@Override
public SourceRecord toSourceRecord(final JMSContext context, final String topic, final boolean messageBodyJms, final Message message) throws JMSException {
public SourceRecord toSourceRecord(final JMSContext context, final String topic, final boolean messageBodyJms,
final Message message) throws JMSException {
return toSourceRecord(context, topic, messageBodyJms, message, null, null);
}

@Override
public SourceRecord toSourceRecord(final JMSContext context, final String topic, final boolean messageBodyJms, final Message message, final Map<String, Long> sourceOffset, final Map<String, String> sourceQueuePartition) throws JMSException {
final SchemaAndValue key = this.getKey(context, topic, message);
final SchemaAndValue value = this.getValue(context, topic, messageBodyJms, message);

if (copyJmsPropertiesFlag && messageBodyJms) {
return new SourceRecord(
sourceQueuePartition,
sourceOffset,
topic,
null,
key.schema(),
key.value(),
value.schema(),
value.value(),
message.getJMSTimestamp(),
jmsToKafkaHeaderConverter.convertJmsPropertiesToKafkaHeaders(message)
);
} else {
return new SourceRecord(
sourceQueuePartition,
sourceOffset,
topic,
key.schema(),
key.value(),
value.schema(),
value.value()
);
public SourceRecord toSourceRecord(final JMSContext context, final String topic, final boolean messageBodyJms,
final Message message, final Map<String, Long> sourceOffset, final Map<String, String> sourceQueuePartition)
throws JMSException {

SchemaAndValue key = new SchemaAndValue(null, null);

try {
// Extract key and value
final SchemaAndValue value = this.getValue(context, topic, messageBodyJms, message);
key = this.getKey(context, topic, message);

// Create and return appropriate record based on configuration
if (copyJmsPropertiesFlag && messageBodyJms) {
return new SourceRecord(
sourceQueuePartition,
sourceOffset,
topic,
null,
key.schema(),
key.value(),
value.schema(),
value.value(),
message.getJMSTimestamp(),
jmsToKafkaHeaderConverter.convertJmsPropertiesToKafkaHeaders(message));
} else {
return new SourceRecord(
sourceQueuePartition,
sourceOffset,
topic,
null,
key.schema(),
key.value(),
value.schema(),
value.value());
}
} catch (final Exception e) {
// Log the error using error handler
errorHandler.logError(e, topic, message);

// If errors are not tolerated, rethrow
if (!errorHandler.shouldTolerateErrors()) {
throw e;
}

// Handle the error based on configured error tolerance
return errorHandler.handleBuildException(message, sourceQueuePartition, sourceOffset, topic, key, e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@

import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.json.JsonConverter;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Loading