Adding upstream version 1.34.4.
Signed-off-by: Daniel Baumann <daniel@debian.org>
This commit is contained in:
parent
e393c3af3f
commit
4978089aab
4963 changed files with 677545 additions and 0 deletions
142
plugins/inputs/kinesis_consumer/README.md
Normal file
142
plugins/inputs/kinesis_consumer/README.md
Normal file
|
@ -0,0 +1,142 @@
|
|||
# Kinesis Consumer Input Plugin
|
||||
|
||||
This service input plugin consumes messages from [AWS Kinesis][kinesis] data
|
||||
stream in one of the supported [data formats][data_formats].
|
||||
|
||||
⭐ Telegraf v1.10.0
|
||||
🏷️ messaging, iot
|
||||
💻 all
|
||||
|
||||
[kinesis]: https://aws.amazon.com/kinesis/
|
||||
[data_formats]: /docs/DATA_FORMATS_INPUT.md
|
||||
|
||||
## Service Input <!-- @/docs/includes/service_input.md -->
|
||||
|
||||
This plugin is a service input. Normal plugins gather metrics determined by the
|
||||
interval setting. Service plugins start a service to listen and wait for
|
||||
metrics or events to occur. Service plugins have two key differences from
|
||||
normal plugins:
|
||||
|
||||
1. The global or plugin specific `interval` setting may not apply
|
||||
2. The CLI options of `--test`, `--test-wait`, and `--once` may not produce
|
||||
output for this plugin
|
||||
|
||||
## Global configuration options <!-- @/docs/includes/plugin_config.md -->
|
||||
|
||||
In addition to the plugin-specific configuration settings, plugins support
|
||||
additional global and plugin configuration settings. These settings are used to
|
||||
modify metrics, tags, and field or create aliases and configure ordering, etc.
|
||||
See the [CONFIGURATION.md][CONFIGURATION.md] for more details.
|
||||
|
||||
[CONFIGURATION.md]: ../../../docs/CONFIGURATION.md#plugins
|
||||
|
||||
## Configuration
|
||||
|
||||
```toml @sample.conf
|
||||
# Configuration for the AWS Kinesis input.
|
||||
[[inputs.kinesis_consumer]]
|
||||
## Amazon REGION of kinesis endpoint.
|
||||
region = "ap-southeast-2"
|
||||
|
||||
## Amazon Credentials
|
||||
## Credentials are loaded in the following order
|
||||
## 1) Web identity provider credentials via STS if role_arn and web_identity_token_file are specified
|
||||
## 2) Assumed credentials via STS if role_arn is specified
|
||||
## 3) explicit credentials from 'access_key' and 'secret_key'
|
||||
## 4) shared profile from 'profile'
|
||||
## 5) environment variables
|
||||
## 6) shared credentials file
|
||||
## 7) EC2 Instance Profile
|
||||
# access_key = ""
|
||||
# secret_key = ""
|
||||
# token = ""
|
||||
# role_arn = ""
|
||||
# web_identity_token_file = ""
|
||||
# role_session_name = ""
|
||||
# profile = ""
|
||||
# shared_credential_file = ""
|
||||
|
||||
## Endpoint to make request against, the correct endpoint is automatically
|
||||
## determined and this option should only be set if you wish to override the
|
||||
## default.
|
||||
## ex: endpoint_url = "http://localhost:8000"
|
||||
# endpoint_url = ""
|
||||
|
||||
## Kinesis StreamName must exist prior to starting telegraf.
|
||||
streamname = "StreamName"
|
||||
|
||||
## Shard iterator type
|
||||
## Available options: 'TRIM_HORIZON' (first in non-expired) and 'LATEST'
|
||||
# shard_iterator_type = "TRIM_HORIZON"
|
||||
|
||||
## Interval for checking for new records
|
||||
## Please consider limits for getting records documented here:
|
||||
## https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html
|
||||
# poll_interval = "250ms"
|
||||
|
||||
## Interval for scanning for new shards created when resharding
|
||||
## If set to zero, shards are only scanned once on startup.
|
||||
# shard_update_interval = "30s"
|
||||
|
||||
## Max undelivered messages
|
||||
## This plugin uses tracking metrics, which ensure messages are read to
|
||||
## outputs before acknowledging them to the original broker to ensure data
|
||||
## is not lost. This option sets the maximum messages to read from the
|
||||
## broker that have not been written by an output.
|
||||
##
|
||||
## This value needs to be picked with awareness of the agent's
|
||||
## metric_batch_size value as well. Setting max undelivered messages too high
|
||||
## can result in a constant stream of data batches to the output. While
|
||||
## setting it too low may never flush the broker's messages.
|
||||
# max_undelivered_messages = 1000
|
||||
|
||||
## Content encoding of the record data
|
||||
## If you are processing a cloudwatch logs kinesis stream then set this to
|
||||
## "gzip" as AWS compresses cloudwatch log data before it is sent to kinesis.
|
||||
# content_encoding = "identity"
|
||||
|
||||
## Data format of the records to consume
|
||||
## See https://github.com/influxdata/telegraf/blob/master/docs/DATA_FORMATS_INPUT.md
|
||||
# data_format = "influx"
|
||||
|
||||
## Optional: Configuration for DynamoDB backend to store positions in the stream
|
||||
# [inputs.kinesis_consumer.checkpoint_dynamodb]
|
||||
# ## Unique name for this consumer
|
||||
# app_name = "default"
|
||||
# ## Table to store the sequence numbers in
|
||||
# table_name = "default"
|
||||
# ## Interval for persisting data to limit write operations
|
||||
# # interval = "10s"
|
||||
```
|
||||
|
||||
### Required AWS IAM permissions
|
||||
|
||||
Kinesis:
|
||||
|
||||
- DescribeStream
|
||||
- GetRecords
|
||||
- GetShardIterator
|
||||
|
||||
DynamoDB:
|
||||
|
||||
- GetItem
|
||||
- PutItem
|
||||
|
||||
### DynamoDB Checkpoint
|
||||
|
||||
The DynamoDB checkpoint stores the last processed record in a DynamoDB. To
|
||||
leverage this functionality, create a table with the following string type keys:
|
||||
|
||||
```shell
|
||||
Partition key: namespace
|
||||
Sort key: shard_id
|
||||
```
|
||||
|
||||
## Metrics
|
||||
|
||||
The plugin accepts arbitrary input and parses it according to the `data_format`
|
||||
setting. There is no predefined metric format.
|
||||
|
||||
## Example Output
|
||||
|
||||
There is no predefined metric format, so output depends on plugin input.
|
355
plugins/inputs/kinesis_consumer/consumer.go
Normal file
355
plugins/inputs/kinesis_consumer/consumer.go
Normal file
|
@ -0,0 +1,355 @@
|
|||
package kinesis_consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/aws"
|
||||
"github.com/aws/aws-sdk-go-v2/service/kinesis"
|
||||
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
)
|
||||
|
||||
type recordHandler func(ctx context.Context, shard string, r *types.Record)
|
||||
|
||||
type shardConsumer struct {
|
||||
seqnr string
|
||||
interval time.Duration
|
||||
log telegraf.Logger
|
||||
|
||||
client *kinesis.Client
|
||||
params *kinesis.GetShardIteratorInput
|
||||
|
||||
onMessage recordHandler
|
||||
}
|
||||
|
||||
func (c *shardConsumer) consume(ctx context.Context, shard string) ([]types.ChildShard, error) {
|
||||
ticker := time.NewTicker(c.interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
// Get the first shard iterator
|
||||
iter, err := c.iterator(ctx)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("getting first shard iterator failed: %w", err)
|
||||
}
|
||||
|
||||
for {
|
||||
// Get new records from the shard
|
||||
resp, err := c.client.GetRecords(ctx, &kinesis.GetRecordsInput{
|
||||
ShardIterator: iter,
|
||||
})
|
||||
if err != nil {
|
||||
// Handle recoverable errors
|
||||
var throughputErr *types.ProvisionedThroughputExceededException
|
||||
var expiredIterErr *types.ExpiredIteratorException
|
||||
switch {
|
||||
case errors.As(err, &throughputErr):
|
||||
// Wait a second before trying again as suggested by
|
||||
// https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html
|
||||
c.log.Tracef("throughput exceeded when getting records for shard %s...", shard)
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
case errors.As(err, &expiredIterErr):
|
||||
c.log.Tracef("iterator expired for shard %s...", shard)
|
||||
if iter, err = c.iterator(ctx); err != nil {
|
||||
return nil, fmt.Errorf("getting shard iterator failed: %w", err)
|
||||
}
|
||||
continue
|
||||
case errors.Is(err, context.Canceled):
|
||||
return nil, nil
|
||||
default:
|
||||
c.log.Tracef("get-records error is of type %T", err)
|
||||
return nil, fmt.Errorf("getting records failed: %w", err)
|
||||
}
|
||||
}
|
||||
c.log.Tracef("read %d records for shard %s...", len(resp.Records), shard)
|
||||
|
||||
// Check if we fully read the shard
|
||||
if resp.NextShardIterator == nil {
|
||||
return resp.ChildShards, nil
|
||||
}
|
||||
iter = resp.NextShardIterator
|
||||
|
||||
// Process the records and keep track of the last sequence number
|
||||
// consumed for recreating the iterator.
|
||||
for _, r := range resp.Records {
|
||||
c.onMessage(ctx, shard, &r)
|
||||
c.seqnr = *r.SequenceNumber
|
||||
if errors.Is(ctx.Err(), context.Canceled) {
|
||||
return nil, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Wait for the poll interval to pass or cancel
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, nil
|
||||
case <-ticker.C:
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *shardConsumer) iterator(ctx context.Context) (*string, error) {
|
||||
for {
|
||||
resp, err := c.client.GetShardIterator(ctx, c.params)
|
||||
if err != nil {
|
||||
var throughputErr *types.ProvisionedThroughputExceededException
|
||||
if errors.As(err, &throughputErr) {
|
||||
// We called the function too often and should wait a bit
|
||||
// until trying again
|
||||
c.log.Tracef("throughput exceeded when getting iterator for shard %s...", *c.params.ShardId)
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
|
||||
return nil, err
|
||||
}
|
||||
c.log.Tracef("successfully updated iterator for shard %s (%s)...", *c.params.ShardId, c.seqnr)
|
||||
return resp.ShardIterator, nil
|
||||
}
|
||||
}
|
||||
|
||||
type consumer struct {
|
||||
config aws.Config
|
||||
stream string
|
||||
iterType types.ShardIteratorType
|
||||
pollInterval time.Duration
|
||||
shardUpdateInterval time.Duration
|
||||
log telegraf.Logger
|
||||
|
||||
onMessage recordHandler
|
||||
position func(shard string) string
|
||||
|
||||
client *kinesis.Client
|
||||
|
||||
shardsConsumed map[string]bool
|
||||
shardConsumers map[string]*shardConsumer
|
||||
|
||||
wg sync.WaitGroup
|
||||
|
||||
sync.Mutex
|
||||
}
|
||||
|
||||
func (c *consumer) init() error {
|
||||
if c.stream == "" {
|
||||
return errors.New("stream cannot be empty")
|
||||
}
|
||||
if c.pollInterval <= 0 {
|
||||
return errors.New("invalid poll interval")
|
||||
}
|
||||
|
||||
if c.onMessage == nil {
|
||||
return errors.New("message handler is undefined")
|
||||
}
|
||||
|
||||
c.shardsConsumed = make(map[string]bool)
|
||||
c.shardConsumers = make(map[string]*shardConsumer)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *consumer) start(ctx context.Context) {
|
||||
// Setup the client
|
||||
c.client = kinesis.NewFromConfig(c.config)
|
||||
|
||||
// Do the initial discovery of shards
|
||||
if err := c.updateShardConsumers(ctx); err != nil {
|
||||
c.log.Errorf("Initializing shards failed: %v", err)
|
||||
}
|
||||
|
||||
// If the consumer has a shard-update interval, use a ticker to update
|
||||
// available shards on a regular basis
|
||||
if c.shardUpdateInterval <= 0 {
|
||||
return
|
||||
}
|
||||
ticker := time.NewTicker(c.shardUpdateInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
if err := c.updateShardConsumers(ctx); err != nil {
|
||||
c.log.Errorf("Updating shards failed: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *consumer) updateShardConsumers(ctx context.Context) error {
|
||||
// List all shards of the given stream
|
||||
var availableShards []types.Shard
|
||||
req := &kinesis.ListShardsInput{StreamName: aws.String(c.stream)}
|
||||
for {
|
||||
resp, err := c.client.ListShards(ctx, req)
|
||||
if err != nil {
|
||||
return fmt.Errorf("listing shards failed: %w", err)
|
||||
}
|
||||
availableShards = append(availableShards, resp.Shards...)
|
||||
|
||||
if resp.NextToken == nil {
|
||||
break
|
||||
}
|
||||
|
||||
req = &kinesis.ListShardsInput{NextToken: resp.NextToken}
|
||||
}
|
||||
c.log.Tracef("got %d shards during update", len(availableShards))
|
||||
|
||||
// All following operations need to be locked to create a consistent
|
||||
// state of the shards and consumers
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
// Filter out all shards actively consumed already
|
||||
inactiveShards := make([]types.Shard, 0, len(availableShards))
|
||||
for _, shard := range availableShards {
|
||||
id := *shard.ShardId
|
||||
if _, found := c.shardConsumers[id]; found {
|
||||
c.log.Tracef("shard %s is actively consumed...", id)
|
||||
continue
|
||||
}
|
||||
c.log.Tracef("shard %s is not actively consumed...", id)
|
||||
inactiveShards = append(inactiveShards, shard)
|
||||
}
|
||||
|
||||
// Fill the shards already consumed and get the positions if the consumer
|
||||
// is backed by an iterator store
|
||||
newShards := make([]types.Shard, 0, len(inactiveShards))
|
||||
seqnrs := make(map[string]string, len(inactiveShards))
|
||||
for _, shard := range inactiveShards {
|
||||
id := *shard.ShardId
|
||||
|
||||
if c.shardsConsumed[id] {
|
||||
c.log.Tracef("shard %s is already fully consumed...", id)
|
||||
continue
|
||||
}
|
||||
c.log.Tracef("shard %s is not fully consumed...", id)
|
||||
|
||||
// Retrieve the shard position from the store
|
||||
if c.position != nil {
|
||||
seqnr := c.position(id)
|
||||
if seqnr == "" {
|
||||
// A truely new shard
|
||||
newShards = append(newShards, shard)
|
||||
c.log.Tracef("shard %s is new...", id)
|
||||
continue
|
||||
}
|
||||
seqnrs[id] = seqnr
|
||||
|
||||
// Check if we already fully consumed for closed shards
|
||||
end := shard.SequenceNumberRange.EndingSequenceNumber
|
||||
if end != nil && *end == seqnr {
|
||||
c.log.Tracef("shard %s is closed and already fully consumed...", id)
|
||||
c.shardsConsumed[id] = true
|
||||
continue
|
||||
}
|
||||
c.log.Tracef("shard %s is not yet fully consumed...", id)
|
||||
}
|
||||
|
||||
// The shard is not fully consumed yet so save the sequence number
|
||||
// and the shard as "new".
|
||||
newShards = append(newShards, shard)
|
||||
}
|
||||
|
||||
// Filter all shards already fully consumed and create a new consumer for
|
||||
// every remaining new shard respecting resharding artifacts
|
||||
for _, shard := range newShards {
|
||||
id := *shard.ShardId
|
||||
|
||||
// Handle resharding by making sure all parents are consumed already
|
||||
// before starting a consumer on a child shard. If parents are not
|
||||
// consumed fully we ignore this shard here as it will be reported
|
||||
// by the call to `GetRecords` as a child later.
|
||||
if shard.ParentShardId != nil && *shard.ParentShardId != "" {
|
||||
pid := *shard.ParentShardId
|
||||
if !c.shardsConsumed[pid] {
|
||||
c.log.Tracef("shard %s has parent %s which is not fully consumed yet...", id, pid)
|
||||
continue
|
||||
}
|
||||
}
|
||||
if shard.AdjacentParentShardId != nil && *shard.AdjacentParentShardId != "" {
|
||||
pid := *shard.AdjacentParentShardId
|
||||
if !c.shardsConsumed[pid] {
|
||||
c.log.Tracef("shard %s has adjacent parent %s which is not fully consumed yet...", id, pid)
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// Create a new consumer and start it
|
||||
c.wg.Add(1)
|
||||
go func(shardID string) {
|
||||
defer c.wg.Done()
|
||||
c.startShardConsumer(ctx, shardID, seqnrs[shardID])
|
||||
}(id)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *consumer) startShardConsumer(ctx context.Context, id, seqnr string) {
|
||||
c.log.Tracef("starting consumer for shard %s at sequence number %q...", id, seqnr)
|
||||
sc := &shardConsumer{
|
||||
seqnr: seqnr,
|
||||
interval: c.pollInterval,
|
||||
log: c.log,
|
||||
onMessage: c.onMessage,
|
||||
client: c.client,
|
||||
params: &kinesis.GetShardIteratorInput{
|
||||
ShardId: &id,
|
||||
ShardIteratorType: c.iterType,
|
||||
StreamName: &c.stream,
|
||||
},
|
||||
}
|
||||
if seqnr != "" {
|
||||
sc.params.ShardIteratorType = types.ShardIteratorTypeAfterSequenceNumber
|
||||
sc.params.StartingSequenceNumber = &seqnr
|
||||
}
|
||||
c.shardConsumers[id] = sc
|
||||
|
||||
childs, err := sc.consume(ctx, id)
|
||||
if err != nil {
|
||||
c.log.Errorf("Consuming shard %s failed: %v", id, err)
|
||||
return
|
||||
}
|
||||
c.log.Tracef("finished consuming shard %s", id)
|
||||
|
||||
c.Lock()
|
||||
defer c.Unlock()
|
||||
|
||||
c.shardsConsumed[id] = true
|
||||
delete(c.shardConsumers, id)
|
||||
|
||||
for _, shard := range childs {
|
||||
cid := *shard.ShardId
|
||||
|
||||
startable := true
|
||||
for _, pid := range shard.ParentShards {
|
||||
startable = startable && c.shardsConsumed[pid]
|
||||
}
|
||||
if !startable {
|
||||
c.log.Tracef("child shard %s of shard %s is not startable as parents are fully consumed yet...", cid, id)
|
||||
continue
|
||||
}
|
||||
c.log.Tracef("child shard %s of shard %s is startable...", cid, id)
|
||||
|
||||
var cseqnr string
|
||||
if c.position != nil {
|
||||
cseqnr = c.position(cid)
|
||||
}
|
||||
c.wg.Add(1)
|
||||
go func() {
|
||||
defer c.wg.Done()
|
||||
c.startShardConsumer(ctx, cid, cseqnr)
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
func (c *consumer) stop() {
|
||||
c.wg.Wait()
|
||||
}
|
45
plugins/inputs/kinesis_consumer/encoding.go
Normal file
45
plugins/inputs/kinesis_consumer/encoding.go
Normal file
|
@ -0,0 +1,45 @@
|
|||
package kinesis_consumer
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"compress/gzip"
|
||||
"compress/zlib"
|
||||
"fmt"
|
||||
"io"
|
||||
)
|
||||
|
||||
type decodingFunc func([]byte) ([]byte, error)
|
||||
|
||||
func processGzip(data []byte) ([]byte, error) {
|
||||
zipData, err := gzip.NewReader(bytes.NewReader(data))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer zipData.Close()
|
||||
return io.ReadAll(zipData)
|
||||
}
|
||||
|
||||
func processZlib(data []byte) ([]byte, error) {
|
||||
zlibData, err := zlib.NewReader(bytes.NewReader(data))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer zlibData.Close()
|
||||
return io.ReadAll(zlibData)
|
||||
}
|
||||
|
||||
func processNoOp(data []byte) ([]byte, error) {
|
||||
return data, nil
|
||||
}
|
||||
|
||||
func getDecodingFunc(encoding string) (decodingFunc, error) {
|
||||
switch encoding {
|
||||
case "gzip":
|
||||
return processGzip, nil
|
||||
case "zlib":
|
||||
return processZlib, nil
|
||||
case "none", "identity", "":
|
||||
return processNoOp, nil
|
||||
}
|
||||
return nil, fmt.Errorf("unknown content encoding %q", encoding)
|
||||
}
|
278
plugins/inputs/kinesis_consumer/kinesis_consumer.go
Normal file
278
plugins/inputs/kinesis_consumer/kinesis_consumer.go
Normal file
|
@ -0,0 +1,278 @@
|
|||
//go:generate ../../../tools/readme_config_includer/generator
|
||||
package kinesis_consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
_ "embed"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/aws"
|
||||
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/config"
|
||||
"github.com/influxdata/telegraf/internal"
|
||||
common_aws "github.com/influxdata/telegraf/plugins/common/aws"
|
||||
"github.com/influxdata/telegraf/plugins/inputs"
|
||||
)
|
||||
|
||||
//go:embed sample.conf
|
||||
var sampleConfig string
|
||||
|
||||
var once sync.Once
|
||||
|
||||
type KinesisConsumer struct {
|
||||
StreamName string `toml:"streamname"`
|
||||
ShardIteratorType string `toml:"shard_iterator_type"`
|
||||
PollInterval config.Duration `toml:"poll_interval"`
|
||||
ShardUpdateInterval config.Duration `toml:"shard_update_interval"`
|
||||
DynamoDB *dynamoDB `toml:"checkpoint_dynamodb"`
|
||||
MaxUndeliveredMessages int `toml:"max_undelivered_messages"`
|
||||
ContentEncoding string `toml:"content_encoding"`
|
||||
Log telegraf.Logger `toml:"-"`
|
||||
common_aws.CredentialConfig
|
||||
|
||||
acc telegraf.TrackingAccumulator
|
||||
parser telegraf.Parser
|
||||
|
||||
cfg aws.Config
|
||||
consumer *consumer
|
||||
cancel context.CancelFunc
|
||||
sem chan struct{}
|
||||
|
||||
iteratorStore *store
|
||||
|
||||
records map[telegraf.TrackingID]iterator
|
||||
recordsTex sync.Mutex
|
||||
|
||||
wg sync.WaitGroup
|
||||
|
||||
contentDecodingFunc decodingFunc
|
||||
}
|
||||
|
||||
type dynamoDB struct {
|
||||
AppName string `toml:"app_name"`
|
||||
TableName string `toml:"table_name"`
|
||||
Interval config.Duration `toml:"interval"`
|
||||
}
|
||||
|
||||
func (*KinesisConsumer) SampleConfig() string {
|
||||
return sampleConfig
|
||||
}
|
||||
|
||||
func (k *KinesisConsumer) SetParser(parser telegraf.Parser) {
|
||||
k.parser = parser
|
||||
}
|
||||
|
||||
func (k *KinesisConsumer) Init() error {
|
||||
// Set defaults
|
||||
if k.MaxUndeliveredMessages < 1 {
|
||||
k.MaxUndeliveredMessages = 1000
|
||||
}
|
||||
|
||||
if k.ShardIteratorType == "" {
|
||||
k.ShardIteratorType = "TRIM_HORIZON"
|
||||
}
|
||||
if k.ContentEncoding == "" {
|
||||
k.ContentEncoding = "identity"
|
||||
}
|
||||
|
||||
// Check input params
|
||||
if k.StreamName == "" {
|
||||
return errors.New("stream name cannot be empty")
|
||||
}
|
||||
|
||||
f, err := getDecodingFunc(k.ContentEncoding)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
k.contentDecodingFunc = f
|
||||
|
||||
if k.DynamoDB != nil {
|
||||
if k.DynamoDB.Interval <= 0 {
|
||||
k.DynamoDB.Interval = config.Duration(10 * time.Second)
|
||||
}
|
||||
k.iteratorStore = newStore(k.DynamoDB.AppName, k.DynamoDB.TableName, time.Duration(k.DynamoDB.Interval), k.Log)
|
||||
}
|
||||
|
||||
k.records = make(map[telegraf.TrackingID]iterator, k.MaxUndeliveredMessages)
|
||||
k.sem = make(chan struct{}, k.MaxUndeliveredMessages)
|
||||
|
||||
// Setup the client to connect to the Kinesis service
|
||||
cfg, err := k.CredentialConfig.Credentials()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if k.EndpointURL != "" {
|
||||
cfg.BaseEndpoint = &k.EndpointURL
|
||||
}
|
||||
if k.Log.Level().Includes(telegraf.Trace) {
|
||||
logWrapper := &telegrafLoggerWrapper{k.Log}
|
||||
cfg.Logger = logWrapper
|
||||
cfg.ClientLogMode = aws.LogRetries
|
||||
}
|
||||
k.cfg = cfg
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (k *KinesisConsumer) Start(acc telegraf.Accumulator) error {
|
||||
k.acc = acc.WithTracking(k.MaxUndeliveredMessages)
|
||||
|
||||
// Start the store if necessary
|
||||
if k.iteratorStore != nil {
|
||||
if err := k.iteratorStore.run(context.Background()); err != nil {
|
||||
return fmt.Errorf("starting DynamoDB store failed: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
ctx := context.Background()
|
||||
ctx, k.cancel = context.WithCancel(ctx)
|
||||
|
||||
// Setup the consumer
|
||||
k.consumer = &consumer{
|
||||
config: k.cfg,
|
||||
stream: k.StreamName,
|
||||
iterType: types.ShardIteratorType(k.ShardIteratorType),
|
||||
pollInterval: time.Duration(k.PollInterval),
|
||||
shardUpdateInterval: time.Duration(k.ShardUpdateInterval),
|
||||
log: k.Log,
|
||||
onMessage: func(ctx context.Context, shard string, r *types.Record) {
|
||||
// Checking for number of messages in flight and wait for a free
|
||||
// slot in case there are too many
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case k.sem <- struct{}{}:
|
||||
break
|
||||
}
|
||||
|
||||
if err := k.onMessage(k.acc, shard, r); err != nil {
|
||||
seqnr := *r.SequenceNumber
|
||||
k.Log.Errorf("Processing message with sequence number %q in shard %s failed: %v", seqnr, shard, err)
|
||||
<-k.sem
|
||||
}
|
||||
},
|
||||
}
|
||||
|
||||
// Link in the backing iterator store
|
||||
if k.iteratorStore != nil {
|
||||
k.consumer.position = func(shard string) string {
|
||||
seqnr, err := k.iteratorStore.get(ctx, k.StreamName, shard)
|
||||
if err != nil && !errors.Is(err, errNotFound) {
|
||||
k.Log.Errorf("retrieving sequence number for shard %q failed: %s", shard, err)
|
||||
}
|
||||
|
||||
return seqnr
|
||||
}
|
||||
}
|
||||
if err := k.consumer.init(); err != nil {
|
||||
return fmt.Errorf("initializing consumer failed: %w", err)
|
||||
}
|
||||
|
||||
// Start the go-routine handling metrics delivered to the output
|
||||
k.wg.Add(1)
|
||||
go func() {
|
||||
defer k.wg.Done()
|
||||
k.onDelivery(ctx)
|
||||
}()
|
||||
|
||||
// Start the go-routine handling message consumption
|
||||
k.wg.Add(1)
|
||||
go func() {
|
||||
defer k.wg.Done()
|
||||
k.consumer.start(ctx)
|
||||
}()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (*KinesisConsumer) Gather(telegraf.Accumulator) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (k *KinesisConsumer) Stop() {
|
||||
k.cancel()
|
||||
k.wg.Wait()
|
||||
k.consumer.stop()
|
||||
|
||||
if k.iteratorStore != nil {
|
||||
k.iteratorStore.stop()
|
||||
}
|
||||
}
|
||||
|
||||
// onMessage is called for new messages consumed from Kinesis
|
||||
func (k *KinesisConsumer) onMessage(acc telegraf.TrackingAccumulator, shard string, r *types.Record) error {
|
||||
data, err := k.contentDecodingFunc(r.Data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
metrics, err := k.parser.Parse(data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(metrics) == 0 {
|
||||
once.Do(func() {
|
||||
k.Log.Debug(internal.NoMetricsCreatedMsg)
|
||||
})
|
||||
}
|
||||
|
||||
seqnr := *r.SequenceNumber
|
||||
|
||||
k.recordsTex.Lock()
|
||||
defer k.recordsTex.Unlock()
|
||||
|
||||
id := acc.AddTrackingMetricGroup(metrics)
|
||||
k.records[id] = iterator{shard: shard, seqnr: seqnr}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// onDelivery is called for every metric successfully delivered to the outputs
|
||||
func (k *KinesisConsumer) onDelivery(ctx context.Context) {
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case info := <-k.acc.Delivered():
|
||||
// Store the metric iterator in DynamoDB if configured
|
||||
if k.iteratorStore != nil {
|
||||
k.storeDelivered(info.ID())
|
||||
}
|
||||
|
||||
// Reduce the number of undelivered messages by reading from the channel
|
||||
<-k.sem
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (k *KinesisConsumer) storeDelivered(id telegraf.TrackingID) {
|
||||
k.recordsTex.Lock()
|
||||
defer k.recordsTex.Unlock()
|
||||
|
||||
// Find the iterator belonging to the delivered message
|
||||
iter, ok := k.records[id]
|
||||
if !ok {
|
||||
k.Log.Debugf("No iterator found for delivered metric %v!", id)
|
||||
return
|
||||
}
|
||||
|
||||
// Remove metric
|
||||
delete(k.records, id)
|
||||
|
||||
// Store the iterator in the database
|
||||
k.iteratorStore.set(k.StreamName, iter.shard, iter.seqnr)
|
||||
}
|
||||
|
||||
func init() {
|
||||
inputs.Add("kinesis_consumer", func() telegraf.Input {
|
||||
return &KinesisConsumer{
|
||||
PollInterval: config.Duration(250 * time.Millisecond),
|
||||
ShardUpdateInterval: config.Duration(30 * time.Second),
|
||||
}
|
||||
})
|
||||
}
|
164
plugins/inputs/kinesis_consumer/kinesis_consumer_test.go
Normal file
164
plugins/inputs/kinesis_consumer/kinesis_consumer_test.go
Normal file
|
@ -0,0 +1,164 @@
|
|||
package kinesis_consumer
|
||||
|
||||
import (
|
||||
"encoding/base64"
|
||||
"testing"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/aws"
|
||||
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
"github.com/influxdata/telegraf/plugins/parsers/json"
|
||||
"github.com/influxdata/telegraf/testutil"
|
||||
)
|
||||
|
||||
func TestInvalidCoding(t *testing.T) {
|
||||
plugin := &KinesisConsumer{
|
||||
StreamName: "foo",
|
||||
ContentEncoding: "notsupported",
|
||||
}
|
||||
require.ErrorContains(t, plugin.Init(), "unknown content encoding")
|
||||
}
|
||||
|
||||
func TestOnMessage(t *testing.T) {
|
||||
// Prepare messages
|
||||
zlibBytpes, err := base64.StdEncoding.DecodeString(
|
||||
"eF5FjlFrgzAUhf9KuM+2aNB2zdsQ2xe3whQGW8qIeqdhaiSJK0P874u1Y4+Hc/jON0GHxoga858BgUF8fs5fzunHU5Jlj6cEPFDXHvXStGqsrsKWTapq44pW1SetxsF1a8qsRtGt0Yy" +
|
||||
"FKbUcrFT9UbYWtQH2frntkm/s7RInkNU6t9JpWNE5WBAFPo3CcHeg+9D703OziUOhCg6MQ/yakrspuZsyEjdYfsm+Jg2K1jZEfZLKQWUvFglylBobZXDLwSP8//EGpD4NNj7dUJpT6" +
|
||||
"hQY3W33h/AhCt84zDBf5l/MDl08",
|
||||
)
|
||||
require.NoError(t, err)
|
||||
|
||||
gzippedBytes, err := base64.StdEncoding.DecodeString(
|
||||
"H4sIAAFXNGAAA0WOUWuDMBSF/0q4z7Zo0HbN2xDbF7fCFAZbyoh6p2FqJIkrQ/zvi7Vjj4dz+M43QYfGiBrznwGBQXx+zl/O6cdTkmWPpwQ8UNce9dK0aqyuwpZNqmrjilbVJ63GwXVr" +
|
||||
"yqxG0a3RjIUptRysVP1Rtha1AfZ+ue2Sb+ztEieQ1Tq30mlY0TlYEAU+jcJwd6D70PvTc7OJQ6EKDoxD/JqSuym5mzISN1h+yb4mDYrWNkR9kspBZS8WCXKUGhtlcMvBI/z/8QakPg02" +
|
||||
"Pt1QmlPqFBjdbfeH8CEK3zjMMF/mX0TaxZUpAQAA",
|
||||
)
|
||||
require.NoError(t, err)
|
||||
|
||||
notZippedBytes := []byte(`
|
||||
{
|
||||
"messageType": "CONTROL_MESSAGE",
|
||||
"owner": "CloudwatchLogs",
|
||||
"logGroup": "",
|
||||
"logStream": "",
|
||||
"subscriptionFilters": [],
|
||||
"logEvents": [
|
||||
{
|
||||
"id": "",
|
||||
"timestamp": 1510254469274,
|
||||
"message": "{\"bob\":\"CWL CONTROL MESSAGE: Checking health of destination Firehose.\", \"timestamp\":\"2021-02-22T22:15:26.794854Z\"},"
|
||||
},
|
||||
{
|
||||
"id": "",
|
||||
"timestamp": 1510254469274,
|
||||
"message": "{\"bob\":\"CWL CONTROL MESSAGE: Checking health of destination Firehose.\", \"timestamp\":\"2021-02-22T22:15:26.794854Z\"}"
|
||||
}
|
||||
]
|
||||
}
|
||||
`)
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
encoding string
|
||||
record *types.Record
|
||||
expectedNumber int
|
||||
expectedContent string
|
||||
}{
|
||||
{
|
||||
name: "test no compression",
|
||||
encoding: "none",
|
||||
record: &types.Record{
|
||||
Data: notZippedBytes,
|
||||
SequenceNumber: aws.String("anything"),
|
||||
},
|
||||
expectedNumber: 2,
|
||||
expectedContent: "bob",
|
||||
},
|
||||
{
|
||||
name: "test no compression via empty string for ContentEncoding",
|
||||
record: &types.Record{
|
||||
Data: notZippedBytes,
|
||||
SequenceNumber: aws.String("anything"),
|
||||
},
|
||||
expectedNumber: 2,
|
||||
expectedContent: "bob",
|
||||
},
|
||||
{
|
||||
name: "test no compression via identity ContentEncoding",
|
||||
encoding: "identity",
|
||||
record: &types.Record{
|
||||
Data: notZippedBytes,
|
||||
SequenceNumber: aws.String("anything"),
|
||||
},
|
||||
expectedNumber: 2,
|
||||
expectedContent: "bob",
|
||||
},
|
||||
{
|
||||
name: "test no compression via no ContentEncoding",
|
||||
record: &types.Record{
|
||||
Data: notZippedBytes,
|
||||
SequenceNumber: aws.String("anything"),
|
||||
},
|
||||
expectedNumber: 2,
|
||||
expectedContent: "bob",
|
||||
},
|
||||
{
|
||||
name: "test gzip compression",
|
||||
encoding: "gzip",
|
||||
record: &types.Record{
|
||||
Data: gzippedBytes,
|
||||
SequenceNumber: aws.String("anything"),
|
||||
},
|
||||
expectedNumber: 1,
|
||||
expectedContent: "bob",
|
||||
},
|
||||
{
|
||||
name: "test zlib compression",
|
||||
encoding: "zlib",
|
||||
record: &types.Record{
|
||||
Data: zlibBytpes,
|
||||
SequenceNumber: aws.String("anything"),
|
||||
},
|
||||
expectedNumber: 1,
|
||||
expectedContent: "bob",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
// Prepare JSON parser
|
||||
parser := &json.Parser{
|
||||
MetricName: "json_test",
|
||||
Query: "logEvents",
|
||||
StringFields: []string{"message"},
|
||||
}
|
||||
require.NoError(t, parser.Init())
|
||||
|
||||
// Setup plugin
|
||||
plugin := &KinesisConsumer{
|
||||
StreamName: "foo",
|
||||
ContentEncoding: tt.encoding,
|
||||
Log: &testutil.Logger{},
|
||||
parser: parser,
|
||||
records: make(map[telegraf.TrackingID]iterator),
|
||||
}
|
||||
require.NoError(t, plugin.Init())
|
||||
|
||||
var acc testutil.Accumulator
|
||||
require.NoError(t, plugin.onMessage(acc.WithTracking(tt.expectedNumber), "test", tt.record))
|
||||
|
||||
actual := acc.GetTelegrafMetrics()
|
||||
require.Len(t, actual, tt.expectedNumber)
|
||||
|
||||
for _, metric := range actual {
|
||||
raw, found := metric.GetField("message")
|
||||
require.True(t, found, "no message present")
|
||||
message, ok := raw.(string)
|
||||
require.Truef(t, ok, "message not a string but %T", raw)
|
||||
require.Contains(t, message, tt.expectedContent)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
29
plugins/inputs/kinesis_consumer/logging.go
Normal file
29
plugins/inputs/kinesis_consumer/logging.go
Normal file
|
@ -0,0 +1,29 @@
|
|||
package kinesis_consumer
|
||||
|
||||
import (
|
||||
"github.com/aws/smithy-go/logging"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
)
|
||||
|
||||
type telegrafLoggerWrapper struct {
|
||||
telegraf.Logger
|
||||
}
|
||||
|
||||
// Log logs messages at the trace level.
|
||||
func (t *telegrafLoggerWrapper) Log(args ...interface{}) {
|
||||
t.Trace(args...)
|
||||
}
|
||||
|
||||
// Logf logs formatted messages with a specific classification.
|
||||
func (t *telegrafLoggerWrapper) Logf(classification logging.Classification, format string, v ...interface{}) {
|
||||
switch classification {
|
||||
case logging.Debug:
|
||||
format = "DEBUG " + format
|
||||
case logging.Warn:
|
||||
format = "WARN" + format
|
||||
default:
|
||||
format = "INFO " + format
|
||||
}
|
||||
t.Logger.Tracef(format, v...)
|
||||
}
|
74
plugins/inputs/kinesis_consumer/sample.conf
Normal file
74
plugins/inputs/kinesis_consumer/sample.conf
Normal file
|
@ -0,0 +1,74 @@
|
|||
# Configuration for the AWS Kinesis input.
|
||||
[[inputs.kinesis_consumer]]
|
||||
## Amazon REGION of kinesis endpoint.
|
||||
region = "ap-southeast-2"
|
||||
|
||||
## Amazon Credentials
|
||||
## Credentials are loaded in the following order
|
||||
## 1) Web identity provider credentials via STS if role_arn and web_identity_token_file are specified
|
||||
## 2) Assumed credentials via STS if role_arn is specified
|
||||
## 3) explicit credentials from 'access_key' and 'secret_key'
|
||||
## 4) shared profile from 'profile'
|
||||
## 5) environment variables
|
||||
## 6) shared credentials file
|
||||
## 7) EC2 Instance Profile
|
||||
# access_key = ""
|
||||
# secret_key = ""
|
||||
# token = ""
|
||||
# role_arn = ""
|
||||
# web_identity_token_file = ""
|
||||
# role_session_name = ""
|
||||
# profile = ""
|
||||
# shared_credential_file = ""
|
||||
|
||||
## Endpoint to make request against, the correct endpoint is automatically
|
||||
## determined and this option should only be set if you wish to override the
|
||||
## default.
|
||||
## ex: endpoint_url = "http://localhost:8000"
|
||||
# endpoint_url = ""
|
||||
|
||||
## Kinesis StreamName must exist prior to starting telegraf.
|
||||
streamname = "StreamName"
|
||||
|
||||
## Shard iterator type
|
||||
## Available options: 'TRIM_HORIZON' (first in non-expired) and 'LATEST'
|
||||
# shard_iterator_type = "TRIM_HORIZON"
|
||||
|
||||
## Interval for checking for new records
|
||||
## Please consider limits for getting records documented here:
|
||||
## https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html
|
||||
# poll_interval = "250ms"
|
||||
|
||||
## Interval for scanning for new shards created when resharding
|
||||
## If set to zero, shards are only scanned once on startup.
|
||||
# shard_update_interval = "30s"
|
||||
|
||||
## Max undelivered messages
|
||||
## This plugin uses tracking metrics, which ensure messages are read to
|
||||
## outputs before acknowledging them to the original broker to ensure data
|
||||
## is not lost. This option sets the maximum messages to read from the
|
||||
## broker that have not been written by an output.
|
||||
##
|
||||
## This value needs to be picked with awareness of the agent's
|
||||
## metric_batch_size value as well. Setting max undelivered messages too high
|
||||
## can result in a constant stream of data batches to the output. While
|
||||
## setting it too low may never flush the broker's messages.
|
||||
# max_undelivered_messages = 1000
|
||||
|
||||
## Content encoding of the record data
|
||||
## If you are processing a cloudwatch logs kinesis stream then set this to
|
||||
## "gzip" as AWS compresses cloudwatch log data before it is sent to kinesis.
|
||||
# content_encoding = "identity"
|
||||
|
||||
## Data format of the records to consume
|
||||
## See https://github.com/influxdata/telegraf/blob/master/docs/DATA_FORMATS_INPUT.md
|
||||
# data_format = "influx"
|
||||
|
||||
## Optional: Configuration for DynamoDB backend to store positions in the stream
|
||||
# [inputs.kinesis_consumer.checkpoint_dynamodb]
|
||||
# ## Unique name for this consumer
|
||||
# app_name = "default"
|
||||
# ## Table to store the sequence numbers in
|
||||
# table_name = "default"
|
||||
# ## Interval for persisting data to limit write operations
|
||||
# # interval = "10s"
|
179
plugins/inputs/kinesis_consumer/store.go
Normal file
179
plugins/inputs/kinesis_consumer/store.go
Normal file
|
@ -0,0 +1,179 @@
|
|||
package kinesis_consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/aws"
|
||||
"github.com/aws/aws-sdk-go-v2/config"
|
||||
"github.com/aws/aws-sdk-go-v2/service/dynamodb"
|
||||
"github.com/aws/aws-sdk-go-v2/service/dynamodb/types"
|
||||
|
||||
"github.com/influxdata/telegraf"
|
||||
)
|
||||
|
||||
var errNotFound = errors.New("no iterator found")
|
||||
|
||||
type iterator struct {
|
||||
stream string
|
||||
shard string
|
||||
seqnr string
|
||||
modified bool
|
||||
}
|
||||
|
||||
type store struct {
|
||||
app string
|
||||
table string
|
||||
interval time.Duration
|
||||
log telegraf.Logger
|
||||
|
||||
client *dynamodb.Client
|
||||
iterators map[string]iterator
|
||||
|
||||
wg sync.WaitGroup
|
||||
cancel context.CancelFunc
|
||||
|
||||
sync.Mutex
|
||||
}
|
||||
|
||||
func newStore(app, table string, interval time.Duration, log telegraf.Logger) *store {
|
||||
s := &store{
|
||||
app: app,
|
||||
table: table,
|
||||
interval: interval,
|
||||
log: log,
|
||||
}
|
||||
|
||||
// Initialize the iterator states
|
||||
s.iterators = make(map[string]iterator)
|
||||
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *store) run(ctx context.Context) error {
|
||||
rctx, cancel := context.WithCancel(ctx)
|
||||
s.cancel = cancel
|
||||
|
||||
// Create a client to connect to DynamoDB
|
||||
cfg, err := config.LoadDefaultConfig(rctx)
|
||||
if err != nil {
|
||||
return fmt.Errorf("loading default config failed: %w", err)
|
||||
}
|
||||
s.client = dynamodb.NewFromConfig(cfg)
|
||||
|
||||
// Start the go-routine that pushes the states out to DynamoDB on a
|
||||
// regular interval
|
||||
s.wg.Add(1)
|
||||
go func() {
|
||||
defer s.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(s.interval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-rctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
s.write(rctx)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *store) stop() {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), s.interval)
|
||||
defer cancel()
|
||||
s.write(ctx)
|
||||
|
||||
s.cancel()
|
||||
s.wg.Wait()
|
||||
}
|
||||
|
||||
func (s *store) write(ctx context.Context) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
for k, iter := range s.iterators {
|
||||
// Only write iterators modified since the last write
|
||||
if !iter.modified {
|
||||
continue
|
||||
}
|
||||
|
||||
if _, err := s.client.PutItem(
|
||||
ctx,
|
||||
&dynamodb.PutItemInput{
|
||||
TableName: aws.String(s.table),
|
||||
Item: map[string]types.AttributeValue{
|
||||
"namespace": &types.AttributeValueMemberS{Value: s.app + "-" + iter.stream},
|
||||
"shard_id": &types.AttributeValueMemberS{Value: iter.shard},
|
||||
"sequence_number": &types.AttributeValueMemberS{Value: iter.seqnr},
|
||||
},
|
||||
}); err != nil {
|
||||
s.log.Errorf("storing iterator %s-%s/%s/%s failed: %v", s.app, iter.stream, iter.shard, iter.seqnr, err)
|
||||
}
|
||||
|
||||
// Mark state as saved
|
||||
iter.modified = false
|
||||
s.iterators[k] = iter
|
||||
}
|
||||
}
|
||||
|
||||
func (s *store) set(stream, shard, seqnr string) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
s.iterators[stream+"/"+shard] = iterator{
|
||||
stream: stream,
|
||||
shard: shard,
|
||||
seqnr: seqnr,
|
||||
modified: true,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *store) get(ctx context.Context, stream, shard string) (string, error) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
// Return the cached result if possible
|
||||
if iter, found := s.iterators[stream+"/"+shard]; found {
|
||||
return iter.seqnr, nil
|
||||
}
|
||||
|
||||
// Retrieve the information from the database
|
||||
resp, err := s.client.GetItem(ctx, &dynamodb.GetItemInput{
|
||||
TableName: aws.String(s.table),
|
||||
ConsistentRead: aws.Bool(true),
|
||||
Key: map[string]types.AttributeValue{
|
||||
"namespace": &types.AttributeValueMemberS{Value: s.app + "-" + stream},
|
||||
"shard_id": &types.AttributeValueMemberS{Value: shard},
|
||||
},
|
||||
})
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
// Extract the sequence number
|
||||
raw, found := resp.Item["sequence_number"]
|
||||
if !found {
|
||||
return "", fmt.Errorf("%w for %s-%s/%s", errNotFound, s.app, stream, shard)
|
||||
}
|
||||
seqnr, ok := raw.(*types.AttributeValueMemberS)
|
||||
if !ok {
|
||||
return "", fmt.Errorf("sequence number for %s-%s/%s is of unexpected type %T", s.app, stream, shard, raw)
|
||||
}
|
||||
|
||||
// Fill the cache
|
||||
s.iterators[stream+"/"+shard] = iterator{
|
||||
stream: stream,
|
||||
shard: shard,
|
||||
seqnr: seqnr.Value,
|
||||
}
|
||||
|
||||
return seqnr.Value, nil
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue