17
17
package org .apache .kafka .streams .errors ;
18
18
19
19
import org .apache .kafka .clients .consumer .ConsumerRecord ;
20
+ import org .apache .kafka .clients .producer .ProducerRecord ;
20
21
import org .apache .kafka .common .Configurable ;
21
22
import org .apache .kafka .streams .errors .internals .DefaultErrorHandlerContext ;
22
23
import org .apache .kafka .streams .processor .ProcessorContext ;
23
24
25
+ import java .util .Collections ;
26
+ import java .util .List ;
27
+
24
28
/**
25
29
* Interface that specifies how an exception from source node deserialization
26
30
* (e.g., reading from Kafka) should be handled.
@@ -63,16 +67,35 @@ default DeserializationHandlerResponse handle(final ProcessorContext context,
63
67
* The actual exception.
64
68
*
65
69
* @return Whether to continue or stop processing.
70
+ *
71
+ * @deprecated Use {@link #handleError(ErrorHandlerContext, ConsumerRecord, Exception)} instead.
66
72
*/
73
+ @ Deprecated
67
74
default DeserializationHandlerResponse handle (final ErrorHandlerContext context ,
68
75
final ConsumerRecord <byte [], byte []> record ,
69
76
final Exception exception ) {
70
77
return handle (((DefaultErrorHandlerContext ) context ).processorContext ().orElse (null ), record , exception );
71
78
}
72
79
80
+ /**
81
+ * Inspects a record and the exception received during deserialization.
82
+ *
83
+ * @param context
84
+ * Error handler context.
85
+ * @param record
86
+ * Record that failed deserialization.
87
+ * @param exception
88
+ * The actual exception.
89
+ *
90
+ * @return a {@link Response} object
91
+ */
92
+ default Response handleError (final ErrorHandlerContext context , final ConsumerRecord <byte [], byte []> record , final Exception exception ) {
93
+ return new Response (Result .from (handle (context , record , exception )), Collections .emptyList ());
94
+ }
73
95
/**
74
96
* Enumeration that describes the response from the exception handler.
75
97
*/
98
+ @ Deprecated
76
99
enum DeserializationHandlerResponse {
77
100
/** Continue processing. */
78
101
CONTINUE (0 , "CONTINUE" ),
@@ -95,4 +118,137 @@ enum DeserializationHandlerResponse {
95
118
}
96
119
}
97
120
121
+ /**
122
+ * Enumeration that describes the response from the exception handler.
123
+ */
124
+ enum Result {
125
+ /** Continue processing. */
126
+ RESUME (0 , "RESUME" ),
127
+ /** Fail processing. */
128
+ FAIL (1 , "FAIL" );
129
+
130
+ /**
131
+ * An english description for the used option. This is for debugging only and may change.
132
+ */
133
+ public final String name ;
134
+
135
+ /**
136
+ * The permanent and immutable id for the used option. This can't change ever.
137
+ */
138
+ public final int id ;
139
+
140
+ Result (final int id , final String name ) {
141
+ this .id = id ;
142
+ this .name = name ;
143
+ }
144
+
145
+ /**
146
+ * Converts the deprecated enum DeserializationHandlerResponse into the new Result enum.
147
+ *
148
+ * @param value the old DeserializationHandlerResponse enum value
149
+ * @return a {@link Result} enum value
150
+ * @throws IllegalArgumentException if the provided value does not map to a valid {@link Result}
151
+ */
152
+ private static DeserializationExceptionHandler .Result from (final DeserializationHandlerResponse value ) {
153
+ switch (value ) {
154
+ case FAIL :
155
+ return Result .FAIL ;
156
+ case CONTINUE :
157
+ return Result .RESUME ;
158
+ default :
159
+ throw new IllegalArgumentException ("No Result enum found for old value: " + value );
160
+ }
161
+ }
162
+ }
163
+
164
+ /**
165
+ * Represents the result of handling a deserialization exception.
166
+ * <p>
167
+ * The {@code Response} class encapsulates a {@link Result},
168
+ * indicating whether processing should continue or fail, along with an optional list of
169
+ * {@link ProducerRecord} instances to be sent to a dead letter queue.
170
+ * </p>
171
+ */
172
+ class Response {
173
+
174
+ private final Result result ;
175
+
176
+ private final List <ProducerRecord <byte [], byte []>> deadLetterQueueRecords ;
177
+
178
+ /**
179
+ * Constructs a new {@code DeserializationExceptionResponse} object.
180
+ *
181
+ * @param result the result indicating whether processing should continue or fail;
182
+ * must not be {@code null}.
183
+ * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}.
184
+ */
185
+ private Response (final Result result ,
186
+ final List <ProducerRecord <byte [], byte []>> deadLetterQueueRecords ) {
187
+ this .result = result ;
188
+ this .deadLetterQueueRecords = deadLetterQueueRecords ;
189
+ }
190
+
191
+ /**
192
+ * Creates a {@code Response} indicating that processing should fail.
193
+ *
194
+ * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}.
195
+ * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#FAIL} status.
196
+ */
197
+ public static Response fail (final List <ProducerRecord <byte [], byte []>> deadLetterQueueRecords ) {
198
+ return new Response (Result .FAIL , deadLetterQueueRecords );
199
+ }
200
+
201
+ /**
202
+ * Creates a {@code Response} indicating that processing should fail.
203
+ *
204
+ * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#FAIL} status.
205
+ */
206
+ public static Response fail () {
207
+ return fail (Collections .emptyList ());
208
+ }
209
+
210
+ /**
211
+ * Creates a {@code Response} indicating that processing should continue.
212
+ *
213
+ * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}.
214
+ * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#RESUME} status.
215
+ */
216
+ public static Response resume (final List <ProducerRecord <byte [], byte []>> deadLetterQueueRecords ) {
217
+ return new Response (Result .RESUME , deadLetterQueueRecords );
218
+ }
219
+
220
+ /**
221
+ * Creates a {@code Response} indicating that processing should continue.
222
+ *
223
+ * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#RESUME} status.
224
+ */
225
+ public static Response resume () {
226
+ return resume (Collections .emptyList ());
227
+ }
228
+
229
+ /**
230
+ * Retrieves the deserialization handler result.
231
+ *
232
+ * @return the {@link Result} indicating whether processing should continue or fail.
233
+ */
234
+ public Result result () {
235
+ return result ;
236
+ }
237
+
238
+ /**
239
+ * Retrieves an unmodifiable list of records to be sent to the dead letter queue.
240
+ * <p>
241
+ * If the list is {@code null}, an empty list is returned.
242
+ * </p>
243
+ *
244
+ * @return an unmodifiable list of {@link ProducerRecord} instances
245
+ * for the dead letter queue, or an empty list if no records are available.
246
+ */
247
+ public List <ProducerRecord <byte [], byte []>> deadLetterQueueRecords () {
248
+ if (deadLetterQueueRecords == null ) {
249
+ return Collections .emptyList ();
250
+ }
251
+ return Collections .unmodifiableList (deadLetterQueueRecords );
252
+ }
253
+ }
98
254
}
0 commit comments