All Downloads are FREE. Search and download functionalities are using the official Maven repository.

software.amazon.kinesis.lifecycle.ConsumerStates Maven / Gradle / Ivy

Go to download

The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis.

There is a newer version: 3.0.1
Show newest version
/*
 * Copyright 2019 Amazon.com, Inc. or its affiliates.
 * Licensed 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 software.amazon.kinesis.lifecycle;

import lombok.Getter;
import lombok.experimental.Accessors;
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
import software.amazon.kinesis.retrieval.ThrottlingReporter;

/**
 * Top level container for all the possible states a {@link ShardConsumer} can be in. The logic for creation of tasks,
 * and state transitions is contained within the {@link ConsumerState} objects.
 *
 * 

State Diagram

* *
 *       +-------------------+
 *       | Waiting on Parent |                               +------------------+
 *  +----+       Shard       |                               |     Shutdown     |
 *  |    |                   |          +--------------------+   Notification   |
 *  |    +----------+--------+          |  Shutdown:         |     Requested    |
 *  |               | Success           |   Requested        +-+-------+--------+
 *  |               |                   |                      |       |
 *  |        +------+-------------+     |                      |       | Shutdown:
 *  |        |    Initializing    +-----+                      |       |  Requested
 *  |        |                    |     |                      |       |
 *  |        |                    +-----+-------+              |       |
 *  |        +---------+----------+     |       | Shutdown:    | +-----+-------------+
 *  |                  | Success        |       |  Terminated  | |     Shutdown      |
 *  |                  |                |       |  Zombie      | |   Notification    +-------------+
 *  |           +------+-------------+  |       |              | |     Complete      |             |
 *  |           |     Processing     +--+       |              | ++-----------+------+             |
 *  |       +---+                    |          |              |  |           |                    |
 *  |       |   |                    +----------+              |  |           | Shutdown:          |
 *  |       |   +------+-------------+          |              \  /           |  Requested         |
 *  |       |          |                        |               \/            +--------------------+
 *  |       |          |                        |               ||
 *  |       | Success  |                        |               || Shutdown:
 *  |       +----------+                        |               ||  Terminated
 *  |                                           |               ||  Zombie
 *  |                                           |               ||
 *  |                                           |               ||
 *  |                                           |           +---++--------------+
 *  |                                           |           |   Shutting Down   |
 *  |                                           +-----------+                   |
 *  |                                                       |                   |
 *  |                                                       +--------+----------+
 *  |                                                                |
 *  |                                                                | Shutdown:
 *  |                                                                |  All Reasons
 *  |                                                                |
 *  |                                                                |
 *  |      Shutdown:                                        +--------+----------+
 *  |        All Reasons                                    |     Shutdown      |
 *  +-------------------------------------------------------+     Complete      |
 *                                                          |                   |
 *                                                          +-------------------+
 * 
*/ class ConsumerStates { /** * Enumerates processing states when working on a shard. */ enum ShardConsumerState { // @formatter:off WAITING_ON_PARENT_SHARDS(new BlockedOnParentState()), INITIALIZING(new InitializingState()), PROCESSING(new ProcessingState()), SHUTDOWN_REQUESTED(new ShutdownNotificationState()), SHUTTING_DOWN(new ShuttingDownState()), SHUTDOWN_COMPLETE(new ShutdownCompleteState()); //@formatter:on @Getter @Accessors(fluent = true) private final ConsumerState consumerState; ShardConsumerState(ConsumerState consumerState) { this.consumerState = consumerState; } } /** * The initial state that any {@link ShardConsumer} should start in. */ static final ConsumerState INITIAL_STATE = ShardConsumerState.WAITING_ON_PARENT_SHARDS.consumerState(); /** * This is the initial state of a shard consumer. This causes the consumer to remain blocked until the all parent * shards have been completed. * *

Valid Transitions

*
*
Success
*
Transition to the initializing state to allow the record processor to be initialized in preparation of * processing.
*
Shutdown
*
*
*
All Reasons
*
Transitions to {@link ShutdownCompleteState}. Since the record processor was never initialized it can't be * informed of the shutdown.
*
*
*
*/ static class BlockedOnParentState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument consumerArgument, ShardConsumer consumer, ProcessRecordsInput input) { return new BlockOnParentShardTask(consumerArgument.shardInfo(), consumerArgument.leaseCoordinator().leaseRefresher(), consumerArgument.parentShardPollIntervalMillis()); } @Override public ConsumerState successTransition() { return ShardConsumerState.INITIALIZING.consumerState(); } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { return ShardConsumerState.SHUTTING_DOWN.consumerState(); } @Override public TaskType taskType() { return TaskType.BLOCK_ON_PARENT_SHARDS; } @Override public ShardConsumerState state() { return ShardConsumerState.WAITING_ON_PARENT_SHARDS; } @Override public boolean isTerminal() { return false; } } /** * This state is responsible for initializing the record processor with the shard information. *

Valid Transitions

*
*
Success
*
Transitions to the processing state which will begin to send records to the record processor
*
Shutdown
*
At this point the record processor has been initialized, but hasn't processed any records. This requires that * the record processor be notified of the shutdown, even though there is almost no actions the record processor * could take. *
*
{@link ShutdownReason#REQUESTED}
*
Transitions to the {@link ShutdownNotificationState}
*
{@link ShutdownReason#LEASE_LOST}
*
Transitions to the {@link ShuttingDownState}
*
{@link ShutdownReason#SHARD_END}
*
*

* This reason should not occur, since terminate is triggered after reaching the end of a shard. Initialize never * makes an requests to Kinesis for records, so it can't reach the end of a shard. *

*

* Transitions to the {@link ShuttingDownState} *

*
*
*
*
*/ static class InitializingState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument argument, ShardConsumer consumer, ProcessRecordsInput input) { return new InitializeTask(argument.shardInfo(), argument.shardRecordProcessor(), argument.checkpoint(), argument.recordProcessorCheckpointer(), argument.initialPositionInStream(), argument.recordsPublisher(), argument.taskBackoffTimeMillis(), argument.metricsFactory()); } @Override public ConsumerState successTransition() { return ShardConsumerState.PROCESSING.consumerState(); } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { return shutdownReason.shutdownState(); } @Override public TaskType taskType() { return TaskType.INITIALIZE; } @Override public ShardConsumerState state() { return ShardConsumerState.INITIALIZING; } @Override public boolean isTerminal() { return false; } } /** * This state is responsible for retrieving records from Kinesis, and dispatching them to the record processor. * While in this state the only way a transition will occur is if a shutdown has been triggered. *

Valid Transitions

*
*
Success
*
Doesn't actually transition, but instead returns the same state
*
Shutdown
*
At this point records are being retrieved, and processed. It's now possible for the consumer to reach the end * of the shard triggering a {@link ShutdownReason#SHARD_END}. *
*
{@link ShutdownReason#REQUESTED}
*
Transitions to the {@link ShutdownNotificationState}
*
{@link ShutdownReason#LEASE_LOST}
*
Transitions to the {@link ShuttingDownState}
*
{@link ShutdownReason#SHARD_END}
*
Transitions to the {@link ShuttingDownState}
*
*
*
*/ static class ProcessingState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument argument, ShardConsumer consumer, ProcessRecordsInput input) { ThrottlingReporter throttlingReporter = new ThrottlingReporter(5, argument.shardInfo().shardId()); return new ProcessTask(argument.shardInfo(), argument.shardRecordProcessor(), argument.recordProcessorCheckpointer(), argument.taskBackoffTimeMillis(), argument.skipShardSyncAtWorkerInitializationIfLeasesExist(), argument.shardDetector(), throttlingReporter, input, argument.shouldCallProcessRecordsEvenForEmptyRecordList(), argument.idleTimeInMilliseconds(), argument.aggregatorUtil(), argument.metricsFactory(), argument.schemaRegistryDecoder() ); } @Override public ConsumerState successTransition() { return ShardConsumerState.PROCESSING.consumerState(); } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { return shutdownReason.shutdownState(); } @Override public TaskType taskType() { return TaskType.PROCESS; } @Override public ShardConsumerState state() { return ShardConsumerState.PROCESSING; } @Override public boolean isTerminal() { return false; } @Override public boolean requiresDataAvailability() { return true; } } static final ConsumerState SHUTDOWN_REQUEST_COMPLETION_STATE = new ShutdownNotificationCompletionState(); /** * This state occurs when a shutdown has been explicitly requested. This shutdown allows the record processor a * chance to checkpoint and prepare to be shutdown via the normal method. This state can only be reached by a * shutdown on the {@link InitializingState} or {@link ProcessingState}. * *

Valid Transitions

*
*
Success
*
Success shouldn't normally be called since the {@link ShardConsumer} is marked for shutdown.
*
Shutdown
*
At this point records are being retrieved, and processed. An explicit shutdown will allow the record * processor one last chance to checkpoint, and then the {@link ShardConsumer} will be held in an idle state. *
*
{@link ShutdownReason#REQUESTED}
*
Remains in the {@link ShardConsumerState#SHUTDOWN_REQUESTED}, but the state implementation changes to * {@link ShutdownNotificationCompletionState}
*
{@link ShutdownReason#LEASE_LOST}
*
Transitions to the {@link ShuttingDownState}
*
{@link ShutdownReason#SHARD_END}
*
Transitions to the {@link ShuttingDownState}
*
*
*
*/ static class ShutdownNotificationState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument argument, ShardConsumer consumer, ProcessRecordsInput input) { // TODO: notify shutdownrequested return new ShutdownNotificationTask(argument.shardRecordProcessor(), argument.recordProcessorCheckpointer(), consumer.shutdownNotification(), argument.shardInfo()); } @Override public ConsumerState successTransition() { return SHUTDOWN_REQUEST_COMPLETION_STATE; } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { if (shutdownReason == ShutdownReason.REQUESTED) { return SHUTDOWN_REQUEST_COMPLETION_STATE; } return shutdownReason.shutdownState(); } @Override public TaskType taskType() { return TaskType.SHUTDOWN_NOTIFICATION; } @Override public ShardConsumerState state() { return ShardConsumerState.SHUTDOWN_REQUESTED; } @Override public boolean isTerminal() { return false; } } /** * Once the {@link ShutdownNotificationState} has been completed the {@link ShardConsumer} must not re-enter any of the * processing states. This state idles the {@link ShardConsumer} until the worker triggers the final shutdown state. * *

Valid Transitions

*
*
Success
*
*

* Success shouldn't normally be called since the {@link ShardConsumer} is marked for shutdown. *

*

* Remains in the {@link ShutdownNotificationCompletionState} *

*
*
Shutdown
*
At this point the {@link ShardConsumer} has notified the record processor of the impending shutdown, and is * waiting that notification. While waiting for the notification no further processing should occur on the * {@link ShardConsumer}. *
*
{@link ShutdownReason#REQUESTED}
*
Remains in the {@link ShardConsumerState#SHUTDOWN_REQUESTED}, and the state implementation remains * {@link ShutdownNotificationCompletionState}
*
{@link ShutdownReason#LEASE_LOST}
*
Transitions to the {@link ShuttingDownState}
*
{@link ShutdownReason#SHARD_END}
*
Transitions to the {@link ShuttingDownState}
*
*
*
*/ static class ShutdownNotificationCompletionState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument argument, ShardConsumer consumer, ProcessRecordsInput input) { return null; } @Override public ConsumerState successTransition() { return this; } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { if (shutdownReason != ShutdownReason.REQUESTED) { return shutdownReason.shutdownState(); } return this; } @Override public TaskType taskType() { return TaskType.SHUTDOWN_NOTIFICATION; } @Override public ShardConsumerState state() { return ShardConsumerState.SHUTDOWN_REQUESTED; } @Override public boolean isTerminal() { return false; } @Override public boolean requiresAwake() { return true; } } /** * This state is entered if the {@link ShardConsumer} loses its lease, or reaches the end of the shard. * *

Valid Transitions

*
*
Success
*
*

* Success shouldn't normally be called since the {@link ShardConsumer} is marked for shutdown. *

*

* Transitions to the {@link ShutdownCompleteState} *

*
*
Shutdown
*
At this point the record processor has processed the final shutdown indication, and depending on the shutdown * reason taken the correct course of action. From this point on there should be no more interactions with the * record processor or {@link ShardConsumer}. *
*
{@link ShutdownReason#REQUESTED}
*
*

* This should not occur as all other {@link ShutdownReason}s take priority over it. *

*

* Transitions to {@link ShutdownCompleteState} *

*
*
{@link ShutdownReason#LEASE_LOST}
*
Transitions to the {@link ShutdownCompleteState}
*
{@link ShutdownReason#SHARD_END}
*
Transitions to the {@link ShutdownCompleteState}
*
*
*
*/ static class ShuttingDownState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument argument, ShardConsumer consumer, ProcessRecordsInput input) { // TODO: set shutdown reason return new ShutdownTask(argument.shardInfo(), argument.shardDetector(), argument.shardRecordProcessor(), argument.recordProcessorCheckpointer(), consumer.shutdownReason(), argument.initialPositionInStream(), argument.cleanupLeasesOfCompletedShards(), argument.ignoreUnexpectedChildShards(), argument.leaseCoordinator(), argument.taskBackoffTimeMillis(), argument.recordsPublisher(), argument.hierarchicalShardSyncer(), argument.metricsFactory(), input == null ? null : input.childShards(), argument.streamIdentifier(), argument.leaseCleanupManager()); } @Override public ConsumerState successTransition() { return ShardConsumerState.SHUTDOWN_COMPLETE.consumerState(); } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { return ShardConsumerState.SHUTDOWN_COMPLETE.consumerState(); } @Override public TaskType taskType() { return TaskType.SHUTDOWN; } @Override public ShardConsumerState state() { return ShardConsumerState.SHUTTING_DOWN; } @Override public boolean isTerminal() { return false; } } /** * This is the final state for the {@link ShardConsumer}. This occurs once all shutdown activities are completed. * *

Valid Transitions

*
*
Success
*
*

* Success shouldn't normally be called since the {@link ShardConsumer} is marked for shutdown. *

*

* Remains in the {@link ShutdownCompleteState} *

*
*
Shutdown
*
At this point the all shutdown activites are completed, and the {@link ShardConsumer} should not take any * further actions. *
*
{@link ShutdownReason#REQUESTED}
*
*

* This should not occur as all other {@link ShutdownReason}s take priority over it. *

*

* Remains in {@link ShutdownCompleteState} *

*
*
{@link ShutdownReason#LEASE_LOST}
*
Remains in {@link ShutdownCompleteState}
*
{@link ShutdownReason#SHARD_END}
*
Remains in {@link ShutdownCompleteState}
*
*
*
*/ static class ShutdownCompleteState implements ConsumerState { @Override public ConsumerTask createTask(ShardConsumerArgument argument, ShardConsumer consumer, ProcessRecordsInput input) { if (consumer.shutdownNotification() != null) { consumer.shutdownNotification().shutdownComplete(); } return null; } @Override public ConsumerState successTransition() { return this; } @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { return this; } @Override public TaskType taskType() { return TaskType.SHUTDOWN_COMPLETE; } @Override public ShardConsumerState state() { return ShardConsumerState.SHUTDOWN_COMPLETE; } @Override public boolean isTerminal() { return true; } } }




© 2015 - 2025 Weber Informatics LLC | Privacy Policy