-
Notifications
You must be signed in to change notification settings - Fork 4.9k
/
KafkaFetchRecords.java
567 lines (485 loc) · 22.6 KB
/
KafkaFetchRecords.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
/*
* 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.camel.component.kafka;
import java.time.Duration;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantLock;
import java.util.regex.Pattern;
import org.apache.camel.component.kafka.consumer.CommitManager;
import org.apache.camel.component.kafka.consumer.CommitManagers;
import org.apache.camel.component.kafka.consumer.errorhandler.KafkaConsumerListener;
import org.apache.camel.component.kafka.consumer.errorhandler.KafkaErrorStrategies;
import org.apache.camel.component.kafka.consumer.support.KafkaConsumerResumeStrategy;
import org.apache.camel.component.kafka.consumer.support.KafkaRecordProcessorFacade;
import org.apache.camel.component.kafka.consumer.support.PartitionAssignmentListener;
import org.apache.camel.component.kafka.consumer.support.ProcessingResult;
import org.apache.camel.component.kafka.consumer.support.ResumeStrategyFactory;
import org.apache.camel.support.BridgeExceptionHandlerToErrorHandler;
import org.apache.camel.support.task.ForegroundTask;
import org.apache.camel.support.task.Tasks;
import org.apache.camel.support.task.budget.Budgets;
import org.apache.camel.util.IOHelper;
import org.apache.camel.util.ReflectionHelper;
import org.apache.camel.util.TimeUtils;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.errors.WakeupException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class KafkaFetchRecords implements Runnable {
/*
This keeps track of the state the record fetcher is. Because the Kafka consumer is not thread safe, it may take
some time between the pause or resume request is triggered and it is actually set.
Some of the states may be set but not read. This is done deliberately to avoid the code to enter multiple times
the branches that handle the *_REQUESTED states.
*/
private enum State {
RUNNING,
PAUSE_REQUESTED,
PAUSED,
RESUME_REQUESTED,
}
private static final Logger LOG = LoggerFactory.getLogger(KafkaFetchRecords.class);
private final KafkaConsumer kafkaConsumer;
private org.apache.kafka.clients.consumer.Consumer consumer;
private String clientId;
private final String topicName;
private final Pattern topicPattern;
private final String threadId;
private final Properties kafkaProps;
private final Map<String, Long> lastProcessedOffset = new HashMap<>();
private final PollExceptionStrategy pollExceptionStrategy;
private final BridgeExceptionHandlerToErrorHandler bridge;
private final ReentrantLock lock = new ReentrantLock();
private CommitManager commitManager;
private Exception lastError;
private final KafkaConsumerListener consumerListener;
// The following set of volatile variables are usually read from other threads,
// like from the health check thread. They are usually only read on those contexts.
private volatile boolean terminated;
private volatile long currentBackoffInterval;
private volatile boolean reconnect; // The reconnect must be false at init (this is the policy whether to reconnect).
private volatile boolean connected; // this is the state (connected or not)
private volatile State state = State.RUNNING;
KafkaFetchRecords(KafkaConsumer kafkaConsumer,
BridgeExceptionHandlerToErrorHandler bridge, String topicName, Pattern topicPattern, String id,
Properties kafkaProps, KafkaConsumerListener consumerListener) {
this.kafkaConsumer = kafkaConsumer;
this.bridge = bridge;
this.topicName = topicName;
this.topicPattern = topicPattern;
this.consumerListener = consumerListener;
this.threadId = topicName + "-" + "Thread " + id;
this.kafkaProps = kafkaProps;
this.pollExceptionStrategy = KafkaErrorStrategies.strategies(this, kafkaConsumer.getEndpoint(), consumer);
}
@Override
public void run() {
if (!isKafkaConsumerRunnable()) {
return;
}
do {
terminated = false;
if (!isConnected()) {
// task that deals with creating kafka consumer
currentBackoffInterval = kafkaConsumer.getEndpoint().getComponent().getCreateConsumerBackoffInterval();
ForegroundTask task = Tasks.foregroundTask()
.withName("Create KafkaConsumer")
.withBudget(Budgets.iterationBudget()
.withMaxIterations(
kafkaConsumer.getEndpoint().getComponent().getCreateConsumerBackoffMaxAttempts())
.withInitialDelay(Duration.ZERO)
.withInterval(Duration.ofMillis(currentBackoffInterval))
.build())
.build();
boolean success = task.run(this::createConsumerTask);
if (!success) {
int max = kafkaConsumer.getEndpoint().getComponent().getCreateConsumerBackoffMaxAttempts();
setupCreateConsumerException(task, max);
// give up and terminate this consumer
terminated = true;
break;
}
// task that deals with subscribing kafka consumer
currentBackoffInterval = kafkaConsumer.getEndpoint().getComponent().getSubscribeConsumerBackoffInterval();
task = Tasks.foregroundTask()
.withName("Subscribe KafkaConsumer")
.withBudget(Budgets.iterationBudget()
.withMaxIterations(
kafkaConsumer.getEndpoint().getComponent().getSubscribeConsumerBackoffMaxAttempts())
.withInitialDelay(Duration.ZERO)
.withInterval(Duration.ofMillis(currentBackoffInterval))
.build())
.build();
success = task.run(this::initializeConsumerTask);
if (!success) {
int max = kafkaConsumer.getEndpoint().getComponent().getCreateConsumerBackoffMaxAttempts();
setupInitializeErrorException(task, max);
// give up and terminate this consumer
terminated = true;
break;
}
setConnected(true);
}
lastError = null;
startPolling();
} while ((pollExceptionStrategy.canContinue() || isReconnect()) && isKafkaConsumerRunnable());
if (LOG.isInfoEnabled()) {
LOG.info("Terminating KafkaConsumer thread {} receiving from {}", threadId, getPrintableTopic());
}
safeUnsubscribe();
IOHelper.close(consumer);
}
private void setupInitializeErrorException(ForegroundTask task, int max) {
String time = TimeUtils.printDuration(task.elapsed());
String topic = getPrintableTopic();
String msg = "Gave up subscribing org.apache.kafka.clients.consumer.KafkaConsumer " +
threadId + " to " + topic + " after " + max + " attempts (elapsed: " + time + ").";
LOG.warn(msg);
lastError = new KafkaConsumerFatalException(msg, lastError);
}
private void setupCreateConsumerException(ForegroundTask task, int max) {
String time = TimeUtils.printDuration(task.elapsed());
String topic = getPrintableTopic();
String msg = "Gave up creating org.apache.kafka.clients.consumer.KafkaConsumer "
+ threadId + " to " + topic + " after " + max + " attempts (elapsed: " + time + ").";
lastError = new KafkaConsumerFatalException(msg, lastError);
}
private boolean initializeConsumerTask() {
try {
initializeConsumer();
} catch (Exception e) {
setConnected(false);
// ensure this is logged so users can see the problem
LOG.warn("Error subscribing org.apache.kafka.clients.consumer.KafkaConsumer due to: {}", e.getMessage(),
e);
lastError = e;
return false;
}
return true;
}
private boolean createConsumerTask() {
try {
createConsumer();
commitManager
= CommitManagers.createCommitManager(consumer, kafkaConsumer, threadId, getPrintableTopic());
if (consumerListener != null) {
consumerListener.setConsumer(consumer);
SeekPolicy seekPolicy = kafkaConsumer.getEndpoint().getComponent().getConfiguration().getSeekTo();
if (seekPolicy == null) {
seekPolicy = SeekPolicy.BEGINNING;
}
consumerListener.setSeekPolicy(seekPolicy);
}
} catch (Exception e) {
setConnected(false);
// ensure this is logged so users can see the problem
LOG.warn("Error creating org.apache.kafka.clients.consumer.KafkaConsumer due to: {}", e.getMessage(),
e);
lastError = e;
return false;
}
return true;
}
protected void createConsumer() {
// create consumer
ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader();
try {
// Kafka uses reflection for loading authentication settings, use its classloader
Thread.currentThread()
.setContextClassLoader(org.apache.kafka.clients.consumer.KafkaConsumer.class.getClassLoader());
// The Kafka consumer should be null at the first try. For every other reconnection event, it will not
long delay = kafkaConsumer.getEndpoint().getConfiguration().getPollTimeoutMs();
final String prefix = this.consumer == null ? "Connecting" : "Reconnecting";
LOG.info("{} Kafka consumer thread ID {} with poll timeout of {} ms", prefix, threadId, delay);
// this may throw an exception if something is wrong with kafka consumer
this.consumer = kafkaConsumer.getEndpoint().getKafkaClientFactory().getConsumer(kafkaProps);
// init client id which we may need to get from the kafka producer via reflection
if (clientId == null) {
clientId = getKafkaProps().getProperty(CommonClientConfigs.CLIENT_ID_CONFIG);
if (clientId == null) {
try {
clientId = (String) ReflectionHelper
.getField(consumer.getClass().getDeclaredField("clientId"), consumer);
} catch (Exception e) {
// ignore
clientId = "";
}
}
}
} finally {
Thread.currentThread().setContextClassLoader(threadClassLoader);
}
}
private void initializeConsumer() {
subscribe();
// set reconnect to false as the connection and resume is done at this point
setConnected(false);
pollExceptionStrategy.reset();
}
private void subscribe() {
KafkaConsumerResumeStrategy resumeStrategy = ResumeStrategyFactory.newResumeStrategy(kafkaConsumer);
resumeStrategy.setConsumer(consumer);
PartitionAssignmentListener listener = new PartitionAssignmentListener(
threadId, kafkaConsumer.getEndpoint().getConfiguration(), lastProcessedOffset,
this::isRunnable, commitManager, resumeStrategy);
if (LOG.isInfoEnabled()) {
LOG.info("Subscribing {} to {}", threadId, getPrintableTopic());
}
if (topicPattern != null) {
consumer.subscribe(topicPattern, listener);
} else {
consumer.subscribe(Arrays.asList(topicName.split(",")), listener);
}
}
protected void startPolling() {
long partitionLastOffset = -1;
try {
/*
* We lock the processing of the record to avoid raising a WakeUpException as a result to a call
* to stop() or shutdown().
*/
lock.lock();
long pollTimeoutMs = kafkaConsumer.getEndpoint().getConfiguration().getPollTimeoutMs();
if (LOG.isTraceEnabled()) {
LOG.trace("Polling {} from {} with timeout: {}", threadId, getPrintableTopic(), pollTimeoutMs);
}
KafkaRecordProcessorFacade recordProcessorFacade = new KafkaRecordProcessorFacade(
kafkaConsumer, lastProcessedOffset, threadId, commitManager, consumerListener);
Duration pollDuration = Duration.ofMillis(pollTimeoutMs);
while (isKafkaConsumerRunnable() && isConnected() && pollExceptionStrategy.canContinue()) {
ConsumerRecords<Object, Object> allRecords = consumer.poll(pollDuration);
if (consumerListener != null) {
if (!consumerListener.afterConsume(consumer)) {
continue;
}
}
commitManager.processAsyncCommits();
ProcessingResult result = recordProcessorFacade.processPolledRecords(allRecords, consumer);
if (result.isBreakOnErrorHit()) {
LOG.debug("We hit an error ... setting flags to force reconnect");
// force re-connect
setReconnect(true);
setConnected(false);
}
updateTaskState();
}
if (!isConnected()) {
LOG.debug("Not reconnecting, check whether to auto-commit or not ...");
commitManager.commit();
}
safeUnsubscribe();
} catch (InterruptException e) {
kafkaConsumer.getExceptionHandler().handleException("Interrupted while consuming " + threadId + " from kafka topic",
e);
commitManager.commit();
LOG.info("Unsubscribing {} from {}", threadId, getPrintableTopic());
safeUnsubscribe();
Thread.currentThread().interrupt();
} catch (WakeupException e) {
// This is normal: it raises this exception when calling the wakeUp (which happens when we stop)
if (LOG.isTraceEnabled()) {
LOG.trace("The kafka consumer was woken up while polling on thread {} for {}", threadId, getPrintableTopic());
}
safeUnsubscribe();
} catch (Exception e) {
if (LOG.isDebugEnabled()) {
LOG.warn("Exception {} caught while polling {} from kafka {} at offset {}: {}",
e.getClass().getName(), threadId, getPrintableTopic(), lastProcessedOffset, e.getMessage(), e);
} else {
LOG.warn("Exception {} caught while polling {} from kafka {} at offset {}: {}",
e.getClass().getName(), threadId, getPrintableTopic(), lastProcessedOffset, e.getMessage());
}
pollExceptionStrategy.handle(partitionLastOffset, e);
} finally {
// only close if not retry
if (!pollExceptionStrategy.canContinue()) {
LOG.debug("Closing consumer {}", threadId);
safeUnsubscribe();
IOHelper.close(consumer);
}
lock.unlock();
}
}
private void updateTaskState() {
switch (state) {
case PAUSE_REQUESTED:
LOG.info("Pausing the consumer as a response to a pause request");
consumer.pause(consumer.assignment());
state = State.PAUSED;
break;
case RESUME_REQUESTED:
LOG.info("Resuming the consumer as a response to a resume request");
consumer.resume(consumer.assignment());
state = State.RUNNING;
break;
default:
break;
}
}
private void safeUnsubscribe() {
if (consumer == null) {
return;
}
final String printableTopic = getPrintableTopic();
try {
consumer.unsubscribe();
} catch (IllegalStateException e) {
LOG.warn("The consumer is likely already closed. Skipping unsubscribing thread {} from kafka {}", threadId,
printableTopic);
} catch (Exception e) {
kafkaConsumer.getExceptionHandler().handleException(
"Error unsubscribing thread " + threadId + " from kafka " + printableTopic, e);
}
}
/*
* This is only used for presenting log messages that take into consideration that it might be subscribed to a topic
* or a topic pattern.
*/
private String getPrintableTopic() {
if (topicPattern != null) {
return "topic pattern " + topicPattern;
} else {
return "topic " + topicName;
}
}
private boolean isKafkaConsumerRunnable() {
return kafkaConsumer.isRunAllowed() && !kafkaConsumer.isStoppingOrStopped()
&& !kafkaConsumer.isSuspendingOrSuspended();
}
private boolean isRunnable() {
return kafkaConsumer.getEndpoint().getCamelContext().isStopping() && !kafkaConsumer.isRunAllowed();
}
private boolean isReconnect() {
return reconnect;
}
public void setReconnect(boolean value) {
reconnect = value;
}
/*
* This wraps a safe stop procedure that should help ensure a clean termination procedure for consumer code.
* This means that it should wait for the last process call to finish cleanly, including the commit of the
* record being processed at the current moment.
*
* Note: keep in mind that the KafkaConsumer is not thread-safe, so no other call to the consumer instance
* should be made here besides the wakeUp.
*/
private void safeStop() {
if (consumer == null) {
return;
}
long timeout = kafkaConsumer.getEndpoint().getConfiguration().getShutdownTimeout();
try {
/*
Try to wait for the processing to finish before giving up and waking up the Kafka consumer regardless
of whether the processing have finished or not.
*/
LOG.info("Waiting up to {} milliseconds for the processing to finish", timeout);
if (!lock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
LOG.warn("The processing of the current record did not finish within {} seconds", timeout);
}
// As advised in the KAFKA-1894 ticket, calling this wakeup method breaks the infinite loop
consumer.wakeup();
} catch (InterruptedException e) {
consumer.wakeup();
Thread.currentThread().interrupt();
} finally {
if (lock.isHeldByCurrentThread()) {
lock.unlock();
}
}
}
void stop() {
safeStop();
}
public boolean isConnected() {
return connected;
}
public boolean isPaused() {
return !consumer.paused().isEmpty();
}
public void setConnected(boolean connected) {
this.connected = connected;
}
public boolean isReady() {
if (!connected) {
return false;
}
boolean ready = true;
try {
if (consumer instanceof org.apache.kafka.clients.consumer.KafkaConsumer) {
// need to use reflection to access the network client which has API to check if the client has ready
// connections
org.apache.kafka.clients.consumer.KafkaConsumer kc = (org.apache.kafka.clients.consumer.KafkaConsumer) consumer;
ConsumerNetworkClient nc
= (ConsumerNetworkClient) ReflectionHelper.getField(kc.getClass().getDeclaredField("client"), kc);
LOG.trace(
"Health-Check calling org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.hasReadyNode");
ready = nc.hasReadyNodes(System.currentTimeMillis());
}
} catch (Exception e) {
// ignore
LOG.debug("Cannot check hasReadyNodes on KafkaConsumer client (ConsumerNetworkClient) due to: "
+ e.getMessage() + ". This exception is ignored.",
e);
}
return ready;
}
Properties getKafkaProps() {
return kafkaProps;
}
String getClientId() {
return clientId;
}
Exception getLastError() {
return lastError;
}
boolean isTerminated() {
return terminated;
}
boolean isRecoverable() {
return (pollExceptionStrategy.canContinue() || isReconnect()) && isKafkaConsumerRunnable();
}
long getCurrentRecoveryInterval() {
return currentBackoffInterval;
}
public BridgeExceptionHandlerToErrorHandler getBridge() {
return bridge;
}
/*
* This is for manually pausing the consumer. This is mostly used for directly calling pause from Java code
* or via JMX
*/
public void pause() {
LOG.info("A pause request was issued and the consumer thread will pause after current processing has finished");
state = State.PAUSE_REQUESTED;
}
/*
* This is for manually resuming the consumer (not to be confused w/ the Resume API). This is
* mostly used for directly calling resume from Java code or via JMX
*/
public void resume() {
LOG.info("A resume request was issued and the consumer thread will resume after current processing has finished");
state = State.RESUME_REQUESTED;
}
}