2017-11-20 16:21:40 +00:00
|
|
|
package consumer
|
2016-02-03 05:04:22 +00:00
|
|
|
|
|
|
|
|
import (
|
2017-11-20 16:21:40 +00:00
|
|
|
"context"
|
2019-04-07 23:29:12 +00:00
|
|
|
"errors"
|
2017-11-20 16:21:40 +00:00
|
|
|
"fmt"
|
2018-07-29 05:53:33 +00:00
|
|
|
"io/ioutil"
|
|
|
|
|
"log"
|
2019-08-14 16:33:35 +00:00
|
|
|
"sync"
|
|
|
|
|
"time"
|
2016-02-03 05:04:22 +00:00
|
|
|
|
2021-09-22 05:00:14 +00:00
|
|
|
"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"
|
2016-02-03 05:04:22 +00:00
|
|
|
)
|
|
|
|
|
|
2020-07-30 21:18:38 +00:00
|
|
|
// Record wraps the record returned from the Kinesis library and
|
|
|
|
|
// extends to include the shard id.
|
|
|
|
|
type Record struct {
|
2021-09-22 05:00:14 +00:00
|
|
|
types.Record
|
2020-08-02 05:05:17 +00:00
|
|
|
ShardID string
|
|
|
|
|
MillisBehindLatest *int64
|
2020-07-30 21:18:38 +00:00
|
|
|
}
|
2017-11-21 16:58:16 +00:00
|
|
|
|
2017-11-20 16:21:40 +00:00
|
|
|
// New creates a kinesis consumer with default settings. Use Option to override
|
|
|
|
|
// any of the optional attributes.
|
2017-11-27 00:00:11 +00:00
|
|
|
func New(streamName string, opts ...Option) (*Consumer, error) {
|
|
|
|
|
if streamName == "" {
|
2021-02-05 17:10:34 +00:00
|
|
|
return nil, errors.New("must provide stream name")
|
2017-11-20 16:21:40 +00:00
|
|
|
}
|
|
|
|
|
|
2019-07-29 04:18:40 +00:00
|
|
|
// new consumer with noop storage, counter, and logger
|
2017-11-20 16:21:40 +00:00
|
|
|
c := &Consumer{
|
2018-12-29 03:39:47 +00:00
|
|
|
streamName: streamName,
|
2021-09-22 05:00:14 +00:00
|
|
|
initialShardIteratorType: types.ShardIteratorTypeLatest,
|
2019-07-29 04:18:40 +00:00
|
|
|
store: &noopStore{},
|
2018-12-29 03:39:47 +00:00
|
|
|
counter: &noopCounter{},
|
2018-07-29 05:53:33 +00:00
|
|
|
logger: &noopLogger{
|
|
|
|
|
logger: log.New(ioutil.Discard, "", log.LstdFlags),
|
|
|
|
|
},
|
2020-01-17 18:22:10 +00:00
|
|
|
scanInterval: 250 * time.Millisecond,
|
2020-04-28 04:12:20 +00:00
|
|
|
maxRecords: 10000,
|
2017-11-20 16:21:40 +00:00
|
|
|
}
|
|
|
|
|
|
2017-11-27 00:00:11 +00:00
|
|
|
// override defaults
|
2017-11-20 16:21:40 +00:00
|
|
|
for _, opt := range opts {
|
2018-07-29 05:53:33 +00:00
|
|
|
opt(c)
|
|
|
|
|
}
|
|
|
|
|
|
2019-07-29 04:18:40 +00:00
|
|
|
// default client
|
2018-07-29 05:53:33 +00:00
|
|
|
if c.client == nil {
|
2021-09-22 05:00:14 +00:00
|
|
|
cfg, err := config.LoadDefaultConfig(context.TODO())
|
2018-09-03 16:59:39 +00:00
|
|
|
if err != nil {
|
2021-09-22 05:00:14 +00:00
|
|
|
log.Fatalf("unable to load SDK config, %v", err)
|
2018-09-03 16:59:39 +00:00
|
|
|
}
|
2021-09-22 05:00:14 +00:00
|
|
|
c.client = kinesis.NewFromConfig(cfg)
|
2017-11-20 16:21:40 +00:00
|
|
|
}
|
|
|
|
|
|
2019-07-29 04:18:40 +00:00
|
|
|
// default group consumes all shards
|
2019-06-09 20:42:25 +00:00
|
|
|
if c.group == nil {
|
2019-07-29 04:18:40 +00:00
|
|
|
c.group = NewAllGroup(c.client, c.store, streamName, c.logger)
|
2019-06-09 20:42:25 +00:00
|
|
|
}
|
|
|
|
|
|
2017-11-20 16:21:40 +00:00
|
|
|
return c, nil
|
2016-02-03 05:04:22 +00:00
|
|
|
}
|
|
|
|
|
|
2017-10-16 00:45:38 +00:00
|
|
|
// Consumer wraps the interaction with the Kinesis stream
|
2016-02-03 05:04:22 +00:00
|
|
|
type Consumer struct {
|
2018-12-29 03:39:47 +00:00
|
|
|
streamName string
|
2021-09-22 05:00:14 +00:00
|
|
|
initialShardIteratorType types.ShardIteratorType
|
2019-08-14 16:33:35 +00:00
|
|
|
initialTimestamp *time.Time
|
2021-09-22 05:00:14 +00:00
|
|
|
client kinesisClient
|
2018-12-29 03:39:47 +00:00
|
|
|
counter Counter
|
2019-07-29 04:18:40 +00:00
|
|
|
group Group
|
|
|
|
|
logger Logger
|
|
|
|
|
store Store
|
2020-01-17 18:22:10 +00:00
|
|
|
scanInterval time.Duration
|
2020-04-28 04:12:20 +00:00
|
|
|
maxRecords int64
|
2020-10-14 03:41:18 +00:00
|
|
|
isAggregated bool
|
2021-07-30 21:16:15 +00:00
|
|
|
shardClosedHandler ShardClosedHandler
|
2016-05-01 05:23:35 +00:00
|
|
|
}
|
|
|
|
|
|
2019-04-07 23:29:12 +00:00
|
|
|
// 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
|
2019-07-28 17:54:01 +00:00
|
|
|
// function returns the special value ErrSkipCheckpoint.
|
2019-04-07 23:29:12 +00:00
|
|
|
type ScanFunc func(*Record) error
|
|
|
|
|
|
2019-07-28 17:54:01 +00:00
|
|
|
// ErrSkipCheckpoint is used as a return value from ScanFunc to indicate that
|
2019-04-07 23:29:12 +00:00
|
|
|
// the current checkpoint should be skipped skipped. It is not returned
|
|
|
|
|
// as an error by any function.
|
2019-07-28 17:54:01 +00:00
|
|
|
var ErrSkipCheckpoint = errors.New("skip checkpoint")
|
2019-04-07 23:29:12 +00:00
|
|
|
|
|
|
|
|
// 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 {
|
2019-06-09 20:42:25 +00:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
|
defer cancel()
|
|
|
|
|
|
2019-04-10 05:03:12 +00:00
|
|
|
var (
|
|
|
|
|
errc = make(chan error, 1)
|
2021-09-22 05:00:14 +00:00
|
|
|
shardc = make(chan types.Shard, 1)
|
2019-04-10 05:03:12 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
|
|
go func() {
|
2019-06-09 20:42:25 +00:00
|
|
|
c.group.Start(ctx, shardc)
|
2019-04-10 05:03:12 +00:00
|
|
|
<-ctx.Done()
|
|
|
|
|
close(shardc)
|
|
|
|
|
}()
|
2017-11-23 19:29:58 +00:00
|
|
|
|
2019-08-14 16:33:35 +00:00
|
|
|
wg := new(sync.WaitGroup)
|
2019-04-10 05:03:12 +00:00
|
|
|
// process each of the shards
|
|
|
|
|
for shard := range shardc {
|
2019-08-14 16:33:35 +00:00
|
|
|
wg.Add(1)
|
2017-11-20 16:21:40 +00:00
|
|
|
go func(shardID string) {
|
2019-08-14 16:33:35 +00:00
|
|
|
defer wg.Done()
|
Maintain parent/child shard ordering across shard splits/merges. (#155)
Kinesis allows clients to rely on an invariant that, for a given partition key, the order of records added to the stream will be maintained. IE: given an input `pkey=x,val=1 pkey=x,val=2 pkey=x,val=3`, the values `1,2,3` will be seen in that order when processed by clients, so long as clients are careful. It does so by putting all records for a single partition key into a single shard, then maintaining ordering within that shard.
However, shards can be split and merge, to distribute load better and handle per-shard throughput limits. Kinesis does this currently by (one or many times) splitting a single shard into two or by merging two adjacent shards into one. When this occurs, Kinesis still allows for ordering consistency by detailing shard parent/child relationships within its `listShards` outputs. A split shard A will create children B and C, both with `ParentShardId=A`. A merging of shards A and B into C will create a new shard C with `ParentShardId=A,AdjacentParentShardId=B`. So long as clients fully process all records in parents (including adjacent parents) before processing the new shard, ordering will be maintained.
`kinesis-consumer` currently doesn't do this. Instead, upon the initial (and subsequent) `listShards` call, all visible shards immediately begin processing. Considering this case, where shards split, then merge, and each shard `X` contains a single record `rX`:
```
time ->
B
/ \
A D
\ /
C
```
record `rD` should be processed after both `rB` and `rC` are processed, and both `rB` and `rC` should wait for `rA` to be processed. By starting goroutines immediately, any ordering of `{rA,rB,rC,rD}` might occur within the original code.
This PR utilizes the `AllGroup` as a book-keeper of fully processed shards, with the `Consumer` calling `CloseShard` once it has finished a shard. `AllGroup` doesn't release a shard for processing until its parents have fully been processed, and the consumer just processes the shards it receives as it used to.
This PR created a new `CloseableGroup` interface rather than append to the existing `Group` interface to maintain backwards compatibility in existing code that may already implement the `Group` interface elsewhere. Different `Group` implementations don't get the ordering described above, but the default `Consumer` does.
2024-06-06 15:37:42 +00:00
|
|
|
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 {
|
2017-11-23 16:49:37 +00:00
|
|
|
select {
|
2021-02-05 17:10:34 +00:00
|
|
|
case errc <- fmt.Errorf("shard %s error: %w", shardID, err):
|
2019-04-10 05:03:12 +00:00
|
|
|
cancel()
|
2017-11-23 16:49:37 +00:00
|
|
|
default:
|
|
|
|
|
}
|
|
|
|
|
}
|
2021-09-22 05:00:14 +00:00
|
|
|
}(aws.ToString(shard.ShardId))
|
2016-02-03 05:04:22 +00:00
|
|
|
}
|
2017-11-20 16:21:40 +00:00
|
|
|
|
2019-08-14 16:33:35 +00:00
|
|
|
go func() {
|
|
|
|
|
wg.Wait()
|
|
|
|
|
close(errc)
|
|
|
|
|
}()
|
|
|
|
|
|
2017-11-23 16:49:37 +00:00
|
|
|
return <-errc
|
2016-02-03 05:04:22 +00:00
|
|
|
}
|
|
|
|
|
|
Maintain parent/child shard ordering across shard splits/merges. (#155)
Kinesis allows clients to rely on an invariant that, for a given partition key, the order of records added to the stream will be maintained. IE: given an input `pkey=x,val=1 pkey=x,val=2 pkey=x,val=3`, the values `1,2,3` will be seen in that order when processed by clients, so long as clients are careful. It does so by putting all records for a single partition key into a single shard, then maintaining ordering within that shard.
However, shards can be split and merge, to distribute load better and handle per-shard throughput limits. Kinesis does this currently by (one or many times) splitting a single shard into two or by merging two adjacent shards into one. When this occurs, Kinesis still allows for ordering consistency by detailing shard parent/child relationships within its `listShards` outputs. A split shard A will create children B and C, both with `ParentShardId=A`. A merging of shards A and B into C will create a new shard C with `ParentShardId=A,AdjacentParentShardId=B`. So long as clients fully process all records in parents (including adjacent parents) before processing the new shard, ordering will be maintained.
`kinesis-consumer` currently doesn't do this. Instead, upon the initial (and subsequent) `listShards` call, all visible shards immediately begin processing. Considering this case, where shards split, then merge, and each shard `X` contains a single record `rX`:
```
time ->
B
/ \
A D
\ /
C
```
record `rD` should be processed after both `rB` and `rC` are processed, and both `rB` and `rC` should wait for `rA` to be processed. By starting goroutines immediately, any ordering of `{rA,rB,rC,rD}` might occur within the original code.
This PR utilizes the `AllGroup` as a book-keeper of fully processed shards, with the `Consumer` calling `CloseShard` once it has finished a shard. `AllGroup` doesn't release a shard for processing until its parents have fully been processed, and the consumer just processes the shards it receives as it used to.
This PR created a new `CloseableGroup` interface rather than append to the existing `Group` interface to maintain backwards compatibility in existing code that may already implement the `Group` interface elsewhere. Different `Group` implementations don't get the ordering described above, but the default `Consumer` does.
2024-06-06 15:37:42 +00:00
|
|
|
func (c *Consumer) scanSingleShard(ctx context.Context, shardID string, fn ScanFunc) error {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2019-04-10 05:03:12 +00:00
|
|
|
// ScanShard loops over records on a specific shard, calls the callback func
|
|
|
|
|
// for each record and checkpoints the progress of scan.
|
2019-04-07 23:29:12 +00:00
|
|
|
func (c *Consumer) ScanShard(ctx context.Context, shardID string, fn ScanFunc) error {
|
|
|
|
|
// get last seq number from checkpoint
|
2019-06-09 20:42:25 +00:00
|
|
|
lastSeqNum, err := c.group.GetCheckpoint(c.streamName, shardID)
|
2017-11-21 16:58:16 +00:00
|
|
|
if err != nil {
|
2021-02-05 17:10:34 +00:00
|
|
|
return fmt.Errorf("get checkpoint error: %w", err)
|
2017-11-21 16:58:16 +00:00
|
|
|
}
|
|
|
|
|
|
2018-07-29 05:53:33 +00:00
|
|
|
// get shard iterator
|
2019-08-14 16:33:35 +00:00
|
|
|
shardIterator, err := c.getShardIterator(ctx, c.streamName, shardID, lastSeqNum)
|
2017-11-23 16:49:37 +00:00
|
|
|
if err != nil {
|
2021-02-05 17:10:34 +00:00
|
|
|
return fmt.Errorf("get shard iterator error: %w", err)
|
2017-10-16 00:40:30 +00:00
|
|
|
}
|
2018-06-08 15:40:42 +00:00
|
|
|
|
2019-06-09 20:42:25 +00:00
|
|
|
c.logger.Log("[CONSUMER] start scan:", shardID, lastSeqNum)
|
2019-04-10 05:03:12 +00:00
|
|
|
defer func() {
|
2019-06-09 20:42:25 +00:00
|
|
|
c.logger.Log("[CONSUMER] stop scan:", shardID)
|
2019-04-10 05:03:12 +00:00
|
|
|
}()
|
2021-09-22 05:00:14 +00:00
|
|
|
|
2020-01-17 18:22:10 +00:00
|
|
|
scanTicker := time.NewTicker(c.scanInterval)
|
|
|
|
|
defer scanTicker.Stop()
|
2017-10-16 00:40:30 +00:00
|
|
|
|
2018-07-29 05:53:33 +00:00
|
|
|
for {
|
2021-09-22 05:00:14 +00:00
|
|
|
resp, err := c.client.GetRecords(ctx, &kinesis.GetRecordsInput{
|
|
|
|
|
Limit: aws.Int32(int32(c.maxRecords)),
|
2020-08-01 22:45:37 +00:00
|
|
|
ShardIterator: shardIterator,
|
|
|
|
|
})
|
2017-10-16 00:40:30 +00:00
|
|
|
|
2021-09-22 05:00:14 +00:00
|
|
|
// attempt to recover from GetRecords error
|
2020-08-01 22:45:37 +00:00
|
|
|
if err != nil {
|
|
|
|
|
c.logger.Log("[CONSUMER] get records error:", err.Error())
|
2019-07-29 04:33:19 +00:00
|
|
|
|
2021-09-22 05:00:14 +00:00
|
|
|
if !isRetriableError(err) {
|
|
|
|
|
return fmt.Errorf("get records error: %v", err.Error())
|
2018-07-29 05:53:33 +00:00
|
|
|
}
|
|
|
|
|
|
2020-08-01 22:45:37 +00:00
|
|
|
shardIterator, err = c.getShardIterator(ctx, c.streamName, shardID, lastSeqNum)
|
|
|
|
|
if err != nil {
|
2021-02-05 17:10:34 +00:00
|
|
|
return fmt.Errorf("get shard iterator error: %w", err)
|
2020-08-01 22:45:37 +00:00
|
|
|
}
|
|
|
|
|
} else {
|
2019-04-07 23:29:12 +00:00
|
|
|
// loop over records, call callback func
|
2021-09-22 05:00:14 +00:00
|
|
|
var records []types.Record
|
|
|
|
|
|
|
|
|
|
// deaggregate records
|
2020-10-14 03:41:18 +00:00
|
|
|
if c.isAggregated {
|
2021-09-22 05:00:14 +00:00
|
|
|
records, err = deaggregateRecords(resp.Records)
|
2020-10-14 03:41:18 +00:00
|
|
|
if err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
records = resp.Records
|
|
|
|
|
}
|
2021-09-22 05:00:14 +00:00
|
|
|
|
2020-10-14 03:41:18 +00:00
|
|
|
for _, r := range records {
|
2019-04-07 23:29:12 +00:00
|
|
|
select {
|
|
|
|
|
case <-ctx.Done():
|
2018-07-29 05:53:33 +00:00
|
|
|
return nil
|
2019-04-07 23:29:12 +00:00
|
|
|
default:
|
2020-08-02 05:05:17 +00:00
|
|
|
err := fn(&Record{r, shardID, resp.MillisBehindLatest})
|
2019-07-28 17:54:01 +00:00
|
|
|
if err != nil && err != ErrSkipCheckpoint {
|
2019-04-07 23:29:12 +00:00
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
|
2019-07-28 17:54:01 +00:00
|
|
|
if err != ErrSkipCheckpoint {
|
2019-06-09 20:42:25 +00:00
|
|
|
if err := c.group.SetCheckpoint(c.streamName, shardID, *r.SequenceNumber); err != nil {
|
2019-04-07 23:29:12 +00:00
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
c.counter.Add("records", 1)
|
|
|
|
|
lastSeqNum = *r.SequenceNumber
|
2018-07-29 05:53:33 +00:00
|
|
|
}
|
2018-06-08 15:40:42 +00:00
|
|
|
}
|
2018-07-29 05:53:33 +00:00
|
|
|
|
2018-09-03 16:59:39 +00:00
|
|
|
if isShardClosed(resp.NextShardIterator, shardIterator) {
|
2019-06-09 20:42:25 +00:00
|
|
|
c.logger.Log("[CONSUMER] shard closed:", shardID)
|
2021-09-22 05:00:14 +00:00
|
|
|
|
2021-07-30 21:16:15 +00:00
|
|
|
if c.shardClosedHandler != nil {
|
Maintain parent/child shard ordering across shard splits/merges. (#155)
Kinesis allows clients to rely on an invariant that, for a given partition key, the order of records added to the stream will be maintained. IE: given an input `pkey=x,val=1 pkey=x,val=2 pkey=x,val=3`, the values `1,2,3` will be seen in that order when processed by clients, so long as clients are careful. It does so by putting all records for a single partition key into a single shard, then maintaining ordering within that shard.
However, shards can be split and merge, to distribute load better and handle per-shard throughput limits. Kinesis does this currently by (one or many times) splitting a single shard into two or by merging two adjacent shards into one. When this occurs, Kinesis still allows for ordering consistency by detailing shard parent/child relationships within its `listShards` outputs. A split shard A will create children B and C, both with `ParentShardId=A`. A merging of shards A and B into C will create a new shard C with `ParentShardId=A,AdjacentParentShardId=B`. So long as clients fully process all records in parents (including adjacent parents) before processing the new shard, ordering will be maintained.
`kinesis-consumer` currently doesn't do this. Instead, upon the initial (and subsequent) `listShards` call, all visible shards immediately begin processing. Considering this case, where shards split, then merge, and each shard `X` contains a single record `rX`:
```
time ->
B
/ \
A D
\ /
C
```
record `rD` should be processed after both `rB` and `rC` are processed, and both `rB` and `rC` should wait for `rA` to be processed. By starting goroutines immediately, any ordering of `{rA,rB,rC,rD}` might occur within the original code.
This PR utilizes the `AllGroup` as a book-keeper of fully processed shards, with the `Consumer` calling `CloseShard` once it has finished a shard. `AllGroup` doesn't release a shard for processing until its parents have fully been processed, and the consumer just processes the shards it receives as it used to.
This PR created a new `CloseableGroup` interface rather than append to the existing `Group` interface to maintain backwards compatibility in existing code that may already implement the `Group` interface elsewhere. Different `Group` implementations don't get the ordering described above, but the default `Consumer` does.
2024-06-06 15:37:42 +00:00
|
|
|
if err := c.shardClosedHandler(c.streamName, shardID); err != nil {
|
2021-07-30 21:16:15 +00:00
|
|
|
return fmt.Errorf("shard closed handler error: %w", err)
|
|
|
|
|
}
|
|
|
|
|
}
|
2018-07-29 05:53:33 +00:00
|
|
|
return nil
|
|
|
|
|
}
|
2019-04-07 23:29:12 +00:00
|
|
|
|
2018-07-29 05:53:33 +00:00
|
|
|
shardIterator = resp.NextShardIterator
|
2017-11-27 00:00:11 +00:00
|
|
|
}
|
2020-08-01 22:45:37 +00:00
|
|
|
|
|
|
|
|
// Wait for next scan
|
|
|
|
|
select {
|
|
|
|
|
case <-ctx.Done():
|
|
|
|
|
return nil
|
|
|
|
|
case <-scanTicker.C:
|
|
|
|
|
continue
|
|
|
|
|
}
|
2016-02-03 05:04:22 +00:00
|
|
|
}
|
2018-07-29 05:53:33 +00:00
|
|
|
}
|
2017-10-16 00:40:30 +00:00
|
|
|
|
2021-09-22 05:00:14 +00:00
|
|
|
// 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)
|
|
|
|
|
}
|
2019-07-31 02:48:20 +00:00
|
|
|
|
2021-09-22 05:00:14 +00:00
|
|
|
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
|
2018-09-03 16:59:39 +00:00
|
|
|
}
|
|
|
|
|
|
2019-08-14 16:33:35 +00:00
|
|
|
func (c *Consumer) getShardIterator(ctx context.Context, streamName, shardID, seqNum string) (*string, error) {
|
2018-07-29 05:53:33 +00:00
|
|
|
params := &kinesis.GetShardIteratorInput{
|
|
|
|
|
ShardId: aws.String(shardID),
|
|
|
|
|
StreamName: aws.String(streamName),
|
|
|
|
|
}
|
|
|
|
|
|
2019-04-07 23:29:12 +00:00
|
|
|
if seqNum != "" {
|
2021-09-22 05:00:14 +00:00
|
|
|
params.ShardIteratorType = types.ShardIteratorTypeAfterSequenceNumber
|
2019-04-07 23:29:12 +00:00
|
|
|
params.StartingSequenceNumber = aws.String(seqNum)
|
2019-08-14 16:33:35 +00:00
|
|
|
} else if c.initialTimestamp != nil {
|
2021-09-22 05:00:14 +00:00
|
|
|
params.ShardIteratorType = types.ShardIteratorTypeAtTimestamp
|
2019-08-14 16:33:35 +00:00
|
|
|
params.Timestamp = c.initialTimestamp
|
2018-07-29 05:53:33 +00:00
|
|
|
} else {
|
2021-09-22 05:00:14 +00:00
|
|
|
params.ShardIteratorType = types.ShardIteratorType(c.initialShardIteratorType)
|
2018-07-29 05:53:33 +00:00
|
|
|
}
|
|
|
|
|
|
2021-09-22 05:00:14 +00:00
|
|
|
res, err := c.client.GetShardIterator(ctx, params)
|
2019-04-07 23:29:12 +00:00
|
|
|
return res.ShardIterator, err
|
2016-02-03 05:04:22 +00:00
|
|
|
}
|
2021-09-22 05:00:14 +00:00
|
|
|
|
|
|
|
|
func isRetriableError(err error) bool {
|
|
|
|
|
switch err.(type) {
|
|
|
|
|
case *types.ExpiredIteratorException:
|
|
|
|
|
return true
|
|
|
|
|
case *types.ProvisionedThroughputExceededException:
|
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
return false
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func isShardClosed(nextShardIterator, currentShardIterator *string) bool {
|
|
|
|
|
return nextShardIterator == nil || currentShardIterator == nextShardIterator
|
|
|
|
|
}
|