github.com/Jeffail/benthos/v3@v3.65.0/lib/output/writer/sqs.go (about) 1 package writer 2 3 import ( 4 "context" 5 "fmt" 6 "regexp" 7 "sort" 8 "strconv" 9 "strings" 10 "sync" 11 "time" 12 13 "github.com/Jeffail/benthos/v3/internal/bloblang/field" 14 "github.com/Jeffail/benthos/v3/internal/interop" 15 "github.com/Jeffail/benthos/v3/internal/metadata" 16 "github.com/Jeffail/benthos/v3/lib/log" 17 "github.com/Jeffail/benthos/v3/lib/message/batch" 18 "github.com/Jeffail/benthos/v3/lib/metrics" 19 "github.com/Jeffail/benthos/v3/lib/types" 20 sess "github.com/Jeffail/benthos/v3/lib/util/aws/session" 21 "github.com/Jeffail/benthos/v3/lib/util/retries" 22 "github.com/aws/aws-sdk-go/aws" 23 "github.com/aws/aws-sdk-go/service/sqs" 24 "github.com/aws/aws-sdk-go/service/sqs/sqsiface" 25 "github.com/cenkalti/backoff/v4" 26 ) 27 28 //------------------------------------------------------------------------------ 29 30 const ( 31 sqsMaxRecordsCount = 10 32 ) 33 34 //------------------------------------------------------------------------------ 35 36 // AmazonSQSConfig contains configuration fields for the output AmazonSQS type. 37 type AmazonSQSConfig struct { 38 sessionConfig `json:",inline" yaml:",inline"` 39 URL string `json:"url" yaml:"url"` 40 MessageGroupID string `json:"message_group_id" yaml:"message_group_id"` 41 MessageDeduplicationID string `json:"message_deduplication_id" yaml:"message_deduplication_id"` 42 Metadata metadata.ExcludeFilterConfig `json:"metadata" yaml:"metadata"` 43 MaxInFlight int `json:"max_in_flight" yaml:"max_in_flight"` 44 retries.Config `json:",inline" yaml:",inline"` 45 Batching batch.PolicyConfig `json:"batching" yaml:"batching"` 46 } 47 48 // NewAmazonSQSConfig creates a new Config with default values. 49 func NewAmazonSQSConfig() AmazonSQSConfig { 50 rConf := retries.NewConfig() 51 rConf.Backoff.InitialInterval = "1s" 52 rConf.Backoff.MaxInterval = "5s" 53 rConf.Backoff.MaxElapsedTime = "30s" 54 55 return AmazonSQSConfig{ 56 sessionConfig: sessionConfig{ 57 Config: sess.NewConfig(), 58 }, 59 URL: "", 60 MessageGroupID: "", 61 MessageDeduplicationID: "", 62 Metadata: metadata.NewExcludeFilterConfig(), 63 MaxInFlight: 1, 64 Config: rConf, 65 Batching: batch.NewPolicyConfig(), 66 } 67 } 68 69 //------------------------------------------------------------------------------ 70 71 // AmazonSQS is a benthos writer.Type implementation that writes messages to an 72 // Amazon SQS queue. 73 type AmazonSQS struct { 74 conf AmazonSQSConfig 75 sqs sqsiface.SQSAPI 76 77 backoffCtor func() backoff.BackOff 78 79 groupID *field.Expression 80 dedupeID *field.Expression 81 metaFilter *metadata.ExcludeFilter 82 83 closer sync.Once 84 closeChan chan struct{} 85 86 log log.Modular 87 stats metrics.Type 88 } 89 90 // NewAmazonSQS creates a new Amazon SQS writer.Type. 91 // 92 // Deprecated: use the V2 API instead. 93 func NewAmazonSQS( 94 conf AmazonSQSConfig, 95 log log.Modular, 96 stats metrics.Type, 97 ) (*AmazonSQS, error) { 98 return NewAmazonSQSV2(conf, types.NoopMgr(), log, stats) 99 } 100 101 // NewAmazonSQSV2 creates a new Amazon SQS writer.Type. 102 func NewAmazonSQSV2( 103 conf AmazonSQSConfig, 104 mgr types.Manager, 105 log log.Modular, 106 stats metrics.Type, 107 ) (*AmazonSQS, error) { 108 s := &AmazonSQS{ 109 conf: conf, 110 log: log, 111 stats: stats, 112 closeChan: make(chan struct{}), 113 } 114 115 var err error 116 if id := conf.MessageGroupID; len(id) > 0 { 117 if s.groupID, err = interop.NewBloblangField(mgr, id); err != nil { 118 return nil, fmt.Errorf("failed to parse group ID expression: %v", err) 119 } 120 } 121 if id := conf.MessageDeduplicationID; len(id) > 0 { 122 if s.dedupeID, err = interop.NewBloblangField(mgr, id); err != nil { 123 return nil, fmt.Errorf("failed to parse dedupe ID expression: %v", err) 124 } 125 } 126 if s.metaFilter, err = conf.Metadata.Filter(); err != nil { 127 return nil, fmt.Errorf("failed to construct metadata filter: %w", err) 128 } 129 130 if s.backoffCtor, err = conf.Config.GetCtor(); err != nil { 131 return nil, err 132 } 133 return s, nil 134 } 135 136 // ConnectWithContext attempts to establish a connection to the target SQS 137 // queue. 138 func (a *AmazonSQS) ConnectWithContext(ctx context.Context) error { 139 return a.Connect() 140 } 141 142 // Connect attempts to establish a connection to the target SQS queue. 143 func (a *AmazonSQS) Connect() error { 144 if a.sqs != nil { 145 return nil 146 } 147 148 sess, err := a.conf.GetSession() 149 if err != nil { 150 return err 151 } 152 153 a.sqs = sqs.New(sess) 154 a.log.Infof("Sending messages to Amazon SQS URL: %v\n", a.conf.URL) 155 return nil 156 } 157 158 type sqsAttributes struct { 159 attrMap map[string]*sqs.MessageAttributeValue 160 groupID *string 161 dedupeID *string 162 content *string 163 } 164 165 var sqsAttributeKeyInvalidCharRegexp = regexp.MustCompile(`(^\.)|(\.\.)|(^aws\.)|(^amazon\.)|(\.$)|([^a-z0-9_\-.]+)`) 166 167 func isValidSQSAttribute(k, v string) bool { 168 return len(sqsAttributeKeyInvalidCharRegexp.FindStringIndex(strings.ToLower(k))) == 0 169 } 170 171 func (a *AmazonSQS) getSQSAttributes(msg types.Message, i int) sqsAttributes { 172 p := msg.Get(i) 173 keys := []string{} 174 a.metaFilter.Iter(p.Metadata(), func(k, v string) error { 175 if isValidSQSAttribute(k, v) { 176 keys = append(keys, k) 177 } else { 178 a.log.Debugf("Rejecting metadata key '%v' due to invalid characters\n", k) 179 } 180 return nil 181 }) 182 var values map[string]*sqs.MessageAttributeValue 183 if len(keys) > 0 { 184 sort.Strings(keys) 185 values = map[string]*sqs.MessageAttributeValue{} 186 187 for i, k := range keys { 188 values[k] = &sqs.MessageAttributeValue{ 189 DataType: aws.String("String"), 190 StringValue: aws.String(p.Metadata().Get(k)), 191 } 192 if i == 9 { 193 break 194 } 195 } 196 } 197 198 var groupID, dedupeID *string 199 if a.groupID != nil { 200 groupID = aws.String(a.groupID.String(i, msg)) 201 } 202 if a.dedupeID != nil { 203 dedupeID = aws.String(a.dedupeID.String(i, msg)) 204 } 205 206 return sqsAttributes{ 207 attrMap: values, 208 groupID: groupID, 209 dedupeID: dedupeID, 210 content: aws.String(string(p.Get())), 211 } 212 } 213 214 // Write attempts to write message contents to a target SQS. 215 func (a *AmazonSQS) Write(msg types.Message) error { 216 return a.WriteWithContext(context.Background(), msg) 217 } 218 219 // WriteWithContext attempts to write message contents to a target SQS. 220 func (a *AmazonSQS) WriteWithContext(ctx context.Context, msg types.Message) error { 221 if a.sqs == nil { 222 return types.ErrNotConnected 223 } 224 225 backOff := a.backoffCtor() 226 227 entries := []*sqs.SendMessageBatchRequestEntry{} 228 attrMap := map[string]sqsAttributes{} 229 msg.Iter(func(i int, p types.Part) error { 230 id := strconv.Itoa(i) 231 attrs := a.getSQSAttributes(msg, i) 232 attrMap[id] = attrs 233 234 entries = append(entries, &sqs.SendMessageBatchRequestEntry{ 235 Id: aws.String(id), 236 MessageBody: attrs.content, 237 MessageAttributes: attrs.attrMap, 238 MessageGroupId: attrs.groupID, 239 MessageDeduplicationId: attrs.dedupeID, 240 }) 241 return nil 242 }) 243 244 input := &sqs.SendMessageBatchInput{ 245 QueueUrl: aws.String(a.conf.URL), 246 Entries: entries, 247 } 248 249 // trim input input length to max sqs batch size 250 if len(entries) > sqsMaxRecordsCount { 251 input.Entries, entries = entries[:sqsMaxRecordsCount], entries[sqsMaxRecordsCount:] 252 } else { 253 entries = nil 254 } 255 256 var err error 257 for len(input.Entries) > 0 { 258 wait := backOff.NextBackOff() 259 260 var batchResult *sqs.SendMessageBatchOutput 261 if batchResult, err = a.sqs.SendMessageBatch(input); err != nil { 262 a.log.Warnf("SQS error: %v\n", err) 263 // bail if a message is too large or all retry attempts expired 264 if wait == backoff.Stop { 265 return err 266 } 267 select { 268 case <-time.After(wait): 269 case <-ctx.Done(): 270 return types.ErrTimeout 271 case <-a.closeChan: 272 return err 273 } 274 continue 275 } 276 277 if unproc := batchResult.Failed; len(unproc) > 0 { 278 input.Entries = []*sqs.SendMessageBatchRequestEntry{} 279 for _, v := range unproc { 280 if *v.SenderFault { 281 err = fmt.Errorf("record failed with code: %v, message: %v", *v.Code, *v.Message) 282 a.log.Errorf("SQS record error: %v\n", err) 283 return err 284 } 285 aMap := attrMap[*v.Id] 286 input.Entries = append(input.Entries, &sqs.SendMessageBatchRequestEntry{ 287 Id: v.Id, 288 MessageBody: aMap.content, 289 MessageAttributes: aMap.attrMap, 290 MessageGroupId: aMap.groupID, 291 MessageDeduplicationId: aMap.dedupeID, 292 }) 293 } 294 err = fmt.Errorf("failed to send %v messages", len(unproc)) 295 } else { 296 input.Entries = nil 297 } 298 299 if err != nil { 300 if wait == backoff.Stop { 301 break 302 } 303 select { 304 case <-time.After(wait): 305 case <-ctx.Done(): 306 return types.ErrTimeout 307 case <-a.closeChan: 308 return err 309 } 310 } 311 312 // add remaining records to batch 313 l := len(input.Entries) 314 if n := len(entries); n > 0 && l < sqsMaxRecordsCount { 315 if remaining := sqsMaxRecordsCount - l; remaining < n { 316 input.Entries, entries = append(input.Entries, entries[:remaining]...), entries[remaining:] 317 } else { 318 input.Entries, entries = append(input.Entries, entries...), nil 319 } 320 } 321 } 322 323 return err 324 } 325 326 // CloseAsync begins cleaning up resources used by this reader asynchronously. 327 func (a *AmazonSQS) CloseAsync() { 328 a.closer.Do(func() { 329 close(a.closeChan) 330 }) 331 } 332 333 // WaitForClose will block until either the reader is closed or a specified 334 // timeout occurs. 335 func (a *AmazonSQS) WaitForClose(time.Duration) error { 336 return nil 337 } 338 339 //------------------------------------------------------------------------------