Compare commits
50 commits
hw-group-i
...
master
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
baf8258298 | ||
|
|
9b6b643efb | ||
|
|
43900507c9 | ||
|
|
8d10ac8dac | ||
|
|
553e2392fd | ||
|
|
6720a01733 | ||
|
|
188bdff278 | ||
|
|
af2db0d43f | ||
|
|
c2b9f79d7a | ||
|
|
6cbda0f706 | ||
|
|
3b3b252fa5 | ||
|
|
61fa84eca6 | ||
|
|
8257129066 | ||
|
|
a334486111 | ||
|
|
c75a9237b6 | ||
|
|
27055f2ace | ||
|
|
799ccf2d40 | ||
|
|
e60d217333 | ||
|
|
3f2519e51c | ||
|
|
97ffabeaa5 | ||
|
|
bae065cf53 | ||
|
|
3b95644d77 | ||
|
|
ef5ce02f91 | ||
|
|
89db667ce5 | ||
|
|
400ef07463 | ||
|
|
dcd9d048fb | ||
|
|
bc5c012fd3 | ||
|
|
217999854b | ||
|
|
f85f25c15e | ||
|
|
b87510458e | ||
|
|
dfb0faaa87 | ||
|
|
b451fc4cc2 | ||
|
|
e3ee95b282 | ||
|
|
4c2aaf78a2 | ||
|
|
3ae979bf82 | ||
|
|
1a141cfbaa | ||
|
|
3a98baa012 | ||
|
|
ed40b5d9b4 | ||
|
|
a252eb38c6 | ||
|
|
71bbc397e2 | ||
|
|
14db23eaf3 | ||
|
|
81a8ac4221 | ||
|
|
35c48ef1c9 | ||
|
|
5da0865ac1 | ||
|
|
a9c97d3b93 | ||
|
|
d2cf65fa7a | ||
|
|
00b5f64fa7 | ||
|
|
c72f561abd | ||
|
|
d05d6c2d5e | ||
|
|
7018c0c47e |
52 changed files with 2491 additions and 923 deletions
1
.gitignore
vendored
1
.gitignore
vendored
|
|
@ -43,3 +43,4 @@ prof.mem
|
|||
|
||||
# Goland files
|
||||
.idea/
|
||||
tmp/**
|
||||
|
|
|
|||
|
|
@ -2,7 +2,7 @@ language: go
|
|||
services:
|
||||
- redis-server
|
||||
go:
|
||||
- "1.12"
|
||||
- "1.13"
|
||||
branches:
|
||||
only:
|
||||
- master
|
||||
|
|
|
|||
98
README.md
98
README.md
|
|
@ -1,10 +1,6 @@
|
|||
# Golang Kinesis Consumer
|
||||
|
||||
[](https://travis-ci.com/harlow/kinesis-consumer) [](https://godoc.org/github.com/harlow/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 expect breaking changes in master over the next few months.
|
||||
|
||||
Latest stable release https://github.com/harlow/kinesis-consumer/releases/tag/v0.3.2
|
||||
 [](https://travis-ci.com/harlow/kinesis-consumer) [](https://godoc.org/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.
|
||||
|
||||
|
|
@ -20,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._
|
||||
|
||||
|
|
@ -83,7 +82,7 @@ return consumer.SkipCheckpoint
|
|||
return errors.New("my error, exit all scans")
|
||||
```
|
||||
|
||||
Use context cancel to signal the scan to exit without error. For example if we wanted to gracefulloy exit the scan on interrupt.
|
||||
Use context cancel to signal the scan to exit without error. For example if we wanted to gracefully exit the scan on interrupt.
|
||||
|
||||
```go
|
||||
// trap SIGINT, wait to trigger shutdown
|
||||
|
|
@ -104,9 +103,13 @@ err := c.Scan(ctx, func(r *consumer.Record) error {
|
|||
})
|
||||
```
|
||||
|
||||
## Checkpoint
|
||||
## Options
|
||||
|
||||
To record the progress of the consumer in the stream we use a checkpoint to store the last sequence number the consumer has read from a particular shard. The boolean value SkipCheckpoint of consumer.ScanError determines if checkpoint will be activated. ScanError is returned by the record processing callback.
|
||||
The consumer allows the following optional overrides.
|
||||
|
||||
### Store
|
||||
|
||||
To record the progress of the consumer in the stream (checkpoint) we use a storage layer to persist the last sequence number the consumer has read from a particular shard. The boolean value ErrSkipCheckpoint of consumer.ScanError determines if checkpoint will be activated. ScanError is returned by the record processing callback.
|
||||
|
||||
This will allow consumers to re-launch and pick up at the position in the stream where they left off.
|
||||
|
||||
|
|
@ -114,33 +117,42 @@ The uniq identifier for a consumer is `[appName, streamName, shardID]`
|
|||
|
||||
<img width="722" alt="kinesis-checkpoints" src="https://user-images.githubusercontent.com/739782/33085867-d8336122-ce9a-11e7-8c8a-a8afeb09dff1.png">
|
||||
|
||||
Note: The default checkpoint is 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.
|
||||
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.
|
||||
|
||||
To persist scan progress choose one of the following checkpoints:
|
||||
|
||||
### Redis Checkpoint
|
||||
|
||||
The Redis checkpoint requries App Name, and Stream Name:
|
||||
The consumer accepts a `WithStore` option to set the storage layer:
|
||||
|
||||
```go
|
||||
import checkpoint "github.com/harlow/kinesis-consumer/checkpoint/redis"
|
||||
c, err := consumer.New(*stream, consumer.WithStore(db))
|
||||
if err != nil {
|
||||
log.Log("consumer error: %v", err)
|
||||
}
|
||||
```
|
||||
|
||||
To persist scan progress choose one of the following storage layers:
|
||||
|
||||
#### Redis
|
||||
|
||||
The Redis checkpoint requires App Name, and Stream Name:
|
||||
|
||||
```go
|
||||
import store "github.com/harlow/kinesis-consumer/store/redis"
|
||||
|
||||
// redis checkpoint
|
||||
ck, err := checkpoint.New(appName)
|
||||
db, err := store.New(appName)
|
||||
if err != nil {
|
||||
log.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
```
|
||||
|
||||
### DynamoDB Checkpoint
|
||||
#### DynamoDB
|
||||
|
||||
The DynamoDB checkpoint requires Table Name, App Name, and Stream Name:
|
||||
|
||||
```go
|
||||
import checkpoint "github.com/harlow/kinesis-consumer/checkpoint/ddb"
|
||||
import store "github.com/harlow/kinesis-consumer/store/ddb"
|
||||
|
||||
// ddb checkpoint
|
||||
ck, err := checkpoint.New(appName, tableName)
|
||||
db, err := store.New(appName, tableName)
|
||||
if err != nil {
|
||||
log.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
|
|
@ -154,7 +166,7 @@ myDynamoDbClient := dynamodb.New(session.New(aws.NewConfig()))
|
|||
// Region: aws.String("us-west-2"),
|
||||
// })
|
||||
|
||||
ck, err := checkpoint.New(*app, *table, checkpoint.WithDynamoClient(myDynamoDbClient))
|
||||
db, err := store.New(*app, *table, checkpoint.WithDynamoClient(myDynamoDbClient))
|
||||
if err != nil {
|
||||
log.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
|
|
@ -173,15 +185,15 @@ Sort key: shard_id
|
|||
|
||||
<img width="727" alt="screen shot 2017-11-22 at 7 59 36 pm" src="https://user-images.githubusercontent.com/739782/33158557-b90e4228-cfbf-11e7-9a99-73b56a446f5f.png">
|
||||
|
||||
### Postgres Checkpoint
|
||||
#### Postgres
|
||||
|
||||
The Postgres checkpoint requires Table Name, App Name, Stream Name and ConnectionString:
|
||||
|
||||
```go
|
||||
import checkpoint "github.com/harlow/kinesis-consumer/checkpoint/postgres"
|
||||
import store "github.com/harlow/kinesis-consumer/store/postgres"
|
||||
|
||||
// postgres checkpoint
|
||||
ck, err := checkpoint.New(app, table, connStr)
|
||||
db, err := store.New(app, table, connStr)
|
||||
if err != nil {
|
||||
log.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
|
|
@ -201,15 +213,15 @@ CREATE TABLE kinesis_consumer (
|
|||
|
||||
The table name has to be the same that you specify when creating the checkpoint. The primary key composed by namespace and shard_id is mandatory in order to the checkpoint run without issues and also to ensure data integrity.
|
||||
|
||||
### Mysql Checkpoint
|
||||
#### Mysql
|
||||
|
||||
The Mysql checkpoint requires Table Name, App Name, Stream Name and ConnectionString (just like the Postgres checkpoint!):
|
||||
|
||||
```go
|
||||
import checkpoint "github.com/harlow/kinesis-consumer/checkpoint/mysql"
|
||||
import store "github.com/harlow/kinesis-consumer/store/mysql"
|
||||
|
||||
// mysql checkpoint
|
||||
ck, err := checkpoint.New(app, table, connStr)
|
||||
db, err := store.New(app, table, connStr)
|
||||
if err != nil {
|
||||
log.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
|
|
@ -229,10 +241,6 @@ CREATE TABLE kinesis_consumer (
|
|||
|
||||
The table name has to be the same that you specify when creating the checkpoint. The primary key composed by namespace and shard_id is mandatory in order to the checkpoint run without issues and also to ensure data integrity.
|
||||
|
||||
## Options
|
||||
|
||||
The consumer allows the following optional overrides.
|
||||
|
||||
### Kinesis Client
|
||||
|
||||
Override the Kinesis client if there is any special config needed:
|
||||
|
|
@ -259,10 +267,10 @@ c, err := consumer.New(streamName, consumer.WithCounter(counter))
|
|||
|
||||
The [expvar package](https://golang.org/pkg/expvar/) will display consumer counts:
|
||||
|
||||
```
|
||||
```json
|
||||
"counters": {
|
||||
"checkpoints": 3,
|
||||
"records": 13005
|
||||
"checkpoints": 3,
|
||||
"records": 13005
|
||||
},
|
||||
```
|
||||
|
||||
|
|
@ -284,12 +292,12 @@ 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.
|
||||
|
||||
```
|
||||
```go
|
||||
// A myLogger provides a minimalistic logger satisfying the Logger interface.
|
||||
type myLogger struct {
|
||||
logger *log.Logger
|
||||
|
|
@ -315,7 +323,7 @@ c, err := consumer.New(streamName, consumer.WithLogger(logger))
|
|||
|
||||
To use a more complicated logging library, e.g. apex log
|
||||
|
||||
```
|
||||
```go
|
||||
type myLogger struct {
|
||||
logger *log.Logger
|
||||
}
|
||||
|
|
@ -333,6 +341,22 @@ func main() {
|
|||
}
|
||||
```
|
||||
|
||||
# Examples
|
||||
|
||||
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).
|
||||
|
||||
$ kinesalite &
|
||||
|
||||
Produce data to the stream:
|
||||
|
||||
$ cat examples/producer/users.txt | go run examples/producer/main.go --stream myStream
|
||||
|
||||
Consume data from the stream:
|
||||
|
||||
$ go run examples/consumer/main.go --stream myStream
|
||||
|
||||
## Contributing
|
||||
|
||||
Please see [CONTRIBUTING.md] for more information. Thank you, [contributors]!
|
||||
|
|
|
|||
144
allgroup.go
Normal file
144
allgroup.go
Normal file
|
|
@ -0,0 +1,144 @@
|
|||
package consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
|
||||
)
|
||||
|
||||
// NewAllGroup returns an initialized AllGroup for consuming
|
||||
// all shards on a stream
|
||||
func NewAllGroup(ksis kinesisClient, store Store, streamName string, logger Logger) *AllGroup {
|
||||
return &AllGroup{
|
||||
ksis: ksis,
|
||||
shards: make(map[string]types.Shard),
|
||||
shardsClosed: make(map[string]chan struct{}),
|
||||
streamName: streamName,
|
||||
logger: logger,
|
||||
Store: store,
|
||||
}
|
||||
}
|
||||
|
||||
// AllGroup is used to consume all shards from a single consumer. It
|
||||
// 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 kinesisClient
|
||||
streamName string
|
||||
logger Logger
|
||||
Store
|
||||
|
||||
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 types.Shard) error {
|
||||
// Note: while ticker is a rather naive approach to this problem,
|
||||
// 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 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 nil
|
||||
case <-ticker.C:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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(ctx context.Context, shardC chan types.Shard) error {
|
||||
g.shardMu.Lock()
|
||||
defer g.shardMu.Unlock()
|
||||
|
||||
g.logger.Log("[GROUP]", "fetching shards")
|
||||
|
||||
shards, err := listShards(ctx, g.ksis, g.streamName)
|
||||
if err != nil {
|
||||
g.logger.Log("[GROUP] error:", err)
|
||||
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
|
||||
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
|
||||
}
|
||||
114
broker.go
114
broker.go
|
|
@ -1,114 +0,0 @@
|
|||
package consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/aws/aws-sdk-go/aws"
|
||||
"github.com/aws/aws-sdk-go/service/kinesis"
|
||||
"github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
|
||||
)
|
||||
|
||||
func newBroker(
|
||||
client kinesisiface.KinesisAPI,
|
||||
streamName string,
|
||||
shardc chan *kinesis.Shard,
|
||||
logger Logger,
|
||||
) *broker {
|
||||
return &broker{
|
||||
client: client,
|
||||
shards: make(map[string]*kinesis.Shard),
|
||||
streamName: streamName,
|
||||
shardc: shardc,
|
||||
logger: logger,
|
||||
}
|
||||
}
|
||||
|
||||
// broker caches a local list of the shards we are already processing
|
||||
// and routinely polls the stream looking for new shards to process
|
||||
type broker struct {
|
||||
client kinesisiface.KinesisAPI
|
||||
streamName string
|
||||
shardc chan *kinesis.Shard
|
||||
logger Logger
|
||||
|
||||
shardMu sync.Mutex
|
||||
shards map[string]*kinesis.Shard
|
||||
}
|
||||
|
||||
// start is a blocking operation which will loop and attempt to find new
|
||||
// shards on a regular cadence.
|
||||
func (b *broker) start(ctx context.Context) {
|
||||
b.findNewShards()
|
||||
ticker := time.NewTicker(30 * time.Second)
|
||||
|
||||
// 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 might be worth refactoring this flow to allow the consumer to
|
||||
// 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.
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
ticker.Stop()
|
||||
return
|
||||
case <-ticker.C:
|
||||
b.findNewShards()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 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 (b *broker) findNewShards() {
|
||||
b.shardMu.Lock()
|
||||
defer b.shardMu.Unlock()
|
||||
|
||||
b.logger.Log("[BROKER]", "fetching shards")
|
||||
|
||||
shards, err := b.listShards()
|
||||
if err != nil {
|
||||
b.logger.Log("[BROKER]", err)
|
||||
return
|
||||
}
|
||||
|
||||
for _, shard := range shards {
|
||||
if _, ok := b.shards[*shard.ShardId]; ok {
|
||||
continue
|
||||
}
|
||||
b.shards[*shard.ShardId] = shard
|
||||
b.shardc <- shard
|
||||
}
|
||||
}
|
||||
|
||||
// listShards pulls a list of shard IDs from the kinesis api
|
||||
func (b *broker) listShards() ([]*kinesis.Shard, error) {
|
||||
var ss []*kinesis.Shard
|
||||
var listShardsInput = &kinesis.ListShardsInput{
|
||||
StreamName: aws.String(b.streamName),
|
||||
}
|
||||
|
||||
for {
|
||||
resp, err := b.client.ListShards(listShardsInput)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("ListShards error: %v", err)
|
||||
}
|
||||
ss = append(ss, resp.Shards...)
|
||||
|
||||
if resp.NextToken == nil {
|
||||
return ss, nil
|
||||
}
|
||||
|
||||
listShardsInput = &kinesis.ListShardsInput{
|
||||
NextToken: resp.NextToken,
|
||||
StreamName: aws.String(b.streamName),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,13 +0,0 @@
|
|||
package consumer
|
||||
|
||||
// Checkpoint interface used track consumer progress in the stream
|
||||
type Checkpoint interface {
|
||||
Get(streamName, shardID string) (string, error)
|
||||
Set(streamName, shardID, sequenceNumber string) error
|
||||
}
|
||||
|
||||
// noopCheckpoint implements the checkpoint interface with discard
|
||||
type noopCheckpoint struct{}
|
||||
|
||||
func (n noopCheckpoint) Set(string, string, string) error { return nil }
|
||||
func (n noopCheckpoint) Get(string, string) (string, error) { return "", nil }
|
||||
|
|
@ -1,61 +0,0 @@
|
|||
package redis
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
redis "gopkg.in/redis.v5"
|
||||
)
|
||||
|
||||
const localhost = "127.0.0.1:6379"
|
||||
|
||||
// New returns a checkpoint that uses Redis for underlying storage
|
||||
func New(appName string) (*Checkpoint, error) {
|
||||
addr := os.Getenv("REDIS_URL")
|
||||
if addr == "" {
|
||||
addr = localhost
|
||||
}
|
||||
|
||||
client := redis.NewClient(&redis.Options{Addr: addr})
|
||||
|
||||
// verify we can ping server
|
||||
_, err := client.Ping().Result()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &Checkpoint{
|
||||
appName: appName,
|
||||
client: client,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Checkpoint stores and retreives the last evaluated key from a DDB scan
|
||||
type Checkpoint struct {
|
||||
appName string
|
||||
client *redis.Client
|
||||
}
|
||||
|
||||
// Get fetches the checkpoint for a particular Shard.
|
||||
func (c *Checkpoint) Get(streamName, shardID string) (string, error) {
|
||||
val, _ := c.client.Get(c.key(streamName, shardID)).Result()
|
||||
return val, nil
|
||||
}
|
||||
|
||||
// Set stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// Upon failover, record processing is resumed from this point.
|
||||
func (c *Checkpoint) Set(streamName, shardID, sequenceNumber string) error {
|
||||
if sequenceNumber == "" {
|
||||
return fmt.Errorf("sequence number should not be empty")
|
||||
}
|
||||
err := c.client.Set(c.key(streamName, shardID), sequenceNumber, 0).Err()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// key generates a unique Redis key for storage of Checkpoint.
|
||||
func (c *Checkpoint) key(streamName, shardID string) string {
|
||||
return fmt.Sprintf("%v:checkpoint:%v:%v", c.appName, streamName, shardID)
|
||||
}
|
||||
14
client.go
Normal file
14
client.go
Normal 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)
|
||||
}
|
||||
282
consumer.go
282
consumer.go
|
|
@ -4,34 +4,44 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"io"
|
||||
"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/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 no-op checkpoint, counter, and logger
|
||||
// new consumer with noop storage, counter, and logger
|
||||
c := &Consumer{
|
||||
streamName: streamName,
|
||||
initialShardIteratorType: kinesis.ShardIteratorTypeLatest,
|
||||
checkpoint: &noopCheckpoint{},
|
||||
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,
|
||||
}
|
||||
|
||||
// override defaults
|
||||
|
|
@ -39,13 +49,18 @@ func New(streamName string, opts ...Option) (*Consumer, error) {
|
|||
opt(c)
|
||||
}
|
||||
|
||||
// default client if none provided
|
||||
// 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
|
||||
if c.group == nil {
|
||||
c.group = NewAllGroup(c.client, c.store, streamName, c.logger)
|
||||
}
|
||||
|
||||
return c, nil
|
||||
|
|
@ -54,119 +69,162 @@ func New(streamName string, opts ...Option) (*Consumer, error) {
|
|||
// Consumer wraps the interaction with the Kinesis stream
|
||||
type Consumer struct {
|
||||
streamName string
|
||||
initialShardIteratorType string
|
||||
client kinesisiface.KinesisAPI
|
||||
logger Logger
|
||||
checkpoint Checkpoint
|
||||
initialShardIteratorType types.ShardIteratorType
|
||||
initialTimestamp *time.Time
|
||||
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
|
||||
// from the stream. The record argument contains the original record
|
||||
// returned from the AWS Kinesis library.
|
||||
//
|
||||
// If an error is returned, scanning stops. The sole exception is when the
|
||||
// function returns the special value SkipCheckpoint.
|
||||
// function returns the special value ErrSkipCheckpoint.
|
||||
type ScanFunc func(*Record) error
|
||||
|
||||
// SkipCheckpoint is used as a return value from ScanFunc to indicate that
|
||||
// the current checkpoint should be skipped skipped. It is not returned
|
||||
// ErrSkipCheckpoint is used as a return value from ScanFunc to indicate that
|
||||
// the current checkpoint should be skipped. It is not returned
|
||||
// as an error by any function.
|
||||
var SkipCheckpoint = errors.New("skip checkpoint")
|
||||
var ErrSkipCheckpoint = errors.New("skip checkpoint")
|
||||
|
||||
// Scan launches a goroutine to process each of the shards in the stream. The ScanFunc
|
||||
// is passed through to each of the goroutines and called with each message pulled from
|
||||
// the stream.
|
||||
func (c *Consumer) Scan(ctx context.Context, fn ScanFunc) error {
|
||||
var (
|
||||
errc = make(chan error, 1)
|
||||
shardc = make(chan *kinesis.Shard, 1)
|
||||
broker = newBroker(c.client, c.streamName, shardc, c.logger)
|
||||
)
|
||||
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
|
||||
go broker.start(ctx)
|
||||
var (
|
||||
errC = make(chan error, 1)
|
||||
shardC = make(chan types.Shard, 1)
|
||||
)
|
||||
|
||||
go func() {
|
||||
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) {
|
||||
if err := c.ScanShard(ctx, shardID, fn); err != nil {
|
||||
s.addShard(shardID)
|
||||
defer func() {
|
||||
s.deleteShard(shardID)
|
||||
}()
|
||||
defer wg.Done()
|
||||
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 occured
|
||||
}
|
||||
}
|
||||
}(aws.StringValue(shard.ShardId))
|
||||
}(shardId)
|
||||
}
|
||||
|
||||
close(errc)
|
||||
go func() {
|
||||
wg.Wait()
|
||||
close(errC)
|
||||
}()
|
||||
|
||||
return <-errc
|
||||
return <-errC
|
||||
}
|
||||
|
||||
// ScanShard loops over records on a specific shard, calls the callback func
|
||||
// for each record and checkpoints the progress of scan.
|
||||
func (c *Consumer) ScanShard(ctx context.Context, shardID string, fn ScanFunc) error {
|
||||
// get last seq number from checkpoint
|
||||
lastSeqNum, err := c.checkpoint.Get(c.streamName, shardID)
|
||||
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(c.streamName, shardID, lastSeqNum)
|
||||
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("[START]\t", shardID, lastSeqNum)
|
||||
c.logger.Log("[CONSUMER] start scan:", shardID, lastSeqNum)
|
||||
defer func() {
|
||||
c.logger.Log("[STOP]\t", shardID)
|
||||
c.logger.Log("[CONSUMER] stop scan:", shardID)
|
||||
}()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil
|
||||
default:
|
||||
resp, err := c.client.GetRecords(&kinesis.GetRecordsInput{
|
||||
ShardIterator: shardIterator,
|
||||
})
|
||||
scanTicker := time.NewTicker(c.scanInterval)
|
||||
defer scanTicker.Stop()
|
||||
|
||||
// attempt to recover from GetRecords error by getting new shard iterator
|
||||
if err != nil {
|
||||
shardIterator, err = c.getShardIterator(c.streamName, shardID, lastSeqNum)
|
||||
if err != nil {
|
||||
return fmt.Errorf("get shard iterator error: %v", err)
|
||||
}
|
||||
continue
|
||||
for {
|
||||
resp, err := c.client.GetRecords(ctx, &kinesis.GetRecordsInput{
|
||||
Limit: aws.Int32(int32(c.maxRecords)),
|
||||
ShardIterator: shardIterator,
|
||||
})
|
||||
|
||||
// attempt to recover from GetRecords error
|
||||
if err != nil {
|
||||
c.logger.Log("[CONSUMER] get records error:", err.Error())
|
||||
|
||||
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 != SkipCheckpoint {
|
||||
err := fn(&Record{r, shardID, resp.MillisBehindLatest})
|
||||
if err != nil && !errors.Is(err, ErrSkipCheckpoint) {
|
||||
return err
|
||||
}
|
||||
|
||||
if err != SkipCheckpoint {
|
||||
if err := c.checkpoint.Set(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)
|
||||
|
|
@ -175,32 +233,102 @@ func (c *Consumer) ScanShard(ctx context.Context, shardID string, fn ScanFunc) e
|
|||
}
|
||||
|
||||
if isShardClosed(resp.NextShardIterator, shardIterator) {
|
||||
c.logger.Log("[CLOSED]\t", shardID)
|
||||
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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func isShardClosed(nextShardIterator, currentShardIterator *string) bool {
|
||||
return nextShardIterator == nil || currentShardIterator == nextShardIterator
|
||||
// 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)
|
||||
}
|
||||
|
||||
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(streamName, shardID, seqNum string) (*string, error) {
|
||||
func (c *Consumer) getShardIterator(ctx context.Context, streamName, shardID, seqNum string) (*string, error) {
|
||||
params := &kinesis.GetShardIteratorInput{
|
||||
ShardId: aws.String(shardID),
|
||||
StreamName: aws.String(streamName),
|
||||
}
|
||||
|
||||
if seqNum != "" {
|
||||
params.ShardIteratorType = aws.String(kinesis.ShardIteratorTypeAfterSequenceNumber)
|
||||
params.ShardIteratorType = types.ShardIteratorTypeAfterSequenceNumber
|
||||
params.StartingSequenceNumber = aws.String(seqNum)
|
||||
} else if c.initialTimestamp != nil {
|
||||
params.ShardIteratorType = types.ShardIteratorTypeAtTimestamp
|
||||
params.Timestamp = c.initialTimestamp
|
||||
} else {
|
||||
params.ShardIteratorType = aws.String(c.initialShardIteratorType)
|
||||
params.ShardIteratorType = c.initialShardIteratorType
|
||||
}
|
||||
|
||||
res, err := c.client.GetShardIterator(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)
|
||||
}
|
||||
|
|
|
|||
447
consumer_test.go
447
consumer_test.go
|
|
@ -2,16 +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/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"
|
||||
|
||||
store "github.com/harlow/kinesis-consumer/store/memory"
|
||||
)
|
||||
|
||||
var records = []*kinesis.Record{
|
||||
var records = []types.Record{
|
||||
{
|
||||
Data: []byte("firstData"),
|
||||
SequenceNumber: aws.String("firstSeqNum"),
|
||||
|
|
@ -22,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)
|
||||
|
|
@ -30,34 +44,35 @@ 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
|
||||
},
|
||||
}
|
||||
var (
|
||||
cp = &fakeCheckpoint{cache: map[string]string{}}
|
||||
cp = store.New()
|
||||
ctr = &fakeCounter{}
|
||||
)
|
||||
|
||||
c, err := New("myStreamName",
|
||||
WithClient(client),
|
||||
WithCounter(ctr),
|
||||
WithCheckpoint(cp),
|
||||
WithStore(cp),
|
||||
WithLogger(&testLogger{t}),
|
||||
)
|
||||
if err != nil {
|
||||
t.Fatalf("new consumer error: %v", err)
|
||||
|
|
@ -90,20 +105,85 @@ func TestScan(t *testing.T) {
|
|||
t.Errorf("counter error expected %d, got %d", 2, val)
|
||||
}
|
||||
|
||||
val, err := cp.Get("myStreamName", "myShard")
|
||||
val, err := cp.GetCheckpoint("myStreamName", "myShard")
|
||||
if err != nil && val != "lastSeqNum" {
|
||||
t.Errorf("checkout error expected %s, got %s", "lastSeqNum", val)
|
||||
}
|
||||
}
|
||||
|
||||
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,
|
||||
|
|
@ -112,14 +192,15 @@ func TestScanShard(t *testing.T) {
|
|||
}
|
||||
|
||||
var (
|
||||
cp = &fakeCheckpoint{cache: map[string]string{}}
|
||||
cp = store.New()
|
||||
ctr = &fakeCounter{}
|
||||
)
|
||||
|
||||
c, err := New("myStreamName",
|
||||
WithClient(client),
|
||||
WithCounter(ctr),
|
||||
WithCheckpoint(cp),
|
||||
WithStore(cp),
|
||||
WithLogger(&testLogger{t}),
|
||||
)
|
||||
if err != nil {
|
||||
t.Fatalf("new consumer error: %v", err)
|
||||
|
|
@ -156,7 +237,7 @@ func TestScanShard(t *testing.T) {
|
|||
}
|
||||
|
||||
// sets checkpoint
|
||||
val, err := cp.Get("myStreamName", "myShard")
|
||||
val, err := cp.GetCheckpoint("myStreamName", "myShard")
|
||||
if err != nil && val != "lastSeqNum" {
|
||||
t.Fatalf("checkout error expected %s, got %s", "lastSeqNum", val)
|
||||
}
|
||||
|
|
@ -164,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,
|
||||
|
|
@ -204,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,
|
||||
|
|
@ -217,9 +298,9 @@ func TestScanShard_SkipCheckpoint(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
var cp = &fakeCheckpoint{cache: map[string]string{}}
|
||||
var cp = store.New()
|
||||
|
||||
c, err := New("myStreamName", WithClient(client), WithCheckpoint(cp))
|
||||
c, err := New("myStreamName", WithClient(client), WithStore(cp))
|
||||
if err != nil {
|
||||
t.Fatalf("new consumer error: %v", err)
|
||||
}
|
||||
|
|
@ -227,9 +308,9 @@ 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 SkipCheckpoint
|
||||
return ErrSkipCheckpoint
|
||||
}
|
||||
|
||||
return nil
|
||||
|
|
@ -240,7 +321,7 @@ func TestScanShard_SkipCheckpoint(t *testing.T) {
|
|||
t.Fatalf("scan shard error: %v", err)
|
||||
}
|
||||
|
||||
val, err := cp.Get("myStreamName", "myShard")
|
||||
val, err := cp.GetCheckpoint("myStreamName", "myShard")
|
||||
if err != nil && val != "firstSeqNum" {
|
||||
t.Fatalf("checkout error expected %s, got %s", "firstSeqNum", val)
|
||||
}
|
||||
|
|
@ -248,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
|
||||
},
|
||||
}
|
||||
|
|
@ -276,46 +357,92 @@ func TestScanShard_ShardIsClosed(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestScanShard_ShardIsClosed_WithShardClosedHandler(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: make([]types.Record, 0),
|
||||
}, nil
|
||||
},
|
||||
}
|
||||
|
||||
var fn = func(r *Record) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
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 == 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)
|
||||
}
|
||||
|
||||
// implementation of checkpoint
|
||||
type fakeCheckpoint struct {
|
||||
cache map[string]string
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
func (fc *fakeCheckpoint) Set(streamName, shardID, sequenceNumber string) error {
|
||||
fc.mu.Lock()
|
||||
defer fc.mu.Unlock()
|
||||
|
||||
key := fmt.Sprintf("%s-%s", streamName, shardID)
|
||||
fc.cache[key] = sequenceNumber
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fc *fakeCheckpoint) Get(streamName, shardID string) (string, error) {
|
||||
fc.mu.Lock()
|
||||
defer fc.mu.Unlock()
|
||||
|
||||
key := fmt.Sprintf("%s-%s", streamName, shardID)
|
||||
return fc.cache[key], nil
|
||||
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
|
||||
|
|
@ -337,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)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
7
examples/consumer-dynamodb/README.md
Normal file
7
examples/consumer-dynamodb/README.md
Normal file
|
|
@ -0,0 +1,7 @@
|
|||
# Consumer
|
||||
|
||||
Read records from the Kinesis stream
|
||||
|
||||
### Run the consumer
|
||||
|
||||
$ go run main.go --app appName --stream streamName --table tableName
|
||||
207
examples/consumer-dynamodb/main.go
Normal file
207
examples/consumer-dynamodb/main.go
Normal 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")),
|
||||
)
|
||||
}
|
||||
|
|
@ -2,16 +2,6 @@
|
|||
|
||||
Read records from the Kinesis stream using mysql as checkpoint
|
||||
|
||||
## Environment Variables
|
||||
|
||||
Export the required environment vars for connecting to the Kinesis stream:
|
||||
|
||||
```shell
|
||||
export AWS_ACCESS_KEY=
|
||||
export AWS_REGION=
|
||||
export AWS_SECRET_KEY=
|
||||
```
|
||||
|
||||
## Run the consumer
|
||||
|
||||
go run main.go --app <appName> --stream <streamName> --table <tableName> --connection <connectionString>
|
||||
95
examples/consumer-mysql/main.go
Normal file
95
examples/consumer-mysql/main.go
Normal file
|
|
@ -0,0 +1,95 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"expvar"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
func main() {
|
||||
var (
|
||||
app = flag.String("app", "", "Consumer app name")
|
||||
stream = flag.String("stream", "", "Stream name")
|
||||
table = flag.String("table", "", "Table name")
|
||||
connStr = flag.String("connection", "", "Connection Str")
|
||||
kinesisEndpoint = flag.String("endpoint", "http://localhost:4567", "Kinesis endpoint")
|
||||
awsRegion = flag.String("region", "us-west-2", "AWS Region")
|
||||
)
|
||||
flag.Parse()
|
||||
|
||||
// mysql checkpoint
|
||||
store, err := store.New(*app, *table, *connStr)
|
||||
if err != nil {
|
||||
log.Fatalf("checkpoint error: %v", err)
|
||||
}
|
||||
|
||||
var counter = expvar.NewMap("counters")
|
||||
|
||||
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
|
||||
return aws.Endpoint{
|
||||
PartitionID: "aws",
|
||||
URL: *kinesisEndpoint,
|
||||
SigningRegion: *awsRegion,
|
||||
}, nil
|
||||
})
|
||||
|
||||
// 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(
|
||||
*stream,
|
||||
consumer.WithClient(client),
|
||||
consumer.WithStore(store),
|
||||
consumer.WithCounter(counter),
|
||||
)
|
||||
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
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
log.Fatalf("scan error: %v", err)
|
||||
}
|
||||
|
||||
if err := store.Shutdown(); err != nil {
|
||||
log.Fatalf("store shutdown error: %v", err)
|
||||
}
|
||||
}
|
||||
|
|
@ -2,16 +2,6 @@
|
|||
|
||||
Read records from the Kinesis stream using postgres as checkpoint
|
||||
|
||||
## Environment Variables
|
||||
|
||||
Export the required environment vars for connecting to the Kinesis stream:
|
||||
|
||||
```shell
|
||||
export AWS_ACCESS_KEY=
|
||||
export AWS_REGION=
|
||||
export AWS_SECRET_KEY=
|
||||
```
|
||||
|
||||
## Run the consumer
|
||||
|
||||
go run main.go --app appName --stream streamName --table tableName --connection connectionString
|
||||
95
examples/consumer-postgres/main.go
Normal file
95
examples/consumer-postgres/main.go
Normal file
|
|
@ -0,0 +1,95 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"expvar"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
func main() {
|
||||
var (
|
||||
app = flag.String("app", "", "Consumer app name")
|
||||
stream = flag.String("stream", "", "Stream name")
|
||||
table = flag.String("table", "", "Table name")
|
||||
connStr = flag.String("connection", "", "Connection Str")
|
||||
kinesisEndpoint = flag.String("endpoint", "http://localhost:4567", "Kinesis endpoint")
|
||||
awsRegion = flag.String("region", "us-west-2", "AWS Region")
|
||||
)
|
||||
flag.Parse()
|
||||
|
||||
// postgres checkpoint
|
||||
store, err := store.New(*app, *table, *connStr)
|
||||
if err != nil {
|
||||
log.Fatalf("checkpoint error: %v", err)
|
||||
}
|
||||
|
||||
var counter = expvar.NewMap("counters")
|
||||
|
||||
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
|
||||
return aws.Endpoint{
|
||||
PartitionID: "aws",
|
||||
URL: *kinesisEndpoint,
|
||||
SigningRegion: *awsRegion,
|
||||
}, nil
|
||||
})
|
||||
|
||||
// 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(
|
||||
*stream,
|
||||
consumer.WithClient(client),
|
||||
consumer.WithStore(store),
|
||||
consumer.WithCounter(counter),
|
||||
)
|
||||
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 := store.Shutdown(); err != nil {
|
||||
log.Fatalf("store shutdown error: %v", err)
|
||||
}
|
||||
}
|
||||
7
examples/consumer-redis/README.md
Normal file
7
examples/consumer-redis/README.md
Normal file
|
|
@ -0,0 +1,7 @@
|
|||
# Consumer
|
||||
|
||||
Read records from the Kinesis stream
|
||||
|
||||
### Run the consumer
|
||||
|
||||
$ go run main.go --app appName --stream streamName
|
||||
101
examples/consumer-redis/main.go
Normal file
101
examples/consumer-redis/main.go
Normal file
|
|
@ -0,0 +1,101 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
// A myLogger provides a minimalistic logger satisfying the Logger interface.
|
||||
type myLogger struct {
|
||||
logger *log.Logger
|
||||
}
|
||||
|
||||
// Log logs the parameters to the stdlib logger. See log.Println.
|
||||
func (l *myLogger) Log(args ...interface{}) {
|
||||
l.logger.Println(args...)
|
||||
}
|
||||
|
||||
func main() {
|
||||
var (
|
||||
app = flag.String("app", "", "Consumer app name")
|
||||
stream = 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()
|
||||
|
||||
// redis checkpoint store
|
||||
store, err := store.New(*app)
|
||||
if err != nil {
|
||||
log.Fatalf("store error: %v", err)
|
||||
}
|
||||
|
||||
// logger
|
||||
logger := &myLogger{
|
||||
logger: log.New(os.Stdout, "consumer-example: ", log.LstdFlags),
|
||||
}
|
||||
|
||||
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
|
||||
return aws.Endpoint{
|
||||
PartitionID: "aws",
|
||||
URL: *kinesisEndpoint,
|
||||
SigningRegion: *awsRegion,
|
||||
}, nil
|
||||
})
|
||||
|
||||
// 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(
|
||||
*stream,
|
||||
consumer.WithClient(client),
|
||||
consumer.WithStore(store),
|
||||
consumer.WithLogger(logger),
|
||||
)
|
||||
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
|
||||
fmt.Println("caught exit signal, cancelling context!")
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
|
@ -1,17 +0,0 @@
|
|||
# Consumer
|
||||
|
||||
Read records from the Kinesis stream
|
||||
|
||||
### Environment Variables
|
||||
|
||||
Export the required environment vars for connecting to the Kinesis stream:
|
||||
|
||||
```
|
||||
export AWS_ACCESS_KEY=
|
||||
export AWS_REGION=
|
||||
export AWS_SECRET_KEY=
|
||||
```
|
||||
|
||||
### Run the consumer
|
||||
|
||||
$ go run main.go --app appName --stream streamName --table tableName
|
||||
|
|
@ -1,135 +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"
|
||||
checkpoint "github.com/harlow/kinesis-consumer/checkpoint/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")
|
||||
)
|
||||
flag.Parse()
|
||||
|
||||
sess, err := session.NewSession(aws.NewConfig())
|
||||
if err != nil {
|
||||
log.Log("new session error: %v", err)
|
||||
}
|
||||
|
||||
// New Kinesis and DynamoDB clients (if you need custom config)
|
||||
myKsis := kinesis.New(sess)
|
||||
myDdbClient := dynamodb.New(sess)
|
||||
|
||||
// ddb checkpoint
|
||||
ck, err := checkpoint.New(*app, *table, checkpoint.WithDynamoClient(myDdbClient), checkpoint.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.WithCheckpoint(ck),
|
||||
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 := ck.Shutdown(); err != nil {
|
||||
log.Log("checkpoint shutdown error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// MyRetryer used for checkpointing
|
||||
type MyRetryer struct {
|
||||
checkpoint.Retryer
|
||||
}
|
||||
|
||||
// ShouldRetry implements custom logic for when a checkpont 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
|
||||
}
|
||||
|
|
@ -1,68 +0,0 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"expvar"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
|
||||
consumer "github.com/harlow/kinesis-consumer"
|
||||
checkpoint "github.com/harlow/kinesis-consumer/checkpoint/mysql"
|
||||
)
|
||||
|
||||
func main() {
|
||||
var (
|
||||
app = flag.String("app", "", "Consumer app name")
|
||||
stream = flag.String("stream", "", "Stream name")
|
||||
table = flag.String("table", "", "Table name")
|
||||
connStr = flag.String("connection", "", "Connection Str")
|
||||
)
|
||||
flag.Parse()
|
||||
|
||||
// mysql checkpoint
|
||||
ck, err := checkpoint.New(*app, *table, *connStr)
|
||||
if err != nil {
|
||||
log.Fatalf("checkpoint error: %v", err)
|
||||
}
|
||||
|
||||
var counter = expvar.NewMap("counters")
|
||||
|
||||
// consumer
|
||||
c, err := consumer.New(
|
||||
*stream,
|
||||
consumer.WithCheckpoint(ck),
|
||||
consumer.WithCounter(counter),
|
||||
)
|
||||
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
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
log.Fatalf("scan error: %v", err)
|
||||
}
|
||||
|
||||
if err := ck.Shutdown(); err != nil {
|
||||
log.Fatalf("checkpoint shutdown error: %v", err)
|
||||
}
|
||||
}
|
||||
|
|
@ -1,68 +0,0 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"expvar"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
|
||||
consumer "github.com/harlow/kinesis-consumer"
|
||||
checkpoint "github.com/harlow/kinesis-consumer/checkpoint/postgres"
|
||||
)
|
||||
|
||||
func main() {
|
||||
var (
|
||||
app = flag.String("app", "", "Consumer app name")
|
||||
stream = flag.String("stream", "", "Stream name")
|
||||
table = flag.String("table", "", "Table name")
|
||||
connStr = flag.String("connection", "", "Connection Str")
|
||||
)
|
||||
flag.Parse()
|
||||
|
||||
// postgres checkpoint
|
||||
ck, err := checkpoint.New(*app, *table, *connStr)
|
||||
if err != nil {
|
||||
log.Fatalf("checkpoint error: %v", err)
|
||||
}
|
||||
|
||||
var counter = expvar.NewMap("counters")
|
||||
|
||||
// consumer
|
||||
c, err := consumer.New(
|
||||
*stream,
|
||||
consumer.WithCheckpoint(ck),
|
||||
consumer.WithCounter(counter),
|
||||
)
|
||||
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 := ck.Shutdown(); err != nil {
|
||||
log.Fatalf("checkpoint shutdown error: %v", err)
|
||||
}
|
||||
}
|
||||
|
|
@ -1,17 +0,0 @@
|
|||
# Consumer
|
||||
|
||||
Read records from the Kinesis stream
|
||||
|
||||
### Environment Variables
|
||||
|
||||
Export the required environment vars for connecting to the Kinesis stream and Redis for checkpoint:
|
||||
|
||||
```
|
||||
export AWS_PROFILE=
|
||||
export AWS_REGION=
|
||||
export REDIS_URL=
|
||||
```
|
||||
|
||||
### Run the consumer
|
||||
|
||||
$ go run main.go --app appName --stream streamName
|
||||
|
|
@ -1,74 +0,0 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
|
||||
consumer "github.com/harlow/kinesis-consumer"
|
||||
checkpoint "github.com/harlow/kinesis-consumer/checkpoint/redis"
|
||||
)
|
||||
|
||||
// A myLogger provides a minimalistic logger satisfying the Logger interface.
|
||||
type myLogger struct {
|
||||
logger *log.Logger
|
||||
}
|
||||
|
||||
// Log logs the parameters to the stdlib logger. See log.Println.
|
||||
func (l *myLogger) Log(args ...interface{}) {
|
||||
l.logger.Println(args...)
|
||||
}
|
||||
|
||||
func main() {
|
||||
var (
|
||||
app = flag.String("app", "", "Consumer app name")
|
||||
stream = flag.String("stream", "", "Stream name")
|
||||
)
|
||||
flag.Parse()
|
||||
|
||||
// redis checkpoint
|
||||
ck, err := checkpoint.New(*app)
|
||||
if err != nil {
|
||||
log.Fatalf("checkpoint error: %v", err)
|
||||
}
|
||||
|
||||
// logger
|
||||
logger := &myLogger{
|
||||
logger: log.New(os.Stdout, "consumer-example: ", log.LstdFlags),
|
||||
}
|
||||
|
||||
// consumer
|
||||
c, err := consumer.New(
|
||||
*stream,
|
||||
consumer.WithCheckpoint(ck),
|
||||
consumer.WithLogger(logger),
|
||||
)
|
||||
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
|
||||
fmt.Println("caught exit signal, cancelling context!")
|
||||
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)
|
||||
}
|
||||
}
|
||||
89
examples/consumer/main.go
Normal file
89
examples/consumer/main.go
Normal file
|
|
@ -0,0 +1,89 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"flag"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"os/signal"
|
||||
"syscall"
|
||||
|
||||
"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"
|
||||
)
|
||||
|
||||
// A myLogger provides a minimalistic logger satisfying the Logger interface.
|
||||
type myLogger struct {
|
||||
logger *log.Logger
|
||||
}
|
||||
|
||||
// Log logs the parameters to the stdlib logger. See log.Println.
|
||||
func (l *myLogger) Log(args ...interface{}) {
|
||||
l.logger.Println(args...)
|
||||
}
|
||||
|
||||
func main() {
|
||||
var (
|
||||
stream = 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()
|
||||
|
||||
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
|
||||
return aws.Endpoint{
|
||||
PartitionID: "aws",
|
||||
URL: *kinesisEndpoint,
|
||||
SigningRegion: *awsRegion,
|
||||
}, nil
|
||||
})
|
||||
|
||||
// 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(
|
||||
*stream,
|
||||
consumer.WithClient(client),
|
||||
)
|
||||
if err != nil {
|
||||
log.Fatalf("consumer error: %v", err)
|
||||
}
|
||||
|
||||
// scan
|
||||
ctx := trap()
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
func trap() context.Context {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
sigs := make(chan os.Signal, 1)
|
||||
signal.Notify(sigs, os.Interrupt, syscall.SIGTERM, syscall.SIGQUIT)
|
||||
|
||||
go func() {
|
||||
sig := <-sigs
|
||||
log.Printf("received %s", sig)
|
||||
cancel()
|
||||
}()
|
||||
|
||||
return ctx
|
||||
}
|
||||
|
|
@ -2,15 +2,6 @@
|
|||
|
||||
A prepopulated file with JSON users is available on S3 for seeing the stream.
|
||||
|
||||
### Environment Variables
|
||||
## Running the code
|
||||
|
||||
Export the required environment vars for connecting to the Kinesis stream:
|
||||
|
||||
```
|
||||
export AWS_PROFILE=
|
||||
export AWS_REGION_NAME=
|
||||
```
|
||||
|
||||
### Running the code
|
||||
|
||||
$ go run main.go --stream streamName
|
||||
$ cat users.txt | go run main.go --stream streamName
|
||||
|
|
|
|||
|
|
@ -2,40 +2,59 @@ package main
|
|||
|
||||
import (
|
||||
"bufio"
|
||||
"context"
|
||||
"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"
|
||||
"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")
|
||||
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()
|
||||
|
||||
// open dummy user data
|
||||
f, err := os.Open("users.txt")
|
||||
if err != nil {
|
||||
log.Fatal("Cannot open users.txt file")
|
||||
}
|
||||
defer f.Close()
|
||||
var records []types.PutRecordsRequestEntry
|
||||
|
||||
var records []*kinesis.PutRecordsRequestEntry
|
||||
resolver := aws.EndpointResolverFunc(func(service, region string) (aws.Endpoint, error) {
|
||||
return aws.Endpoint{
|
||||
PartitionID: "aws",
|
||||
URL: *kinesisEndpoint,
|
||||
SigningRegion: *awsRegion,
|
||||
}, nil
|
||||
})
|
||||
|
||||
sess, err := session.NewSession(aws.NewConfig())
|
||||
cfg, err := config.LoadDefaultConfig(
|
||||
context.TODO(),
|
||||
config.WithRegion(*awsRegion),
|
||||
config.WithEndpointResolver(resolver),
|
||||
config.WithCredentialsProvider(credentials.NewStaticCredentialsProvider("user", "pass", "token")),
|
||||
)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
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)
|
||||
}
|
||||
var client = kinesis.New(sess)
|
||||
|
||||
// loop over file data
|
||||
b := bufio.NewScanner(f)
|
||||
b := bufio.NewScanner(os.Stdin)
|
||||
|
||||
for b.Scan() {
|
||||
records = append(records, &kinesis.PutRecordsRequestEntry{
|
||||
records = append(records, types.PutRecordsRequestEntry{
|
||||
Data: b.Bytes(),
|
||||
PartitionKey: aws.String(time.Now().Format(time.RFC3339Nano)),
|
||||
})
|
||||
|
|
@ -51,13 +70,47 @@ func main() {
|
|||
}
|
||||
}
|
||||
|
||||
func putRecords(client *kinesis.Kinesis, streamName *string, records []*kinesis.PutRecordsRequestEntry) {
|
||||
_, err := client.PutRecords(&kinesis.PutRecordsInput{
|
||||
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(".")
|
||||
}
|
||||
|
|
|
|||
30
go.mod
30
go.mod
|
|
@ -1,13 +1,25 @@
|
|||
module github.com/harlow/kinesis-consumer
|
||||
|
||||
require (
|
||||
github.com/apex/log v1.0.0
|
||||
github.com/aws/aws-sdk-go v1.15.0
|
||||
github.com/go-ini/ini v1.38.1
|
||||
github.com/go-sql-driver/mysql v1.4.1
|
||||
github.com/jmespath/go-jmespath v0.0.0-20160202185014-0b12d6b521d8
|
||||
github.com/lib/pq v0.0.0-20180523175426-90697d60dd84
|
||||
github.com/pkg/errors v0.8.0
|
||||
gopkg.in/DATA-DOG/go-sqlmock.v1 v1.3.0
|
||||
gopkg.in/redis.v5 v5.2.9
|
||||
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.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 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
|
||||
|
|
|
|||
297
go.sum
297
go.sum
|
|
@ -1,19 +1,278 @@
|
|||
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/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-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/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/pkg/errors v0.8.0 h1:WdK/asTD0HN+q6hsWO3/vpuAkAr+tw6aNJNDFFf0+qw=
|
||||
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||
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=
|
||||
gopkg.in/redis.v5 v5.2.9 h1:MNZYOLPomQzZMfpN3ZtD1uyJ2IDonTTlxYiV/pEApiw=
|
||||
gopkg.in/redis.v5 v5.2.9/go.mod h1:6gtv0/+A4iM08kdRfocWYB3bLX2tebpNtfKlFT6H4mY=
|
||||
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.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/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/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/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/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.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/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/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/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-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-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
|
||||
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-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/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-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/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/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/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=
|
||||
|
|
|
|||
21
group.go
Normal file
21
group.go
Normal file
|
|
@ -0,0 +1,21 @@
|
|||
package consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"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 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
|
||||
}
|
||||
6
internal/deaggregator/README.md
Normal file
6
internal/deaggregator/README.md
Normal 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.
|
||||
94
internal/deaggregator/deaggregator.go
Normal file
94
internal/deaggregator/deaggregator.go
Normal 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,
|
||||
}
|
||||
}
|
||||
202
internal/deaggregator/deaggregator_test.go
Normal file
202
internal/deaggregator/deaggregator_test.go
Normal 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: ¤tTime,
|
||||
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.")
|
||||
}
|
||||
34
kinesis.go
Normal file
34
kinesis.go
Normal file
|
|
@ -0,0 +1,34 @@
|
|||
package consumer
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"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(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(ctx, listShardsInput)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("ListShards error: %w", err)
|
||||
}
|
||||
ss = append(ss, resp.Shards...)
|
||||
|
||||
if resp.NextToken == nil {
|
||||
return ss, nil
|
||||
}
|
||||
|
||||
listShardsInput = &kinesis.ListShardsInput{
|
||||
NextToken: resp.NextToken,
|
||||
}
|
||||
}
|
||||
}
|
||||
65
options.go
65
options.go
|
|
@ -1,14 +1,25 @@
|
|||
package consumer
|
||||
|
||||
import "github.com/aws/aws-sdk-go/service/kinesis/kinesisiface"
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/aws/aws-sdk-go-v2/service/kinesis/types"
|
||||
)
|
||||
|
||||
// Option is used to override defaults when creating a new Consumer
|
||||
type Option func(*Consumer)
|
||||
|
||||
// WithCheckpoint overrides the default checkpoint
|
||||
func WithCheckpoint(checkpoint Checkpoint) Option {
|
||||
// WithGroup overrides the default storage
|
||||
func WithGroup(group Group) Option {
|
||||
return func(c *Consumer) {
|
||||
c.checkpoint = checkpoint
|
||||
c.group = group
|
||||
}
|
||||
}
|
||||
|
||||
// WithStore overrides the default storage
|
||||
func WithStore(store Store) Option {
|
||||
return func(c *Consumer) {
|
||||
c.store = store
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -27,15 +38,55 @@ 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
|
||||
}
|
||||
}
|
||||
|
||||
// ShardIteratorType overrides the starting point for the consumer
|
||||
// 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)
|
||||
}
|
||||
}
|
||||
|
||||
// WithTimestamp overrides the starting point for the consumer
|
||||
func WithTimestamp(t time.Time) Option {
|
||||
return func(c *Consumer) {
|
||||
c.initialTimestamp = &t
|
||||
}
|
||||
}
|
||||
|
||||
// WithScanInterval overrides the scan interval for the consumer
|
||||
func WithScanInterval(d time.Duration) Option {
|
||||
return func(c *Consumer) {
|
||||
c.scanInterval = d
|
||||
}
|
||||
}
|
||||
|
||||
// WithMaxRecords overrides the maximum number of records to be
|
||||
// returned in a single GetRecords call for the consumer (specify a
|
||||
// value of up to 10,000)
|
||||
func WithMaxRecords(n int64) Option {
|
||||
return func(c *Consumer) {
|
||||
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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
13
store.go
Normal file
13
store.go
Normal file
|
|
@ -0,0 +1,13 @@
|
|||
package consumer
|
||||
|
||||
// Store interface used to persist scan progress
|
||||
type Store interface {
|
||||
GetCheckpoint(streamName, shardID string) (string, error)
|
||||
SetCheckpoint(streamName, shardID, sequenceNumber string) error
|
||||
}
|
||||
|
||||
// noopStore implements the storage interface with discard
|
||||
type noopStore struct{}
|
||||
|
||||
func (n noopStore) GetCheckpoint(string, string) (string, error) { return "", nil }
|
||||
func (n noopStore) SetCheckpoint(string, string, string) error { return nil }
|
||||
|
|
@ -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,12 +40,9 @@ func WithRetryer(r Retryer) Option {
|
|||
|
||||
// New returns a checkpoint that uses DynamoDB for underlying storage
|
||||
func New(appName, tableName string, opts ...Option) (*Checkpoint, error) {
|
||||
client := dynamodb.New(session.New(aws.NewConfig()))
|
||||
|
||||
ck := &Checkpoint{
|
||||
tableName: tableName,
|
||||
appName: appName,
|
||||
client: client,
|
||||
maxInterval: time.Duration(1 * time.Minute),
|
||||
done: make(chan struct{}),
|
||||
mu: &sync.Mutex{},
|
||||
|
|
@ -56,6 +54,15 @@ func New(appName, tableName string, opts ...Option) (*Checkpoint, error) {
|
|||
opt(ck)
|
||||
}
|
||||
|
||||
// default client
|
||||
if ck.client == nil {
|
||||
cfg, err := config.LoadDefaultConfig(context.TODO())
|
||||
if err != nil {
|
||||
log.Fatalf("unable to load SDK config, %v", err)
|
||||
}
|
||||
ck.client = dynamodb.NewFromConfig(cfg)
|
||||
}
|
||||
|
||||
go ck.loop()
|
||||
|
||||
return ck, nil
|
||||
|
|
@ -65,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
|
||||
|
|
@ -74,51 +81,47 @@ 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"`
|
||||
}
|
||||
|
||||
// Get determines if a checkpoint for a particular Shard exists.
|
||||
// GetCheckpoint determines if a checkpoint for a particular Shard exists.
|
||||
// Typically used to determine whether we should start processing the shard with
|
||||
// TRIM_HORIZON or AFTER_SEQUENCE_NUMBER (if checkpoint exists).
|
||||
func (c *Checkpoint) Get(streamName, shardID string) (string, error) {
|
||||
func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
|
||||
namespace := fmt.Sprintf("%s-%s", c.appName, streamName)
|
||||
|
||||
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.Get(streamName, shardID)
|
||||
return c.GetCheckpoint(streamName, shardID)
|
||||
}
|
||||
return "", err
|
||||
}
|
||||
|
||||
var i item
|
||||
dynamodbattribute.UnmarshalMap(resp.Item, &i)
|
||||
attributevalue.UnmarshalMap(resp.Item, &i)
|
||||
return i.SequenceNumber, nil
|
||||
}
|
||||
|
||||
// Set stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// SetCheckpoint stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// Upon failover, record processing is resumed from this point.
|
||||
func (c *Checkpoint) Set(streamName, shardID, sequenceNumber string) error {
|
||||
func (c *Checkpoint) SetCheckpoint(streamName, shardID, sequenceNumber string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
|
|
@ -127,8 +130,8 @@ func (c *Checkpoint) Set(streamName, shardID, sequenceNumber string) error {
|
|||
}
|
||||
|
||||
key := key{
|
||||
streamName: streamName,
|
||||
shardID: shardID,
|
||||
StreamName: streamName,
|
||||
ShardID: shardID,
|
||||
}
|
||||
c.checkpoints[key] = sequenceNumber
|
||||
|
||||
|
|
@ -161,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 {
|
||||
|
|
@ -171,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
|
||||
|
|
@ -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)
|
||||
|
|
@ -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
|
||||
|
|
@ -10,15 +9,16 @@ type Retryer interface {
|
|||
ShouldRetry(error) bool
|
||||
}
|
||||
|
||||
// DefaultRetryer .
|
||||
type DefaultRetryer struct {
|
||||
Retryer
|
||||
}
|
||||
|
||||
// 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
|
||||
}
|
||||
|
|
@ -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)
|
||||
33
store/memory/store.go
Normal file
33
store/memory/store.go
Normal file
|
|
@ -0,0 +1,33 @@
|
|||
// The memory store provides a store that can be used for testing and single-threaded applications.
|
||||
// DO NOT USE this in a production application where persistence beyond a single application lifecycle is necessary
|
||||
// or when there are multiple consumers.
|
||||
package store
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
)
|
||||
|
||||
func New() *Store {
|
||||
return &Store{}
|
||||
}
|
||||
|
||||
type Store struct {
|
||||
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
|
||||
}
|
||||
|
||||
func (c *Store) GetCheckpoint(streamName, shardID string) (string, error) {
|
||||
val, ok := c.Load(streamName + ":" + shardID)
|
||||
if !ok {
|
||||
return "", nil
|
||||
}
|
||||
return val.(string), nil
|
||||
}
|
||||
30
store/memory/store_test.go
Normal file
30
store/memory/store_test.go
Normal file
|
|
@ -0,0 +1,30 @@
|
|||
package store
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func Test_CheckpointLifecycle(t *testing.T) {
|
||||
c := New()
|
||||
|
||||
// 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)
|
||||
}
|
||||
}
|
||||
|
||||
func Test_SetEmptySeqNum(t *testing.T) {
|
||||
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")
|
||||
}
|
||||
}
|
||||
|
|
@ -77,10 +77,10 @@ func (c *Checkpoint) GetMaxInterval() time.Duration {
|
|||
return c.maxInterval
|
||||
}
|
||||
|
||||
// Get determines if a checkpoint for a particular Shard exists.
|
||||
// GetCheckpoint determines if a checkpoint for a particular Shard exists.
|
||||
// Typically used to determine whether we should start processing the shard with
|
||||
// TRIM_HORIZON or AFTER_SEQUENCE_NUMBER (if checkpoint exists).
|
||||
func (c *Checkpoint) Get(streamName, shardID string) (string, error) {
|
||||
func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
|
||||
namespace := fmt.Sprintf("%s-%s", c.appName, streamName)
|
||||
|
||||
var sequenceNumber string
|
||||
|
|
@ -97,9 +97,9 @@ func (c *Checkpoint) Get(streamName, shardID string) (string, error) {
|
|||
return sequenceNumber, nil
|
||||
}
|
||||
|
||||
// Set stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// SetCheckpoint stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// Upon failover, record processing is resumed from this point.
|
||||
func (c *Checkpoint) Set(streamName, shardID, sequenceNumber string) error {
|
||||
func (c *Checkpoint) SetCheckpoint(streamName, shardID, sequenceNumber string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
|
|
@ -6,8 +6,8 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
sqlmock "github.com/DATA-DOG/go-sqlmock"
|
||||
"github.com/pkg/errors"
|
||||
sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
|
||||
)
|
||||
|
||||
func TestNew(t *testing.T) {
|
||||
|
|
@ -72,7 +72,7 @@ func TestNew_WithMaxIntervalOption(t *testing.T) {
|
|||
ck.Shutdown()
|
||||
}
|
||||
|
||||
func TestCheckpoint_Get(t *testing.T) {
|
||||
func TestCheckpoint_GetCheckpoint(t *testing.T) {
|
||||
appName := "streamConsumer"
|
||||
tableName := "checkpoint"
|
||||
connString := "user:password@/dbname"
|
||||
|
|
@ -98,7 +98,7 @@ func TestCheckpoint_Get(t *testing.T) {
|
|||
tableName)
|
||||
mock.ExpectQuery(expectedSQLRegexString).WithArgs(namespace, shardID).WillReturnRows(expectedRows)
|
||||
|
||||
gotSequenceNumber, err := ck.Get(streamName, shardID)
|
||||
gotSequenceNumber, err := ck.GetCheckpoint(streamName, shardID)
|
||||
|
||||
if gotSequenceNumber != expectedSequenceNumber {
|
||||
t.Errorf("expected sequence number equals %v, but got %v", expectedSequenceNumber, gotSequenceNumber)
|
||||
|
|
@ -134,7 +134,7 @@ func TestCheckpoint_Get_NoRows(t *testing.T) {
|
|||
tableName)
|
||||
mock.ExpectQuery(expectedSQLRegexString).WithArgs(namespace, shardID).WillReturnError(sql.ErrNoRows)
|
||||
|
||||
gotSequenceNumber, err := ck.Get(streamName, shardID)
|
||||
gotSequenceNumber, err := ck.GetCheckpoint(streamName, shardID)
|
||||
|
||||
if gotSequenceNumber != "" {
|
||||
t.Errorf("expected sequence number equals empty, but got %v", gotSequenceNumber)
|
||||
|
|
@ -170,7 +170,7 @@ func TestCheckpoint_Get_QueryError(t *testing.T) {
|
|||
tableName)
|
||||
mock.ExpectQuery(expectedSQLRegexString).WithArgs(namespace, shardID).WillReturnError(errors.New("an error"))
|
||||
|
||||
gotSequenceNumber, err := ck.Get(streamName, shardID)
|
||||
gotSequenceNumber, err := ck.GetCheckpoint(streamName, shardID)
|
||||
|
||||
if gotSequenceNumber != "" {
|
||||
t.Errorf("expected sequence number equals empty, but got %v", gotSequenceNumber)
|
||||
|
|
@ -184,7 +184,7 @@ func TestCheckpoint_Get_QueryError(t *testing.T) {
|
|||
ck.Shutdown()
|
||||
}
|
||||
|
||||
func TestCheckpoint_Set(t *testing.T) {
|
||||
func TestCheckpoint_SetCheckpoint(t *testing.T) {
|
||||
appName := "streamConsumer"
|
||||
tableName := "checkpoint"
|
||||
connString := "user:password@/dbname"
|
||||
|
|
@ -197,7 +197,7 @@ func TestCheckpoint_Set(t *testing.T) {
|
|||
t.Fatalf("error occurred during the checkpoint creation. cause: %v", err)
|
||||
}
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err != nil {
|
||||
t.Errorf("expected error equals nil, but got %v", err)
|
||||
|
|
@ -218,7 +218,7 @@ func TestCheckpoint_Set_SequenceNumberEmpty(t *testing.T) {
|
|||
t.Fatalf("error occurred during the checkpoint creation. cause: %v", err)
|
||||
}
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err == nil {
|
||||
t.Errorf("expected error equals not nil, but got %v", err)
|
||||
|
|
@ -249,7 +249,7 @@ func TestCheckpoint_Shutdown(t *testing.T) {
|
|||
result := sqlmock.NewResult(0, 1)
|
||||
mock.ExpectExec(expectedSQLRegexString).WithArgs(namespace, shardID, expectedSequenceNumber).WillReturnResult(result)
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("unable to set checkpoint for data initialization. cause: %v", err)
|
||||
|
|
@ -287,7 +287,7 @@ func TestCheckpoint_Shutdown_SaveError(t *testing.T) {
|
|||
expectedSQLRegexString := fmt.Sprintf(`REPLACE INTO %s \(namespace, shard_id, sequence_number\) VALUES \(\?, \?, \?\)`, tableName)
|
||||
mock.ExpectExec(expectedSQLRegexString).WithArgs(namespace, shardID, expectedSequenceNumber).WillReturnError(errors.New("an error"))
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("unable to set checkpoint for data initialization. cause: %v", err)
|
||||
|
|
@ -6,6 +6,7 @@ import (
|
|||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
// this is the postgres package so it makes sense to be here
|
||||
_ "github.com/lib/pq"
|
||||
)
|
||||
|
|
@ -77,10 +78,10 @@ func (c *Checkpoint) GetMaxInterval() time.Duration {
|
|||
return c.maxInterval
|
||||
}
|
||||
|
||||
// Get determines if a checkpoint for a particular Shard exists.
|
||||
// GetCheckpoint determines if a checkpoint for a particular Shard exists.
|
||||
// Typically used to determine whether we should start processing the shard with
|
||||
// TRIM_HORIZON or AFTER_SEQUENCE_NUMBER (if checkpoint exists).
|
||||
func (c *Checkpoint) Get(streamName, shardID string) (string, error) {
|
||||
func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
|
||||
namespace := fmt.Sprintf("%s-%s", c.appName, streamName)
|
||||
|
||||
var sequenceNumber string
|
||||
|
|
@ -97,9 +98,9 @@ func (c *Checkpoint) Get(streamName, shardID string) (string, error) {
|
|||
return sequenceNumber, nil
|
||||
}
|
||||
|
||||
// Set stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// SetCheckpoint stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// Upon failover, record processing is resumed from this point.
|
||||
func (c *Checkpoint) Set(streamName, shardID, sequenceNumber string) error {
|
||||
func (c *Checkpoint) SetCheckpoint(streamName, shardID, sequenceNumber string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
|
|
@ -6,8 +6,8 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
sqlmock "github.com/DATA-DOG/go-sqlmock"
|
||||
"github.com/pkg/errors"
|
||||
sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
|
||||
)
|
||||
|
||||
func TestNew(t *testing.T) {
|
||||
|
|
@ -72,7 +72,7 @@ func TestNew_WithMaxIntervalOption(t *testing.T) {
|
|||
ck.Shutdown()
|
||||
}
|
||||
|
||||
func TestCheckpoint_Get(t *testing.T) {
|
||||
func TestCheckpoint_GetCheckpoint(t *testing.T) {
|
||||
appName := "streamConsumer"
|
||||
tableName := "checkpoint"
|
||||
connString := "UserID=root;Password=myPassword;Host=localhost;Port=5432;Database=myDataBase;"
|
||||
|
|
@ -98,7 +98,7 @@ func TestCheckpoint_Get(t *testing.T) {
|
|||
tableName)
|
||||
mock.ExpectQuery(expectedSQLRegexString).WithArgs(namespace, shardID).WillReturnRows(expectedRows)
|
||||
|
||||
gotSequenceNumber, err := ck.Get(streamName, shardID)
|
||||
gotSequenceNumber, err := ck.GetCheckpoint(streamName, shardID)
|
||||
|
||||
if gotSequenceNumber != expectedSequenceNumber {
|
||||
t.Errorf("expected sequence number equals %v, but got %v", expectedSequenceNumber, gotSequenceNumber)
|
||||
|
|
@ -134,7 +134,7 @@ func TestCheckpoint_Get_NoRows(t *testing.T) {
|
|||
tableName)
|
||||
mock.ExpectQuery(expectedSQLRegexString).WithArgs(namespace, shardID).WillReturnError(sql.ErrNoRows)
|
||||
|
||||
gotSequenceNumber, err := ck.Get(streamName, shardID)
|
||||
gotSequenceNumber, err := ck.GetCheckpoint(streamName, shardID)
|
||||
|
||||
if gotSequenceNumber != "" {
|
||||
t.Errorf("expected sequence number equals empty, but got %v", gotSequenceNumber)
|
||||
|
|
@ -170,7 +170,7 @@ func TestCheckpoint_Get_QueryError(t *testing.T) {
|
|||
tableName)
|
||||
mock.ExpectQuery(expectedSQLRegexString).WithArgs(namespace, shardID).WillReturnError(errors.New("an error"))
|
||||
|
||||
gotSequenceNumber, err := ck.Get(streamName, shardID)
|
||||
gotSequenceNumber, err := ck.GetCheckpoint(streamName, shardID)
|
||||
|
||||
if gotSequenceNumber != "" {
|
||||
t.Errorf("expected sequence number equals empty, but got %v", gotSequenceNumber)
|
||||
|
|
@ -184,7 +184,7 @@ func TestCheckpoint_Get_QueryError(t *testing.T) {
|
|||
ck.Shutdown()
|
||||
}
|
||||
|
||||
func TestCheckpoint_Set(t *testing.T) {
|
||||
func TestCheckpoint_SetCheckpoint(t *testing.T) {
|
||||
appName := "streamConsumer"
|
||||
tableName := "checkpoint"
|
||||
connString := "UserID=root;Password=myPassword;Host=localhost;Port=5432;Database=myDataBase;"
|
||||
|
|
@ -197,7 +197,7 @@ func TestCheckpoint_Set(t *testing.T) {
|
|||
t.Fatalf("error occurred during the checkpoint creation. cause: %v", err)
|
||||
}
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err != nil {
|
||||
t.Errorf("expected error equals nil, but got %v", err)
|
||||
|
|
@ -218,7 +218,7 @@ func TestCheckpoint_Set_SequenceNumberEmpty(t *testing.T) {
|
|||
t.Fatalf("error occurred during the checkpoint creation. cause: %v", err)
|
||||
}
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err == nil {
|
||||
t.Errorf("expected error equals not nil, but got %v", err)
|
||||
|
|
@ -249,7 +249,7 @@ func TestCheckpoint_Shutdown(t *testing.T) {
|
|||
result := sqlmock.NewResult(0, 1)
|
||||
mock.ExpectExec(expectedSQLRegexString).WithArgs(namespace, shardID, expectedSequenceNumber).WillReturnResult(result)
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("unable to set checkpoint for data initialization. cause: %v", err)
|
||||
|
|
@ -287,7 +287,7 @@ func TestCheckpoint_Shutdown_SaveError(t *testing.T) {
|
|||
expectedSQLRegexString := fmt.Sprintf(`INSERT INTO %s \(namespace, shard_id, sequence_number\) VALUES\(\$1, \$2, \$3\) ON CONFLICT \(namespace, shard_id\) DO UPDATE SET sequence_number= \$3;`, tableName)
|
||||
mock.ExpectExec(expectedSQLRegexString).WithArgs(namespace, shardID, expectedSequenceNumber).WillReturnError(errors.New("an error"))
|
||||
|
||||
err = ck.Set(streamName, shardID, expectedSequenceNumber)
|
||||
err = ck.SetCheckpoint(streamName, shardID, expectedSequenceNumber)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("unable to set checkpoint for data initialization. cause: %v", err)
|
||||
13
store/redis/options.go
Normal file
13
store/redis/options.go
Normal file
|
|
@ -0,0 +1,13 @@
|
|||
package 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)
|
||||
|
||||
// WithClient overrides the default client
|
||||
func WithClient(client *redis.Client) Option {
|
||||
return func(c *Checkpoint) {
|
||||
c.client = client
|
||||
}
|
||||
}
|
||||
78
store/redis/redis.go
Normal file
78
store/redis/redis.go
Normal file
|
|
@ -0,0 +1,78 @@
|
|||
package redis
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
redis "github.com/go-redis/redis/v9"
|
||||
)
|
||||
|
||||
const localhost = "127.0.0.1:6379"
|
||||
|
||||
// New returns a checkpoint that uses Redis for underlying storage
|
||||
func New(appName string, opts ...Option) (*Checkpoint, error) {
|
||||
if appName == "" {
|
||||
return nil, fmt.Errorf("must provide app name")
|
||||
}
|
||||
|
||||
c := &Checkpoint{
|
||||
appName: appName,
|
||||
}
|
||||
|
||||
// override defaults
|
||||
for _, opt := range opts {
|
||||
opt(c)
|
||||
}
|
||||
|
||||
// default client if none provided
|
||||
if c.client == nil {
|
||||
addr := os.Getenv("REDIS_URL")
|
||||
if addr == "" {
|
||||
addr = localhost
|
||||
}
|
||||
|
||||
client := redis.NewClient(&redis.Options{Addr: addr})
|
||||
c.client = client
|
||||
}
|
||||
|
||||
// verify we can ping server
|
||||
_, err := c.client.Ping(context.Background()).Result()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return c, nil
|
||||
}
|
||||
|
||||
// Checkpoint stores and retreives the last evaluated key from a DDB scan
|
||||
type Checkpoint struct {
|
||||
appName string
|
||||
client *redis.Client
|
||||
}
|
||||
|
||||
// GetCheckpoint fetches the checkpoint for a particular Shard.
|
||||
func (c *Checkpoint) GetCheckpoint(streamName, shardID string) (string, error) {
|
||||
ctx := context.Background()
|
||||
val, _ := c.client.Get(ctx, c.key(streamName, shardID)).Result()
|
||||
return val, nil
|
||||
}
|
||||
|
||||
// SetCheckpoint stores a checkpoint for a shard (e.g. sequence number of last record processed by application).
|
||||
// Upon failover, record processing is resumed from this point.
|
||||
func (c *Checkpoint) SetCheckpoint(streamName, shardID, sequenceNumber string) error {
|
||||
if sequenceNumber == "" {
|
||||
return fmt.Errorf("sequence number should not be empty")
|
||||
}
|
||||
ctx := context.Background()
|
||||
err := c.client.Set(ctx, c.key(streamName, shardID), sequenceNumber, 0).Err()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// key generates a unique Redis key for storage of Checkpoint.
|
||||
func (c *Checkpoint) key(streamName, shardID string) string {
|
||||
return fmt.Sprintf("%v:checkpoint:%v:%v", c.appName, streamName, shardID)
|
||||
}
|
||||
|
|
@ -2,8 +2,28 @@ package redis
|
|||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/alicebob/miniredis"
|
||||
redis "github.com/go-redis/redis/v9"
|
||||
)
|
||||
|
||||
func Test_CheckpointOptions(t *testing.T) {
|
||||
s, err := miniredis.Run()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
defer s.Close()
|
||||
|
||||
client := redis.NewClient(&redis.Options{
|
||||
Addr: s.Addr(),
|
||||
})
|
||||
|
||||
_, err = New("app", WithClient(client))
|
||||
if err != nil {
|
||||
t.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func Test_CheckpointLifecycle(t *testing.T) {
|
||||
// new
|
||||
c, err := New("app")
|
||||
|
|
@ -12,10 +32,10 @@ func Test_CheckpointLifecycle(t *testing.T) {
|
|||
}
|
||||
|
||||
// set
|
||||
c.Set("streamName", "shardID", "testSeqNum")
|
||||
c.SetCheckpoint("streamName", "shardID", "testSeqNum")
|
||||
|
||||
// get
|
||||
val, err := c.Get("streamName", "shardID")
|
||||
val, err := c.GetCheckpoint("streamName", "shardID")
|
||||
if err != nil {
|
||||
t.Fatalf("get checkpoint error: %v", err)
|
||||
}
|
||||
|
|
@ -30,7 +50,7 @@ func Test_SetEmptySeqNum(t *testing.T) {
|
|||
t.Fatalf("new checkpoint error: %v", err)
|
||||
}
|
||||
|
||||
err = c.Set("streamName", "shardID", "")
|
||||
err = c.SetCheckpoint("streamName", "shardID", "")
|
||||
if err == nil {
|
||||
t.Fatalf("should not allow empty sequence number")
|
||||
}
|
||||
Loading…
Reference in a new issue