Skip to content
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

KAFKA-15309: part 1-Add custom error handler for RecordTooLargeException to Producer #15846

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from
Open
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
Expand Up @@ -262,6 +262,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
private final ApiVersions apiVersions;
private final TransactionManager transactionManager;
private final Optional<ClientTelemetryReporter> clientTelemetryReporter;
private final ProducerExceptionHandler customExceptionHandler;

/**
* A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
Expand Down Expand Up @@ -414,6 +415,7 @@ private void warnIfPartitionerDeprecated() {
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
this.customExceptionHandler = config.getProducerExceptionHandler();

this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG);
int deliveryTimeoutMs = configureDeliveryTimeout(config, log);
Expand Down Expand Up @@ -502,6 +504,7 @@ private void warnIfPartitionerDeprecated() {
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
this.customExceptionHandler = config.getProducerExceptionHandler();
this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG);
this.partitionerIgnoreKeys = config.getBoolean(ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG);
this.apiVersions = new ApiVersions();
Expand Down Expand Up @@ -1081,7 +1084,20 @@ private Future<RecordMetadata> doSend(ProducerRecord<K, V> record, Callback call
this.errors.record();
this.interceptors.onSendError(record, appendCallbacks.topicPartition(), e);
if (transactionManager != null) {
transactionManager.maybeTransitionToErrorState(e);
ProducerExceptionHandler.Response response;
if (e instanceof RecordTooLargeException && customExceptionHandler != null && (response = customExceptionHandler.handle(record, e)) != null) {
if (response == ProducerExceptionHandler.Response.SWALLOW) {
log.info("The custom handler drops the too large record. Processing continues despite the too large record.");
} else {
if (response == ProducerExceptionHandler.Response.RETRY) {
log.error("RecordTooLargeException is not retriable.");
}
transactionManager.maybeTransitionToErrorState(e);
}

} else {
transactionManager.maybeTransitionToErrorState(e);
}
}
return new FutureFailure(e);
} catch (InterruptedException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,9 @@ public class ProducerConfig extends AbstractConfig {
"By default the TransactionId is not configured, which means transactions cannot be used. " +
"Note that, by default, transactions require a cluster of at least three brokers which is the recommended setting for production; for development you can change this, by adjusting broker setting <code>transaction.state.log.replication.factor</code>.";

public static final String CUSTOM_EXCEPTION_HANDLER_CLASS_CONFIG = "custom.exception.handler";
private static final String CUSTOM_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the <code>org.apache.kafka.clients.producer.ProducerExceptionHandler</code> interface.";

/**
* <code>security.providers</code>
*/
Expand Down Expand Up @@ -510,7 +513,16 @@ public class ProducerConfig extends AbstractConfig {
null,
new ConfigDef.NonEmptyString(),
Importance.LOW,
TRANSACTIONAL_ID_DOC);
TRANSACTIONAL_ID_DOC)
.define(CUSTOM_EXCEPTION_HANDLER_CLASS_CONFIG,
Type.CLASS,
null,
Importance.MEDIUM,
CUSTOM_EXCEPTION_HANDLER_CLASS_DOC);
}

public ProducerExceptionHandler getProducerExceptionHandler() {
return getConfiguredInstance(CUSTOM_EXCEPTION_HANDLER_CLASS_CONFIG, ProducerExceptionHandler.class);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.kafka.clients.producer;

import org.apache.kafka.common.Configurable;

/**
* Interface that specifies how an the RecordTooLargeException and/or UnknownTopicOrPartitionException should be handled.
* The accepted responses for RecordTooLargeException are FAIL and SWALLOW. Therefore, RETRY will be interpreted and executed as FAIL.
*/
public interface ProducerExceptionHandler extends Configurable {

/**
* Determine whether to stop processing, keep retrying internally, or swallow the error by dropping the record.
* For RecordTooLargeException RETRY will be interpreted and executed as FAIL.
*
* @param record The record that failed to produce
* @param exception The exception that occurred during production
*/
Response handle(final ProducerRecord record, final Exception exception);

enum Response {
/* stop processing: fail */
FAIL(0, "FAIL"),
/* continue: keep retrying */
RETRY(1, "RETRY"),
/* continue: swallow the error */
SWALLOW(2, "SWALLOW");

/**
* an english description of the api--this is for debugging and can change
*/
public final String name;

/**
* the permanent and immutable id of an API--this can't change ever
*/
public final int id;

Response(final int id, final String name) {
this.id = id;
this.name = name;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1406,6 +1406,38 @@ public void testCommitTransactionWithRecordTooLargeException() throws Exception
assertThrows(KafkaException.class, producer::commitTransaction);
}
}
@Test
public void testCommitTransactionWithRecordTooLargeExceptionAndCustomHandler() throws Exception {
Map<String, Object> configs = new HashMap<>();
configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "some.id");
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
configs.put(ProducerConfig.MAX_REQUEST_SIZE_CONFIG, 1000);
configs.put(ProducerConfig.CUSTOM_EXCEPTION_HANDLER_CLASS_CONFIG,
SwallowRecordTooLargeExceptionHandler.class.getName());

Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = mock(ProducerMetadata.class);
MockClient client = new MockClient(time, metadata);
client.updateMetadata(initialUpdateResponse);
client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", NODE));
client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE));

when(metadata.fetch()).thenReturn(onePartitionCluster);

String largeString = IntStream.range(0, 1000).mapToObj(i -> "*").collect(Collectors.joining());
ProducerRecord<String, String> largeRecord = new ProducerRecord<>(topic, "large string", largeString);

try (KafkaProducer<String, String> producer = kafkaProducer(configs, new StringSerializer(),
new StringSerializer(), metadata, client, null, time)) {
producer.initTransactions();

client.prepareResponse(endTxnResponse(Errors.NONE));
producer.beginTransaction();
TestUtils.assertFutureError(producer.send(largeRecord), RecordTooLargeException.class);
assertDoesNotThrow(producer::commitTransaction);
}
}

@Test
public void testCommitTransactionWithMetadataTimeoutForMissingTopic() throws Exception {
Expand Down Expand Up @@ -2498,4 +2530,19 @@ void testDeliveryTimeoutAndLingerMsConfig() {
assertDoesNotThrow(() -> new KafkaProducer<>(configs, new StringSerializer(), new StringSerializer()).close());
}

public static class SwallowRecordTooLargeExceptionHandler implements ProducerExceptionHandler {
@Override
public Response handle(ProducerRecord record, Exception exception) {
if (exception instanceof RecordTooLargeException) {
return Response.SWALLOW;
}
return null;
}

@Override
public void configure(Map<String, ?> configs) {

}
}

}