amazon-kinesis-client/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangProtocol.java

296 lines
13 KiB
Java
Raw Normal View History

/*
* Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved.
*
* Licensed under the Amazon Software License (the "License").
* You may not use this file except in compliance with the License.
* A copy of the License is located at
*
* http://aws.amazon.com/asl/
*
* or in the "license" file accompanying this file. This file 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.multilang;
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import lombok.extern.slf4j.Slf4j;
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
import software.amazon.kinesis.exceptions.InvalidStateException;
import software.amazon.kinesis.lifecycle.events.InitializationInput;
import software.amazon.kinesis.lifecycle.events.LeaseLostInput;
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
import software.amazon.kinesis.lifecycle.events.ShardEndedInput;
import software.amazon.kinesis.multilang.config.MultiLangDaemonConfiguration;
import software.amazon.kinesis.multilang.messages.CheckpointMessage;
import software.amazon.kinesis.multilang.messages.InitializeMessage;
import software.amazon.kinesis.multilang.messages.LeaseLostMessage;
import software.amazon.kinesis.multilang.messages.Message;
import software.amazon.kinesis.multilang.messages.ProcessRecordsMessage;
import software.amazon.kinesis.multilang.messages.ShardEndedMessage;
import software.amazon.kinesis.multilang.messages.ShutdownRequestedMessage;
import software.amazon.kinesis.multilang.messages.StatusMessage;
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
/**
* An implementation of the multi language protocol.
*/
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
@Slf4j
class MultiLangProtocol {
private final InitializationInput initializationInput;
private final Optional<Integer> timeoutInSeconds;
private MessageReader messageReader;
private MessageWriter messageWriter;
private MultiLangDaemonConfiguration configuration;
/**
* Constructor.
*
* @param messageReader
* A message reader.
* @param messageWriter
* A message writer.
* @param initializationInput
* information about the shard this processor is starting to process
*/
MultiLangProtocol(MessageReader messageReader, MessageWriter messageWriter,
InitializationInput initializationInput, MultiLangDaemonConfiguration configuration) {
this.messageReader = messageReader;
this.messageWriter = messageWriter;
this.initializationInput = initializationInput;
this.configuration = configuration;
this.timeoutInSeconds = Optional.ofNullable(configuration.getTimeoutInSeconds());
}
/**
* Writes an {@link InitializeMessage} to the child process's STDIN and waits for the child process to respond with
* a {@link StatusMessage} on its STDOUT.
*
* @return Whether or not this operation succeeded.
*/
boolean initialize() {
/*
* Call and response to child process.
*/
Future<Boolean> writeFuture = messageWriter.writeInitializeMessage(initializationInput);
return waitForStatusMessage(InitializeMessage.ACTION, null, writeFuture);
}
/**
* Writes a {@link ProcessRecordsMessage} to the child process's STDIN and waits for the child process to respond
* with a {@link StatusMessage} on its STDOUT.
*
* @param processRecordsInput
* The records, and associated metadata, to process.
* @return Whether or not this operation succeeded.
*/
boolean processRecords(ProcessRecordsInput processRecordsInput) {
Future<Boolean> writeFuture = messageWriter.writeProcessRecordsMessage(processRecordsInput);
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
return waitForStatusMessage(ProcessRecordsMessage.ACTION, processRecordsInput.checkpointer(), writeFuture);
}
/**
* Notifies the client process that the lease has been lost, and it needs to shutdown.
*
* @param leaseLostInput
* the lease lost input that is passed to the {@link MessageWriter}
* @return true if the message was successfully writtem
*/
boolean leaseLost(LeaseLostInput leaseLostInput) {
return waitForStatusMessage(LeaseLostMessage.ACTION, null, messageWriter.writeLeaseLossMessage(leaseLostInput));
}
/**
*
* @param shardEndedInput
* @return
*/
boolean shardEnded(ShardEndedInput shardEndedInput) {
return waitForStatusMessage(ShardEndedMessage.ACTION, shardEndedInput.checkpointer(),
messageWriter.writeShardEndedMessage(shardEndedInput));
}
/**
* Writes a {@link ShutdownRequestedMessage} to the child process's STDIN and waits for the child process to respond with a
* {@link StatusMessage} on its STDOUT.
*
* @param checkpointer A checkpointer.
* @return Whether or not this operation succeeded.
*/
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
boolean shutdownRequested(RecordProcessorCheckpointer checkpointer) {
Future<Boolean> writeFuture = messageWriter.writeShutdownRequestedMessage();
return waitForStatusMessage(ShutdownRequestedMessage.ACTION, checkpointer, writeFuture);
}
/**
* Waits for a {@link StatusMessage} for a particular action. If a {@link CheckpointMessage} is received, then this
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
* method will attempt to checkpoint with the provided {@link RecordProcessorCheckpointer}. This method returns
* true if writing to the child process succeeds and the status message received back was for the correct action and
* all communications with the child process regarding checkpointing were successful. Note that whether or not the
* checkpointing itself was successful is not the concern of this method. This method simply cares whether it was
* able to successfully communicate the results of its attempts to checkpoint.
*
* @param action
* What action is being waited on.
* @param checkpointer
* the checkpointer from the process records, or shutdown request
* @param writeFuture
* The writing task.
* @return Whether or not this operation succeeded.
*/
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
private boolean waitForStatusMessage(String action, RecordProcessorCheckpointer checkpointer,
Future<Boolean> writeFuture) {
boolean statusWasCorrect = waitForStatusMessage(action, checkpointer);
// Examine whether or not we failed somewhere along the line.
try {
boolean writerIsStillOpen = writeFuture.get();
return statusWasCorrect && writerIsStillOpen;
} catch (InterruptedException e) {
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
log.error("Interrupted while writing {} message for shard {}", action, initializationInput.shardId());
return false;
} catch (ExecutionException e) {
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
log.error("Failed to write {} message for shard {}", action, initializationInput.shardId(), e);
return false;
}
}
/**
* Waits for status message and verifies it against the expectation
*
* @param action
* What action is being waited on.
* @param checkpointer
* the original process records request
* @return Whether or not this operation succeeded.
*/
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
boolean waitForStatusMessage(String action, RecordProcessorCheckpointer checkpointer) {
Optional<StatusMessage> statusMessage = Optional.empty();
while (!statusMessage.isPresent()) {
Future<Message> future = this.messageReader.getNextMessageFromSTDOUT();
Optional<Message> message = timeoutInSeconds
.map(second -> futureMethod(() -> future.get(second, TimeUnit.SECONDS), action))
.orElse(futureMethod(future::get, action));
if (!message.isPresent()) {
return false;
}
Optional<Boolean> checkpointFailed = message.filter(m -> m instanceof CheckpointMessage)
.map(m -> (CheckpointMessage) m)
.flatMap(m -> futureMethod(() -> checkpoint(m, checkpointer).get(), "Checkpoint"))
.map(checkpointSuccess -> !checkpointSuccess);
if (checkpointFailed.orElse(false)) {
return false;
}
statusMessage = message.filter(m -> m instanceof StatusMessage).map(m -> (StatusMessage) m );
}
return this.validateStatusMessage(statusMessage.get(), action);
}
private interface FutureMethod<T> {
T get() throws InterruptedException, TimeoutException, ExecutionException;
}
private <T> Optional<T> futureMethod(FutureMethod<T> fm, String action) {
try {
return Optional.of(fm.get());
} catch (InterruptedException e) {
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
log.error("Interrupted while waiting for {} message for shard {}", action,
initializationInput.shardId(), e);
} catch (ExecutionException e) {
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
log.error("Failed to get status message for {} action for shard {}", action,
initializationInput.shardId(), e);
} catch (TimeoutException e) {
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
log.error("Timedout to get status message for {} action for shard {}. Terminating...",
action,
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
initializationInput.shardId(),
e);
haltJvm(1);
}
return Optional.empty();
}
/**
* This method is used to halt the JVM. Use this method with utmost caution, since this method will kill the JVM
* without calling the Shutdown hooks.
*
* @param exitStatus The exit status with which the JVM is to be halted.
*/
protected void haltJvm(int exitStatus) {
Runtime.getRuntime().halt(exitStatus);
}
/**
* Utility for confirming that the status message is for the provided action.
*
* @param statusMessage The status of the child process.
* @param action The action that was being waited on.
* @return Whether or not this operation succeeded.
*/
private boolean validateStatusMessage(StatusMessage statusMessage, String action) {
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
log.info("Received response {} from subprocess while waiting for {}"
+ " while processing shard {}", statusMessage, action, initializationInput.shardId());
return !(statusMessage == null || statusMessage.getResponseFor() == null || !statusMessage.getResponseFor()
.equals(action));
}
/**
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
* Attempts to checkpoint with the provided {@link RecordProcessorCheckpointer} at the sequence number in the
* provided {@link CheckpointMessage}. If no sequence number is provided, i.e. the sequence number is null, then
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
* this method will call {@link RecordProcessorCheckpointer#checkpoint()}. The method returns a future representing
* the attempt to write the result of this checkpoint attempt to the child process.
*
* @param checkpointMessage A checkpoint message.
* @param checkpointer A checkpointer.
* @return Whether or not this operation succeeded.
*/
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
private Future<Boolean> checkpoint(CheckpointMessage checkpointMessage, RecordProcessorCheckpointer checkpointer) {
String sequenceNumber = checkpointMessage.getSequenceNumber();
Long subSequenceNumber = checkpointMessage.getSubSequenceNumber();
try {
if (checkpointer != null) {
log.debug(logCheckpointMessage(sequenceNumber, subSequenceNumber));
if (sequenceNumber != null) {
if (subSequenceNumber != null) {
checkpointer.checkpoint(sequenceNumber, subSequenceNumber);
} else {
checkpointer.checkpoint(sequenceNumber);
}
} else {
checkpointer.checkpoint();
}
return this.messageWriter.writeCheckpointMessageWithError(sequenceNumber, subSequenceNumber, null);
} else {
String message =
String.format("Was asked to checkpoint at %s but no checkpointer was provided for shard %s",
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
sequenceNumber, initializationInput.shardId());
log.error(message);
return this.messageWriter.writeCheckpointMessageWithError(sequenceNumber, subSequenceNumber,
new InvalidStateException(
message));
}
} catch (Throwable t) {
return this.messageWriter.writeCheckpointMessageWithError(sequenceNumber, subSequenceNumber, t);
}
}
private String logCheckpointMessage(String sequenceNumber, Long subSequenceNumber) {
return String.format("Attempting to checkpoint shard %s @ sequence number %s, and sub sequence number %s",
Release 2.0.0 of the Amazon Kinesis Client for Java * Added support for Enhanced Fan Out. Enhanced Fan Out provides for lower end to end latency, and increased number of consumers per stream. * Records are now delivered via streaming, reducing end-to-end latency. * The Amazon Kinesis Client will automatically register a new consumer if required. When registering a new consumer, the Kinesis Client will default to the application name unless configured otherwise. * New configuration options are available to configure Enhanced Fan Out. * `SubscribeToShard` maintains long lived connections with Kinesis, which in the AWS Java SDK 2.0 is limited by default. The `KinesisClientUtil` has been added to assist configuring the `maxConcurrency` of the `KinesisAsyncClient`. __WARNING: The Amazon Kinesis Client may see significantly increased latency, unless the `KinesisAsyncClient` is configured to have a `maxConcurrency` high enough to allow all leases plus additional usages of the `KinesisAsyncClient`.__ | Name | Default | Description | |-----------------|---------|---------------------------------------------------------------------------------------------------------------------| | consumerArn | Unset | The ARN for an already created consumer. If this is set, the Kinesis Client will not attempt to create a consumer. | | streamName | Unset | The name of the stream that a consumer should be create for if necessary | | consumerName | Unset | The name of the consumer to create. If this is not set the applicationName will be used instead. | | applicationName | Unset | The name of the application. This is used as the name of the consumer unless consumerName is set. | * Modular Configuration of the Kinesis Client The Kinesis Client has migrated to a modular configuration system, and the `KinesisClientLibConfiguration` class has been removed. Configuration has been split into 7 classes. Default versions of the configuration can be created from the `ConfigsBuilder`. Please see the migration guide for more information * `CheckpointConfig` * `CoordinatorConfig` * `LeaseManagementConfig` * `LifecycleConfig` * `MetricsConfig` * `ProcessorConfig` * `RetrievalConfig` * Upgraded to AWS Java SDK 2.0 The Kinesis Client now uses the AWS Java SDK 2.0. The dependency on AWS Java SDK 1.11 has been removed. All configurations will only accept 2.0 clients. * When configuring the `KinesisAsyncClient` the `KinesisClientUtil#createKinesisAsyncClient` can be used to configure the Kinesis Client * __If you need support for AWS Java SDK 1.11 you will need to add a direct dependency.__ __When adding a dependency you must ensure that the 1.11 versions of Jackson dependencies are excluded__ Please see the migration guide for more information * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`.
2018-08-02 17:57:11 +00:00
initializationInput.shardId(), sequenceNumber, subSequenceNumber);
}
}