3131import org .apache .nifi .components .PropertyDescriptor ;
3232import org .apache .nifi .controller .NodeTypeProvider ;
3333import org .apache .nifi .flowfile .FlowFile ;
34+ import org .apache .nifi .logging .ComponentLog ;
3435import org .apache .nifi .processor .AbstractProcessor ;
3536import org .apache .nifi .processor .DataUnit ;
3637import org .apache .nifi .processor .ProcessContext ;
6667import software .amazon .kinesis .common .InitialPositionInStream ;
6768import software .amazon .kinesis .common .InitialPositionInStreamExtended ;
6869import software .amazon .kinesis .coordinator .Scheduler ;
70+ import software .amazon .kinesis .coordinator .WorkerStateChangeListener ;
6971import software .amazon .kinesis .lifecycle .events .InitializationInput ;
7072import software .amazon .kinesis .lifecycle .events .LeaseLostInput ;
7173import software .amazon .kinesis .lifecycle .events .ProcessRecordsInput ;
@@ -320,6 +322,7 @@ Ensure that the credentials provided have access to Kinesis, DynamoDB and (optio
320322 private volatile RecordBuffer .ForProcessor <Lease > recordBuffer ;
321323
322324 private volatile @ Nullable ReaderRecordProcessor readerRecordProcessor ;
325+ private volatile SchedulerChangeListener errorListener ;
323326
324327 // An instance filed, so that it can be read in getRelationships.
325328 private volatile ProcessingStrategy processingStrategy = ProcessingStrategy .from (
@@ -387,6 +390,8 @@ public void setup(final ProcessContext context) {
387390 recordBuffer = memoryBoundRecordBuffer ;
388391 final ShardRecordProcessorFactory recordProcessorFactory = () -> new ConsumeKinesisRecordProcessor (memoryBoundRecordBuffer );
389392
393+ errorListener = new SchedulerChangeListener (getLogger ());
394+
390395 final String applicationName = context .getProperty (APPLICATION_NAME ).getValue ();
391396 final String workerId = generateWorkerId ();
392397 final ConfigsBuilder configsBuilder = new ConfigsBuilder (streamTracker , applicationName , kinesisClient , dynamoDbClient , cloudWatchClient , workerId , recordProcessorFactory );
@@ -395,7 +400,7 @@ public void setup(final ProcessContext context) {
395400
396401 kinesisScheduler = new Scheduler (
397402 configsBuilder .checkpointConfig (),
398- configsBuilder .coordinatorConfig (),
403+ configsBuilder .coordinatorConfig (). workerStateChangeListener ( errorListener ) ,
399404 configsBuilder .leaseManagementConfig (),
400405 configsBuilder .lifecycleConfig (),
401406 configsBuilder .metricsConfig ().metricsFactory (metricsFactory ),
@@ -528,6 +533,7 @@ public void onStopped() {
528533 }
529534
530535 recordBuffer = null ;
536+ errorListener = null ;
531537 readerRecordProcessor = null ;
532538 }
533539
@@ -560,6 +566,12 @@ private void shutdownScheduler() {
560566
561567 @ Override
562568 public void onTrigger (final ProcessContext context , final ProcessSession session ) throws ProcessException {
569+ if (errorListener .holdsInitializationFailure ()) {
570+ getLogger ().error ("Failed to initialize the processor. Correct the error and restart the processor." , errorListener .failure ());
571+ context .yield ();
572+ return ;
573+ }
574+
563575 final Optional <Lease > leaseAcquired = recordBuffer .acquireBufferLease ();
564576
565577 leaseAcquired .ifPresentOrElse (
@@ -684,6 +696,35 @@ public void shutdownRequested(final ShutdownRequestedInput shutdownRequestedInpu
684696 }
685697 }
686698
699+ private static final class SchedulerChangeListener implements WorkerStateChangeListener {
700+
701+ private final ComponentLog logger ;
702+
703+ private volatile @ Nullable Throwable initializationFailure ;
704+
705+ SchedulerChangeListener (final ComponentLog logger ) {
706+ this .logger = logger ;
707+ }
708+
709+ @ Override
710+ public void onWorkerStateChange (final WorkerState newState ) {
711+ logger .debug ("Kinesis Scheduler changed state to: {}" , newState );
712+ }
713+
714+ @ Override
715+ public void onAllInitializationAttemptsFailed (final Throwable e ) {
716+ initializationFailure = e ;
717+ }
718+
719+ boolean holdsInitializationFailure () {
720+ return initializationFailure != null ;
721+ }
722+
723+ @ Nullable Throwable failure () {
724+ return initializationFailure ;
725+ }
726+ }
727+
687728 enum ProcessingStrategy implements DescribedValue {
688729 FLOW_FILE ("Write one FlowFile for each consumed Kinesis Record" ),
689730 RECORD ("Write one FlowFile containing multiple consumed Kinesis Records processed with Record Reader and Record Writer" );
0 commit comments