AWS Kinesis

Stateful Functions offers an AWS Kinesis I/O Module for reading from and writing to Kinesis streams. It is based on Apache Flink’s Kinesis connector. The Kinesis I/O Module is configurable in Yaml or Java.

Dependency

To use the Kinesis I/O Module in Java, please include the following dependency in your pom.

  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>statefun-kinesis-io</artifactId>
  4. <version>2.1.0</version>
  5. <scope>provided</scope>
  6. </dependency>

Kinesis Ingress Spec

A KinesisIngressSpec declares an ingress spec for consuming from Kinesis stream.

It accepts the following arguments:

  1. The AWS region
  2. An AWS credentials provider
  3. A KinesisIngressDeserializer for deserializing data from Kinesis (Java only)
  4. The stream start position
  5. Properties for the Kinesis client
  6. The name of the stream to consume from
  1. package org.apache.flink.statefun.docs.io.kinesis;
  2. import org.apache.flink.statefun.docs.models.User;
  3. import org.apache.flink.statefun.sdk.io.IngressIdentifier;
  4. import org.apache.flink.statefun.sdk.io.IngressSpec;
  5. import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
  6. import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressBuilder;
  7. import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressStartupPosition;
  8. public class IngressSpecs {
  9. public static final IngressIdentifier<User> ID =
  10. new IngressIdentifier<>(User.class, "example", "input-ingress");
  11. public static final IngressSpec<User> kinesisIngress =
  12. KinesisIngressBuilder.forIdentifier(ID)
  13. .withAwsRegion("us-west-1")
  14. .withAwsCredentials(AwsCredentials.fromDefaultProviderChain())
  15. .withDeserializer(UserDeserializer.class)
  16. .withStream("stream-name")
  17. .withStartupPosition(KinesisIngressStartupPosition.fromEarliest())
  18. .withClientConfigurationProperty("key", "value")
  19. .build();
  20. }
  1. version: "1.0"
  2. module:
  3. meta:
  4. type: remote
  5. spec:
  6. ingresses:
  7. - ingress:
  8. meta:
  9. type: statefun.kinesis.io/routable-protobuf-ingress
  10. id: example-namespace/messages
  11. spec:
  12. awsRegion:
  13. type: specific
  14. id: us-west-1
  15. awsCredentials:
  16. type: basic
  17. accessKeyId: my_access_key_id
  18. secretAccessKey: my_secret_access_key
  19. startupPosition:
  20. type: earliest
  21. streams:
  22. - stream: stream-1
  23. typeUrl: com.googleapis/org.apache.flink.statefun.docs.models.User
  24. targets:
  25. - example-namespace/my-function-1
  26. - example-namespace/my-function-2
  27. - stream: stream-2
  28. typeUrl: com.googleapis/org.apache.flink.statefun.docs.models.User
  29. targets:
  30. - example-namespace/my-function-1
  31. clientConfigProperties:
  32. - SocketTimeout: 9999
  33. - MaxConnections: 15

The ingress also accepts properties to directly configure the Kinesis client, using KinesisIngressBuilder#withClientConfigurationProperty(). Please refer to the Kinesis client configuration documentation for the full list of available properties. Note that configuration passed using named methods will have higher precedence and overwrite their respective settings in the provided properties.

Startup Position

The ingress allows configuring the startup position to be one of the following:

Latest (default)

Start consuming from the latest position, i.e. head of the stream shards.

  1. KinesisIngressStartupPosition#fromLatest();
  1. startupPosition:
  2. type: latest

Earlist

Start consuming from the earliest position possible.

  1. KinesisIngressStartupPosition#fromEarliest();
  1. startupPosition:
  2. type: earliest

Date

Starts from offsets that have an ingestion time larger than or equal to a specified date.

  1. KinesisIngressStartupPosition#fromDate(ZonedDateTime.now());
  1. startupPosition:
  2. type: date
  3. date: 2020-02-01 04:15:00.00 Z

Kinesis Deserializer

The Kinesis ingress needs to know how to turn the binary data in Kinesis into Java objects. The KinesisIngressDeserializer allows users to specify such a schema. The T deserialize(IngressRecord ingressRecord) method gets called for each Kinesis record, passing the binary data and metadata from Kinesis.

  1. package org.apache.flink.statefun.docs.io.kinesis;
  2. import com.fasterxml.jackson.databind.ObjectMapper;
  3. import java.io.IOException;
  4. import org.apache.flink.statefun.docs.models.User;
  5. import org.apache.flink.statefun.sdk.kinesis.ingress.IngressRecord;
  6. import org.apache.flink.statefun.sdk.kinesis.ingress.KinesisIngressDeserializer;
  7. import org.slf4j.Logger;
  8. import org.slf4j.LoggerFactory;
  9. public class UserDeserializer implements KinesisIngressDeserializer<User> {
  10. private static Logger LOG = LoggerFactory.getLogger(UserDeserializer.class);
  11. private final ObjectMapper mapper = new ObjectMapper();
  12. @Override
  13. public User deserialize(IngressRecord ingressRecord) {
  14. try {
  15. return mapper.readValue(ingressRecord.getData(), User.class);
  16. } catch (IOException e) {
  17. LOG.debug("Failed to deserialize record", e);
  18. return null;
  19. }
  20. }
  21. }

Kinesis Egress Spec

A KinesisEgressBuilder declares an egress spec for writing data out to a Kinesis stream.

It accepts the following arguments:

  1. The egress identifier associated with this egress
  2. The AWS credentials provider
  3. A KinesisEgressSerializer for serializing data into Kinesis (Java only)
  4. The AWS region
  5. Properties for the Kinesis client
  6. The number of max outstanding records before backpressure is applied
  1. package org.apache.flink.statefun.docs.io.kinesis;
  2. import org.apache.flink.statefun.docs.models.User;
  3. import org.apache.flink.statefun.sdk.io.EgressIdentifier;
  4. import org.apache.flink.statefun.sdk.io.EgressSpec;
  5. import org.apache.flink.statefun.sdk.kinesis.auth.AwsCredentials;
  6. import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressBuilder;
  7. public class EgressSpecs {
  8. public static final EgressIdentifier<User> ID =
  9. new EgressIdentifier<>("example", "output-egress", User.class);
  10. public static final EgressSpec<User> kinesisEgress =
  11. KinesisEgressBuilder.forIdentifier(ID)
  12. .withAwsCredentials(AwsCredentials.fromDefaultProviderChain())
  13. .withAwsRegion("us-west-1")
  14. .withMaxOutstandingRecords(100)
  15. .withClientConfigurationProperty("key", "value")
  16. .withSerializer(UserSerializer.class)
  17. .build();
  18. }
  1. version: "1.0"
  2. module:
  3. meta:
  4. type: remote
  5. spec:
  6. egresses:
  7. - egress:
  8. meta:
  9. type: statefun.kinesis.io/generic-egress
  10. id: example/output-messages
  11. spec:
  12. awsRegion:
  13. type: custom-endpoint
  14. endpoint: https://localhost:4567
  15. id: us-west-1
  16. awsCredentials:
  17. type: profile
  18. profileName: john-doe
  19. profilePath: /path/to/profile/config
  20. maxOutstandingRecords: 9999
  21. clientConfigProperties:
  22. - ThreadingModel: POOLED
  23. - ThreadPoolSize: 10

Please refer to the Kinesis producer default configuration properties documentation for the full list of available properties.

Kinesis Serializer

The Kinesis egress needs to know how to turn Java objects into binary data. The KinesisEgressSerializer allows users to specify such a schema. The EgressRecord serialize(T value) method gets called for each message, allowing users to set a value, and other metadata.

  1. package org.apache.flink.statefun.docs.io.kinesis;
  2. import com.fasterxml.jackson.databind.ObjectMapper;
  3. import java.io.IOException;
  4. import org.apache.flink.statefun.docs.models.User;
  5. import org.apache.flink.statefun.sdk.kinesis.egress.EgressRecord;
  6. import org.apache.flink.statefun.sdk.kinesis.egress.KinesisEgressSerializer;
  7. import org.slf4j.Logger;
  8. import org.slf4j.LoggerFactory;
  9. public class UserSerializer implements KinesisEgressSerializer<User> {
  10. private static final Logger LOG = LoggerFactory.getLogger(UserSerializer.class);
  11. private static final String STREAM = "user-stream";
  12. private final ObjectMapper mapper = new ObjectMapper();
  13. @Override
  14. public EgressRecord serialize(User value) {
  15. try {
  16. return EgressRecord.newBuilder()
  17. .withPartitionKey(value.getUserId())
  18. .withData(mapper.writeValueAsBytes(value))
  19. .withStream(STREAM)
  20. .build();
  21. } catch (IOException e) {
  22. LOG.info("Failed to serializer user", e);
  23. return null;
  24. }
  25. }
  26. }

AWS Region

Both the Kinesis ingress and egress can be configured to a specific AWS region.

Default Provider Chain (default)

Consults AWS’s default provider chain to determine the AWS region.

  1. AwsRegion.fromDefaultProviderChain();
  1. awsCredentials:
  2. type: default

Specific

Specifies an AWS region using the region’s unique id.

  1. AwsRegion.of("us-west-1");
  1. awsCredentials:
  2. type: specific
  3. id: us-west-1

Custom Endpoint

Connects to an AWS region through a non-standard AWS service endpoint. This is typically used only for development and testing purposes.

  1. AwsRegion.ofCustomEndpoint("https://localhost:4567", "us-west-1");
  1. awsCredentials:
  2. type: custom-endpoint
  3. endpoint: https://localhost:4567
  4. id: us-west-1

AWS Credentials

Both the Kinesis ingress and egress can be configured using standard AWS credential providers.

Default Provider Chain (default)

Consults AWS’s default provider chain to determine the AWS credentials.

  1. AwsCredentials.fromDefaultProviderChain();
  1. awsCredentials:
  2. type: default

Basic

Specifies the AWS credentials directly with provided access key ID and secret access key strings.

  1. AwsCredentials.basic("accessKeyId", "secretAccessKey");
  1. awsCredentials:
  2. type: basic
  3. accessKeyId: access-key-id
  4. secretAccessKey: secret-access-key

Profile

Specifies the AWS credentials using an AWS configuration profile, along with the profile’s configuration path.

  1. AwsCredentials.profile("profile-name", "/path/to/profile/config");
  1. awsCredentials:
  2. type: basic
  3. profileName: profile-name
  4. profilePath: /path/to/profile/config