-
Notifications
You must be signed in to change notification settings - Fork 2.9k
NIFI-15180 Log initialization errors in ConsumeKinesis #10526
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,6 +31,7 @@ | |
| import org.apache.nifi.components.PropertyDescriptor; | ||
| import org.apache.nifi.controller.NodeTypeProvider; | ||
| import org.apache.nifi.flowfile.FlowFile; | ||
| import org.apache.nifi.logging.ComponentLog; | ||
| import org.apache.nifi.processor.AbstractProcessor; | ||
| import org.apache.nifi.processor.DataUnit; | ||
| import org.apache.nifi.processor.ProcessContext; | ||
|
|
@@ -66,6 +67,7 @@ | |
| import software.amazon.kinesis.common.InitialPositionInStream; | ||
| import software.amazon.kinesis.common.InitialPositionInStreamExtended; | ||
| import software.amazon.kinesis.coordinator.Scheduler; | ||
| import software.amazon.kinesis.coordinator.WorkerStateChangeListener; | ||
| import software.amazon.kinesis.lifecycle.events.InitializationInput; | ||
| import software.amazon.kinesis.lifecycle.events.LeaseLostInput; | ||
| import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; | ||
|
|
@@ -89,7 +91,9 @@ | |
| import java.util.Optional; | ||
| import java.util.Set; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.TimeoutException; | ||
|
|
||
| import static java.util.concurrent.TimeUnit.NANOSECONDS; | ||
|
|
@@ -113,7 +117,7 @@ | |
| Consumes data from the specified AWS Kinesis stream and outputs a FlowFile for every processed Record (raw) | ||
| or a FlowFile for a batch of processed records if a Record Reader and Record Writer are configured. | ||
| The processor may take a few minutes on the first start and several seconds on subsequent starts | ||
| to initialise before starting to fetch data. | ||
| to initialize before starting to fetch data. | ||
| Uses DynamoDB for check pointing and coordination, and (optional) CloudWatch for metrics. | ||
| """) | ||
| @WritesAttributes({ | ||
|
|
@@ -163,6 +167,11 @@ public class ConsumeKinesis extends AbstractProcessor { | |
| private static final int KINESIS_HTTP_CLIENT_WINDOW_SIZE_BYTES = 512 * 1024; // 512 KiB | ||
| private static final Duration KINESIS_HTTP_HEALTH_CHECK_PERIOD = Duration.ofMinutes(1); | ||
|
|
||
| /** | ||
| * Using a large enough value to ensure we don't wait infinitely for the initialization. | ||
| * Actual initialization shouldn't take that long. | ||
| */ | ||
| private static final Duration KINESIS_SCHEDULER_INITIALIZATION_TIMEOUT = Duration.ofMinutes(15); | ||
| private static final Duration KINESIS_SCHEDULER_GRACEFUL_SHUTDOWN_TIMEOUT = Duration.ofMinutes(3); | ||
|
|
||
| static final PropertyDescriptor STREAM_NAME = new PropertyDescriptor.Builder() | ||
|
|
@@ -393,9 +402,11 @@ public void setup(final ProcessContext context) { | |
|
|
||
| final MetricsFactory metricsFactory = configureMetricsFactory(context); | ||
|
|
||
| final InitializationStateChangeListener initializationListener = new InitializationStateChangeListener(getLogger()); | ||
|
|
||
| kinesisScheduler = new Scheduler( | ||
| configsBuilder.checkpointConfig(), | ||
| configsBuilder.coordinatorConfig(), | ||
| configsBuilder.coordinatorConfig().workerStateChangeListener(initializationListener), | ||
| configsBuilder.leaseManagementConfig(), | ||
| configsBuilder.lifecycleConfig(), | ||
| configsBuilder.metricsConfig().metricsFactory(metricsFactory), | ||
|
|
@@ -409,8 +420,32 @@ public void setup(final ProcessContext context) { | |
| schedulerThread.start(); | ||
| // The thread is stopped when kinesisScheduler is shutdown in the onStopped method. | ||
|
|
||
| getLogger().info("Started Kinesis Scheduler for stream [{}] with application name [{}] and workerId [{}]", | ||
| streamName, applicationName, workerId); | ||
| final InitializationResult result; | ||
| try { | ||
| result = initializationListener.result().get(KINESIS_SCHEDULER_INITIALIZATION_TIMEOUT.getSeconds(), SECONDS); | ||
| } catch (final InterruptedException | ExecutionException | TimeoutException e) { | ||
| if (e instanceof InterruptedException) { | ||
| Thread.currentThread().interrupt(); | ||
| } | ||
| cleanUpState(); | ||
| throw new ProcessException(e); | ||
| } | ||
|
|
||
| switch (result) { | ||
| case InitializationResult.Success ignored -> | ||
| getLogger().info( | ||
| "Started Kinesis Scheduler for stream [{}] with application name [{}] and workerId [{}]", | ||
| streamName, applicationName, workerId); | ||
| case InitializationResult.Failure failure -> { | ||
| cleanUpState(); | ||
|
|
||
| final ProcessException ex = failure.error() | ||
| .map(err -> new ProcessException("Failed to initialize the processor.", err)) | ||
exceptionfactory marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| .orElseGet(() -> new ProcessException("Failed to initialize the processor due to an unknown failure. Check application logs for more details.")); | ||
|
||
|
|
||
| throw ex; | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -510,10 +545,15 @@ private String generateWorkerId() { | |
|
|
||
| @OnStopped | ||
| public void onStopped() { | ||
| cleanUpState(); | ||
| } | ||
|
|
||
| private void cleanUpState() { | ||
| if (kinesisScheduler != null) { | ||
| shutdownScheduler(); | ||
| kinesisScheduler = null; | ||
| } | ||
|
|
||
| if (kinesisClient != null) { | ||
| kinesisClient.close(); | ||
| kinesisClient = null; | ||
|
|
@@ -532,6 +572,10 @@ public void onStopped() { | |
| } | ||
|
|
||
| private void shutdownScheduler() { | ||
| if (kinesisScheduler.shutdownComplete()) { | ||
| return; | ||
| } | ||
|
|
||
| final long start = System.nanoTime(); | ||
| getLogger().debug("Shutting down Kinesis Scheduler"); | ||
|
|
||
|
|
@@ -684,6 +728,49 @@ public void shutdownRequested(final ShutdownRequestedInput shutdownRequestedInpu | |
| } | ||
| } | ||
|
|
||
| private static final class InitializationStateChangeListener implements WorkerStateChangeListener { | ||
|
|
||
| private final ComponentLog logger; | ||
|
|
||
| private final CompletableFuture<InitializationResult> resultFuture = new CompletableFuture<>(); | ||
|
|
||
| private volatile @Nullable Throwable initializationFailure; | ||
|
|
||
| InitializationStateChangeListener(final ComponentLog logger) { | ||
| this.logger = logger; | ||
| } | ||
|
|
||
| @Override | ||
| public void onWorkerStateChange(final WorkerState newState) { | ||
| logger.info("Processor state changed to: {}", newState); | ||
exceptionfactory marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| if (newState == WorkerState.STARTED) { | ||
| resultFuture.complete(new InitializationResult.Success()); | ||
| } else if (newState == WorkerState.SHUT_DOWN) { | ||
| resultFuture.complete(new InitializationResult.Failure(Optional.ofNullable(initializationFailure))); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void onAllInitializationAttemptsFailed(final Throwable e) { | ||
| // This method is called before the SHUT_DOWN_STARTED phase. | ||
| // Memorizing the error until the Scheduler is SHUT_DOWN. | ||
| initializationFailure = e; | ||
exceptionfactory marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| Future<InitializationResult> result() { | ||
| return resultFuture; | ||
| } | ||
| } | ||
|
|
||
| private sealed interface InitializationResult { | ||
| record Success() implements InitializationResult { | ||
| } | ||
|
|
||
| record Failure(Optional<Throwable> error) implements InitializationResult { | ||
| } | ||
| } | ||
|
|
||
| enum ProcessingStrategy implements DescribedValue { | ||
| FLOW_FILE("Write one FlowFile for each consumed Kinesis Record"), | ||
| RECORD("Write one FlowFile containing multiple consumed Kinesis Records processed with Record Reader and Record Writer"); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.