software.amazon.kinesis.lifecycle.ConsumerStates Maven / Gradle / Ivy
Show all versions of amazon-kinesis-client Show documentation
/*
* 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;
}
}
}