Compare commits

...

24 commits

Author SHA1 Message Date
Guy A Molinari
baf8258298
Fix issue #167 - Concurrent write/write of in-flight shard map (#168) 2024-12-07 11:54:45 -08:00
Sanket Deshpande
9b6b643efb
fixed concurrent map rw panic for shardsInProgress map (#163)
Co-authored-by: Sanket Deshpande <sanket@clearblade.com>
2024-09-25 12:23:08 -07:00
lrs
43900507c9
fix isRetriableError (#159)
fix issues-158
2024-09-16 12:27:23 -07:00
Mikhail Konovalov
8d10ac8dac
Fix ProvisionedThroughputExceededException error (#161)
Fixes #158. Seems the bug was introduced in #155. See #155 (comment)
2024-09-16 12:25:49 -07:00
Jarrad
553e2392fd
fix nil pointer dereference on AWS errors (#148)
* fix nil pointer dereference on AWS errors

* return Start errors to Scan consumer

before the previous commit e465b09, client errors panicked the
reader, so consumers would pick up sharditerator errors by virtue of
their server crashing and burning.

Now that client errors are properly returned, the behaviour of
listShards is problematic because it absorbs any client errors it gets.

The result of these two things now is that if you hit an aws error, your server will go into an
endless scan loop you can't detect and can't easily recover from.

To avoid that, listShards will now stop if it hits a client error.

---------

Co-authored-by: Jarrad Whitaker <jwhitaker 📧 swift-nav.com>
2024-06-06 08:38:16 -07:00
gram-signal
6720a01733
Maintain parent/child shard ordering across shard splits/merges. (#155)
Kinesis allows clients to rely on an invariant that, for a given partition key, the order of records added to the stream will be maintained.  IE: given an input `pkey=x,val=1  pkey=x,val=2  pkey=x,val=3`, the values `1,2,3` will be seen in that order when processed by clients, so long as clients are careful.  It does so by putting all records for a single partition key into a single shard, then maintaining ordering within that shard.

However, shards can be split and merge, to distribute load better and handle per-shard throughput limits.  Kinesis does this currently by (one or many times) splitting a single shard into two or by merging two adjacent shards into one.  When this occurs, Kinesis still allows for ordering consistency by detailing shard parent/child relationships within its `listShards` outputs.  A split shard A will create children B and C, both with `ParentShardId=A`.  A merging of shards A and B into C will create a new shard C with `ParentShardId=A,AdjacentParentShardId=B`.  So long as clients fully process all records in parents (including adjacent parents) before processing the new shard, ordering will be maintained.

`kinesis-consumer` currently doesn't do this.  Instead, upon the initial (and subsequent) `listShards` call, all visible shards immediately begin processing.  Considering this case, where shards split, then merge, and each shard `X` contains a single record `rX`:

```
time ->
  B
 / \
A   D
 \ /
  C
```

record `rD` should be processed after both `rB` and `rC` are processed, and both `rB` and `rC` should wait for `rA` to be processed.  By starting goroutines immediately, any ordering of `{rA,rB,rC,rD}` might occur within the original code.

This PR utilizes the `AllGroup` as a book-keeper of fully processed shards, with the `Consumer` calling `CloseShard` once it has finished a shard.  `AllGroup` doesn't release a shard for processing until its parents have fully been processed, and the consumer just processes the shards it receives as it used to.

This PR created a new `CloseableGroup` interface rather than append to the existing `Group` interface to maintain backwards compatibility in existing code that may already implement the `Group` interface elsewhere.  Different `Group` implementations don't get the ordering described above, but the default `Consumer` does.
2024-06-06 08:37:42 -07:00
guangwu
188bdff278
fix: typo (#156) 2023-12-14 16:36:20 -08:00
Dario Emerson
af2db0d43f
update go-redis to v9 (#150) 2023-06-16 11:20:07 -07:00
Harlow Ward
c2b9f79d7a Put aws-skd-v2 note in installation section 2021-12-04 13:43:18 -08:00
Harlow Ward
6cbda0f706
Update ddb checkpoint item to use dynamodbav (#144)
Update the DynamoDB checkpoint item to use the `dynamodbav` value for marshaling. 

Fixes: https://github.com/harlow/kinesis-consumer/issues/142

Minor changes:
* Update the DDB example consumer to support a local version of DDB for streamlined testing
2021-12-04 13:40:26 -08:00
Harlow Ward
3b3b252fa5 Add all examples to example directory 2021-12-04 10:42:17 -08:00
Harlow Ward
61fa84eca6
Update to use aws-sdk-go-v2 (#141) 2021-09-21 22:00:14 -07:00
Harlow Ward
8257129066
Remove note about consumer groups (#139) 2021-09-02 11:20:14 -07:00
jonandrewj
a334486111
Implement a WithShardClosedHandler option for the consumer (#135) 2021-07-30 14:16:15 -07:00
jonandrewj
c75a9237b6
Add InternalFailureException to the list of retriable errors (#132)
* Add InternalFailureException to the list of retriable errors
2021-02-25 14:54:48 -08:00
Frank Meyer
27055f2ace
Wrap underlying errors via %w verb (#130)
As introduced in Go 1.13. This enables user of this library
to check for an underlying wrapped error type via errors.Is and
errors.As functions.
2021-02-05 09:10:34 -08:00
James Regovich
799ccf2d40
Add support for aggregated records (#127)
Add config option for aggregated records and deaggregation on records in ScanShard

This PR adds an option to consume aggregated records.
2020-10-13 20:41:18 -07:00
Jason Tackaberry
e60d217333
Include MillisBehindLatest in Record for ScanFunc (#124) 2020-08-01 22:05:17 -07:00
Jason Tackaberry
3f2519e51c
Run initial scan immediately (#123)
Rather than starting the shard scan loop and waiting for next scan tick before
fetching any data, do the first poll immediately, and then wait.
2020-08-01 15:45:37 -07:00
Jason Tackaberry
97ffabeaa5
Include ShardID in Record passed to ScanFunc (#121)
* Include ShardID in Record passed to ScanFunc
* Update mock to explicitly use kinesis.Record

Supports change wherein consumer.Record is changed from an alias of
kinesis.Record to a composition containing it.
2020-07-30 14:18:38 -07:00
Harlow Ward
bae065cf53 Make sure gofmt run on all files 2020-07-21 20:31:38 -07:00
Harlow Ward
3b95644d77 Move license to file without prefix 2020-07-21 20:31:04 -07:00
Nicolas Augusto Sassi
ef5ce02f91
Update README.md (#119)
fix typo
2020-07-21 20:27:21 -07:00
Harlow Ward
89db667ce5
Bump AWS SDK to v1.33.3 (#118)
* Bump AWS SDK to v1.33.3
* Bump redis version
2020-07-21 20:27:03 -07:00
39 changed files with 1733 additions and 599 deletions

1
.gitignore vendored
View file

@ -43,3 +43,4 @@ prof.mem
# Goland files
.idea/
tmp/**

View file

@ -2,7 +2,7 @@ language: go
services:
- redis-server
go:
- "1.12"
- "1.13"
branches:
only:
- master

View file

@ -1,10 +1,5 @@
# Golang Kinesis Consumer
__Note:__ This repo is under active development adding [Consumer Groups #42](https://github.com/harlow/kinesis-consumer/issues/42). Master should always be deployable, but there may be interface changes in master over the next few months.
Latest stable release https://github.com/harlow/kinesis-consumer/releases/tag/v0.3.2
![technology Go](https://img.shields.io/badge/technology-go-blue.svg) [![Build Status](https://travis-ci.com/harlow/kinesis-consumer.svg?branch=master)](https://travis-ci.com/harlow/kinesis-consumer) [![GoDoc](https://godoc.org/github.com/harlow/kinesis-consumer?status.svg)](https://godoc.org/github.com/harlow/kinesis-consumer) [![GoReportCard](https://goreportcard.com/badge/github.com/harlow/kinesis-consumer)](https://goreportcard.com/report/harlow/kinesis-consumer)
Kinesis consumer applications written in Go. This library is intended to be a lightweight wrapper around the Kinesis API to read records, save checkpoints (with swappable backends), and gracefully recover from service timeouts/errors.
@ -21,11 +16,14 @@ Get the package source:
$ go get github.com/harlow/kinesis-consumer
Note: This repo now requires the AWS SDK V2 package. If you are still using
AWS SDK V1 then use: https://github.com/harlow/kinesis-consumer/releases/tag/v0.3.5
## Overview
The consumer leverages a handler func that accepts a Kinesis record. The `Scan` method will consume all shards concurrently and call the callback func as it receives records from the stream.
_Important 1: The `Scan` func will also poll the stream to check for new shards, it will automatcially start consuming new shards added to the stream._
_Important 1: The `Scan` func will also poll the stream to check for new shards, it will automatically start consuming new shards added to the stream._
_Important 2: The default Log, Counter, and Checkpoint are no-op which means no logs, counts, or checkpoints will be emitted when scanning the stream. See the options below to override these defaults._
@ -121,7 +119,7 @@ The uniq identifier for a consumer is `[appName, streamName, shardID]`
Note: The default storage is in-memory (no-op). Which means the scan will not persist any state and the consumer will start from the beginning of the stream each time it is re-started.
The consumer accpets a `WithStore` option to set the storage layer:
The consumer accepts a `WithStore` option to set the storage layer:
```go
c, err := consumer.New(*stream, consumer.WithStore(db))
@ -134,7 +132,7 @@ To persist scan progress choose one of the following storage layers:
#### Redis
The Redis checkpoint requries App Name, and Stream Name:
The Redis checkpoint requires App Name, and Stream Name:
```go
import store "github.com/harlow/kinesis-consumer/store/redis"
@ -294,7 +292,7 @@ c, err := consumer.New(
### Logging
Logging supports the basic built-in logging library or use thrid party external one, so long as
Logging supports the basic built-in logging library or use third party external one, so long as
it implements the Logger interface.
For example, to use the builtin logging package, we wrap it with myLogger structure.
@ -345,7 +343,7 @@ func main() {
# Examples
There are example Produder and Consumer code in `/cmd` directory. These should help give end-to-end examples of setting up consumers with different checkpoint strategies.
There are examples of producer and comsumer in the `/examples` directory. These should help give end-to-end examples of setting up consumers with different checkpoint strategies.
The examples run locally against [Kinesis Lite](https://github.com/mhart/kinesalite).
@ -353,11 +351,11 @@ The examples run locally against [Kinesis Lite](https://github.com/mhart/kinesal
Produce data to the stream:
$ cat cmd/producer/users.txt | go run cmd/producer/main.go --stream myStream
$ cat examples/producer/users.txt | go run examples/producer/main.go --stream myStream
Consume data from the stream:
$ go run cmd/consumer/main.go --stream myStream
$ go run examples/consumer/main.go --stream myStream
## Contributing

View file

@ -2,22 +2,23 @@ package consumer
import (
"context"
"fmt"
"sync"
"time"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
)
// NewAllGroup returns an intitialized AllGroup for consuming
// NewAllGroup returns an initialized AllGroup for consuming
// all shards on a stream
func NewAllGroup(ksis kinesisiface.KinesisAPI, store Store, streamName string, logger Logger) *AllGroup {
func NewAllGroup(ksis kinesisClient, store Store, streamName string, logger Logger) *AllGroup {
return &AllGroup{
ksis: ksis,
shards: make(map[string]*kinesis.Shard),
streamName: streamName,
logger: logger,
Store: store,
ksis: ksis,
shards: make(map[string]types.Shard),
shardsClosed: make(map[string]chan struct{}),
streamName: streamName,
logger: logger,
Store: store,
}
}
@ -25,61 +26,119 @@ func NewAllGroup(ksis kinesisiface.KinesisAPI, store Store, streamName string, l
// caches a local list of the shards we are already processing
// and routinely polls the stream looking for new shards to process.
type AllGroup struct {
ksis kinesisiface.KinesisAPI
ksis kinesisClient
streamName string
logger Logger
Store
shardMu sync.Mutex
shards map[string]*kinesis.Shard
shardMu sync.Mutex
shards map[string]types.Shard
shardsClosed map[string]chan struct{}
}
// Start is a blocking operation which will loop and attempt to find new
// shards on a regular cadence.
func (g *AllGroup) Start(ctx context.Context, shardc chan *kinesis.Shard) {
var ticker = time.NewTicker(30 * time.Second)
g.findNewShards(shardc)
func (g *AllGroup) Start(ctx context.Context, shardC chan types.Shard) error {
// Note: while ticker is a rather naive approach to this problem,
// it actually simplies a few things. i.e. If we miss a new shard while
// AWS is resharding we'll pick it up max 30 seconds later.
// it actually simplifies a few things. I.e. If we miss a new shard
// while AWS is resharding, we'll pick it up max 30 seconds later.
// It might be worth refactoring this flow to allow the consumer to
// It might be worth refactoring this flow to allow the consumer
// to notify the broker when a shard is closed. However, shards don't
// necessarily close at the same time, so we could potentially get a
// thundering heard of notifications from the consumer.
var ticker = time.NewTicker(30 * time.Second)
for {
if err := g.findNewShards(ctx, shardC); err != nil {
ticker.Stop()
return err
}
select {
case <-ctx.Done():
ticker.Stop()
return
return nil
case <-ticker.C:
g.findNewShards(shardc)
}
}
}
func (g *AllGroup) CloseShard(_ context.Context, shardID string) error {
g.shardMu.Lock()
defer g.shardMu.Unlock()
c, ok := g.shardsClosed[shardID]
if !ok {
return fmt.Errorf("closing unknown shard ID %q", shardID)
}
close(c)
return nil
}
func waitForCloseChannel(ctx context.Context, c <-chan struct{}) bool {
if c == nil {
// no channel means we haven't seen this shard in listShards, so it
// probably fell off the TRIM_HORIZON, and we can assume it's fully processed.
return true
}
select {
case <-ctx.Done():
return false
case <-c:
// the channel has been processed and closed by the consumer (CloseShard has been called)
return true
}
}
// findNewShards pulls the list of shards from the Kinesis API
// and uses a local cache to determine if we are already processing
// a particular shard.
func (g *AllGroup) findNewShards(shardc chan *kinesis.Shard) {
func (g *AllGroup) findNewShards(ctx context.Context, shardC chan types.Shard) error {
g.shardMu.Lock()
defer g.shardMu.Unlock()
g.logger.Log("[GROUP]", "fetching shards")
shards, err := listShards(g.ksis, g.streamName)
shards, err := listShards(ctx, g.ksis, g.streamName)
if err != nil {
g.logger.Log("[GROUP] error:", err)
return
return err
}
// We do two `for` loops, since we have to set up all the `shardClosed`
// channels before we start using any of them. It's highly probable
// that Kinesis provides us the shards in dependency order (parents
// before children), but it doesn't appear to be a guarantee.
newShards := make(map[string]types.Shard)
for _, shard := range shards {
if _, ok := g.shards[*shard.ShardId]; ok {
continue
}
g.shards[*shard.ShardId] = shard
shardc <- shard
g.shardsClosed[*shard.ShardId] = make(chan struct{})
newShards[*shard.ShardId] = shard
}
// only new shards need to be checked for parent dependencies
for _, shard := range newShards {
shard := shard // Shadow shard, since we use it in goroutine
var parent1, parent2 <-chan struct{}
if shard.ParentShardId != nil {
parent1 = g.shardsClosed[*shard.ParentShardId]
}
if shard.AdjacentParentShardId != nil {
parent2 = g.shardsClosed[*shard.AdjacentParentShardId]
}
go func() {
// Asynchronously wait for all parents of this shard to be processed
// before providing it out to our client. Kinesis guarantees that a
// given partition key's data will be provided to clients in-order,
// but when splits or joins happen, we need to process all parents prior
// to processing children or that ordering guarantee is not maintained.
if waitForCloseChannel(ctx, parent1) && waitForCloseChannel(ctx, parent2) {
shardC <- shard
}
}()
}
return nil
}

14
client.go Normal file
View file

@ -0,0 +1,14 @@
package consumer
import (
"context"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
)
// kinesisClient defines the interface of functions needed for the consumer
type kinesisClient interface {
GetRecords(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error)
ListShards(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error)
GetShardIterator(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error)
}

View file

@ -1,142 +0,0 @@
package main
import (
"context"
"expvar"
"flag"
"fmt"
"log"
"net"
"net/http"
"os"
"os/signal"
alog "github.com/apex/log"
"github.com/apex/log/handlers/text"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/dynamodb"
"github.com/aws/aws-sdk-go/service/kinesis"
consumer "github.com/harlow/kinesis-consumer"
storage "github.com/harlow/kinesis-consumer/store/ddb"
)
// kick off a server for exposing scan metrics
func init() {
sock, err := net.Listen("tcp", "localhost:8080")
if err != nil {
log.Printf("net listen error: %v", err)
}
go func() {
fmt.Println("Metrics available at http://localhost:8080/debug/vars")
http.Serve(sock, nil)
}()
}
// A myLogger provides a minimalistic logger satisfying the Logger interface.
type myLogger struct {
logger alog.Logger
}
// Log logs the parameters to the stdlib logger. See log.Println.
func (l *myLogger) Log(args ...interface{}) {
l.logger.Infof("producer: %v", args...)
}
func main() {
// Wrap myLogger around apex logger
log := &myLogger{
logger: alog.Logger{
Handler: text.New(os.Stdout),
Level: alog.DebugLevel,
},
}
var (
app = flag.String("app", "", "Consumer app name")
stream = flag.String("stream", "", "Stream name")
table = flag.String("table", "", "Checkpoint table name")
kinesisEndpoint = flag.String("endpoint", "http://localhost:4567", "Kinesis endpoint")
awsRegion = flag.String("region", "us-west-2", "AWS Region")
)
flag.Parse()
// New Kinesis and DynamoDB clients (if you need custom config)
sess, err := session.NewSession(aws.NewConfig())
if err != nil {
log.Log("new session error: %v", err)
}
myDdbClient := dynamodb.New(sess)
var myKsis = kinesis.New(session.Must(session.NewSession(
aws.NewConfig().
WithEndpoint(*kinesisEndpoint).
WithRegion(*awsRegion).
WithLogLevel(3),
)))
// ddb persitance
ddb, err := storage.New(*app, *table, storage.WithDynamoClient(myDdbClient), storage.WithRetryer(&MyRetryer{}))
if err != nil {
log.Log("checkpoint error: %v", err)
}
// expvar counter
var counter = expvar.NewMap("counters")
// consumer
c, err := consumer.New(
*stream,
consumer.WithStore(ddb),
consumer.WithLogger(log),
consumer.WithCounter(counter),
consumer.WithClient(myKsis),
)
if err != nil {
log.Log("consumer error: %v", err)
}
// use cancel func to signal shutdown
ctx, cancel := context.WithCancel(context.Background())
// trap SIGINT, wait to trigger shutdown
signals := make(chan os.Signal, 1)
signal.Notify(signals, os.Interrupt)
go func() {
<-signals
cancel()
}()
// scan stream
err = c.Scan(ctx, func(r *consumer.Record) error {
fmt.Println(string(r.Data))
return nil // continue scanning
})
if err != nil {
log.Log("scan error: %v", err)
}
if err := ddb.Shutdown(); err != nil {
log.Log("storage shutdown error: %v", err)
}
}
// MyRetryer used for storage
type MyRetryer struct {
storage.Retryer
}
// ShouldRetry implements custom logic for when errors should retry
func (r *MyRetryer) ShouldRetry(err error) bool {
if awsErr, ok := err.(awserr.Error); ok {
switch awsErr.Code() {
case dynamodb.ErrCodeProvisionedThroughputExceededException, dynamodb.ErrCodeLimitExceededException:
return true
default:
return false
}
}
return false
}

View file

@ -1,97 +0,0 @@
package main
import (
"bufio"
"flag"
"fmt"
"log"
"os"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kinesis"
)
func main() {
var (
streamName = flag.String("stream", "", "Stream name")
kinesisEndpoint = flag.String("endpoint", "http://localhost:4567", "Kinesis endpoint")
awsRegion = flag.String("region", "us-west-2", "AWS Region")
)
flag.Parse()
var records []*kinesis.PutRecordsRequestEntry
var client = kinesis.New(session.Must(session.NewSession(
aws.NewConfig().
WithEndpoint(*kinesisEndpoint).
WithRegion(*awsRegion).
WithLogLevel(3),
)))
// create stream if doesn't exist
if err := createStream(client, streamName); err != nil {
log.Fatalf("create stream error: %v", err)
}
// loop over file data
b := bufio.NewScanner(os.Stdin)
for b.Scan() {
records = append(records, &kinesis.PutRecordsRequestEntry{
Data: b.Bytes(),
PartitionKey: aws.String(time.Now().Format(time.RFC3339Nano)),
})
if len(records) > 250 {
putRecords(client, streamName, records)
records = nil
}
}
if len(records) > 0 {
putRecords(client, streamName, records)
}
}
func createStream(client *kinesis.Kinesis, streamName *string) error {
resp, err := client.ListStreams(&kinesis.ListStreamsInput{})
if err != nil {
return fmt.Errorf("list streams error: %v", err)
}
for _, val := range resp.StreamNames {
if *streamName == *val {
return nil
}
}
_, err = client.CreateStream(
&kinesis.CreateStreamInput{
StreamName: streamName,
ShardCount: aws.Int64(2),
},
)
if err != nil {
return err
}
return client.WaitUntilStreamExists(
&kinesis.DescribeStreamInput{
StreamName: streamName,
},
)
}
func putRecords(client *kinesis.Kinesis, streamName *string, records []*kinesis.PutRecordsRequestEntry) {
_, err := client.PutRecords(&kinesis.PutRecordsInput{
StreamName: streamName,
Records: records,
})
if err != nil {
log.Fatalf("error putting records: %v", err)
}
fmt.Print(".")
}

View file

@ -4,36 +4,41 @@ import (
"context"
"errors"
"fmt"
"io/ioutil"
"io"
"log"
"sync"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
"github.com/harlow/kinesis-consumer/internal/deaggregator"
)
// Record is an alias of record returned from kinesis library
type Record = kinesis.Record
// Record wraps the record returned from the Kinesis library and
// extends to include the shard id.
type Record struct {
types.Record
ShardID string
MillisBehindLatest *int64
}
// New creates a kinesis consumer with default settings. Use Option to override
// any of the optional attributes.
func New(streamName string, opts ...Option) (*Consumer, error) {
if streamName == "" {
return nil, fmt.Errorf("must provide stream name")
return nil, errors.New("must provide stream name")
}
// new consumer with noop storage, counter, and logger
c := &Consumer{
streamName: streamName,
initialShardIteratorType: kinesis.ShardIteratorTypeLatest,
initialShardIteratorType: types.ShardIteratorTypeLatest,
store: &noopStore{},
counter: &noopCounter{},
logger: &noopLogger{
logger: log.New(ioutil.Discard, "", log.LstdFlags),
logger: log.New(io.Discard, "", log.LstdFlags),
},
scanInterval: 250 * time.Millisecond,
maxRecords: 10000,
@ -46,11 +51,11 @@ func New(streamName string, opts ...Option) (*Consumer, error) {
// default client
if c.client == nil {
newSession, err := session.NewSession(aws.NewConfig())
cfg, err := config.LoadDefaultConfig(context.TODO())
if err != nil {
return nil, err
log.Fatalf("unable to load SDK config, %v", err)
}
c.client = kinesis.New(newSession)
c.client = kinesis.NewFromConfig(cfg)
}
// default group consumes all shards
@ -64,15 +69,17 @@ func New(streamName string, opts ...Option) (*Consumer, error) {
// Consumer wraps the interaction with the Kinesis stream
type Consumer struct {
streamName string
initialShardIteratorType string
initialShardIteratorType types.ShardIteratorType
initialTimestamp *time.Time
client kinesisiface.KinesisAPI
client kinesisClient
counter Counter
group Group
logger Logger
store Store
scanInterval time.Duration
maxRecords int64
isAggregated bool
shardClosedHandler ShardClosedHandler
}
// ScanFunc is the type of the function called for each message read
@ -83,7 +90,7 @@ type Consumer struct {
type ScanFunc func(*Record) error
// ErrSkipCheckpoint is used as a return value from ScanFunc to indicate that
// the current checkpoint should be skipped skipped. It is not returned
// the current checkpoint should be skipped. It is not returned
// as an error by any function.
var ErrSkipCheckpoint = errors.New("skip checkpoint")
@ -95,40 +102,60 @@ func (c *Consumer) Scan(ctx context.Context, fn ScanFunc) error {
defer cancel()
var (
errc = make(chan error, 1)
shardc = make(chan *kinesis.Shard, 1)
errC = make(chan error, 1)
shardC = make(chan types.Shard, 1)
)
go func() {
c.group.Start(ctx, shardc)
err := c.group.Start(ctx, shardC)
if err != nil {
errC <- fmt.Errorf("error starting scan: %w", err)
cancel()
}
<-ctx.Done()
close(shardc)
close(shardC)
}()
wg := new(sync.WaitGroup)
// process each of the shards
for shard := range shardc {
s := newShardsInProcess()
for shard := range shardC {
shardId := aws.ToString(shard.ShardId)
if s.doesShardExist(shardId) {
// safetynet: if shard already in process by another goroutine, just skipping the request
continue
}
wg.Add(1)
go func(shardID string) {
s.addShard(shardID)
defer func() {
s.deleteShard(shardID)
}()
defer wg.Done()
if err := c.ScanShard(ctx, shardID, fn); err != nil {
var err error
if err = c.ScanShard(ctx, shardID, fn); err != nil {
err = fmt.Errorf("shard %s error: %w", shardID, err)
} else if closeable, ok := c.group.(CloseableGroup); !ok {
// group doesn't allow closure, skip calling CloseShard
} else if err = closeable.CloseShard(ctx, shardID); err != nil {
err = fmt.Errorf("shard closed CloseableGroup error: %w", err)
}
if err != nil {
select {
case errc <- fmt.Errorf("shard %s error: %v", shardID, err):
// first error to occur
case errC <- fmt.Errorf("shard %s error: %w", shardID, err):
cancel()
default:
// error has already occurred
}
}
}(aws.StringValue(shard.ShardId))
}(shardId)
}
go func() {
wg.Wait()
close(errc)
close(errC)
}()
return <-errc
return <-errC
}
// ScanShard loops over records on a specific shard, calls the callback func
@ -137,65 +164,67 @@ func (c *Consumer) ScanShard(ctx context.Context, shardID string, fn ScanFunc) e
// get last seq number from checkpoint
lastSeqNum, err := c.group.GetCheckpoint(c.streamName, shardID)
if err != nil {
return fmt.Errorf("get checkpoint error: %v", err)
return fmt.Errorf("get checkpoint error: %w", err)
}
// get shard iterator
shardIterator, err := c.getShardIterator(ctx, c.streamName, shardID, lastSeqNum)
if err != nil {
return fmt.Errorf("get shard iterator error: %v", err)
return fmt.Errorf("get shard iterator error: %w", err)
}
c.logger.Log("[CONSUMER] start scan:", shardID, lastSeqNum)
defer func() {
c.logger.Log("[CONSUMER] stop scan:", shardID)
}()
scanTicker := time.NewTicker(c.scanInterval)
defer scanTicker.Stop()
for {
select {
case <-ctx.Done():
return nil
case <-scanTicker.C:
resp, err := c.client.GetRecords(&kinesis.GetRecordsInput{
Limit: aws.Int64(c.maxRecords),
ShardIterator: shardIterator,
})
resp, err := c.client.GetRecords(ctx, &kinesis.GetRecordsInput{
Limit: aws.Int32(int32(c.maxRecords)),
ShardIterator: shardIterator,
})
// attempt to recover from GetRecords error when expired iterator
if err != nil {
c.logger.Log("[CONSUMER] get records error:", err.Error())
// attempt to recover from GetRecords error
if err != nil {
c.logger.Log("[CONSUMER] get records error:", err.Error())
if awserr, ok := err.(awserr.Error); ok {
if _, ok := retriableErrors[awserr.Code()]; !ok {
return fmt.Errorf("get records error: %v", awserr.Message())
}
}
shardIterator, err = c.getShardIterator(ctx, c.streamName, shardID, lastSeqNum)
if err != nil {
return fmt.Errorf("get shard iterator error: %v", err)
}
continue
if !isRetriableError(err) {
return fmt.Errorf("get records error: %v", err.Error())
}
shardIterator, err = c.getShardIterator(ctx, c.streamName, shardID, lastSeqNum)
if err != nil {
return fmt.Errorf("get shard iterator error: %w", err)
}
} else {
// loop over records, call callback func
for _, r := range resp.Records {
var records []types.Record
// deaggregate records
if c.isAggregated {
records, err = deaggregateRecords(resp.Records)
if err != nil {
return err
}
} else {
records = resp.Records
}
for _, r := range records {
select {
case <-ctx.Done():
return nil
default:
err := fn(r)
if err != nil && err != ErrSkipCheckpoint {
err := fn(&Record{r, shardID, resp.MillisBehindLatest})
if err != nil && !errors.Is(err, ErrSkipCheckpoint) {
return err
}
if err != ErrSkipCheckpoint {
if err := c.group.SetCheckpoint(c.streamName, shardID, *r.SequenceNumber); err != nil {
return err
}
if err := c.group.SetCheckpoint(c.streamName, shardID, *r.SequenceNumber); err != nil {
return err
}
c.counter.Add("records", 1)
@ -205,21 +234,45 @@ func (c *Consumer) ScanShard(ctx context.Context, shardID string, fn ScanFunc) e
if isShardClosed(resp.NextShardIterator, shardIterator) {
c.logger.Log("[CONSUMER] shard closed:", shardID)
if c.shardClosedHandler != nil {
if err := c.shardClosedHandler(c.streamName, shardID); err != nil {
return fmt.Errorf("shard closed handler error: %w", err)
}
}
return nil
}
shardIterator = resp.NextShardIterator
}
// Wait for next scan
select {
case <-ctx.Done():
return nil
case <-scanTicker.C:
continue
}
}
}
var retriableErrors = map[string]struct{}{
kinesis.ErrCodeExpiredIteratorException: struct{}{},
kinesis.ErrCodeProvisionedThroughputExceededException: struct{}{},
}
// temporary conversion func of []types.Record -> DeaggregateRecords([]*types.Record) -> []types.Record
func deaggregateRecords(in []types.Record) ([]types.Record, error) {
var recs []*types.Record
for _, rec := range in {
recs = append(recs, &rec)
}
func isShardClosed(nextShardIterator, currentShardIterator *string) bool {
return nextShardIterator == nil || currentShardIterator == nextShardIterator
deagg, err := deaggregator.DeaggregateRecords(recs)
if err != nil {
return nil, err
}
var out []types.Record
for _, rec := range deagg {
out = append(out, *rec)
}
return out, nil
}
func (c *Consumer) getShardIterator(ctx context.Context, streamName, shardID, seqNum string) (*string, error) {
@ -229,15 +282,53 @@ func (c *Consumer) getShardIterator(ctx context.Context, streamName, shardID, se
}
if seqNum != "" {
params.ShardIteratorType = aws.String(kinesis.ShardIteratorTypeAfterSequenceNumber)
params.ShardIteratorType = types.ShardIteratorTypeAfterSequenceNumber
params.StartingSequenceNumber = aws.String(seqNum)
} else if c.initialTimestamp != nil {
params.ShardIteratorType = aws.String(kinesis.ShardIteratorTypeAtTimestamp)
params.ShardIteratorType = types.ShardIteratorTypeAtTimestamp
params.Timestamp = c.initialTimestamp
} else {
params.ShardIteratorType = aws.String(c.initialShardIteratorType)
params.ShardIteratorType = c.initialShardIteratorType
}
res, err := c.client.GetShardIteratorWithContext(aws.Context(ctx), params)
return res.ShardIterator, err
res, err := c.client.GetShardIterator(ctx, params)
if err != nil {
return nil, err
}
return res.ShardIterator, nil
}
func isRetriableError(err error) bool {
if oe := (*types.ExpiredIteratorException)(nil); errors.As(err, &oe) {
return true
}
if oe := (*types.ProvisionedThroughputExceededException)(nil); errors.As(err, &oe) {
return true
}
return false
}
func isShardClosed(nextShardIterator, currentShardIterator *string) bool {
return nextShardIterator == nil || currentShardIterator == nextShardIterator
}
type shards struct {
shardsInProcess sync.Map
}
func newShardsInProcess() *shards {
return &shards{}
}
func (s *shards) addShard(shardId string) {
s.shardsInProcess.Store(shardId, struct{}{})
}
func (s *shards) doesShardExist(shardId string) bool {
_, ok := s.shardsInProcess.Load(shardId)
return ok
}
func (s *shards) deleteShard(shardId string) {
s.shardsInProcess.Delete(shardId)
}

View file

@ -2,20 +2,21 @@ package consumer
import (
"context"
"errors"
"fmt"
"math/rand"
"sync"
"testing"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/aws/request"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
"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/harlow/kinesis-consumer/store/memory"
store "github.com/harlow/kinesis-consumer/store/memory"
)
var records = []*kinesis.Record{
var records = []types.Record{
{
Data: []byte("firstData"),
SequenceNumber: aws.String("firstSeqNum"),
@ -26,6 +27,15 @@ var records = []*kinesis.Record{
},
}
// Implement logger to wrap testing.T.Log.
type testLogger struct {
t *testing.T
}
func (t *testLogger) Log(args ...interface{}) {
t.t.Log(args...)
}
func TestNew(t *testing.T) {
if _, err := New("myStreamName"); err != nil {
t.Fatalf("new consumer error: %v", err)
@ -34,20 +44,20 @@ func TestNew(t *testing.T) {
func TestScan(t *testing.T) {
client := &kinesisClientMock{
getShardIteratorMock: func(input *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(input *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: records,
}, nil
},
listShardsMock: func(input *kinesis.ListShardsInput) (*kinesis.ListShardsOutput, error) {
listShardsMock: func(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error) {
return &kinesis.ListShardsOutput{
Shards: []*kinesis.Shard{
Shards: []types.Shard{
{ShardId: aws.String("myShard")},
},
}, nil
@ -62,6 +72,7 @@ func TestScan(t *testing.T) {
WithClient(client),
WithCounter(ctr),
WithStore(cp),
WithLogger(&testLogger{t}),
)
if err != nil {
t.Fatalf("new consumer error: %v", err)
@ -100,14 +111,79 @@ func TestScan(t *testing.T) {
}
}
func TestScan_ListShardsError(t *testing.T) {
mockError := errors.New("mock list shards error")
client := &kinesisClientMock{
listShardsMock: func(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error) {
return nil, mockError
},
}
// use cancel func to signal shutdown
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
var res string
var fn = func(r *Record) error {
res += string(r.Data)
cancel() // simulate cancellation while processing first record
return nil
}
c, err := New("myStreamName", WithClient(client))
if err != nil {
t.Fatalf("new consumer error: %v", err)
}
err = c.Scan(ctx, fn)
if !errors.Is(err, mockError) {
t.Errorf("expected an error from listShards, but instead got %v", err)
}
}
func TestScan_GetShardIteratorError(t *testing.T) {
mockError := errors.New("mock get shard iterator error")
client := &kinesisClientMock{
listShardsMock: func(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error) {
return &kinesis.ListShardsOutput{
Shards: []types.Shard{
{ShardId: aws.String("myShard")},
},
}, nil
},
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return nil, mockError
},
}
// use cancel func to signal shutdown
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
var res string
var fn = func(r *Record) error {
res += string(r.Data)
cancel() // simulate cancellation while processing first record
return nil
}
c, err := New("myStreamName", WithClient(client))
if err != nil {
t.Fatalf("new consumer error: %v", err)
}
err = c.Scan(ctx, fn)
if !errors.Is(err, mockError) {
t.Errorf("expected an error from getShardIterator, but instead got %v", err)
}
}
func TestScanShard(t *testing.T) {
var client = &kinesisClientMock{
getShardIteratorMock: func(input *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(input *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: records,
@ -124,6 +200,7 @@ func TestScanShard(t *testing.T) {
WithClient(client),
WithCounter(ctr),
WithStore(cp),
WithLogger(&testLogger{t}),
)
if err != nil {
t.Fatalf("new consumer error: %v", err)
@ -168,12 +245,12 @@ func TestScanShard(t *testing.T) {
func TestScanShard_Cancellation(t *testing.T) {
var client = &kinesisClientMock{
getShardIteratorMock: func(input *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(input *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: records,
@ -208,12 +285,12 @@ func TestScanShard_Cancellation(t *testing.T) {
func TestScanShard_SkipCheckpoint(t *testing.T) {
var client = &kinesisClientMock{
getShardIteratorMock: func(input *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(input *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: records,
@ -231,7 +308,7 @@ func TestScanShard_SkipCheckpoint(t *testing.T) {
var ctx, cancel = context.WithCancel(context.Background())
var fn = func(r *Record) error {
if aws.StringValue(r.SequenceNumber) == "lastSeqNum" {
if aws.ToString(r.SequenceNumber) == "lastSeqNum" {
cancel()
return ErrSkipCheckpoint
}
@ -252,15 +329,15 @@ func TestScanShard_SkipCheckpoint(t *testing.T) {
func TestScanShard_ShardIsClosed(t *testing.T) {
var client = &kinesisClientMock{
getShardIteratorMock: func(input *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(input *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: make([]*Record, 0),
Records: make([]types.Record, 0),
}, nil
},
}
@ -280,22 +357,18 @@ func TestScanShard_ShardIsClosed(t *testing.T) {
}
}
func TestScanShard_GetRecordsError(t *testing.T) {
func TestScanShard_ShardIsClosed_WithShardClosedHandler(t *testing.T) {
var client = &kinesisClientMock{
getShardIteratorMock: func(input *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(input *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: nil,
}, awserr.New(
kinesis.ErrCodeInvalidArgumentException,
"aws error message",
fmt.Errorf("error message"),
)
NextShardIterator: nil,
Records: make([]types.Record, 0),
}, nil
},
}
@ -303,38 +376,73 @@ func TestScanShard_GetRecordsError(t *testing.T) {
return nil
}
c, err := New("myStreamName", WithClient(client))
c, err := New("myStreamName",
WithClient(client),
WithShardClosedHandler(func(streamName, shardID string) error {
return fmt.Errorf("closed shard error")
}),
WithLogger(&testLogger{t}))
if err != nil {
t.Fatalf("new consumer error: %v", err)
}
err = c.ScanShard(context.Background(), "myShard", fn)
if err.Error() != "get records error: aws error message" {
if err == nil {
t.Fatal("expected an error but didn't get one")
}
if err.Error() != "shard closed handler error: closed shard error" {
t.Fatalf("unexpected error: %s", err.Error())
}
}
func TestScanShard_GetRecordsError(t *testing.T) {
var client = &kinesisClientMock{
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: nil,
},
&types.InvalidArgumentException{Message: aws.String("aws error message")}
},
}
var fn = func(r *Record) error {
return nil
}
c, err := New("myStreamName", WithClient(client), WithLogger(&testLogger{t}))
if err != nil {
t.Fatalf("new consumer error: %v", err)
}
err = c.ScanShard(context.Background(), "myShard", fn)
if err.Error() != "get records error: InvalidArgumentException: aws error message" {
t.Fatalf("unexpected error: %v", err)
}
}
type kinesisClientMock struct {
kinesisiface.KinesisAPI
getShardIteratorMock func(*kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error)
getRecordsMock func(*kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error)
listShardsMock func(*kinesis.ListShardsInput) (*kinesis.ListShardsOutput, error)
kinesis.Client
getShardIteratorMock func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error)
getRecordsMock func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error)
listShardsMock func(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error)
}
func (c *kinesisClientMock) ListShards(in *kinesis.ListShardsInput) (*kinesis.ListShardsOutput, error) {
return c.listShardsMock(in)
func (c *kinesisClientMock) ListShards(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error) {
return c.listShardsMock(ctx, params)
}
func (c *kinesisClientMock) GetRecords(in *kinesis.GetRecordsInput) (*kinesis.GetRecordsOutput, error) {
return c.getRecordsMock(in)
func (c *kinesisClientMock) GetRecords(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return c.getRecordsMock(ctx, params)
}
func (c *kinesisClientMock) GetShardIterator(in *kinesis.GetShardIteratorInput) (*kinesis.GetShardIteratorOutput, error) {
return c.getShardIteratorMock(in)
}
func (c *kinesisClientMock) GetShardIteratorWithContext(ctx aws.Context, in *kinesis.GetShardIteratorInput, options ...request.Option) (*kinesis.GetShardIteratorOutput, error) {
return c.getShardIteratorMock(in)
func (c *kinesisClientMock) GetShardIterator(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return c.getShardIteratorMock(ctx, params)
}
// implementation of counter
@ -356,3 +464,201 @@ func (fc *fakeCounter) Add(streamName string, count int64) {
fc.counter += count
}
func TestScan_PreviousParentsBeforeTrimHorizon(t *testing.T) {
client := &kinesisClientMock{
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String("49578481031144599192696750682534686652010819674221576194"),
}, nil
},
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: records,
}, nil
},
listShardsMock: func(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error) {
return &kinesis.ListShardsOutput{
Shards: []types.Shard{
{
ShardId: aws.String("myShard"),
ParentShardId: aws.String("myOldParent"),
AdjacentParentShardId: aws.String("myOldAdjacentParent"),
},
},
}, nil
},
}
var (
cp = store.New()
ctr = &fakeCounter{}
)
c, err := New("myStreamName",
WithClient(client),
WithCounter(ctr),
WithStore(cp),
WithLogger(&testLogger{t}),
)
if err != nil {
t.Fatalf("new consumer error: %v", err)
}
var (
ctx, cancel = context.WithCancel(context.Background())
res string
)
var fn = func(r *Record) error {
res += string(r.Data)
if string(r.Data) == "lastData" {
cancel()
}
return nil
}
if err := c.Scan(ctx, fn); err != nil {
t.Errorf("scan returned unexpected error %v", err)
}
if res != "firstDatalastData" {
t.Errorf("callback error expected %s, got %s", "firstDatalastData", res)
}
if val := ctr.Get(); val != 2 {
t.Errorf("counter error expected %d, got %d", 2, val)
}
val, err := cp.GetCheckpoint("myStreamName", "myShard")
if err != nil && val != "lastSeqNum" {
t.Errorf("checkout error expected %s, got %s", "lastSeqNum", val)
}
}
func TestScan_ParentChildOrdering(t *testing.T) {
// We create a set of shards where shard1 split into (shard2,shard3), then (shard2,shard3) merged into shard4.
client := &kinesisClientMock{
getShardIteratorMock: func(ctx context.Context, params *kinesis.GetShardIteratorInput, optFns ...func(*kinesis.Options)) (*kinesis.GetShardIteratorOutput, error) {
return &kinesis.GetShardIteratorOutput{
ShardIterator: aws.String(*params.ShardId + "iter"),
}, nil
},
getRecordsMock: func(ctx context.Context, params *kinesis.GetRecordsInput, optFns ...func(*kinesis.Options)) (*kinesis.GetRecordsOutput, error) {
switch *params.ShardIterator {
case "shard1iter":
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: []types.Record{
{
Data: []byte("shard1data"),
SequenceNumber: aws.String("shard1num"),
},
},
}, nil
case "shard2iter":
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: []types.Record{},
}, nil
case "shard3iter":
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: []types.Record{
{
Data: []byte("shard3data"),
SequenceNumber: aws.String("shard3num"),
},
},
}, nil
case "shard4iter":
return &kinesis.GetRecordsOutput{
NextShardIterator: nil,
Records: []types.Record{
{
Data: []byte("shard4data"),
SequenceNumber: aws.String("shard4num"),
},
},
}, nil
default:
panic("got unexpected iterator")
}
},
listShardsMock: func(ctx context.Context, params *kinesis.ListShardsInput, optFns ...func(*kinesis.Options)) (*kinesis.ListShardsOutput, error) {
// Intentionally misorder these to test resiliance to ordering issues from ListShards.
return &kinesis.ListShardsOutput{
Shards: []types.Shard{
{
ShardId: aws.String("shard3"),
ParentShardId: aws.String("shard1"),
},
{
ShardId: aws.String("shard1"),
ParentShardId: aws.String("shard0"), // not otherwise referenced, parent ordering should ignore this
},
{
ShardId: aws.String("shard4"),
ParentShardId: aws.String("shard2"),
AdjacentParentShardId: aws.String("shard3"),
},
{
ShardId: aws.String("shard2"),
ParentShardId: aws.String("shard1"),
},
},
}, nil
},
}
var (
cp = store.New()
ctr = &fakeCounter{}
)
c, err := New("myStreamName",
WithClient(client),
WithCounter(ctr),
WithStore(cp),
WithLogger(&testLogger{t}),
)
if err != nil {
t.Fatalf("new consumer error: %v", err)
}
var (
ctx, cancel = context.WithCancel(context.Background())
res string
)
rand.Seed(time.Now().UnixNano())
var fn = func(r *Record) error {
res += string(r.Data)
time.Sleep(time.Duration(rand.Int()%100) * time.Millisecond)
if string(r.Data) == "shard4data" {
cancel()
}
return nil
}
if err := c.Scan(ctx, fn); err != nil {
t.Errorf("scan returned unexpected error %v", err)
}
if want := "shard1datashard3datashard4data"; res != want {
t.Errorf("callback error expected %s, got %s", want, res)
}
if val := ctr.Get(); val != 3 {
t.Errorf("counter error expected %d, got %d", 2, val)
}
val, err := cp.GetCheckpoint("myStreamName", "shard4data")
if err != nil && val != "shard4num" {
t.Errorf("checkout error expected %s, got %s", "shard4num", val)
}
}

View file

@ -0,0 +1,207 @@
package main
import (
"context"
"expvar"
"flag"
"fmt"
"log"
"net"
"net/http"
"os"
"os/signal"
"time"
alog "github.com/apex/log"
"github.com/apex/log/handlers/text"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/credentials"
"github.com/aws/aws-sdk-go-v2/service/dynamodb"
ddbtypes "github.com/aws/aws-sdk-go-v2/service/dynamodb/types"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
consumer "github.com/harlow/kinesis-consumer"
storage "github.com/harlow/kinesis-consumer/store/ddb"
)
// kick off a server for exposing scan metrics
func init() {
sock, err := net.Listen("tcp", "localhost:8080")
if err != nil {
log.Printf("net listen error: %v", err)
}
go func() {
fmt.Println("Metrics available at http://localhost:8080/debug/vars")
http.Serve(sock, nil)
}()
}
// A myLogger provides a minimalistic logger satisfying the Logger interface.
type myLogger struct {
logger alog.Logger
}
// Log logs the parameters to the stdlib logger. See log.Println.
func (l *myLogger) Log(args ...interface{}) {
l.logger.Infof("producer: %v", args...)
}
func main() {
// Wrap myLogger around apex logger
mylog := &myLogger{
logger: alog.Logger{
Handler: text.New(os.Stdout),
Level: alog.DebugLevel,
},
}
var (
app = flag.String("app", "", "Consumer app name")
stream = flag.String("stream", "", "Stream name")
tableName = flag.String("table", "", "Checkpoint table name")
ddbEndpoint = flag.String("ddb-endpoint", "http://localhost:8000", "DynamoDB endpoint")
kinesisEndpoint = flag.String("ksis-endpoint", "http://localhost:4567", "Kinesis endpoint")
awsRegion = flag.String("region", "us-west-2", "AWS Region")
)
flag.Parse()
// set up clients
kcfg, err := newConfig(*kinesisEndpoint, *awsRegion)
if err != nil {
log.Fatalf("new kinesis config error: %v", err)
}
var myKsis = kinesis.NewFromConfig(kcfg)
dcfg, err := newConfig(*ddbEndpoint, *awsRegion)
if err != nil {
log.Fatalf("new ddb config error: %v", err)
}
var myDdbClient = dynamodb.NewFromConfig(dcfg)
// ddb checkpoint table
if err := createTable(myDdbClient, *tableName); err != nil {
log.Fatalf("create ddb table error: %v", err)
}
// ddb persitance
ddb, err := storage.New(*app, *tableName, storage.WithDynamoClient(myDdbClient), storage.WithRetryer(&MyRetryer{}))
if err != nil {
log.Fatalf("checkpoint error: %v", err)
}
// expvar counter
var counter = expvar.NewMap("counters")
// consumer
c, err := consumer.New(
*stream,
consumer.WithStore(ddb),
consumer.WithLogger(mylog),
consumer.WithCounter(counter),
consumer.WithClient(myKsis),
)
if err != nil {
log.Fatalf("consumer error: %v", err)
}
// use cancel func to signal shutdown
ctx, cancel := context.WithCancel(context.Background())
// trap SIGINT, wait to trigger shutdown
signals := make(chan os.Signal, 1)
signal.Notify(signals, os.Interrupt)
go func() {
<-signals
cancel()
}()
// scan stream
err = c.Scan(ctx, func(r *consumer.Record) error {
fmt.Println(string(r.Data))
return nil // continue scanning
})
if err != nil {
log.Fatalf("scan error: %v", err)
}
if err := ddb.Shutdown(); err != nil {
log.Fatalf("storage shutdown error: %v", err)
}
}
func createTable(client *dynamodb.Client, tableName string) error {
resp, err := client.ListTables(context.Background(), &dynamodb.ListTablesInput{})
if err != nil {
return fmt.Errorf("list streams error: %v", err)
}
for _, val := range resp.TableNames {
if tableName == val {
return nil
}
}
_, err = client.CreateTable(
context.Background(),
&dynamodb.CreateTableInput{
TableName: aws.String(tableName),
AttributeDefinitions: []ddbtypes.AttributeDefinition{
{AttributeName: aws.String("namespace"), AttributeType: "S"},
{AttributeName: aws.String("shard_id"), AttributeType: "S"},
},
KeySchema: []ddbtypes.KeySchemaElement{
{AttributeName: aws.String("namespace"), KeyType: ddbtypes.KeyTypeHash},
{AttributeName: aws.String("shard_id"), KeyType: ddbtypes.KeyTypeRange},
},
ProvisionedThroughput: &ddbtypes.ProvisionedThroughput{
ReadCapacityUnits: aws.Int64(1),
WriteCapacityUnits: aws.Int64(1),
},
},
)
if err != nil {
return err
}
waiter := dynamodb.NewTableExistsWaiter(client)
return waiter.Wait(
context.Background(),
&dynamodb.DescribeTableInput{
TableName: aws.String(tableName),
},
5*time.Second,
)
}
// MyRetryer used for storage
type MyRetryer struct {
storage.Retryer
}
// ShouldRetry implements custom logic for when errors should retry
func (r *MyRetryer) ShouldRetry(err error) bool {
switch err.(type) {
case *types.ProvisionedThroughputExceededException, *types.LimitExceededException:
return true
}
return false
}
func newConfig(url, region string) (aws.Config, error) {
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
return aws.Endpoint{
PartitionID: "aws",
URL: url,
SigningRegion: region,
}, nil
})
return config.LoadDefaultConfig(
context.TODO(),
config.WithRegion(region),
config.WithEndpointResolver(resolver),
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
)
}

View file

@ -9,9 +9,10 @@ import (
"os"
"os/signal"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/credentials"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
consumer "github.com/harlow/kinesis-consumer"
store "github.com/harlow/kinesis-consumer/store/mysql"
)
@ -35,13 +36,25 @@ func main() {
var counter = expvar.NewMap("counters")
// client
cfg := aws.NewConfig().
WithEndpoint(*kinesisEndpoint).
WithRegion(*awsRegion).
WithLogLevel(3)
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
return aws.Endpoint{
PartitionID: "aws",
URL: *kinesisEndpoint,
SigningRegion: *awsRegion,
}, nil
})
var client = kinesis.New(session.Must(session.NewSession(cfg)))
// client
cfg, err := config.LoadDefaultConfig(
context.TODO(),
config.WithRegion(*awsRegion),
config.WithEndpointResolver(resolver),
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
)
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var client = kinesis.NewFromConfig(cfg)
// consumer
c, err := consumer.New(

View file

@ -9,9 +9,10 @@ import (
"os"
"os/signal"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/credentials"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
consumer "github.com/harlow/kinesis-consumer"
store "github.com/harlow/kinesis-consumer/store/postgres"
)
@ -35,13 +36,25 @@ func main() {
var counter = expvar.NewMap("counters")
// client
cfg := aws.NewConfig().
WithEndpoint(*kinesisEndpoint).
WithRegion(*awsRegion).
WithLogLevel(3)
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
return aws.Endpoint{
PartitionID: "aws",
URL: *kinesisEndpoint,
SigningRegion: *awsRegion,
}, nil
})
var client = kinesis.New(session.Must(session.NewSession(cfg)))
// client
cfg, err := config.LoadDefaultConfig(
context.TODO(),
config.WithRegion(*awsRegion),
config.WithEndpointResolver(resolver),
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
)
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var client = kinesis.NewFromConfig(cfg)
// consumer
c, err := consumer.New(

View file

@ -8,9 +8,10 @@ import (
"os"
"os/signal"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/credentials"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
consumer "github.com/harlow/kinesis-consumer"
store "github.com/harlow/kinesis-consumer/store/redis"
)
@ -45,13 +46,25 @@ func main() {
logger: log.New(os.Stdout, "consumer-example: ", log.LstdFlags),
}
// client
cfg := aws.NewConfig().
WithEndpoint(*kinesisEndpoint).
WithRegion(*awsRegion).
WithLogLevel(3)
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
return aws.Endpoint{
PartitionID: "aws",
URL: *kinesisEndpoint,
SigningRegion: *awsRegion,
}, nil
})
var client = kinesis.New(session.Must(session.NewSession(cfg)))
// client
cfg, err := config.LoadDefaultConfig(
context.TODO(),
config.WithRegion(*awsRegion),
config.WithEndpointResolver(resolver),
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
)
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var client = kinesis.NewFromConfig(cfg)
// consumer
c, err := consumer.New(

View file

@ -9,9 +9,10 @@ import (
"os/signal"
"syscall"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/credentials"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
consumer "github.com/harlow/kinesis-consumer"
)
@ -33,12 +34,25 @@ func main() {
)
flag.Parse()
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
return aws.Endpoint{
PartitionID: "aws",
URL: *kinesisEndpoint,
SigningRegion: *awsRegion,
}, nil
})
// client
var client = kinesis.New(session.Must(session.NewSession(
aws.NewConfig().
WithEndpoint(*kinesisEndpoint).
WithRegion(*awsRegion),
)))
cfg, err := config.LoadDefaultConfig(
context.TODO(),
config.WithRegion(*awsRegion),
config.WithEndpointResolver(resolver),
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
)
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var client = kinesis.NewFromConfig(cfg)
// consumer
c, err := consumer.New(

116
examples/producer/main.go Normal file
View file

@ -0,0 +1,116 @@
package main
import (
"bufio"
"context"
"flag"
"fmt"
"log"
"os"
"time"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/credentials"
"github.com/aws/aws-sdk-go-v2/service/kinesis"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
)
func main() {
var (
streamName = flag.String("stream", "", "Stream name")
kinesisEndpoint = flag.String("endpoint", "http://localhost:4567", "Kinesis endpoint")
awsRegion = flag.String("region", "us-west-2", "AWS Region")
)
flag.Parse()
var records []types.PutRecordsRequestEntry
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
return aws.Endpoint{
PartitionID: "aws",
URL: *kinesisEndpoint,
SigningRegion: *awsRegion,
}, nil
})
cfg, err := config.LoadDefaultConfig(
context.TODO(),
config.WithRegion(*awsRegion),
config.WithEndpointResolver(resolver),
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
)
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var client = kinesis.NewFromConfig(cfg)
// create stream if doesn't exist
if err := createStream(client, *streamName); err != nil {
log.Fatalf("create stream error: %v", err)
}
// loop over file data
b := bufio.NewScanner(os.Stdin)
for b.Scan() {
records = append(records, types.PutRecordsRequestEntry{
Data: b.Bytes(),
PartitionKey: aws.String(time.Now().Format(time.RFC3339Nano)),
})
if len(records) > 250 {
putRecords(client, streamName, records)
records = nil
}
}
if len(records) > 0 {
putRecords(client, streamName, records)
}
}
func createStream(client *kinesis.Client, streamName string) error {
resp, err := client.ListStreams(context.Background(), &kinesis.ListStreamsInput{})
if err != nil {
return fmt.Errorf("list streams error: %v", err)
}
for _, val := range resp.StreamNames {
if streamName == val {
return nil
}
}
_, err = client.CreateStream(
context.Background(),
&kinesis.CreateStreamInput{
StreamName: aws.String(streamName),
ShardCount: aws.Int32(2),
},
)
if err != nil {
return err
}
waiter := kinesis.NewStreamExistsWaiter(client)
return waiter.Wait(
context.Background(),
&kinesis.DescribeStreamInput{
StreamName: aws.String(streamName),
},
30*time.Second,
)
}
func putRecords(client *kinesis.Client, streamName *string, records []types.PutRecordsRequestEntry) {
_, err := client.PutRecords(context.Background(), &kinesis.PutRecordsInput{
StreamName: streamName,
Records: records,
})
if err != nil {
log.Fatalf("error putting records: %v", err)
}
fmt.Print(".")
}

32
go.mod
View file

@ -1,23 +1,25 @@
module github.com/harlow/kinesis-consumer
require (
github.com/DATA-DOG/go-sqlmock v1.3.3
github.com/alicebob/gopher-json v0.0.0-20180125190556-5a6b3ba71ee6 // indirect
github.com/DATA-DOG/go-sqlmock v1.4.1
github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a // indirect
github.com/alicebob/miniredis v2.5.0+incompatible
github.com/apex/log v1.0.0
github.com/aws/aws-sdk-go v1.15.0
github.com/go-ini/ini v1.38.1 // indirect
github.com/go-redis/redis v6.15.2+incompatible
github.com/go-sql-driver/mysql v1.4.1
github.com/apex/log v1.6.0
github.com/aws/aws-sdk-go-v2 v1.11.2
github.com/aws/aws-sdk-go-v2/config v1.6.1
github.com/aws/aws-sdk-go-v2/credentials v1.3.3
github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.2.0
github.com/aws/aws-sdk-go-v2/service/dynamodb v1.5.0
github.com/aws/aws-sdk-go-v2/service/kinesis v1.6.0
github.com/awslabs/kinesis-aggregation/go v0.0.0-20210630091500-54e17340d32f
github.com/go-redis/redis/v9 v9.0.0-rc.2
github.com/go-sql-driver/mysql v1.5.0
github.com/golang/protobuf v1.5.2
github.com/gomodule/redigo v2.0.0+incompatible // indirect
github.com/lib/pq v0.0.0-20180523175426-90697d60dd84
github.com/onsi/ginkgo v1.8.0 // indirect
github.com/onsi/gomega v1.5.0 // indirect
github.com/pkg/errors v0.8.0
github.com/stretchr/testify v1.3.0 // indirect
github.com/yuin/gopher-lua v0.0.0-20190514113301-1cd887cd7036 // indirect
google.golang.org/appengine v1.6.1 // indirect
gopkg.in/redis.v5 v5.2.9 // indirect
github.com/lib/pq v1.7.0
github.com/pkg/errors v0.9.1
github.com/stretchr/testify v1.8.1
github.com/yuin/gopher-lua v0.0.0-20200603152657-dc2b0ca8b37e // indirect
)
go 1.13

288
go.sum
View file

@ -1,80 +1,278 @@
github.com/DATA-DOG/go-sqlmock v1.3.3 h1:CWUqKXe0s8A2z6qCgkP4Kru7wC11YoAnoupUKFDnH08=
github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM=
github.com/alicebob/gopher-json v0.0.0-20180125190556-5a6b3ba71ee6 h1:45bxf7AZMwWcqkLzDAQugVEwedisr5nRJ1r+7LYnv0U=
github.com/alicebob/gopher-json v0.0.0-20180125190556-5a6b3ba71ee6/go.mod h1:SGnFV6hVsYE877CKEZ6tDNTjaSXYUk6QqoIK6PrAtcc=
github.com/DATA-DOG/go-sqlmock v1.4.1 h1:ThlnYciV1iM/V0OSF/dtkqWb6xo5qITT1TJBG1MRDJM=
github.com/DATA-DOG/go-sqlmock v1.4.1/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM=
github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a h1:HbKu58rmZpUGpz5+4FfNmIU+FmZg2P3Xaj2v2bfNWmk=
github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a/go.mod h1:SGnFV6hVsYE877CKEZ6tDNTjaSXYUk6QqoIK6PrAtcc=
github.com/alicebob/miniredis v2.5.0+incompatible h1:yBHoLpsyjupjz3NL3MhKMVkR41j82Yjf3KFv7ApYzUI=
github.com/alicebob/miniredis v2.5.0+incompatible/go.mod h1:8HZjEj4yU0dwhYHky+DxYx+6BMjkBbe5ONFIF1MXffk=
github.com/apex/log v1.0.0 h1:5UWeZC54mWVtOGSCjtuvDPgY/o0QxmjQgvYZ27pLVGQ=
github.com/apex/log v1.0.0/go.mod h1:yA770aXIDQrhVOIGurT/pVdfCpSq1GQV/auzMN5fzvY=
github.com/aws/aws-sdk-go v1.15.0 h1:uxi9gcf4jxEX7r8oWYMEkYB4kziKet+1cHPmq52LjC4=
github.com/aws/aws-sdk-go v1.15.0/go.mod h1:mFuSZ37Z9YOHbQEwBWztmVzqXrEkub65tZoCYDt7FT0=
github.com/apex/log v1.6.0 h1:Y50wF1PBIIexIgTm0/7G6gcLitkO5jHK5Mb6wcMY0UI=
github.com/apex/log v1.6.0/go.mod h1:x7s+P9VtvFBXge9Vbn+8TrqKmuzmD35TTkeBHul8UtY=
github.com/apex/logs v1.0.0/go.mod h1:XzxuLZ5myVHDy9SAmYpamKKRNApGj54PfYLcFrXqDwo=
github.com/aphistic/golf v0.0.0-20180712155816-02c07f170c5a/go.mod h1:3NqKYiepwy8kCu4PNA+aP7WUV72eXWJeP9/r3/K9aLE=
github.com/aphistic/sweet v0.2.0/go.mod h1:fWDlIh/isSE9n6EPsRmC0det+whmX6dJid3stzu0Xys=
github.com/aws/aws-sdk-go v1.19.48/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
github.com/aws/aws-sdk-go v1.20.6/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
github.com/aws/aws-sdk-go-v2 v1.8.1/go.mod h1:xEFuWz+3TYdlPRuo+CqATbeDWIWyaT5uAPwPaWtgse0=
github.com/aws/aws-sdk-go-v2 v1.9.0/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4=
github.com/aws/aws-sdk-go-v2 v1.11.2 h1:SDiCYqxdIYi6HgQfAWRhgdZrdnOuGyLDJVRSWLeHWvs=
github.com/aws/aws-sdk-go-v2 v1.11.2/go.mod h1:SQfA+m2ltnu1cA0soUkj4dRSsmITiVQUJvBIZjzfPyQ=
github.com/aws/aws-sdk-go-v2/config v1.6.1 h1:qrZINaORyr78syO1zfD4l7r4tZjy0Z1l0sy4jiysyOM=
github.com/aws/aws-sdk-go-v2/config v1.6.1/go.mod h1:t/y3UPu0XEDy0cEw6mvygaBQaPzWiYAxfP2SzgtvclA=
github.com/aws/aws-sdk-go-v2/credentials v1.3.3 h1:A13QPatmUl41SqUfnuT3V0E3XiNGL6qNTOINbE8cZL4=
github.com/aws/aws-sdk-go-v2/credentials v1.3.3/go.mod h1:oVieKMT3m9BSfqhOfuQ+E0j/yN84ZAJ7Qv8Sfume/ak=
github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.2.0 h1:8kvinmbIDObqsWegKP0JjeanYPiA4GUVpAtciNWE+jw=
github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue v1.2.0/go.mod h1:UVFtSYSWCHj2+brBLDHUdlJXmz8LxUpZhA+Ewypc+xQ=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.4.1 h1:rc+fRGvlKbeSd9IFhFS1KWBs0XjTkq0CfK5xqyLgIp0=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.4.1/go.mod h1:+GTydg3uHmVlQdkRoetz6VHKbOMEYof70m19IpMLifc=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.0.4 h1:IM9b6hlCcVFJFydPoyphs/t7YrHfqKy7T4/7AG5Eprs=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.0.4/go.mod h1:W5gGbtNXFpF9/ssYZTaItzG/B+j0bjTnwStiCP2AtWU=
github.com/aws/aws-sdk-go-v2/internal/ini v1.2.1 h1:IkqRRUZTKaS16P2vpX+FNc2jq3JWa3c478gykQp4ow4=
github.com/aws/aws-sdk-go-v2/internal/ini v1.2.1/go.mod h1:Pv3WenDjI0v2Jl7UaMFIIbPOBbhn33RmmAmGgkXDoqY=
github.com/aws/aws-sdk-go-v2/service/dynamodb v1.5.0 h1:SGwKUQaJudQQZE72dDQlL2FGuHNAEK1CyqKLTjh6mqE=
github.com/aws/aws-sdk-go-v2/service/dynamodb v1.5.0/go.mod h1:XY5YhCS9SLul3JSQ08XG/nfxXxrkh6RR21XPq/J//NY=
github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.4.0 h1:QbFWJr2SAyVYvyoOHvJU6sCGLnqNT94ZbWElJMEI1JY=
github.com/aws/aws-sdk-go-v2/service/dynamodbstreams v1.4.0/go.mod h1:bYsEP8w5YnbYyrx/Zi5hy4hTwRRQISSJS3RWrsGRijg=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0 h1:gceOysEWNNwLd6cki65IMBZ4WAM0MwgBQq2n7kejoT8=
github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.3.0/go.mod h1:v8ygadNyATSm6elwJ/4gzJwcFhri9RqS8skgHKiwXPU=
github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.1.0 h1:QCPbsMPMcM4iGbui5SH6O4uxvZffPoBJ4CIGX7dU0l4=
github.com/aws/aws-sdk-go-v2/service/internal/endpoint-discovery v1.1.0/go.mod h1:enkU5tq2HoXY+ZMiQprgF3Q83T3PbO77E83yXXzRZWE=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.3 h1:VxFCgxsqWe7OThOwJ5IpFX3xrObtuIH9Hg/NW7oot1Y=
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.3/go.mod h1:7gcsONBmFoCcKrAqrm95trrMd2+C/ReYKP7Vfu8yHHA=
github.com/aws/aws-sdk-go-v2/service/kinesis v1.6.0 h1:hb+NupVMUzINGUCfDs2+YqMkWKu47dBIQHpulM0XWh4=
github.com/aws/aws-sdk-go-v2/service/kinesis v1.6.0/go.mod h1:9O7UG2pELnP0hq35+Gd7XDjOLBkg7tmgRQ0y14ZjoJI=
github.com/aws/aws-sdk-go-v2/service/sso v1.3.3 h1:K2gCnGvAASpz+jqP9iyr+F/KNjmTYf8aWOtTQzhmZ5w=
github.com/aws/aws-sdk-go-v2/service/sso v1.3.3/go.mod h1:Jgw5O+SK7MZ2Yi9Yvzb4PggAPYaFSliiQuWR0hNjexk=
github.com/aws/aws-sdk-go-v2/service/sts v1.6.2 h1:l504GWCoQi1Pk68vSUFGLmDIEMzRfVGNgLakDK+Uj58=
github.com/aws/aws-sdk-go-v2/service/sts v1.6.2/go.mod h1:RBhoMJB8yFToaCnbe0jNq5Dcdy0jp6LhHqg55rjClkM=
github.com/aws/smithy-go v1.7.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E=
github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E=
github.com/aws/smithy-go v1.9.0 h1:c7FUdEqrQA1/UVKKCNDFQPNKGp4FQg3YW4Ck5SLTG58=
github.com/aws/smithy-go v1.9.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E=
github.com/awslabs/kinesis-aggregation/go v0.0.0-20210630091500-54e17340d32f h1:Pf0BjJDga7C98f0vhw+Ip5EaiE07S3lTKpIYPNS0nMo=
github.com/awslabs/kinesis-aggregation/go v0.0.0-20210630091500-54e17340d32f/go.mod h1:SghidfnxvX7ribW6nHI7T+IBbc9puZ9kk5Tx/88h8P4=
github.com/aybabtme/rgbterm v0.0.0-20170906152045-cc83f3b3ce59/go.mod h1:q/89r3U2H7sSsE2t6Kca0lfwTK8JdoNGS/yzM/4iH5I=
github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE=
github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI=
github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI=
github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU=
github.com/davecgh/go-spew v1.1.0 h1:ZDRjVQ15GmhC3fiQ8ni8+OwkZQO4DARzQgrnXU1Liz8=
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78=
github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc=
github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
github.com/go-ini/ini v1.25.4/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8=
github.com/go-ini/ini v1.38.1 h1:hbtfM8emWUVo9GnXSloXYyFbXxZ+tG6sbepSStoe1FY=
github.com/go-ini/ini v1.38.1/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8=
github.com/go-redis/redis v6.15.2+incompatible h1:9SpNVG76gr6InJGxoZ6IuuxaCOQwDAhzyXg+Bs+0Sb4=
github.com/go-redis/redis v6.15.2+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA=
github.com/go-sql-driver/mysql v1.4.1 h1:g24URVg0OFbNUTx9qqY1IRZ9D9z3iPyi5zKhQZpNwpA=
github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w=
github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4=
github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0=
github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
github.com/go-redis/redis/v9 v9.0.0-rc.2 h1:IN1eI8AvJJeWHjMW/hlFAv2sAfvTun2DVksDDJ3a6a0=
github.com/go-redis/redis/v9 v9.0.0-rc.2/go.mod h1:cgBknjwcBJa2prbnuHH/4k/Mlj4r0pWNV2HBanHujfY=
github.com/go-sql-driver/mysql v1.5.0 h1:ozyZYNQW3x3HtqT1jira07DN2PArx2v7/mN66gGcHOs=
github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg=
github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE=
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw=
github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
github.com/gomodule/redigo v2.0.0+incompatible h1:K/R+8tc58AaqLkqG2Ol3Qk+DR/TlNuhuh457pBFPtt0=
github.com/gomodule/redigo v2.0.0+incompatible/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4=
github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY=
github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38=
github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY=
github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE=
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
github.com/jmespath/go-jmespath v0.0.0-20160202185014-0b12d6b521d8 h1:12VvqtR6Aowv3l/EQUlocDHW2Cp4G9WJVH7uyH8QFJE=
github.com/jmespath/go-jmespath v0.0.0-20160202185014-0b12d6b521d8/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
github.com/lib/pq v0.0.0-20180523175426-90697d60dd84 h1:it29sI2IM490luSc3RAhp5WuCYnc6RtbfLVAB7nmC5M=
github.com/lib/pq v0.0.0-20180523175426-90697d60dd84/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo=
github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc=
github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg=
github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo=
github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8=
github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U=
github.com/jpillora/backoff v0.0.0-20180909062703-3050d21c67d7/go.mod h1:2iMrUgbbvHEiQClaW2NsSzMyGHqN+rDFqY705q49KG0=
github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs=
github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE=
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
github.com/lib/pq v1.7.0 h1:h93mCPfUSkaul3Ka/VG8uZdmW1uMHDGxzu0NWHuJmHY=
github.com/lib/pq v1.7.0/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o=
github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ=
github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE=
github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE=
github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE=
github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU=
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/ginkgo v1.8.0 h1:VkHVNpR4iVnU8XQR6DBm8BqYjN7CRzw+xKUbVVbbW9w=
github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/gomega v1.5.0 h1:izbySO9zDPmjJ8rDjLvkA2zJHIo+HkYXHnf7eN7SSyo=
github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0=
github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE=
github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU=
github.com/onsi/ginkgo/v2 v2.1.3/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c=
github.com/onsi/ginkgo/v2 v2.1.4/go.mod h1:um6tUpWM/cxCK3/FK8BXqEiUMUwRgSM4JXG47RKZmLU=
github.com/onsi/ginkgo/v2 v2.1.6/go.mod h1:MEH45j8TBi6u9BMogfbp0stKC5cdGjumZj5Y7AG4VIk=
github.com/onsi/ginkgo/v2 v2.3.0/go.mod h1:Eew0uilEqZmIEZr8JrvYlvOM7Rr6xzTmMV8AyFNU9d0=
github.com/onsi/ginkgo/v2 v2.4.0/go.mod h1:iHkDK1fKGcBoEHT5W7YBq4RFWaQulw+caOMkAt4OrFo=
github.com/onsi/ginkgo/v2 v2.5.0 h1:TRtrvv2vdQqzkwrQ1ke6vtXf7IK34RBUJafIy1wMwls=
github.com/onsi/ginkgo/v2 v2.5.0/go.mod h1:Luc4sArBICYCS8THh8v3i3i5CuSZO+RaQRaJoeNwomw=
github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/pkg/errors v0.8.0 h1:WdK/asTD0HN+q6hsWO3/vpuAkAr+tw6aNJNDFFf0+qw=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY=
github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9yPro=
github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo=
github.com/onsi/gomega v1.21.1/go.mod h1:iYAIXgPSaDHak0LCMA+AWBpIKBr8WZicMxnE8luStNc=
github.com/onsi/gomega v1.22.1/go.mod h1:x6n7VNe4hw0vkyYUM4mjIXx3JbLiPaBPNgB7PRQ1tuM=
github.com/onsi/gomega v1.24.0/go.mod h1:Z/NWtiqwBrwUt4/2loMmHL63EDLnYHmVbuBpDr2vQAg=
github.com/onsi/gomega v1.24.1 h1:KORJXNNTzJXzu4ScJWssJfJMnJ+2QJqhoQSRwNlze9E=
github.com/onsi/gomega v1.24.1/go.mod h1:3AOiACssS3/MajrniINInwbfOOtfZvplPzuRSmvt1jM=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/rogpeppe/fastuuid v1.1.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/smartystreets/assertions v1.0.0/go.mod h1:kHHU4qYBaI3q23Pp3VPrmWhuIUrLW/7eUrw0BU5VaoM=
github.com/smartystreets/go-aws-auth v0.0.0-20180515143844-0c1422d1fdb9/go.mod h1:SnhjPscd9TpLiy1LpzGSKh3bXCfxxXuqd9xmQJy3slM=
github.com/smartystreets/gunit v1.0.0/go.mod h1:qwPWnhz6pn0NnRBP++URONOVyNkPyr4SauJk4cUOwJs=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q=
github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
github.com/yuin/gopher-lua v0.0.0-20190514113301-1cd887cd7036 h1:1b6PAtenNyhsmo/NKXVe34h7JEZKva1YB/ne7K7mqKM=
github.com/yuin/gopher-lua v0.0.0-20190514113301-1cd887cd7036/go.mod h1:gqRgreBUhTSL0GeU64rtZ3Uq3wtjOa/TB2YfrtkCbVQ=
github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk=
github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
github.com/tj/assert v0.0.0-20171129193455-018094318fb0/go.mod h1:mZ9/Rh9oLWpLLDRpvE+3b7gP/C2YyLFYxNmcLnPTMe0=
github.com/tj/assert v0.0.3 h1:Df/BlaZ20mq6kuai7f5z2TvPFiwC3xaWJSDQNiIS3Rk=
github.com/tj/assert v0.0.3/go.mod h1:Ne6X72Q+TB1AteidzQncjw9PabbMp4PBMZ1k+vd1Pvk=
github.com/tj/go-buffer v1.0.1/go.mod h1:iyiJpfFcR2B9sXu7KvjbT9fpM4mOelRSDTbntVj52Uc=
github.com/tj/go-elastic v0.0.0-20171221160941-36157cbbebc2/go.mod h1:WjeM0Oo1eNAjXGDx2yma7uG2XoyRZTq1uv3M/o7imD0=
github.com/tj/go-kinesis v0.0.0-20171128231115-08b17f58cb1b/go.mod h1:/yhzCV0xPfx6jb1bBgRFjl5lytqVqZXEaeqWP8lTEao=
github.com/tj/go-spin v1.1.0/go.mod h1:Mg1mzmePZm4dva8Qz60H2lHwmJ2loum4VIrLgVnKwh4=
github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY=
github.com/yuin/gopher-lua v0.0.0-20200603152657-dc2b0ca8b37e h1:oIpIX9VKxSCFrfjsKpluGbNPBGq9iNnT9crH781j9wY=
github.com/yuin/gopher-lua v0.0.0-20200603152657-dc2b0ca8b37e/go.mod h1:gqRgreBUhTSL0GeU64rtZ3Uq3wtjOa/TB2YfrtkCbVQ=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20190426145343-a29dc8fdc734/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw=
golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.6.0-dev.0.20220106191415-9b9b3d81d5e3/go.mod h1:3p9vT2HGsQu2K1YbXdKPJLVgG5VJdoTa1poYQBtP1AY=
golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4=
golang.org/x/mod v0.6.0/go.mod h1:4mET923SAdbXp2ki8ey+zGs1SLqsuM2Y0uvdZR/fUNI=
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190603091049-60506f45cf65 h1:+rhAzEzT3f4JtomfC371qB+0Ola2caSKcY69NUBZrRQ=
golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk=
golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk=
golang.org/x/net v0.0.0-20220425223048-2871e0cb64e4/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk=
golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
golang.org/x/net v0.1.0/go.mod h1:Cx3nUiGt4eDBEyega/BKRp+/AlGL8hYe7U9odMt2Cco=
golang.org/x/net v0.2.0 h1:sZfSu1wtKLGlWI4ZZayP0ck9Y73K1ynO6gqzTdBVdPU=
golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY=
golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190204203706-41f3e6584952/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190606165138-5da285871e9c h1:+EXw7AwNOKzPFXMZ1yNjO40aWCh3PIquJB2fYlv9wcs=
golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220319134239-a9b59b0215f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220422013727-9388b58f7150/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.2.0 h1:ljd4t30dBnAvMZaQCevtY0xLLD0A+bRZXbgLMLU1F/A=
golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs=
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
golang.org/x/text v0.4.0 h1:BrVqGRd7+k1DiOgtnFvAkoQEWQvBc25ouMJM6429SFg=
golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
google.golang.org/appengine v1.6.1 h1:QzqyMA1tlu6CgqCDUtU9V+ZKhLFT2dkJuANu5QaxI3I=
google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0=
gopkg.in/DATA-DOG/go-sqlmock.v1 v1.3.0 h1:FVCohIoYO7IJoDDVpV2pdq7SgrMH6wHnuTyrdrxJNoY=
gopkg.in/DATA-DOG/go-sqlmock.v1 v1.3.0/go.mod h1:OdE7CF6DbADk7lN8LIKRzRJTTZXIjtWgA5THM5lhBAw=
golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.1.10/go.mod h1:Uh6Zz+xoGYZom868N8YTex3t7RhtHDBrE8Gzo9bV56E=
golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc=
golang.org/x/tools v0.2.0/go.mod h1:y4OqIKeOV/fWJetJ8bXPU1sEVniLMIyDAZWeHdV+NTA=
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
google.golang.org/protobuf v1.28.0 h1:w43yiav+6bVFTBQFZX0r7ipe9JQ1QsbMgHwbBziscLw=
google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4=
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo=
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
gopkg.in/redis.v5 v5.2.9 h1:MNZYOLPomQzZMfpN3ZtD1uyJ2IDonTTlxYiV/pEApiw=
gopkg.in/redis.v5 v5.2.9/go.mod h1:6gtv0/+A4iM08kdRfocWYB3bLX2tebpNtfKlFT6H4mY=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
gopkg.in/yaml.v2 v2.2.1 h1:mUhvW9EsL+naU5Q3cakzfE91YhliOondGd6ZrsDBHQE=
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.0-20200605160147-a5ece683394c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=

View file

@ -3,12 +3,19 @@ package consumer
import (
"context"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
)
// Group interface used to manage which shard to process
type Group interface {
Start(ctx context.Context, shardc chan *kinesis.Shard)
Start(ctx context.Context, shardc chan types.Shard) error
GetCheckpoint(streamName, shardID string) (string, error)
SetCheckpoint(streamName, shardID, sequenceNumber string) error
}
type CloseableGroup interface {
Group
// Allows shard processors to tell the group when the shard has been
// fully processed. Should be called only once per shardID.
CloseShard(ctx context.Context, shardID string) error
}

View file

@ -0,0 +1,6 @@
# Temporary Deaggregator
Upgrading to aws-sdk-go-v2 was blocked on a PR to introduce a new Deaggregator:
https://github.com/awslabs/kinesis-aggregation/pull/143/files
Once that PR is merged I'll remove this code and pull in the `awslabs/kinesis-aggregation` repo.

View file

@ -0,0 +1,94 @@
// Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
package deaggregator
import (
"crypto/md5"
"fmt"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
"github.com/golang/protobuf/proto"
rec "github.com/awslabs/kinesis-aggregation/go/records"
)
// Magic File Header for a KPL Aggregated Record
var KplMagicHeader = fmt.Sprintf("%q", []byte("\xf3\x89\x9a\xc2"))
const (
KplMagicLen = 4 // Length of magic header for KPL Aggregate Record checking.
DigestSize = 16 // MD5 Message size for protobuf.
)
// DeaggregateRecords takes an array of Kinesis records and expands any Protobuf
// records within that array, returning an array of all records
func DeaggregateRecords(records []*types.Record) ([]*types.Record, error) {
var isAggregated bool
allRecords := make([]*types.Record, 0)
for _, record := range records {
isAggregated = true
var dataMagic string
var decodedDataNoMagic []byte
// Check if record is long enough to have magic file header
if len(record.Data) >= KplMagicLen {
dataMagic = fmt.Sprintf("%q", record.Data[:KplMagicLen])
decodedDataNoMagic = record.Data[KplMagicLen:]
} else {
isAggregated = false
}
// Check if record has KPL Aggregate Record Magic Header and data length
// is correct size
if KplMagicHeader != dataMagic || len(decodedDataNoMagic) <= DigestSize {
isAggregated = false
}
if isAggregated {
messageDigest := fmt.Sprintf("%x", decodedDataNoMagic[len(decodedDataNoMagic)-DigestSize:])
messageData := decodedDataNoMagic[:len(decodedDataNoMagic)-DigestSize]
calculatedDigest := fmt.Sprintf("%x", md5.Sum(messageData))
// Check protobuf MD5 hash matches MD5 sum of record
if messageDigest != calculatedDigest {
isAggregated = false
} else {
aggRecord := &rec.AggregatedRecord{}
err := proto.Unmarshal(messageData, aggRecord)
if err != nil {
return nil, err
}
partitionKeys := aggRecord.PartitionKeyTable
for _, aggrec := range aggRecord.Records {
newRecord := createUserRecord(partitionKeys, aggrec, record)
allRecords = append(allRecords, newRecord)
}
}
}
if !isAggregated {
allRecords = append(allRecords, record)
}
}
return allRecords, nil
}
// createUserRecord takes in the partitionKeys of the aggregated record, the individual
// deaggregated record, and the original aggregated record builds a kinesis.Record and
// returns it
func createUserRecord(partitionKeys []string, aggRec *rec.Record, record *types.Record) *types.Record {
partitionKey := partitionKeys[*aggRec.PartitionKeyIndex]
return &types.Record{
ApproximateArrivalTimestamp: record.ApproximateArrivalTimestamp,
Data: aggRec.Data,
EncryptionType: record.EncryptionType,
PartitionKey: &partitionKey,
SequenceNumber: record.SequenceNumber,
}
}

View file

@ -0,0 +1,202 @@
// Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
// SPDX-License-Identifier: Apache-2.0
package deaggregator_test
import (
"crypto/md5"
"fmt"
"math/rand"
"testing"
"time"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
rec "github.com/awslabs/kinesis-aggregation/go/records"
deagg "github.com/harlow/kinesis-consumer/internal/deaggregator"
)
// Generate an aggregate record in the correct AWS-specified format
// https://github.com/awslabs/amazon-kinesis-producer/blob/master/aggregation-format.md
func generateAggregateRecord(numRecords int) []byte {
aggr := &rec.AggregatedRecord{}
// Start with the magic header
aggRecord := []byte("\xf3\x89\x9a\xc2")
partKeyTable := make([]string, 0)
// Create proto record with numRecords length
for i := 0; i < numRecords; i++ {
var partKey uint64
var hashKey uint64
partKey = uint64(i)
hashKey = uint64(i) * uint64(10)
r := &rec.Record{
PartitionKeyIndex: &partKey,
ExplicitHashKeyIndex: &hashKey,
Data: []byte("Some test data string"),
Tags: make([]*rec.Tag, 0),
}
aggr.Records = append(aggr.Records, r)
partKeyVal := "test" + fmt.Sprint(i)
partKeyTable = append(partKeyTable, partKeyVal)
}
aggr.PartitionKeyTable = partKeyTable
// Marshal to protobuf record, create md5 sum from proto record
// and append both to aggRecord with magic header
data, _ := proto.Marshal(aggr)
md5Hash := md5.Sum(data)
aggRecord = append(aggRecord, data...)
aggRecord = append(aggRecord, md5Hash[:]...)
return aggRecord
}
// Generate a generic kinesis.Record using whatever []byte
// is passed in as the data (can be normal []byte or proto record)
func generateKinesisRecord(data []byte) *types.Record {
currentTime := time.Now()
encryptionType := types.EncryptionTypeNone
partitionKey := "1234"
sequenceNumber := "21269319989900637946712965403778482371"
return &types.Record{
ApproximateArrivalTimestamp: &currentTime,
Data: data,
EncryptionType: encryptionType,
PartitionKey: &partitionKey,
SequenceNumber: &sequenceNumber,
}
}
// This tests to make sure that the data is at least larger than the length
// of the magic header to do some array slicing with index out of bounds
func TestSmallLengthReturnsCorrectNumberOfDeaggregatedRecords(t *testing.T) {
var err error
var kr *types.Record
krs := make([]*types.Record, 0, 1)
smallByte := []byte("No")
kr = generateKinesisRecord(smallByte)
krs = append(krs, kr)
dars, err := deagg.DeaggregateRecords(krs)
if err != nil {
panic(err)
}
// Small byte test, since this is not a deaggregated record, should return 1
// record in the array.
assert.Equal(t, 1, len(dars), "Small Byte test should return length of 1.")
}
// This function tests to make sure that the data starts with the correct magic header
// according to KPL aggregate documentation.
func TestNonMatchingMagicHeaderReturnsSingleRecord(t *testing.T) {
var err error
var kr *types.Record
krs := make([]*types.Record, 0, 1)
min := 1
max := 10
n := rand.Intn(max-min) + min
aggData := generateAggregateRecord(n)
mismatchAggData := aggData[1:]
kr = generateKinesisRecord(mismatchAggData)
krs = append(krs, kr)
dars, err := deagg.DeaggregateRecords(krs)
if err != nil {
panic(err)
}
// A byte record with a magic header that does not match 0xF3 0x89 0x9A 0xC2
// should return a single record.
assert.Equal(t, 1, len(dars), "Mismatch magic header test should return length of 1.")
}
// This function tests that the DeaggregateRecords function returns the correct number of
// deaggregated records from a single aggregated record.
func TestVariableLengthRecordsReturnsCorrectNumberOfDeaggregatedRecords(t *testing.T) {
var err error
var kr *types.Record
krs := make([]*types.Record, 0, 1)
min := 1
max := 10
n := rand.Intn(max-min) + min
aggData := generateAggregateRecord(n)
kr = generateKinesisRecord(aggData)
krs = append(krs, kr)
dars, err := deagg.DeaggregateRecords(krs)
if err != nil {
panic(err)
}
// Variable Length Aggregate Record test has aggregaterd records and should return
// n length.
assertMsg := fmt.Sprintf("Variable Length Aggregate Record should return length %v.", len(dars))
assert.Equal(t, n, len(dars), assertMsg)
}
// This function tests the length of the message after magic file header. If length is less than
// the digest size (16 bytes), it is not an aggregated record.
func TestRecordAfterMagicHeaderWithLengthLessThanDigestSizeReturnsSingleRecord(t *testing.T) {
var err error
var kr *types.Record
krs := make([]*types.Record, 0, 1)
min := 1
max := 10
n := rand.Intn(max-min) + min
aggData := generateAggregateRecord(n)
// Change size of proto message to 15
reducedAggData := aggData[:19]
kr = generateKinesisRecord(reducedAggData)
krs = append(krs, kr)
dars, err := deagg.DeaggregateRecords(krs)
if err != nil {
panic(err)
}
// A byte record with length less than 16 after the magic header should return
// a single record from DeaggregateRecords
assert.Equal(t, 1, len(dars), "Digest size test should return length of 1.")
}
// This function tests the MD5 Sum at the end of the record by comparing MD5 sum
// at end of proto record with MD5 Sum of Proto message. If they do not match,
// it is not an aggregated record.
func TestRecordWithMismatchMd5SumReturnsSingleRecord(t *testing.T) {
var err error
var kr *types.Record
krs := make([]*types.Record, 0, 1)
min := 1
max := 10
n := rand.Intn(max-min) + min
aggData := generateAggregateRecord(n)
// Remove last byte from array to mismatch the MD5 sums
mismatchAggData := aggData[:len(aggData)-1]
kr = generateKinesisRecord(mismatchAggData)
krs = append(krs, kr)
dars, err := deagg.DeaggregateRecords(krs)
if err != nil {
panic(err)
}
// A byte record with an MD5 sum that does not match with the md5.Sum(record)
// will be marked as a non-aggregate record and return a single record
assert.Equal(t, 1, len(dars), "Mismatch md5 sum test should return length of 1.")
}

View file

@ -1,24 +1,25 @@
package consumer
import (
"context"
"fmt"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/service/kinesis"
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
"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"
)
// listShards pulls a list of shard IDs from the kinesis api
func listShards(ksis kinesisiface.KinesisAPI, streamName string) ([]*kinesis.Shard, error) {
var ss []*kinesis.Shard
// listShards pulls a list of Shard IDs from the kinesis api
func listShards(ctx context.Context, ksis kinesisClient, streamName string) ([]types.Shard, error) {
var ss []types.Shard
var listShardsInput = &kinesis.ListShardsInput{
StreamName: aws.String(streamName),
}
for {
resp, err := ksis.ListShards(listShardsInput)
resp, err := ksis.ListShards(ctx, listShardsInput)
if err != nil {
return nil, fmt.Errorf("ListShards error: %v", err)
return nil, fmt.Errorf("ListShards error: %w", err)
}
ss = append(ss, resp.Shards...)

View file

@ -3,7 +3,7 @@ package consumer
import (
"time"
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
)
// Option is used to override defaults when creating a new Consumer
@ -38,7 +38,7 @@ func WithCounter(counter Counter) Option {
}
// WithClient overrides the default client
func WithClient(client kinesisiface.KinesisAPI) Option {
func WithClient(client kinesisClient) Option {
return func(c *Consumer) {
c.client = client
}
@ -47,7 +47,7 @@ func WithClient(client kinesisiface.KinesisAPI) Option {
// WithShardIteratorType overrides the starting point for the consumer
func WithShardIteratorType(t string) Option {
return func(c *Consumer) {
c.initialShardIteratorType = t
c.initialShardIteratorType = types.ShardIteratorType(t)
}
}
@ -73,3 +73,20 @@ func WithMaxRecords(n int64) Option {
c.maxRecords = n
}
}
func WithAggregation(a bool) Option {
return func(c *Consumer) {
c.isAggregated = a
}
}
// ShardClosedHandler is a handler that will be called when the consumer has reached the end of a closed shard.
// No more records for that shard will be provided by the consumer.
// An error can be returned to stop the consumer.
type ShardClosedHandler = func(streamName, shardID string) error
func WithShardClosedHandler(h ShardClosedHandler) Option {
return func(c *Consumer) {
c.shardClosedHandler = h
}
}

View file

@ -1,16 +1,17 @@
package ddb
import (
"context"
"fmt"
"log"
"sync"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/dynamodb"
"github.com/aws/aws-sdk-go/service/dynamodb/dynamodbattribute"
"github.com/aws/aws-sdk-go/service/dynamodb/dynamodbiface"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/feature/dynamodb/attributevalue"
"github.com/aws/aws-sdk-go-v2/service/dynamodb"
"github.com/aws/aws-sdk-go-v2/service/dynamodb/types"
)
// Option is used to override defaults when creating a new Checkpoint
@ -24,7 +25,7 @@ func WithMaxInterval(maxInterval time.Duration) Option {
}
// WithDynamoClient sets the dynamoDb client
func WithDynamoClient(svc dynamodbiface.DynamoDBAPI) Option {
func WithDynamoClient(svc *dynamodb.Client) Option {
return func(c *Checkpoint) {
c.client = svc
}
@ -39,7 +40,6 @@ func WithRetryer(r Retryer) Option {
// New returns a checkpoint that uses DynamoDB for underlying storage
func New(appName, tableName string, opts ...Option) (*Checkpoint, error) {
ck := &Checkpoint{
tableName: tableName,
appName: appName,
@ -56,11 +56,11 @@ func New(appName, tableName string, opts ...Option) (*Checkpoint, error) {
// default client
if ck.client == nil {
newSession, err := session.NewSession(aws.NewConfig())
cfg, err := config.LoadDefaultConfig(context.TODO())
if err != nil {
return nil, err
log.Fatalf("unable to load SDK config, %v", err)
}
ck.client = dynamodb.New(newSession)
ck.client = dynamodb.NewFromConfig(cfg)
}
go ck.loop()
@ -72,7 +72,7 @@ func New(appName, tableName string, opts ...Option) (*Checkpoint, error) {
type Checkpoint struct {
tableName string
appName string
client dynamodbiface.DynamoDBAPI
client *dynamodb.Client
maxInterval time.Duration
mu *sync.Mutex // protects the checkpoints
checkpoints map[key]string
@ -81,14 +81,14 @@ type Checkpoint struct {
}
type key struct {
streamName string
shardID string
StreamName string
ShardID string
}
type item struct {
Namespace string `json:"namespace"`
ShardID string `json:"shard_id"`
SequenceNumber string `json:"sequence_number"`
Namespace string `json:"namespace" dynamodbav:"namespace"`
ShardID string `json:"shard_id" dynamodbav:"shard_id"`
SequenceNumber string `json:"sequence_number" dynamodbav:"sequence_number"`
}
// GetCheckpoint determines if a checkpoint for a particular Shard exists.
@ -100,17 +100,13 @@ func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
params := &dynamodb.GetItemInput{
TableName: aws.String(c.tableName),
ConsistentRead: aws.Bool(true),
Key: map[string]*dynamodb.AttributeValue{
"namespace": &dynamodb.AttributeValue{
S: aws.String(namespace),
},
"shard_id": &dynamodb.AttributeValue{
S: aws.String(shardID),
},
Key: map[string]types.AttributeValue{
"namespace": &types.AttributeValueMemberS{Value: namespace},
"shard_id": &types.AttributeValueMemberS{Value: shardID},
},
}
resp, err := c.client.GetItem(params)
resp, err := c.client.GetItem(context.Background(), params)
if err != nil {
if c.retryer.ShouldRetry(err) {
return c.GetCheckpoint(streamName, shardID)
@ -119,7 +115,7 @@ func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
}
var i item
dynamodbattribute.UnmarshalMap(resp.Item, &i)
attributevalue.UnmarshalMap(resp.Item, &i)
return i.SequenceNumber, nil
}
@ -134,8 +130,8 @@ func (c *Checkpoint) SetCheckpoint(streamName, shardID, sequenceNumber string) e
}
key := key{
streamName: streamName,
shardID: shardID,
StreamName: streamName,
ShardID: shardID,
}
c.checkpoints[key] = sequenceNumber
@ -168,9 +164,9 @@ func (c *Checkpoint) save() error {
defer c.mu.Unlock()
for key, sequenceNumber := range c.checkpoints {
item, err := dynamodbattribute.MarshalMap(item{
Namespace: fmt.Sprintf("%s-%s", c.appName, key.streamName),
ShardID: key.shardID,
item, err := attributevalue.MarshalMap(item{
Namespace: fmt.Sprintf("%s-%s", c.appName, key.StreamName),
ShardID: key.ShardID,
SequenceNumber: sequenceNumber,
})
if err != nil {
@ -178,10 +174,12 @@ func (c *Checkpoint) save() error {
return nil
}
_, err = c.client.PutItem(&dynamodb.PutItemInput{
TableName: aws.String(c.tableName),
Item: item,
})
_, err = c.client.PutItem(
context.TODO(),
&dynamodb.PutItemInput{
TableName: aws.String(c.tableName),
Item: item,
})
if err != nil {
if !c.retryer.ShouldRetry(err) {
return err

View file

@ -1,12 +1,13 @@
package ddb
import (
"context"
"log"
"testing"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/dynamodb"
"github.com/aws/aws-sdk-go-v2/config"
"github.com/aws/aws-sdk-go-v2/service/dynamodb"
)
type fakeRetryer struct {
@ -42,11 +43,12 @@ func TestCheckpointSetting(t *testing.T) {
setRetryer(ckPtr)
// Test WithDyanmoDBClient
var fakeDbClient = dynamodb.New(
session.New(aws.NewConfig()), &aws.Config{
Region: aws.String("us-west-2"),
},
)
cfg, err := config.LoadDefaultConfig(context.TODO())
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var fakeDbClient = dynamodb.NewFromConfig(cfg)
setDDBClient := WithDynamoClient(fakeDbClient)
setDDBClient(ckPtr)
@ -70,11 +72,12 @@ func TestNewCheckpointWithOptions(t *testing.T) {
setRetryer := WithRetryer(&r)
// Test WithDyanmoDBClient
var fakeDbClient = dynamodb.New(
session.New(aws.NewConfig()), &aws.Config{
Region: aws.String("us-west-2"),
},
)
cfg, err := config.LoadDefaultConfig(context.TODO())
if err != nil {
log.Fatalf("unable to load SDK config, %v", err)
}
var fakeDbClient = dynamodb.NewFromConfig(cfg)
setDDBClient := WithDynamoClient(fakeDbClient)
ckPtr, err := New("testapp", "testtable", setInterval, setRetryer, setDDBClient)

View file

@ -1,8 +1,7 @@
package ddb
import (
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/service/dynamodb"
"github.com/aws/aws-sdk-go-v2/service/dynamodb/types"
)
// Retryer interface contains one method that decides whether to retry based on error
@ -17,10 +16,9 @@ type DefaultRetryer struct {
// ShouldRetry when error occured
func (r *DefaultRetryer) ShouldRetry(err error) bool {
if awsErr, ok := err.(awserr.Error); ok {
if awsErr.Code() == dynamodb.ErrCodeProvisionedThroughputExceededException {
return true
}
switch err.(type) {
case *types.ProvisionedThroughputExceededException:
return true
}
return false
}

View file

@ -1,22 +1,21 @@
package ddb
import (
"errors"
"testing"
"github.com/aws/aws-sdk-go/aws/awserr"
"github.com/aws/aws-sdk-go/service/dynamodb"
"github.com/aws/aws-sdk-go-v2/aws"
"github.com/aws/aws-sdk-go-v2/service/dynamodb/types"
)
func TestDefaultRetyer(t *testing.T) {
retryableError := awserr.New(dynamodb.ErrCodeProvisionedThroughputExceededException, "error is retryable", errors.New("don't care what is here"))
retryableError := &types.ProvisionedThroughputExceededException{Message: aws.String("error not retryable")}
// retryer is not nil and should returns according to what error is passed in.
q := &DefaultRetryer{}
if q.ShouldRetry(retryableError) != true {
t.Errorf("expected ShouldRetry returns %v. got %v", false, q.ShouldRetry(retryableError))
}
nonRetryableError := awserr.New(dynamodb.ErrCodeBackupInUseException, "error is not retryable", errors.New("don't care what is here"))
nonRetryableError := &types.BackupInUseException{Message: aws.String("error not retryable")}
shouldRetry := q.ShouldRetry(nonRetryableError)
if shouldRetry != false {
t.Errorf("expected ShouldRetry returns %v. got %v", true, shouldRetry)

View file

@ -4,30 +4,30 @@
package store
import (
"fmt"
"sync"
"fmt"
"sync"
)
func New() *Store {
return &Store{}
return &Store{}
}
type Store struct {
sync.Map
sync.Map
}
func (c *Store) SetCheckpoint(streamName, shardID, sequenceNumber string) error {
if sequenceNumber == "" {
return fmt.Errorf("sequence number should not be empty")
}
c.Store(streamName+":"+shardID, sequenceNumber)
return nil
if sequenceNumber == "" {
return fmt.Errorf("sequence number should not be empty")
}
c.Store(streamName+":"+shardID, sequenceNumber)
return nil
}
func (c *Store) GetCheckpoint(streamName, shardID string) (string, error) {
val, ok := c.Load(streamName + ":" + shardID)
if !ok {
return "", nil
}
return val.(string), nil
val, ok := c.Load(streamName + ":" + shardID)
if !ok {
return "", nil
}
return val.(string), nil
}

View file

@ -1,30 +1,30 @@
package store
import (
"testing"
"testing"
)
func Test_CheckpointLifecycle(t *testing.T) {
c := New()
c := New()
// set
c.SetCheckpoint("streamName", "shardID", "testSeqNum")
// set
c.SetCheckpoint("streamName", "shardID", "testSeqNum")
// get
val, err := c.GetCheckpoint("streamName", "shardID")
if err != nil {
t.Fatalf("get checkpoint error: %v", err)
}
if val != "testSeqNum" {
t.Fatalf("checkpoint exists expected %s, got %s", "testSeqNum", val)
}
// get
val, err := c.GetCheckpoint("streamName", "shardID")
if err != nil {
t.Fatalf("get checkpoint error: %v", err)
}
if val != "testSeqNum" {
t.Fatalf("checkpoint exists expected %s, got %s", "testSeqNum", val)
}
}
func Test_SetEmptySeqNum(t *testing.T) {
c := New()
c := New()
err := c.SetCheckpoint("streamName", "shardID", "")
if err == nil || err.Error() != "sequence number should not be empty" {
t.Fatalf("should not allow empty sequence number")
}
err := c.SetCheckpoint("streamName", "shardID", "")
if err == nil || err.Error() != "sequence number should not be empty" {
t.Fatalf("should not allow empty sequence number")
}
}

View file

@ -1,6 +1,6 @@
package redis
import redis "github.com/go-redis/redis"
import redis "github.com/go-redis/redis/v9"
// Option is used to override defaults when creating a new Redis checkpoint
type Option func(*Checkpoint)

View file

@ -1,10 +1,11 @@
package redis
import (
"context"
"fmt"
"os"
redis "github.com/go-redis/redis"
redis "github.com/go-redis/redis/v9"
)
const localhost = "127.0.0.1:6379"
@ -36,7 +37,7 @@ func New(appName string, opts ...Option) (*Checkpoint, error) {
}
// verify we can ping server
_, err := c.client.Ping().Result()
_, err := c.client.Ping(context.Background()).Result()
if err != nil {
return nil, err
}
@ -52,7 +53,8 @@ type Checkpoint struct {
// GetCheckpoint fetches the checkpoint for a particular Shard.
func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
val, _ := c.client.Get(c.key(streamName, shardID)).Result()
ctx := context.Background()
val, _ := c.client.Get(ctx, c.key(streamName, shardID)).Result()
return val, nil
}
@ -62,7 +64,8 @@ func (c *Checkpoint) SetCheckpoint(streamName, shardID, sequenceNumber string) e
if sequenceNumber == "" {
return fmt.Errorf("sequence number should not be empty")
}
err := c.client.Set(c.key(streamName, shardID), sequenceNumber, 0).Err()
ctx := context.Background()
err := c.client.Set(ctx, c.key(streamName, shardID), sequenceNumber, 0).Err()
if err != nil {
return err
}

View file

@ -4,7 +4,7 @@ import (
"testing"
"github.com/alicebob/miniredis"
redis "github.com/go-redis/redis"
redis "github.com/go-redis/redis/v9"
)
func Test_CheckpointOptions(t *testing.T) {