github.com/Jeffail/benthos/v3@v3.65.0/lib/output/writer/kinesis.go (about) 1 package writer 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/bloblang/field" 10 "github.com/Jeffail/benthos/v3/internal/interop" 11 "github.com/Jeffail/benthos/v3/lib/log" 12 "github.com/Jeffail/benthos/v3/lib/message/batch" 13 "github.com/Jeffail/benthos/v3/lib/metrics" 14 "github.com/Jeffail/benthos/v3/lib/types" 15 sess "github.com/Jeffail/benthos/v3/lib/util/aws/session" 16 "github.com/Jeffail/benthos/v3/lib/util/retries" 17 "github.com/aws/aws-sdk-go/aws" 18 "github.com/aws/aws-sdk-go/aws/session" 19 "github.com/aws/aws-sdk-go/service/kinesis" 20 "github.com/aws/aws-sdk-go/service/kinesis/kinesisiface" 21 "github.com/cenkalti/backoff/v4" 22 ) 23 24 //------------------------------------------------------------------------------ 25 26 const ( 27 kinesisMaxRecordsCount = 500 28 mebibyte = 1048576 29 ) 30 31 type sessionConfig struct { 32 sess.Config `json:",inline" yaml:",inline"` 33 } 34 35 // KinesisConfig contains configuration fields for the Kinesis output type. 36 type KinesisConfig struct { 37 sessionConfig `json:",inline" yaml:",inline"` 38 Stream string `json:"stream" yaml:"stream"` 39 HashKey string `json:"hash_key" yaml:"hash_key"` 40 PartitionKey string `json:"partition_key" yaml:"partition_key"` 41 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 42 retries.Config `json:",inline" yaml:",inline"` 43 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 44 } 45 46 // NewKinesisConfig creates a new Config with default values. 47 func NewKinesisConfig() KinesisConfig { 48 rConf := retries.NewConfig() 49 rConf.Backoff.InitialInterval = "1s" 50 rConf.Backoff.MaxInterval = "5s" 51 rConf.Backoff.MaxElapsedTime = "30s" 52 return KinesisConfig{ 53 sessionConfig: sessionConfig{ 54 Config: sess.NewConfig(), 55 }, 56 Stream: "", 57 HashKey: "", 58 PartitionKey: "", 59 MaxInFlight: 1, 60 Config: rConf, 61 Batching: batch.NewPolicyConfig(), 62 } 63 } 64 65 //------------------------------------------------------------------------------ 66 67 // Kinesis is a benthos writer.Type implementation that writes messages to an 68 // Amazon Kinesis stream. 69 type Kinesis struct { 70 conf KinesisConfig 71 72 session *session.Session 73 kinesis kinesisiface.KinesisAPI 74 75 backoffCtor func() backoff.BackOff 76 hashKey *field.Expression 77 partitionKey *field.Expression 78 streamName *string 79 80 log log.Modular 81 stats metrics.Type 82 83 mThrottled metrics.StatCounter 84 mThrottledF metrics.StatCounter 85 mPartsThrottled metrics.StatCounter 86 mPartsThrottledF metrics.StatCounter 87 } 88 89 // NewKinesis creates a new Amazon Kinesis writer.Type. 90 // 91 // Deprecated: use the V2 API instead. 92 func NewKinesis( 93 conf KinesisConfig, 94 log log.Modular, 95 stats metrics.Type, 96 ) (*Kinesis, error) { 97 return NewKinesisV2(conf, types.NoopMgr(), log, stats) 98 } 99 100 // NewKinesisV2 creates a new Amazon Kinesis writer.Type. 101 func NewKinesisV2( 102 conf KinesisConfig, 103 mgr types.Manager, 104 log log.Modular, 105 stats metrics.Type, 106 ) (*Kinesis, error) { 107 if conf.PartitionKey == "" { 108 return nil, errors.New("partition key must not be empty") 109 } 110 111 k := Kinesis{ 112 conf: conf, 113 log: log, 114 stats: stats, 115 mPartsThrottled: stats.GetCounter("parts.send.throttled"), 116 mThrottled: stats.GetCounter("send.throttled"), 117 streamName: aws.String(conf.Stream), 118 } 119 var err error 120 if k.hashKey, err = interop.NewBloblangField(mgr, conf.HashKey); err != nil { 121 return nil, fmt.Errorf("failed to parse hash key expression: %v", err) 122 } 123 if k.partitionKey, err = interop.NewBloblangField(mgr, conf.PartitionKey); err != nil { 124 return nil, fmt.Errorf("failed to parse partition key expression: %v", err) 125 } 126 if k.backoffCtor, err = conf.Config.GetCtor(); err != nil { 127 return nil, err 128 } 129 return &k, nil 130 } 131 132 //------------------------------------------------------------------------------ 133 134 // toRecords converts an individual benthos message into a slice of Kinesis 135 // batch put entries by promoting each message part into a single part message 136 // and passing each new message through the partition and hash key interpolation 137 // process, allowing the user to define the partition and hash key per message 138 // part. 139 func (a *Kinesis) toRecords(msg types.Message) ([]*kinesis.PutRecordsRequestEntry, error) { 140 entries := make([]*kinesis.PutRecordsRequestEntry, msg.Len()) 141 142 err := msg.Iter(func(i int, p types.Part) error { 143 entry := kinesis.PutRecordsRequestEntry{ 144 Data: p.Get(), 145 PartitionKey: aws.String(a.partitionKey.String(i, msg)), 146 } 147 148 if len(entry.Data) > mebibyte { 149 a.log.Errorf("part %d exceeds the maximum Kinesis payload limit of 1 MiB\n", i) 150 return types.ErrMessageTooLarge 151 } 152 153 if hashKey := a.hashKey.String(i, msg); hashKey != "" { 154 entry.ExplicitHashKey = aws.String(hashKey) 155 } 156 157 entries[i] = &entry 158 return nil 159 }) 160 161 return entries, err 162 } 163 164 //------------------------------------------------------------------------------ 165 166 // Connect creates a new Kinesis client and ensures that the target Kinesis 167 // stream exists. 168 func (a *Kinesis) Connect() error { 169 return a.ConnectWithContext(context.Background()) 170 } 171 172 // ConnectWithContext creates a new Kinesis client and ensures that the target 173 // Kinesis stream exists. 174 func (a *Kinesis) ConnectWithContext(ctx context.Context) error { 175 if a.session != nil { 176 return nil 177 } 178 179 sess, err := a.conf.GetSession() 180 if err != nil { 181 return err 182 } 183 184 a.session = sess 185 a.kinesis = kinesis.New(sess) 186 187 if err := a.kinesis.WaitUntilStreamExists(&kinesis.DescribeStreamInput{ 188 StreamName: a.streamName, 189 }); err != nil { 190 return err 191 } 192 193 a.log.Infof("Sending messages to Kinesis stream: %v\n", a.conf.Stream) 194 return nil 195 } 196 197 // Write attempts to write message contents to a target Kinesis stream in 198 // batches of 500. If throttling is detected, failed messages are retried 199 // according to the configurable backoff settings. 200 func (a *Kinesis) Write(msg types.Message) error { 201 return a.WriteWithContext(context.Background(), msg) 202 } 203 204 // WriteWithContext attempts to write message contents to a target Kinesis 205 // stream in batches of 500. If throttling is detected, failed messages are 206 // retried according to the configurable backoff settings. 207 func (a *Kinesis) WriteWithContext(ctx context.Context, msg types.Message) error { 208 if a.session == nil { 209 return types.ErrNotConnected 210 } 211 212 backOff := a.backoffCtor() 213 214 records, err := a.toRecords(msg) 215 if err != nil { 216 return err 217 } 218 219 input := &kinesis.PutRecordsInput{ 220 Records: records, 221 StreamName: a.streamName, 222 } 223 224 // trim input record length to max kinesis batch size 225 if len(records) > kinesisMaxRecordsCount { 226 input.Records, records = records[:kinesisMaxRecordsCount], records[kinesisMaxRecordsCount:] 227 } else { 228 records = nil 229 } 230 231 var failed []*kinesis.PutRecordsRequestEntry 232 backOff.Reset() 233 for len(input.Records) > 0 { 234 wait := backOff.NextBackOff() 235 236 // batch write to kinesis 237 output, err := a.kinesis.PutRecords(input) 238 if err != nil { 239 a.log.Warnf("kinesis error: %v\n", err) 240 // bail if a message is too large or all retry attempts expired 241 if wait == backoff.Stop { 242 return err 243 } 244 continue 245 } 246 247 // requeue any individual records that failed due to throttling 248 failed = nil 249 if output.FailedRecordCount != nil { 250 for i, entry := range output.Records { 251 if entry.ErrorCode != nil { 252 failed = append(failed, input.Records[i]) 253 switch *entry.ErrorCode { 254 case kinesis.ErrCodeProvisionedThroughputExceededException: 255 a.log.Errorf("Kinesis record write request rate too high, either the frequency or the size of the data exceeds your available throughput.") 256 case kinesis.ErrCodeKMSThrottlingException: 257 a.log.Errorf("Kinesis record write request throttling exception, the send traffic exceeds your request quota.") 258 default: 259 err = fmt.Errorf("record failed with code [%s] %s: %+v", *entry.ErrorCode, *entry.ErrorMessage, input.Records[i]) 260 a.log.Errorf("kinesis record write error: %v\n", err) 261 return err 262 } 263 } 264 } 265 } 266 input.Records = failed 267 268 // if throttling errors detected, pause briefly 269 l := len(failed) 270 if l > 0 { 271 a.mThrottled.Incr(1) 272 a.mPartsThrottled.Incr(int64(l)) 273 a.log.Warnf("scheduling retry of throttled records (%d)\n", l) 274 if wait == backoff.Stop { 275 return types.ErrTimeout 276 } 277 time.Sleep(wait) 278 } 279 280 // add remaining records to batch 281 if n := len(records); n > 0 && l < kinesisMaxRecordsCount { 282 if remaining := kinesisMaxRecordsCount - l; remaining < n { 283 input.Records, records = append(input.Records, records[:remaining]...), records[remaining:] 284 } else { 285 input.Records, records = append(input.Records, records...), nil 286 } 287 } 288 } 289 return err 290 } 291 292 // CloseAsync begins cleaning up resources used by this reader asynchronously. 293 func (a *Kinesis) CloseAsync() { 294 } 295 296 // WaitForClose will block until either the reader is closed or a specified 297 // timeout occurs. 298 func (a *Kinesis) WaitForClose(time.Duration) error { 299 return nil 300 } 301 302 //------------------------------------------------------------------------------