sebastienviale commented on code in PR #17942: URL: https://github.com/apache/kafka/pull/17942#discussion_r2210162603
########## streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java: ########## @@ -0,0 +1,114 @@ +/* + * 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.streams.errors; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; +import org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.InternalMockProcessorContext; +import org.apache.kafka.test.MockRecordCollector; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.Collections; +import java.util.Iterator; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +@ExtendWith(MockitoExtension.class) +public class ExceptionHandlerUtilsTest { + @Test + public void checkDealLetterQueueRecords() { Review Comment: fixed to checkDeadLetterQueueRecords ########## streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java: ########## @@ -0,0 +1,101 @@ +/* + * 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.streams.errors.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.InvalidConfigurationException; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.ErrorHandlerContext; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Collections; +import java.util.List; + +/** + * {@code ExceptionHandlerUtils} Contains utilities method that could be used by all exception handlers + */ +public class ExceptionHandlerUtils { + public static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; + public static final String HEADER_ERRORS_STACKTRACE_NAME = "__streams.errors.stacktrace"; + public static final String HEADER_ERRORS_EXCEPTION_MESSAGE_NAME = "__streams.errors.message"; + public static final String HEADER_ERRORS_TOPIC_NAME = "__streams.errors.topic"; + public static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; + public static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; + + + public static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopicName) { + return deadLetterQueueTopicName != null; + } + + /** + * If required, return Dead Letter Queue records for the provided exception + * + * @param key Serialized key for the records + * @param value Serialized value for the records + * @param context ErrorHandlerContext of the exception + * @param exception Thrown exception + * @return A list of Dead Letter Queue records to produce + */ + public static List<ProducerRecord<byte[], byte[]>> maybeBuildDeadLetterQueueRecords(final String deadLetterQueueTopicName, + final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception exception) { + if (!shouldBuildDeadLetterQueueRecord(deadLetterQueueTopicName)) { + return Collections.emptyList(); + } + + return Collections.singletonList(buildDeadLetterQueueRecord(deadLetterQueueTopicName, key, value, context, exception)); + } + + + /** + * Build dead letter queue record for the provided exception. + * + * @param key Serialized key for the record. + * @param value Serialized value for the record. + * @param context error handler context of the exception. + * @return A dead letter queue record to produce. + */ + public static ProducerRecord<byte[], byte[]> buildDeadLetterQueueRecord(final String deadLetterQueueTopicName, + final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception e) { + if (deadLetterQueueTopicName == null) { + throw new InvalidConfigurationException(String.format("%s cannot be null while building dead letter queue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + } + final ProducerRecord<byte[], byte[]> producerRecord = new ProducerRecord<>(deadLetterQueueTopicName, null, context.timestamp(), key, value); + final StringWriter stackStraceStringWriter = new StringWriter(); + final PrintWriter stackTracePrintWriter = new PrintWriter(stackStraceStringWriter); + e.printStackTrace(stackTracePrintWriter); + + try (final StringSerializer stringSerializer = new StringSerializer()) { + producerRecord.headers().add(HEADER_ERRORS_EXCEPTION_NAME, stringSerializer.serialize(null, e.toString())); + producerRecord.headers().add(HEADER_ERRORS_EXCEPTION_MESSAGE_NAME, stringSerializer.serialize(null, e.getMessage())); + producerRecord.headers().add(HEADER_ERRORS_STACKTRACE_NAME, stringSerializer.serialize(null, stackStraceStringWriter.toString())); Review Comment: fixed to stackTraceStringWriter -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org