-
Notifications
You must be signed in to change notification settings - Fork 2.5k
Expand file tree
/
Copy pathDefaultStreamPoller.java
More file actions
895 lines (797 loc) · 33.3 KB
/
DefaultStreamPoller.java
File metadata and controls
895 lines (797 loc) · 33.3 KB
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
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.indices.pollingingest;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.cluster.ClusterChangedEvent;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.block.ClusterBlockLevel;
import org.opensearch.cluster.metadata.IngestionSource;
import org.opensearch.common.Nullable;
import org.opensearch.common.metrics.CounterMetric;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.IngestionConsumerFactory;
import org.opensearch.index.IngestionShardConsumer;
import org.opensearch.index.IngestionShardPointer;
import org.opensearch.index.Message;
import org.opensearch.index.engine.IngestionEngine;
import org.opensearch.indices.pollingingest.mappers.IngestionMessageMapper;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
/**
* Default implementation of {@link StreamPoller}
*/
public class DefaultStreamPoller implements StreamPoller {
private static final Logger logger = LogManager.getLogger(DefaultStreamPoller.class);
private static final int DEFAULT_POLLER_SLEEP_PERIOD_MS = 100;
private static final int CONSUMER_INIT_RETRY_INTERVAL_MS = 10000;
private volatile State state = State.NONE;
// goal state
private volatile boolean started;
private volatile boolean closed;
private volatile boolean paused;
private volatile IngestionErrorStrategy errorStrategy;
// indicates if a local or global cluster write block is in effect
private volatile boolean isWriteBlockEnabled;
// flag to indicate if consumer needs to be reinitialized
private volatile boolean reinitializeConsumer;
private volatile long lastPolledMessageTimestamp = 0;
private volatile long cachedPointerBasedLag = -1; // -1 indicates poller has not consumed any message yet
private volatile long lastPointerBasedLagUpdateTime = 0;
// Warmup configuration and state
private final IngestionSource.WarmupConfig warmupConfig;
private volatile boolean warmupComplete = false;
private volatile long warmupStartTime = 0;
private final CountDownLatch warmupLatch = new CountDownLatch(1);
@Nullable
private IngestionShardConsumer consumer;
private IngestionConsumerFactory consumerFactory;
private String consumerClientId;
private int shardId;
private ExecutorService consumerThread;
// start of the batch, inclusive
private IngestionShardPointer initialBatchStartPointer;
private ResetState resetState;
private final String resetValue;
private long maxPollSize;
private int pollTimeout;
private long pointerBasedLagUpdateIntervalMs;
private final IngestionMessageMapper messageMapper;
private final String indexName;
private final CounterMetric totalPolledCount = new CounterMetric();
private final CounterMetric totalConsumerErrorCount = new CounterMetric();
private final CounterMetric totalPollerMessageFailureCount = new CounterMetric();
// indicates number of messages dropped due to error
private final CounterMetric totalPollerMessageDroppedCount = new CounterMetric();
private PartitionedBlockingQueueContainer blockingQueueContainer;
// Force the consumer to start reading from this pointer. This is used in case of failures, or during initialization/reinitialization.
private IngestionShardPointer forcedShardPointer = null;
private DefaultStreamPoller(
IngestionShardPointer startPointer,
IngestionConsumerFactory consumerFactory,
String consumerClientId,
int shardId,
IngestionEngine ingestionEngine,
ResetState resetState,
String resetValue,
IngestionErrorStrategy errorStrategy,
State initialState,
long maxPollSize,
int pollTimeout,
int numProcessorThreads,
int blockingQueueSize,
long pointerBasedLagUpdateIntervalMs,
IngestionMessageMapper.MapperType mapperType,
Map<String, Object> mapperSettings,
IngestPipelineExecutor pipelineExecutor,
IngestionSource.WarmupConfig warmupConfig
) {
this(
startPointer,
consumerFactory,
consumerClientId,
shardId,
new PartitionedBlockingQueueContainer(
numProcessorThreads,
shardId,
ingestionEngine,
errorStrategy,
blockingQueueSize,
pipelineExecutor
),
resetState,
resetValue,
errorStrategy,
initialState,
maxPollSize,
pollTimeout,
pointerBasedLagUpdateIntervalMs,
ingestionEngine.config().getIndexSettings(),
IngestionMessageMapper.create(mapperType.getName(), shardId, mapperSettings),
warmupConfig
);
}
/**
* Visible for testing.
*/
DefaultStreamPoller(
IngestionShardPointer startPointer,
IngestionConsumerFactory consumerFactory,
String consumerClientId,
int shardId,
PartitionedBlockingQueueContainer blockingQueueContainer,
ResetState resetState,
String resetValue,
IngestionErrorStrategy errorStrategy,
State initialState,
long maxPollSize,
int pollTimeout,
long pointerBasedLagUpdateIntervalMs,
IndexSettings indexSettings,
IngestionMessageMapper messageMapper,
IngestionSource.WarmupConfig warmupConfig
) {
this.consumerFactory = Objects.requireNonNull(consumerFactory);
this.consumerClientId = Objects.requireNonNull(consumerClientId);
this.shardId = shardId;
this.resetState = resetState;
this.resetValue = resetValue;
this.initialBatchStartPointer = startPointer;
this.state = initialState;
this.maxPollSize = maxPollSize;
this.pollTimeout = pollTimeout;
this.pointerBasedLagUpdateIntervalMs = pointerBasedLagUpdateIntervalMs;
this.blockingQueueContainer = blockingQueueContainer;
this.consumerThread = Executors.newSingleThreadExecutor(
r -> new Thread(r, String.format(Locale.ROOT, "stream-poller-consumer-%d-%d", shardId, System.currentTimeMillis()))
);
this.errorStrategy = errorStrategy;
this.indexName = indexSettings.getIndex().getName();
this.messageMapper = Objects.requireNonNull(messageMapper);
this.warmupConfig = Objects.requireNonNull(warmupConfig);
// handle initial poller states
this.paused = initialState == State.PAUSED;
// If warmup is disabled, mark as complete immediately
if (!warmupConfig.isEnabled()) {
this.warmupComplete = true;
}
}
@Override
public void start() {
if (closed) {
throw new RuntimeException("poller is closed!");
}
if (started) {
throw new RuntimeException("poller is already running");
}
started = true;
consumerThread.submit(this::startPoll);
blockingQueueContainer.startProcessorThreads();
}
/**
* Start the poller. visibile for testing
*/
protected void startPoll() {
if (!started) {
throw new IllegalStateException("poller is not started!");
}
if (closed) {
throw new IllegalStateException("poller is closed!");
}
logger.info("Starting poller for shard {}", shardId);
// Initialize warmup if enabled
if (warmupConfig.isEnabled() && !warmupComplete) {
warmupStartTime = System.currentTimeMillis();
state = State.WARMING_UP;
logger.info("Starting warmup phase for index {} shard {}, waiting for lag to catch up", indexName, shardId);
}
while (true) {
try {
if (closed) {
setStateWithWarmupAwareness(State.CLOSED);
closeConsumer();
break;
}
// Initialize/reinitialization consumer
if (this.consumer == null || reinitializeConsumer) {
handleConsumerInitialization();
continue;
}
// Update lag periodically. Lag is updated even if the poller is paused.
updatePointerBasedLagIfNeeded();
// Check warmup status if not yet complete
if (!warmupComplete && warmupConfig.isEnabled()) {
updateWarmupStatus();
}
if (paused || isWriteBlockEnabled) {
setStateWithWarmupAwareness(State.PAUSED);
try {
Thread.sleep(DEFAULT_POLLER_SLEEP_PERIOD_MS);
} catch (Throwable e) {
logger.error("Error in pausing the poller of shard {}: {}", shardId, e);
}
continue;
}
setStateWithWarmupAwareness(State.POLLING);
List<IngestionShardConsumer.ReadResult<? extends IngestionShardPointer, ? extends Message>> results;
// Force the consumer to start from forcedShardPointer if available
if (forcedShardPointer != null) {
results = consumer.readNext(forcedShardPointer, true, maxPollSize, pollTimeout);
forcedShardPointer = null;
} else {
results = consumer.readNext(maxPollSize, pollTimeout);
}
if (results.isEmpty()) {
// no new records
setLastPolledMessageTimestamp(0);
Thread.sleep(DEFAULT_POLLER_SLEEP_PERIOD_MS);
continue;
}
setStateWithWarmupAwareness(State.PROCESSING);
// processRecords returns failed shard pointers. Update forcedShardPointer to the failed pointer to retry on next iteration
// in case of failures
forcedShardPointer = processRecords(results);
} catch (Exception e) {
// Pause ingestion when an error is encountered while polling the streaming source.
// Currently we do not have a good way to skip past the failing messages.
// The user will have the option to manually update the offset and resume ingestion.
// todo: support retry?
logger.error("Pausing ingestion. Fatal error occurred in polling the shard {} for index {}: {}", shardId, indexName, e);
totalConsumerErrorCount.inc();
pause();
}
}
}
/**
* Process records and write to the blocking queue. In case of error, return the shard pointer of the failed message.
*/
private IngestionShardPointer processRecords(
List<IngestionShardConsumer.ReadResult<? extends IngestionShardPointer, ? extends Message>> results
) {
IngestionShardPointer failedShardPointer = null;
for (IngestionShardConsumer.ReadResult<? extends IngestionShardPointer, ? extends Message> result : results) {
try {
totalPolledCount.inc();
// Use mapper to create ShardUpdateMessage
ShardUpdateMessage shardUpdateMessage = messageMapper.mapAndProcess(result.getPointer(), result.getMessage());
blockingQueueContainer.add(shardUpdateMessage);
setLastPolledMessageTimestamp(result.getMessage().getTimestamp() == null ? 0 : result.getMessage().getTimestamp());
logger.debug(
"Put message {} with pointer {} to the blocking queue",
String.valueOf(result.getMessage().getPayload()),
result.getPointer().asString()
);
} catch (Exception e) {
logger.error(
"[Default Poller] Error processing record. Index={}, Shard={}, pointer={}: error={}",
indexName,
shardId,
result.getPointer().asString(),
e
);
errorStrategy.handleError(e, IngestionErrorStrategy.ErrorStage.POLLING);
totalPollerMessageFailureCount.inc();
if (errorStrategy.shouldIgnoreError(e, IngestionErrorStrategy.ErrorStage.POLLING) == false) {
// Blocking error encountered. Pause poller to stop processing remaining updates.
pause();
failedShardPointer = result.getPointer();
break;
} else {
totalPollerMessageDroppedCount.inc();
}
}
}
return failedShardPointer;
}
@Override
public void pause() {
if (closed) {
throw new RuntimeException("consumer is closed!");
}
paused = true;
}
@Override
public void resume() {
if (closed) {
throw new RuntimeException("consumer is closed!");
}
paused = false;
}
@Override
public void close() {
closed = true;
if (!started) {
logger.info("consumer thread not started");
return;
}
long startTime = System.currentTimeMillis(); // Record the start time
long timeout = 5000;
while (state != State.CLOSED) {
// Check if the timeout has been reached
if (System.currentTimeMillis() - startTime > timeout) {
logger.error("Timeout reached while waiting for shard {} to close", shardId);
break; // Exit the loop if the timeout is reached
}
try {
Thread.sleep(100);
} catch (Throwable e) {
logger.error("Error in closing the poller of shard {}: {}", shardId, e);
}
}
consumerThread.shutdown();
blockingQueueContainer.close();
logger.info("closed the poller of shard {}", shardId);
}
@Override
public boolean isPaused() {
return paused;
}
@Override
public boolean isClosed() {
return closed;
}
@Override
public boolean isWarmupComplete() {
return warmupComplete || !warmupConfig.isEnabled();
}
/**
* Sets the poller state with warmup-aware logic.
* During warmup, POLLING and PROCESSING states are reported as WARMING_UP
* to allow monitoring via the ingestion state API.
*
* @param newState the desired state to set
*/
private void setStateWithWarmupAwareness(State newState) {
// CLOSED and PAUSED always take effect
if (newState == State.CLOSED || newState == State.PAUSED) {
this.state = newState;
return;
}
// During warmup, stay in WARMING_UP instead of POLLING/PROCESSING
if (!isWarmupComplete()) {
this.state = State.WARMING_UP;
return;
}
this.state = newState;
}
@Override
public boolean awaitWarmupComplete(long timeoutMs) throws InterruptedException {
if (!warmupConfig.isEnabled() || isWarmupComplete()) {
return true;
}
boolean completed = warmupLatch.await(timeoutMs, TimeUnit.MILLISECONDS);
if (!completed) {
logger.warn(
"Warmup timeout for index {} shard {} - proceeding with current lag (warmupComplete={})",
indexName,
shardId,
isWarmupComplete()
);
}
return completed;
}
/**
* Check if warmup conditions are met and mark warmup as complete if so.
*
* Warmup uses offset-based lag (cachedPointerBasedLag) which tracks the difference between
* the current consumer position and the end of the stream. This is the preferred mode for
* Kafka and other sources that support offset-based lag calculation.
* Note: cachedPointerBasedLag is -1 by default (indicating no messages consumed yet) and is only updated after updatePointerBasedLagIfNeeded()
* is called.
*/
private void updateWarmupStatus() {
// Skip warmup if poller is paused
if (paused) {
warmupComplete = true;
warmupLatch.countDown();
logger.info("Warmup skipped for index {} shard {} - poller is paused", indexName, shardId);
return;
}
long currentLag = cachedPointerBasedLag;
long threshold = warmupConfig.lagThreshold();
long elapsedTime = System.currentTimeMillis() - warmupStartTime;
boolean lagBelowThreshold = currentLag >= 0 && currentLag <= threshold;
boolean timeoutReached = elapsedTime >= warmupConfig.timeout().millis();
if (lagBelowThreshold) {
warmupComplete = true;
warmupLatch.countDown();
logger.info(
"Warmup complete for index {} shard {} - lag {} is at or below threshold {}",
indexName,
shardId,
currentLag,
threshold
);
} else if (timeoutReached) {
warmupComplete = true;
warmupLatch.countDown();
logger.warn(
"Warmup timeout for index {} shard {} after {}ms - proceeding with lag {} (threshold was {})",
indexName,
shardId,
elapsedTime,
currentLag,
threshold
);
}
}
/**
* Returns the batch start pointer from where the poller can resume in case of shard recovery. The poller and
* processor are decoupled in this implementation, and hence the latest pointer tracked by the processor acts as the
* recovery/start point. In case the processor has not started tracking, then the initial batchStartPointer used by
* the poller acts as the start point. If multiple processor threads are used, the minimum shard pointer across
* processors indicates the start point.
*/
@Override
public IngestionShardPointer getBatchStartPointer() {
return blockingQueueContainer.getCurrentShardPointers()
.stream()
.filter(Objects::nonNull)
.min(Comparator.naturalOrder())
.orElseGet(() -> initialBatchStartPointer);
}
@Override
public PollingIngestStats getStats() {
MessageProcessorRunnable.MessageProcessorMetrics processorMetrics = blockingQueueContainer.getMessageProcessorMetrics();
PollingIngestStats.Builder builder = new PollingIngestStats.Builder();
// set processor stats
builder.setTotalProcessedCount(processorMetrics.processedCounter().count());
builder.setTotalInvalidMessageCount(processorMetrics.invalidMessageCounter().count());
builder.setTotalProcessorVersionConflictsCount(processorMetrics.versionConflictCounter().count());
builder.setTotalProcessorFailedCount(processorMetrics.failedMessageCounter().count());
builder.setTotalProcessorFailuresDroppedCount(processorMetrics.failedMessageDroppedCounter().count());
builder.setTotalProcessorThreadInterruptCount(processorMetrics.processorThreadInterruptCounter().count());
// set consumer stats
builder.setTotalPolledCount(totalPolledCount.count());
builder.setTotalConsumerErrorCount(totalConsumerErrorCount.count());
builder.setTotalPollerMessageFailureCount(totalPollerMessageFailureCount.count());
builder.setTotalPollerMessageDroppedCount(totalPollerMessageDroppedCount.count());
builder.setLagInMillis(computeTimeBasedLag());
builder.setPointerBasedLag(cachedPointerBasedLag);
return builder.build();
}
/**
* Returns the lag in milliseconds since the last polled message
*/
private long computeTimeBasedLag() {
if (lastPolledMessageTimestamp == 0 || paused) {
return 0;
}
return System.currentTimeMillis() - lastPolledMessageTimestamp;
}
private void setLastPolledMessageTimestamp(long timestamp) {
if (lastPolledMessageTimestamp != timestamp) {
lastPolledMessageTimestamp = timestamp;
}
}
/**
* Update the cached pointer-based lag if enough time has elapsed since the last update.
* {@code consumer.getPointerBasedLag()} is called from the poller thread, so it's safe to access the consumer.
* If pointerBasedLagUpdateIntervalMs is 0, pointer-based lag calculation is disabled.
*/
private void updatePointerBasedLagIfNeeded() {
// If interval is 0, pointer-based lag is disabled
if (pointerBasedLagUpdateIntervalMs == 0) {
return;
}
long currentTime = System.currentTimeMillis();
if (consumer != null && (currentTime - lastPointerBasedLagUpdateTime >= pointerBasedLagUpdateIntervalMs)) {
try {
// update the lastPointerBasedLagUpdateTime first, to avoid load on streaming source in case of errors
lastPointerBasedLagUpdateTime = currentTime;
cachedPointerBasedLag = consumer.getPointerBasedLag(initialBatchStartPointer);
} catch (Exception e) {
logger.warn("Failed to update lag for index {} shard {}: {}", indexName, shardId, e.getMessage());
}
}
}
public State getState() {
return this.state;
}
@Override
public IngestionErrorStrategy getErrorStrategy() {
return this.errorStrategy;
}
@Override
public void updateErrorStrategy(IngestionErrorStrategy errorStrategy) {
this.errorStrategy = errorStrategy;
blockingQueueContainer.updateErrorStrategy(errorStrategy);
}
@Override
public boolean isWriteBlockEnabled() {
return isWriteBlockEnabled;
}
@Override
public void setWriteBlockEnabled(boolean isWriteBlockEnabled) {
this.isWriteBlockEnabled = isWriteBlockEnabled;
}
@Nullable
@Override
public IngestionShardConsumer getConsumer() {
return consumer;
}
/**
* Mark the poller's consumer for reinitialization. A new consumer will be initialized and start consuming from the
* latest batchStartPointer. This method also reinitializes the consumer factory with the updated ingestion source.
* @param updatedIngestionSource the updated ingestion source with new configuration parameters
*/
@Override
public synchronized void requestConsumerReinitialization(IngestionSource updatedIngestionSource) {
if (closed) {
logger.warn("Cannot reinitialize consumer for closed poller of shard {}", shardId);
return;
}
// Reinitialize the consumer factory with updated configuration
consumerFactory.initialize(updatedIngestionSource);
logger.info("Configuration parameters updated for index {} shard {}, requesting consumer reinitialization", indexName, shardId);
reinitializeConsumer = true;
}
@Override
public void clusterChanged(ClusterChangedEvent event) {
try {
if (event.blocksChanged() == false) {
return;
}
final ClusterState state = event.state();
isWriteBlockEnabled = state.blocks().indexBlocked(ClusterBlockLevel.WRITE, indexName);
} catch (Exception e) {
logger.error("Error applying cluster state in stream poller", e);
throw e;
}
}
/**
* Handles the reset state logic.
* Returns the resulting IngestionShardPointer for reset or null if no reset required.
*/
private IngestionShardPointer getResetShardPointer() {
IngestionShardPointer resetPointer = null;
if (resetState != ResetState.NONE && consumer != null) {
switch (resetState) {
case EARLIEST:
resetPointer = consumer.earliestPointer();
logger.info("Resetting pointer by seeking to earliest pointer {}", resetPointer.asString());
break;
case LATEST:
resetPointer = consumer.latestPointer();
logger.info("Resetting pointer by seeking to latest pointer {}", resetPointer.asString());
break;
case RESET_BY_OFFSET:
resetPointer = consumer.pointerFromOffset(resetValue);
logger.info("Resetting pointer by seeking to pointer {}", resetPointer.asString());
break;
case RESET_BY_TIMESTAMP:
resetPointer = consumer.pointerFromTimestampMillis(Long.parseLong(resetValue));
logger.info(
"Resetting pointer by seeking to timestamp {}, corresponding pointer {}",
resetValue,
resetPointer.asString()
);
break;
}
resetState = ResetState.NONE;
}
return resetPointer;
}
/**
* Handles consumer initialization and reinitialization logic. Closes existing consumer if available and clears the
* blocking queues before initializing a new consumer. Also forces the consumer to start reading from the initial
* batchStartPointer if first time initialization, or from the latest available batchStartPointer on reinitialization.
*/
private void handleConsumerInitialization() {
closeConsumer();
blockingQueueContainer.clearAllQueues();
initializeConsumer();
// Handle consumer offset reset the first time an index is created. The reset offset takes precedence if available.
IngestionShardPointer resetShardPointer = getResetShardPointer();
if (resetShardPointer != null) {
initialBatchStartPointer = resetShardPointer;
}
// Force the consumer to start from the batchStartPointer. This will be the initialBatchStartPointer for first
// time initialization, or the latest batchStartPointer based on processed messages.
forcedShardPointer = getBatchStartPointer();
}
/**
* Initializes the consumer using the provided consumerFactory. If an error is encountered during initialization,
* the poller thread sleeps for the provided duration before retrying/proceeding with the polling loop.
*/
private synchronized void initializeConsumer() {
try {
reinitializeConsumer = false;
this.consumer = consumerFactory.createShardConsumer(consumerClientId, shardId);
logger.info("Successfully initialized consumer for shard {}", shardId);
} catch (Exception e) {
logger.warn("Failed to create consumer for shard {}: {}", shardId, e.getMessage());
totalConsumerErrorCount.inc();
try {
Thread.sleep(CONSUMER_INIT_RETRY_INTERVAL_MS);
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
}
}
/**
* Closes the consumer gracefully. This should be called when reinitializing or shutting down the poller.
*/
private void closeConsumer() {
if (this.consumer != null) {
try {
logger.info("Closing consumer for index {} shard {}", indexName, shardId);
this.consumer.close();
this.consumer = null;
} catch (Exception e) {
logger.warn("Error closing consumer for index {} shard {}: {}", indexName, shardId, e.getMessage());
}
}
}
/**
* Builder for DefaultStreamPoller
*/
public static class Builder {
private IngestionShardPointer startPointer;
private IngestionConsumerFactory consumerFactory;
private String consumerClientId;
private int shardId;
private IngestionEngine ingestionEngine;
private ResetState resetState = ResetState.LATEST;
private String resetValue = "";
private IngestionErrorStrategy errorStrategy;
private State initialState = State.NONE;
private long maxPollSize = 1000;
private int pollTimeout = 1000;
private int numProcessorThreads = 1;
private int blockingQueueSize = 100;
private long pointerBasedLagUpdateIntervalMs = 10000;
private IngestionMessageMapper.MapperType mapperType = IngestionMessageMapper.MapperType.DEFAULT;
private Map<String, Object> mapperSettings = Collections.emptyMap();
private IngestPipelineExecutor pipelineExecutor;
// Warmup configuration - default matches IndexMetadata settings
private IngestionSource.WarmupConfig warmupConfig = new IngestionSource.WarmupConfig(TimeValue.timeValueMillis(-1), 100L);
/**
* Initialize the builder with mandatory parameters
*/
public Builder(
IngestionShardPointer startPointer,
IngestionConsumerFactory consumerFactory,
String consumerClientId,
int shardId,
IngestionEngine ingestionEngine
) {
this.startPointer = startPointer;
this.consumerFactory = Objects.requireNonNull(consumerFactory);
this.consumerClientId = Objects.requireNonNull(consumerClientId);
this.shardId = shardId;
this.ingestionEngine = Objects.requireNonNull(ingestionEngine);
this.errorStrategy = new DropIngestionErrorStrategy("poller");
}
/**
* Set error strategy
*/
public Builder errorStrategy(IngestionErrorStrategy errorStrategy) {
this.errorStrategy = Objects.requireNonNull(errorStrategy);
return this;
}
/**
* Set reset state
*/
public Builder resetState(ResetState resetState) {
this.resetState = resetState;
return this;
}
/**
* Set reset value
*/
public Builder resetValue(String resetValue) {
this.resetValue = resetValue;
return this;
}
/**
* Set initial state
*/
public Builder initialState(State initialState) {
this.initialState = initialState;
return this;
}
/**
* Set max poll size
*/
public Builder maxPollSize(long maxPollSize) {
this.maxPollSize = maxPollSize;
return this;
}
/**
* Set poll timeout
*/
public Builder pollTimeout(int pollTimeout) {
this.pollTimeout = pollTimeout;
return this;
}
/**
* Set number of processor threads
*/
public Builder numProcessorThreads(int numProcessorThreads) {
this.numProcessorThreads = numProcessorThreads;
return this;
}
/**
* Set blocking queue size
*/
public Builder blockingQueueSize(int blockingQueueSize) {
this.blockingQueueSize = blockingQueueSize;
return this;
}
/**
* Set pointer-based lag update interval in milliseconds
*/
public Builder pointerBasedLagUpdateInterval(long pointerBasedLagUpdateInterval) {
this.pointerBasedLagUpdateIntervalMs = pointerBasedLagUpdateInterval;
return this;
}
/**
* Set mapper type
*/
public Builder mapperType(IngestionMessageMapper.MapperType mapperType) {
this.mapperType = mapperType;
return this;
}
/**
* Set mapper settings
*/
public Builder mapperSettings(Map<String, Object> mapperSettings) {
this.mapperSettings = mapperSettings != null ? mapperSettings : Collections.emptyMap();
return this;
}
/**
* Set pipeline executor for ingest pipeline execution
*/
public Builder pipelineExecutor(IngestPipelineExecutor pipelineExecutor) {
this.pipelineExecutor = pipelineExecutor;
return this;
}
/**
* Set warmup configuration
*/
public Builder warmupConfig(IngestionSource.WarmupConfig warmupConfig) {
this.warmupConfig = Objects.requireNonNull(warmupConfig);
return this;
}
/**
* Build the DefaultStreamPoller instance
*/
public DefaultStreamPoller build() {
return new DefaultStreamPoller(
startPointer,
consumerFactory,
consumerClientId,
shardId,
ingestionEngine,
resetState,
resetValue,
errorStrategy,
initialState,
maxPollSize,
pollTimeout,
numProcessorThreads,
blockingQueueSize,
pointerBasedLagUpdateIntervalMs,
mapperType,
mapperSettings,
pipelineExecutor,
warmupConfig
);
}
}
}