From c2b932579c2bfd42e0b7b92d5069777709573cf8 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 28 Feb 2024 18:38:05 +0900 Subject: [PATCH 01/43] Refactor sqsReader.Receive: move queue ack waiting and message deletion in a separated goroutine --- x-pack/filebeat/input/awss3/input.go | 2 +- .../input/awss3/input_benchmark_test.go | 2 +- x-pack/filebeat/input/awss3/interfaces.go | 4 +- .../input/awss3/mock_interfaces_test.go | 36 ++++--- .../input/awss3/mock_publisher_test.go | 7 +- x-pack/filebeat/input/awss3/sqs.go | 93 +++++++++++++++++-- x-pack/filebeat/input/awss3/sqs_s3_event.go | 92 ++++++++---------- .../filebeat/input/awss3/sqs_s3_event_test.go | 80 +++++++++++----- x-pack/filebeat/input/awss3/sqs_test.go | 29 +++++- 9 files changed, 234 insertions(+), 111 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input.go b/x-pack/filebeat/input/awss3/input.go index 0b33ae042f9..283fd567e10 100644 --- a/x-pack/filebeat/input/awss3/input.go +++ b/x-pack/filebeat/input/awss3/input.go @@ -148,7 +148,7 @@ func (in *s3Input) Run(inputContext v2.Context, pipeline beat.Pipeline) error { // Poll metrics periodically in the background go pollSqsWaitingMetric(ctx, receiver) - if err := receiver.Receive(ctx); err != nil { + if err := receiver.Receive(ctx, pipeline); err != nil { return err } } diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index e05e5b461ca..8d485e5d5de 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -233,7 +233,7 @@ func benchmarkInputSQS(t *testing.T, maxMessagesInflight int) testing.BenchmarkR b.ResetTimer() start := time.Now() - if err := sqsReader.Receive(ctx); err != nil { + if err := sqsReader.Receive(ctx, pipeline); err != nil { if !errors.Is(err, context.DeadlineExceeded) { t.Fatal(err) } diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index 1f1390c4f2f..6e852a0b45e 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -9,6 +9,7 @@ import ( "errors" "fmt" "net/url" + "sync" "time" smithyhttp "github.com/aws/smithy-go/transport/http" @@ -65,7 +66,8 @@ type sqsProcessor interface { // given message and is responsible for updating the message's visibility // timeout while it is being processed and for deleting it when processing // completes successfully. - ProcessSQS(ctx context.Context, msg *types.Message) error + ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *awscommon.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) + DeleteSQS(ctx context.Context, msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error } // ------ diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index b976cf00ebc..5a9158903c3 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -11,15 +7,15 @@ package awss3 import ( context "context" reflect "reflect" + sync "sync" time "time" s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" aws "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" logp "github.com/elastic/elastic-agent-libs/logp" + gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. @@ -276,18 +272,36 @@ func (m *MockSQSProcessor) EXPECT() *MockSQSProcessorMockRecorder { return m.recorder } -// ProcessSQS mocks base method. -func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message) error { +// DeleteSQS mocks base method. +func (m *MockSQSProcessor) DeleteSQS(ctx context.Context, msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg) + ret := m.ctrl.Call(m, "DeleteSQS", ctx, msg, receiveCount, processingErr, handles) ret0, _ := ret[0].(error) return ret0 } +// DeleteSQS indicates an expected call of DeleteSQS. +func (mr *MockSQSProcessorMockRecorder) DeleteSQS(ctx, msg, receiveCount, processingErr, handles interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSQS", reflect.TypeOf((*MockSQSProcessor)(nil).DeleteSQS), ctx, msg, receiveCount, processingErr, handles) +} + +// ProcessSQS mocks base method. +func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *aws.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg, client, acker) + ret0, _ := ret[0].(int) + ret1, _ := ret[1].([]s3ObjectHandler) + ret2, _ := ret[2].(context.CancelFunc) + ret3, _ := ret[3].(*sync.WaitGroup) + ret4, _ := ret[4].(error) + return ret0, ret1, ret2, ret3, ret4 +} + // ProcessSQS indicates an expected call of ProcessSQS. -func (mr *MockSQSProcessorMockRecorder) ProcessSQS(ctx, msg interface{}) *gomock.Call { +func (mr *MockSQSProcessorMockRecorder) ProcessSQS(ctx, msg, client, acker interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessSQS", reflect.TypeOf((*MockSQSProcessor)(nil).ProcessSQS), ctx, msg) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessSQS", reflect.TypeOf((*MockSQSProcessor)(nil).ProcessSQS), ctx, msg, client, acker) } // MockS3API is a mock of s3API interface. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index efbd5bcef97..9c34b58efd1 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -11,9 +7,8 @@ package awss3 import ( reflect "reflect" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" + gomock "github.com/golang/mock/gomock" ) // MockBeatClient is a mock of Client interface. diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index dd454a3bfb9..5b4ae26be1c 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -13,6 +13,7 @@ import ( "github.com/aws/aws-sdk-go-v2/service/sqs/types" + "github.com/elastic/beats/v7/libbeat/beat" awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "github.com/elastic/elastic-agent-libs/logp" "github.com/elastic/go-concert/timed" @@ -32,6 +33,17 @@ type sqsReader struct { metrics *inputMetrics } +type processingOutcome struct { + start time.Time + keepaliveWg *sync.WaitGroup + keepaliveCancel context.CancelFunc + acker *awscommon.EventACKTracker + msg *types.Message + receiveCount int + handles []s3ObjectHandler + processingErr error +} + func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessagesInflight int, msgHandler sqsProcessor) *sqsReader { if metrics == nil { // Metrics are optional. Initialize a stub. @@ -47,11 +59,51 @@ func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessag } } -func (r *sqsReader) Receive(ctx context.Context) error { +func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { // This loop tries to keep the workers busy as much as possible while // honoring the max message cap as opposed to a simpler loop that receives // N messages, waits for them all to finish, then requests N more messages. var workerWg sync.WaitGroup + endingChan := make(chan error, 1) + processingChan := make(chan processingOutcome) + + go func(ctx context.Context) { + // Wait for all workers to finish. + for { + select { + case processOutcome, ok := <-processingChan: + if !ok { + if errors.Is(ctx.Err(), context.Canceled) { + // A canceled context is a normal shutdown. + close(endingChan) + return + } + + endingChan <- ctx.Err() + return + } + + go func(processOutcome processingOutcome) { + // Wait for all events to be ACKed before proceeding. + processOutcome.acker.Wait() + + // Stop keepalive routine before deleting visibility. + processOutcome.keepaliveCancel() + processOutcome.keepaliveWg.Wait() + + err := r.msgHandler.DeleteSQS(ctx, processOutcome.msg, processOutcome.receiveCount, processOutcome.processingErr, processOutcome.handles) + if err != nil { + r.log.Warnw("Failed deleting SQS message.", + "error", err, + "message_id", *processOutcome.msg.MessageId, + "elapsed_time_ns", time.Since(processOutcome.start)) + } + }(processOutcome) + default: + } + } + }(ctx) + for ctx.Err() == nil { // Determine how many SQS workers are available. workers, err := r.workerSem.AcquireContext(r.maxMessagesInflight, ctx) @@ -90,24 +142,49 @@ func (r *sqsReader) Receive(ctx context.Context) error { r.workerSem.Release(1) }() - if err := r.msgHandler.ProcessSQS(ctx, &msg); err != nil { + // Create a pipeline client scoped to this goroutine. + client, err := pipeline.ConnectWith(beat.ClientConfig{ + EventListener: awscommon.NewEventACKHandler(), + Processing: beat.ProcessingConfig{ + // This input only produces events with basic types so normalization + // is not required. + EventNormalization: boolPtr(false), + }, + }) + + if err != nil { r.log.Warnw("Failed processing SQS message.", "error", err, "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) + + return + } + + defer client.Close() + + acker := awscommon.NewEventACKTracker(ctx) + + receiveCount, handles, keepaliveCancel, keepaliveWg, processingErr := r.msgHandler.ProcessSQS(ctx, &msg, client, acker) + + processingChan <- processingOutcome{ + start: start, + keepaliveWg: keepaliveWg, + keepaliveCancel: keepaliveCancel, + acker: acker, + msg: &msg, + receiveCount: receiveCount, + handles: handles, + processingErr: processingErr, } }(msg, time.Now()) } } - // Wait for all workers to finish. workerWg.Wait() + close(processingChan) - if errors.Is(ctx.Err(), context.Canceled) { - // A canceled context is a normal shutdown. - return nil - } - return ctx.Err() + return <-endingChan } func (r *sqsReader) GetApproximateMessageCount(ctx context.Context) (int, error) { diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 7f95cf564c0..395d8218c7f 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -122,47 +122,19 @@ func newSQSS3EventProcessor( } } -func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message) error { - log := p.log.With( - "message_id", *msg.MessageId, - "message_receipt_time", time.Now().UTC()) - - keepaliveCtx, keepaliveCancel := context.WithCancel(ctx) - defer keepaliveCancel() - - // Start SQS keepalive worker. - var keepaliveWg sync.WaitGroup - keepaliveWg.Add(1) - go p.keepalive(keepaliveCtx, log, &keepaliveWg, msg) - - receiveCount := getSQSReceiveCount(msg.Attributes) - if receiveCount == 1 { - // Only contribute to the sqs_lag_time histogram on the first message - // to avoid skewing the metric when processing retries. - if s, found := msg.Attributes[sqsSentTimestampAttribute]; found { - if sentTimeMillis, err := strconv.ParseInt(s, 10, 64); err == nil { - sentTime := time.UnixMilli(sentTimeMillis) - p.metrics.sqsLagTime.Update(time.Since(sentTime).Nanoseconds()) - } - } - } - - handles, processingErr := p.processS3Events(ctx, log, *msg.Body) - - // Stop keepalive routine before changing visibility. - keepaliveCancel() - keepaliveWg.Wait() - +func (p *sqsS3EventProcessor) DeleteSQS(ctx context.Context, msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error { // No error. Delete SQS. if processingErr == nil { - if msgDelErr := p.sqs.DeleteMessage(context.Background(), msg); msgDelErr != nil { + if msgDelErr := p.sqs.DeleteMessage(ctx, msg); msgDelErr != nil { return fmt.Errorf("failed deleting message from SQS queue (it may be reprocessed): %w", msgDelErr) } + p.metrics.sqsMessagesDeletedTotal.Inc() // SQS message finished and deleted, finalize s3 objects if finalizeErr := p.finalizeS3Objects(handles); finalizeErr != nil { return fmt.Errorf("failed finalizing message from SQS queue (manual cleanup is required): %w", finalizeErr) } + return nil } @@ -180,12 +152,12 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message if errors.Is(processingErr, &nonRetryableError{}) { if msgDelErr := p.sqs.DeleteMessage(context.Background(), msg); msgDelErr != nil { return multierr.Combine( - fmt.Errorf("failed processing SQS message (attempted to delete message): %w", processingErr), + fmt.Errorf("failed deleting SQS message (attempted to delete message): %w", processingErr), fmt.Errorf("failed deleting message from SQS queue (it may be reprocessed): %w", msgDelErr), ) } p.metrics.sqsMessagesDeletedTotal.Inc() - return fmt.Errorf("failed processing SQS message (message was deleted): %w", processingErr) + return fmt.Errorf("failed deleting SQS message (message was deleted): %w", processingErr) } // An error that may be resolved by letting the visibility timeout @@ -193,7 +165,37 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message // queue is enabled then the message will eventually placed on the DLQ // after maximum receives is reached. p.metrics.sqsMessagesReturnedTotal.Inc() - return fmt.Errorf("failed processing SQS message (it will return to queue after visibility timeout): %w", processingErr) + return fmt.Errorf("failed deleting SQS message (it will return to queue after visibility timeout): %w", processingErr) + +} +func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *awscommon.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) { + keepaliveCtx, keepaliveCancel := context.WithCancel(ctx) + + // Start SQS keepalive worker. + var keepaliveWg sync.WaitGroup + keepaliveWg.Add(1) + + log := p.log.With( + "message_id", *msg.MessageId, + "message_receipt_time", time.Now().UTC()) + + go p.keepalive(keepaliveCtx, log, &keepaliveWg, msg) + + receiveCount := getSQSReceiveCount(msg.Attributes) + if receiveCount == 1 { + // Only contribute to the sqs_lag_time histogram on the first message + // to avoid skewing the metric when processing retries. + if s, found := msg.Attributes[sqsSentTimestampAttribute]; found { + if sentTimeMillis, err := strconv.ParseInt(s, 10, 64); err == nil { + sentTime := time.UnixMilli(sentTimeMillis) + p.metrics.sqsLagTime.Update(time.Since(sentTime).Nanoseconds()) + } + } + } + + handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) + + return receiveCount, handles, keepaliveCancel, &keepaliveWg, processingErr } func (p *sqsS3EventProcessor) keepalive(ctx context.Context, log *logp.Logger, wg *sync.WaitGroup, msg *types.Message) { @@ -291,7 +293,7 @@ func (*sqsS3EventProcessor) isObjectCreatedEvents(event s3EventV2) bool { return event.EventSource == "aws:s3" && strings.HasPrefix(event.EventName, "ObjectCreated:") } -func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Logger, body string) ([]s3ObjectHandler, error) { +func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Logger, body string, client beat.Client, acker *awscommon.EventACKTracker) ([]s3ObjectHandler, error) { s3Events, err := p.getS3Notifications(body) if err != nil { if errors.Is(err, context.Canceled) { @@ -307,24 +309,6 @@ func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Log return nil, nil } - // Create a pipeline client scoped to this goroutine. - client, err := p.pipeline.ConnectWith(beat.ClientConfig{ - EventListener: awscommon.NewEventACKHandler(), - Processing: beat.ProcessingConfig{ - // This input only produces events with basic types so normalization - // is not required. - EventNormalization: boolPtr(false), - }, - }) - if err != nil { - return nil, err - } - defer client.Close() - - // Wait for all events to be ACKed before proceeding. - acker := awscommon.NewEventACKTracker(ctx) - defer acker.Wait() - var errs []error var handles []s3ObjectHandler for i, event := range s3Events { diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index 5ecd72fc4c9..2a97449959d 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -44,14 +44,20 @@ func TestSQSS3EventProcessor(t *testing.T) { mockBeatPipeline := NewMockBeatPipeline(ctrl) gomock.InOrder( - mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil), - mockClient.EXPECT().Close(), - mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), ) p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - require.NoError(t, p.ProcessSQS(ctx, &msg)) + + acker := awscommon.NewEventACKTracker(ctx) + receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + require.NoError(t, processingErr) + + gomock.InOrder( + mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), + ) + + require.NoError(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) }) t.Run("invalid SQS JSON body does not retry", func(t *testing.T) { @@ -69,14 +75,19 @@ func TestSQSS3EventProcessor(t *testing.T) { body = body[10:] invalidBodyMsg.Body = &body + mockClient := NewMockBeatClient(ctrl) + acker := awscommon.NewEventACKTracker(ctx) + + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) + receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker) + t.Log(processingErr) + require.Error(t, processingErr) + gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&invalidBodyMsg)).Return(nil), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - err := p.ProcessSQS(ctx, &invalidBodyMsg) - require.Error(t, err) - t.Log(err) + require.Error(t, p.DeleteSQS(ctx, &invalidBodyMsg, receiveCount, processingErr, handles)) }) t.Run("zero S3 events in body", func(t *testing.T) { @@ -91,12 +102,18 @@ func TestSQSS3EventProcessor(t *testing.T) { emptyRecordsMsg := newSQSMessage([]s3EventV2{}...) + mockClient := NewMockBeatClient(ctrl) + acker := awscommon.NewEventACKTracker(ctx) + + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) + receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker) + require.NoError(t, processingErr) + gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&emptyRecordsMsg)).Return(nil), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - require.NoError(t, p.ProcessSQS(ctx, &emptyRecordsMsg)) + require.NoError(t, p.DeleteSQS(ctx, &emptyRecordsMsg, receiveCount, processingErr, handles)) }) t.Run("visibility is extended after half expires", func(t *testing.T) { @@ -116,19 +133,25 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().ChangeMessageVisibility(gomock.Any(), gomock.Eq(&msg), gomock.Eq(visibilityTimeout)).AnyTimes().Return(nil) gomock.InOrder( - mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Do(func(ctx context.Context, _ *logp.Logger, _ beat.Client, _ *awscommon.EventACKTracker, _ s3EventV2) { require.NoError(t, timed.Wait(ctx, 5*visibilityTimeout)) }).Return(mockS3Handler), mockS3Handler.EXPECT().ProcessS3Object().Return(nil), - mockClient.EXPECT().Close(), + ) + + acker := awscommon.NewEventACKTracker(ctx) + + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory, 5) + receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + require.NoError(t, processingErr) + + gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), mockS3Handler.EXPECT().FinalizeS3Object().Return(nil), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - require.NoError(t, p.ProcessSQS(ctx, &msg)) + require.NoError(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) }) t.Run("message returns to queue on error", func(t *testing.T) { @@ -144,16 +167,18 @@ func TestSQSS3EventProcessor(t *testing.T) { mockBeatPipeline := NewMockBeatPipeline(ctrl) gomock.InOrder( - mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(mockS3Handler), mockS3Handler.EXPECT().ProcessS3Object().Return(errors.New("fake connectivity problem")), - mockClient.EXPECT().Close(), ) + acker := awscommon.NewEventACKTracker(ctx) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - err := p.ProcessSQS(ctx, &msg) - t.Log(err) - require.Error(t, err) + receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + t.Log(processingErr) + require.Error(t, processingErr) + + require.Error(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) }) t.Run("message is deleted after multiple receives", func(t *testing.T) { @@ -174,17 +199,22 @@ func TestSQSS3EventProcessor(t *testing.T) { } gomock.InOrder( - mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(mockS3Handler), mockS3Handler.EXPECT().ProcessS3Object().Return(errors.New("fake connectivity problem")), - mockClient.EXPECT().Close(), - mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), ) + acker := awscommon.NewEventACKTracker(ctx) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - err := p.ProcessSQS(ctx, &msg) - t.Log(err) - require.Error(t, err) + receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + t.Log(processingErr) + require.Error(t, processingErr) + + gomock.InOrder( + mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), + ) + + require.Error(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) }) } diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 5eda5d1885e..adf51a4fc43 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -9,6 +9,7 @@ import ( "crypto/sha256" "encoding/json" "errors" + "sync" "testing" "time" @@ -65,15 +66,34 @@ func TestSQSReceiver(t *testing.T) { }), ) - // Expect the one message returned to have been processed. + mockClient := NewMockBeatClient(ctrl) + mockBeatPipeline := NewMockBeatPipeline(ctrl) + _, keepaliveCancel := context.WithCancel(ctx) + + // Start SQS keepalive worker. + var keepaliveWg sync.WaitGroup + + gomock.InOrder( + mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), + // Expect the one message returned to have been processed. + mockMsgHandler.EXPECT(). + ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any()). + Times(1). + Return(0, nil, keepaliveCancel, &keepaliveWg, nil), + ) + + // The two expected calls happen in different goroutines, we cannot enforce an oder + // Expect the client to be closed + mockClient.EXPECT().Close() + // Expect the one message returned to have been deleted. mockMsgHandler.EXPECT(). - ProcessSQS(gomock.Any(), gomock.Eq(&msg)). + DeleteSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). Return(nil) // Execute sqsReader and verify calls/state. receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler) - require.NoError(t, receiver.Receive(ctx)) + require.NoError(t, receiver.Receive(ctx, mockBeatPipeline)) assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) @@ -104,9 +124,10 @@ func TestSQSReceiver(t *testing.T) { }), ) + mockBeatPipeline := NewMockBeatPipeline(ctrl) // Execute SQSReceiver and verify calls/state. receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler) - require.NoError(t, receiver.Receive(ctx)) + require.NoError(t, receiver.Receive(ctx, mockBeatPipeline)) assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) } From 6734772c04a53d36a69247883f0c4e28c4c49819 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Thu, 29 Feb 2024 11:25:12 +0900 Subject: [PATCH 02/43] background ctx in DeleteMessage, linting --- x-pack/filebeat/input/awss3/input_benchmark_test.go | 3 +-- .../filebeat/input/awss3/input_integration_test.go | 5 ++--- x-pack/filebeat/input/awss3/interfaces.go | 2 +- x-pack/filebeat/input/awss3/mock_interfaces_test.go | 12 ++++++++---- x-pack/filebeat/input/awss3/mock_publisher_test.go | 4 ++++ x-pack/filebeat/input/awss3/s3_objects_test.go | 7 ++++--- x-pack/filebeat/input/awss3/sqs.go | 7 ++++++- x-pack/filebeat/input/awss3/sqs_s3_event.go | 6 +++--- x-pack/filebeat/input/awss3/sqs_s3_event_test.go | 12 ++++++------ x-pack/filebeat/input/awss3/sqs_test.go | 2 +- 10 files changed, 36 insertions(+), 24 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 8d485e5d5de..842bcaa56ec 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -8,7 +8,6 @@ import ( "context" "errors" "fmt" - "io/ioutil" "os" "path/filepath" "runtime" @@ -132,7 +131,7 @@ type constantS3 struct { var _ s3API = (*constantS3)(nil) func newConstantS3(t testing.TB) *constantS3 { - data, err := ioutil.ReadFile(cloudtrailTestFile) + data, err := os.ReadFile(cloudtrailTestFile) if err != nil { t.Fatal(err) } diff --git a/x-pack/filebeat/input/awss3/input_integration_test.go b/x-pack/filebeat/input/awss3/input_integration_test.go index 62cbc835011..58f03e77687 100644 --- a/x-pack/filebeat/input/awss3/input_integration_test.go +++ b/x-pack/filebeat/input/awss3/input_integration_test.go @@ -12,7 +12,6 @@ import ( "bytes" "context" "fmt" - "io/ioutil" "os" "path" "path/filepath" @@ -64,7 +63,7 @@ func getTerraformOutputs(t *testing.T, isLocalStack bool) terraformOutputData { outputFile = terraformOutputYML } - ymlData, err := ioutil.ReadFile(path.Join(path.Dir(filename), outputFile)) + ymlData, err := os.ReadFile(path.Join(path.Dir(filename), outputFile)) if os.IsNotExist(err) { t.Skipf("Run 'terraform apply' in %v to setup S3 and SQS for the test.", filepath.Dir(outputFile)) } @@ -365,7 +364,7 @@ func uploadS3TestFiles(t *testing.T, region, bucket string, s3Client *s3.Client, _, basefile, _, _ := runtime.Caller(0) basedir := path.Dir(basefile) for _, filename := range filenames { - data, err := ioutil.ReadFile(path.Join(basedir, filename)) + data, err := os.ReadFile(path.Join(basedir, filename)) if err != nil { t.Fatalf("Failed to open file %q, %v", filename, err) } diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index 6e852a0b45e..1aada072b49 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -67,7 +67,7 @@ type sqsProcessor interface { // timeout while it is being processed and for deleting it when processing // completes successfully. ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *awscommon.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) - DeleteSQS(ctx context.Context, msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error + DeleteSQS(msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error } // ------ diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index 5a9158903c3..6507e816db3 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -273,17 +277,17 @@ func (m *MockSQSProcessor) EXPECT() *MockSQSProcessorMockRecorder { } // DeleteSQS mocks base method. -func (m *MockSQSProcessor) DeleteSQS(ctx context.Context, msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error { +func (m *MockSQSProcessor) DeleteSQS(msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteSQS", ctx, msg, receiveCount, processingErr, handles) + ret := m.ctrl.Call(m, "DeleteSQS", msg, receiveCount, processingErr, handles) ret0, _ := ret[0].(error) return ret0 } // DeleteSQS indicates an expected call of DeleteSQS. -func (mr *MockSQSProcessorMockRecorder) DeleteSQS(ctx, msg, receiveCount, processingErr, handles interface{}) *gomock.Call { +func (mr *MockSQSProcessorMockRecorder) DeleteSQS(msg, receiveCount, processingErr, handles interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSQS", reflect.TypeOf((*MockSQSProcessor)(nil).DeleteSQS), ctx, msg, receiveCount, processingErr, handles) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSQS", reflect.TypeOf((*MockSQSProcessor)(nil).DeleteSQS), msg, receiveCount, processingErr, handles) } // ProcessSQS mocks base method. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index 9c34b58efd1..f28976698da 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 6732c12e057..f8ad084a4d6 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -8,7 +8,8 @@ import ( "bytes" "context" "errors" - "io/ioutil" + "io" + "os" "path/filepath" "strings" "testing" @@ -27,7 +28,7 @@ import ( ) func newS3Object(t testing.TB, filename, contentType string) (s3EventV2, *s3.GetObjectOutput) { - data, err := ioutil.ReadFile(filename) + data, err := os.ReadFile(filename) if err != nil { t.Fatal(err) } @@ -39,7 +40,7 @@ func newS3GetObjectResponse(filename string, data []byte, contentType string) *s r := bytes.NewReader(data) getObjectOutput := s3.GetObjectOutput{} getObjectOutput.ContentLength = int64(r.Len()) - getObjectOutput.Body = ioutil.NopCloser(r) + getObjectOutput.Body = io.NopCloser(r) if contentType != "" { getObjectOutput.ContentType = &contentType } diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 5b4ae26be1c..9683ecb541c 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -91,15 +91,20 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { processOutcome.keepaliveCancel() processOutcome.keepaliveWg.Wait() - err := r.msgHandler.DeleteSQS(ctx, processOutcome.msg, processOutcome.receiveCount, processOutcome.processingErr, processOutcome.handles) + err := r.msgHandler.DeleteSQS(processOutcome.msg, processOutcome.receiveCount, processOutcome.processingErr, processOutcome.handles) if err != nil { r.log.Warnw("Failed deleting SQS message.", "error", err, "message_id", *processOutcome.msg.MessageId, "elapsed_time_ns", time.Since(processOutcome.start)) + } else { + r.log.Warnw("Success deleting SQS message.", + "message_id", *processOutcome.msg.MessageId, + "elapsed_time_ns", time.Since(processOutcome.start)) } }(processOutcome) default: + time.Sleep(500 * time.Microsecond) } } }(ctx) diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 395d8218c7f..cf6efc97bef 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -44,7 +44,7 @@ func (e *nonRetryableError) Error() string { } func (e *nonRetryableError) Is(err error) bool { - _, ok := err.(*nonRetryableError) //nolint:errorlint // This is not used directly to detected wrapped errors (errors.Is handles unwrapping). + _, ok := err.(*nonRetryableError) return ok } @@ -122,10 +122,10 @@ func newSQSS3EventProcessor( } } -func (p *sqsS3EventProcessor) DeleteSQS(ctx context.Context, msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error { +func (p *sqsS3EventProcessor) DeleteSQS(msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error { // No error. Delete SQS. if processingErr == nil { - if msgDelErr := p.sqs.DeleteMessage(ctx, msg); msgDelErr != nil { + if msgDelErr := p.sqs.DeleteMessage(context.Background(), msg); msgDelErr != nil { return fmt.Errorf("failed deleting message from SQS queue (it may be reprocessed): %w", msgDelErr) } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index 2a97449959d..fb190c5aed3 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -57,7 +57,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), ) - require.NoError(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) + require.NoError(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) }) t.Run("invalid SQS JSON body does not retry", func(t *testing.T) { @@ -87,7 +87,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&invalidBodyMsg)).Return(nil), ) - require.Error(t, p.DeleteSQS(ctx, &invalidBodyMsg, receiveCount, processingErr, handles)) + require.Error(t, p.DeleteSQS(&invalidBodyMsg, receiveCount, processingErr, handles)) }) t.Run("zero S3 events in body", func(t *testing.T) { @@ -113,7 +113,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&emptyRecordsMsg)).Return(nil), ) - require.NoError(t, p.DeleteSQS(ctx, &emptyRecordsMsg, receiveCount, processingErr, handles)) + require.NoError(t, p.DeleteSQS(&emptyRecordsMsg, receiveCount, processingErr, handles)) }) t.Run("visibility is extended after half expires", func(t *testing.T) { @@ -151,7 +151,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().FinalizeS3Object().Return(nil), ) - require.NoError(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) + require.NoError(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) }) t.Run("message returns to queue on error", func(t *testing.T) { @@ -178,7 +178,7 @@ func TestSQSS3EventProcessor(t *testing.T) { t.Log(processingErr) require.Error(t, processingErr) - require.Error(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) + require.Error(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) }) t.Run("message is deleted after multiple receives", func(t *testing.T) { @@ -214,7 +214,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), ) - require.Error(t, p.DeleteSQS(ctx, &msg, receiveCount, processingErr, handles)) + require.Error(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) }) } diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index adf51a4fc43..609ec05f136 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -87,7 +87,7 @@ func TestSQSReceiver(t *testing.T) { mockClient.EXPECT().Close() // Expect the one message returned to have been deleted. mockMsgHandler.EXPECT(). - DeleteSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). + DeleteSQS(gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). Return(nil) From 826b53c6bdb6256951ea80aede6a2a98c2eb9712 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Thu, 29 Feb 2024 13:08:29 +0900 Subject: [PATCH 03/43] make check --- x-pack/filebeat/input/awss3/mock_interfaces_test.go | 3 ++- x-pack/filebeat/input/awss3/mock_publisher_test.go | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index 6507e816db3..4a64078c898 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -16,10 +16,11 @@ import ( s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" + gomock "github.com/golang/mock/gomock" + beat "github.com/elastic/beats/v7/libbeat/beat" aws "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" logp "github.com/elastic/elastic-agent-libs/logp" - gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index f28976698da..efbd5bcef97 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -11,8 +11,9 @@ package awss3 import ( reflect "reflect" - beat "github.com/elastic/beats/v7/libbeat/beat" gomock "github.com/golang/mock/gomock" + + beat "github.com/elastic/beats/v7/libbeat/beat" ) // MockBeatClient is a mock of Client interface. From e1294f417f9f9828e651213bb03ccdd9545b4253 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 4 Mar 2024 16:32:20 +0900 Subject: [PATCH 04/43] Refactor benchmark, acker with atomic --- .../input/awss3/input_benchmark_test.go | 243 +++++++++++++++--- x-pack/filebeat/input/awss3/metrics.go | 2 + x-pack/filebeat/input/awss3/s3_objects.go | 6 +- .../filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs_s3_event.go | 2 +- .../input/awss3/testdata/aws-cloudtrail.json | 1 + x-pack/libbeat/common/aws/acker.go | 36 +-- x-pack/libbeat/common/aws/acker_test.go | 8 +- 8 files changed, 237 insertions(+), 63 deletions(-) create mode 100644 x-pack/filebeat/input/awss3/testdata/aws-cloudtrail.json diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 842bcaa56ec..0b58ac76b95 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -6,15 +6,21 @@ package awss3 import ( "context" + "encoding/json" "errors" "fmt" + "math" + "math/rand" "os" "path/filepath" "runtime" + "strconv" + "strings" "sync" "testing" "time" + "github.com/elastic/beats/v7/libbeat/common/atomic" "github.com/elastic/beats/v7/libbeat/statestore" "github.com/elastic/beats/v7/libbeat/statestore/storetest" @@ -35,27 +41,84 @@ import ( ) const ( - cloudtrailTestFile = "testdata/aws-cloudtrail.json.gz" + cloudtrailTestFileGz = "testdata/aws-cloudtrail.json.gz" + cloudtrailTestFile = "testdata/aws-cloudtrail.json" totalListingObjects = 10000 totalListingObjectsForInputS3 = totalListingObjects / 5 ) type constantSQS struct { - msgs []sqsTypes.Message + s3API *constantS3 + receiveCallN *atomic.Uint64 + msgs [][]sqsTypes.Message } var _ sqsAPI = (*constantSQS)(nil) -func newConstantSQS() *constantSQS { - return &constantSQS{ - msgs: []sqsTypes.Message{ - newSQSMessage(newS3Event(filepath.Base(cloudtrailTestFile))), - }, +func newConstantSQS(t testing.TB, maxMessages int, totalSqsMessages uint64, s3API *constantS3) *constantSQS { + customRand := rand.New(rand.NewSource(1)) + + var s3ObjN int + var generatedMessages uint64 + + msgs := make([][]sqsTypes.Message, 0) + + c := &constantSQS{s3API: s3API, receiveCallN: atomic.NewUint64(0)} + for { + if generatedMessages == totalSqsMessages { + break + } + + currentMessages := uint64(customRand.Intn(maxMessages)) + 1 + if totalSqsMessages < generatedMessages+currentMessages { + currentMessages = totalSqsMessages - generatedMessages + } + + generatedMessages += currentMessages + + currentMsgs := make([]sqsTypes.Message, 0, currentMessages) + for ; currentMessages > 0; currentMessages-- { + totS3Events := customRand.Intn(9) + 1 + s3Events := make([]s3EventV2, 0, totS3Events) + for ; totS3Events > 0; totS3Events-- { + totRecordsInS3Events := customRand.Intn(160) + 1 + recordsInS3Events := make([]map[string]any, 0, totRecordsInS3Events) + for ; totRecordsInS3Events > 0; totRecordsInS3Events-- { + recordN := customRand.Intn(len(c.s3API.records)-1) + 1 + recordsInS3Events = append(recordsInS3Events, c.s3API.records[recordN]) + } + + s3ObjKey := fmt.Sprintf("%d", s3ObjN) + s3Events = append(s3Events, newS3Event(s3ObjKey)) + + data, err := json.Marshal(struct{ Records []map[string]any }{Records: recordsInS3Events}) + if err != nil { + t.Fatal(err) + } + + c.s3API.objects = append(c.s3API.objects, data) + s3ObjN++ + } + + currentMsgs = append(currentMsgs, newSQSMessage(s3Events...)) + } + + msgs = append(msgs, currentMsgs) } + + c.msgs = msgs + + return c } func (c *constantSQS) ReceiveMessage(ctx context.Context, maxMessages int) ([]sqsTypes.Message, error) { - return c.msgs, nil + receiveCallN := c.receiveCallN.Add(1) + var msgs []sqsTypes.Message + if receiveCallN <= uint64(len(c.msgs)) { + msgs = c.msgs[receiveCallN-1] + } + + return msgs, nil } func (*constantSQS) DeleteMessage(ctx context.Context, msg *sqsTypes.Message) error { @@ -124,6 +187,8 @@ func newS3PagerConstant(listPrefix string) *s3PagerConstant { type constantS3 struct { filename string data []byte + records []map[string]any + objects [][]byte contentType string pagerConstant s3Pager } @@ -131,57 +196,132 @@ type constantS3 struct { var _ s3API = (*constantS3)(nil) func newConstantS3(t testing.TB) *constantS3 { + dataGz, err := os.ReadFile(cloudtrailTestFileGz) + if err != nil { + t.Fatal(err) + } + data, err := os.ReadFile(cloudtrailTestFile) if err != nil { t.Fatal(err) } + var records struct{ Records []map[string]any } + err = json.Unmarshal(data, &records) + if err != nil { + t.Fatal(err) + } + return &constantS3{ - filename: filepath.Base(cloudtrailTestFile), - data: data, + filename: filepath.Base(cloudtrailTestFileGz), + data: dataGz, + records: records.Records, contentType: contentTypeJSON, + objects: make([][]byte, 0), } } -func (c constantS3) GetObject(ctx context.Context, bucket, key string) (*s3.GetObjectOutput, error) { - return newS3GetObjectResponse(c.filename, c.data, c.contentType), nil +func (c *constantS3) GetObject(ctx context.Context, bucket, key string) (*s3.GetObjectOutput, error) { + // direct listing uses gz content + if strings.HasSuffix(key, ".json.gz") { + return newS3GetObjectResponse(c.filename, c.data, c.contentType), nil + } + + // this is s3 sqs notification + keyN, err := strconv.Atoi(key) + if err != nil { + return nil, err + } + + return newS3GetObjectResponse(key, c.objects[keyN], c.contentType), nil } -func (c constantS3) CopyObject(ctx context.Context, from_bucket, to_bucket, from_key, to_key string) (*s3.CopyObjectOutput, error) { +func (c *constantS3) CopyObject(ctx context.Context, from_bucket, to_bucket, from_key, to_key string) (*s3.CopyObjectOutput, error) { return nil, nil } -func (c constantS3) DeleteObject(ctx context.Context, bucket, key string) (*s3.DeleteObjectOutput, error) { +func (c *constantS3) DeleteObject(ctx context.Context, bucket, key string) (*s3.DeleteObjectOutput, error) { return nil, nil } -func (c constantS3) ListObjectsPaginator(bucket, prefix string) s3Pager { +func (c *constantS3) ListObjectsPaginator(bucket, prefix string) s3Pager { return c.pagerConstant } var _ beat.Pipeline = (*fakePipeline)(nil) +func newFakePipeline() *fakePipeline { + fp := &fakePipeline{flush: time.NewTicker(10 * time.Second), log: log, mutex: new(sync.Mutex)} + go func() { + for { + fp.mutex.Lock() + select { + case <-fp.flush.C: + fp.ackPendings() + default: + if fp.pendingEvents > 3200 { + fp.ackPendings() + } + } + + fp.mutex.Unlock() + } + }() + + return fp +} + // fakePipeline returns new ackClients. -type fakePipeline struct{} +type fakePipeline struct { + flush *time.Ticker + mutex *sync.Mutex + pendingEvents int + publishedEvents int + events []*beat.Event + log *logp.Logger +} -func (c *fakePipeline) ConnectWith(clientConfig beat.ClientConfig) (beat.Client, error) { - return &ackClient{}, nil +func (fp *fakePipeline) ackPendings() { + for _, eventToACK := range fp.events { + if eventToACK.Private.(*awscommon.EventACKTracker).PendingACKs.Load() > 0 { + fp.pendingEvents-- + eventToACK.Private.(*awscommon.EventACKTracker).ACK() + } + } + + events := make([]*beat.Event, 0, len(fp.events)) + for _, eventToACK := range fp.events { + if eventToACK.Private.(*awscommon.EventACKTracker).PendingACKs.Load() > 0 { + events = append(events, eventToACK) + } + } + + fp.events = events +} + +func (fp *fakePipeline) ConnectWith(clientConfig beat.ClientConfig) (beat.Client, error) { + return &ackClient{fp: fp}, nil } -func (c *fakePipeline) Connect() (beat.Client, error) { +func (fp *fakePipeline) Connect() (beat.Client, error) { panic("Connect() is not implemented.") } var _ beat.Client = (*ackClient)(nil) // ackClient is a fake beat.Client that ACKs the published messages. -type ackClient struct{} +type ackClient struct { + fp *fakePipeline +} func (c *ackClient) Close() error { return nil } func (c *ackClient) Publish(event beat.Event) { - // Fake the ACK handling. - event.Private.(*awscommon.EventACKTracker).ACK() + c.fp.mutex.Lock() + c.fp.pendingEvents++ + c.fp.publishedEvents++ + c.fp.events = append(c.fp.events, &event) + c.fp.mutex.Unlock() } func (c *ackClient) PublishAll(event []beat.Event) { @@ -197,6 +337,9 @@ file_selectors: - regex: '.json.gz$' expand_event_list_from_field: Records +- + regex: '^[\d]+$' + expand_event_list_from_field: Records `) inputConfig := defaultConfig() @@ -211,27 +354,37 @@ func benchmarkInputSQS(t *testing.T, maxMessagesInflight int) testing.BenchmarkR log := logp.NewLogger(inputName) metricRegistry := monitoring.NewRegistry() metrics := newInputMetrics("test_id", metricRegistry, maxMessagesInflight) - sqsAPI := newConstantSQS() + totalSqsMessages := uint64(math.Ceil(float64(maxMessagesInflight) * 1.1)) s3API := newConstantS3(t) - pipeline := &fakePipeline{} + sqsAPI := newConstantSQS(t, maxMessagesInflight, totalSqsMessages, s3API) + + logSqs := log.Named("sqs") + pipeline := newFakePipeline() + conf := makeBenchmarkConfig(t) s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), metrics, s3API, conf.FileSelectors, backupConfig{}, maxMessagesInflight) sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), metrics, sqsAPI, nil, time.Minute, 5, pipeline, s3EventHandlerFactory, maxMessagesInflight) - sqsReader := newSQSReader(log.Named("sqs"), metrics, sqsAPI, maxMessagesInflight, sqsMessageHandler) + sqsReader := newSQSReader(logSqs, metrics, sqsAPI, maxMessagesInflight, sqsMessageHandler) ctx, cancel := context.WithCancel(context.Background()) b.Cleanup(cancel) + cancelChan := make(chan time.Duration) + start := time.Now() go func() { - for metrics.sqsMessagesReceivedTotal.Get() < uint64(b.N) { + for { + if metrics.sqsMessagesProcessedTotal.Get() == totalSqsMessages { + break + } time.Sleep(5 * time.Millisecond) } + cancel() + cancelChan <- time.Since(start) }() b.ResetTimer() - start := time.Now() if err := sqsReader.Receive(ctx, pipeline); err != nil { if !errors.Is(err, context.DeadlineExceeded) { t.Fatal(err) @@ -239,15 +392,23 @@ func benchmarkInputSQS(t *testing.T, maxMessagesInflight int) testing.BenchmarkR } b.StopTimer() elapsed := time.Since(start) + cancelElapsed := <-cancelChan + deltaElapsedCanceled := elapsed.Seconds() - cancelElapsed.Seconds() b.ReportMetric(float64(maxMessagesInflight), "max_messages_inflight") b.ReportMetric(elapsed.Seconds(), "sec") + b.ReportMetric(cancelElapsed.Seconds(), "cancel_sec") + b.ReportMetric(deltaElapsedCanceled, "delta_sec_from_cancel") + b.ReportMetric(100.*(deltaElapsedCanceled/elapsed.Seconds()), "flushing_time_percentage") b.ReportMetric(float64(metrics.s3EventsCreatedTotal.Get()), "events") - b.ReportMetric(float64(metrics.s3EventsCreatedTotal.Get())/elapsed.Seconds(), "events_per_sec") + b.ReportMetric(float64(metrics.s3EventsCreatedTotal.Get())/cancelElapsed.Seconds(), "events_per_sec") b.ReportMetric(float64(metrics.s3BytesProcessedTotal.Get()), "s3_bytes") - b.ReportMetric(float64(metrics.s3BytesProcessedTotal.Get())/elapsed.Seconds(), "s3_bytes_per_sec") + b.ReportMetric(float64(metrics.s3BytesProcessedTotal.Get())/cancelElapsed.Seconds(), "s3_bytes_per_sec") + + b.ReportMetric(float64(metrics.s3ObjectsRequestedTotal.Get()), "s3_objects") + b.ReportMetric(float64(metrics.s3ObjectsRequestedTotal.Get())/cancelElapsed.Seconds(), "s3_objects_per_sec") b.ReportMetric(float64(metrics.sqsMessagesDeletedTotal.Get()), "sqs_messages") b.ReportMetric(float64(metrics.sqsMessagesDeletedTotal.Get())/elapsed.Seconds(), "sqs_messages_per_sec") @@ -273,18 +434,36 @@ func TestBenchmarkInputSQS(t *testing.T) { headers := []string{ "Max Msgs Inflight", + "Events total", "Events per sec", + "S3 Bytes total", "S3 Bytes per sec", - "Time (sec)", + "S3 Objects total", + "S3 Objects per sec", + "SQS Messages total", + "SQS Messages per sec", + "Full Time (sec)", + "Processing Time (sec)", + "Flushing Time (sec)", + "Flushing time (%)", "CPUs", } data := make([][]string, 0) for _, r := range results { data = append(data, []string{ fmt.Sprintf("%v", r.Extra["max_messages_inflight"]), + fmt.Sprintf("%v", r.Extra["events"]), fmt.Sprintf("%v", r.Extra["events_per_sec"]), + fmt.Sprintf("%v", humanize.Bytes(uint64(r.Extra["s3_bytes"]))), fmt.Sprintf("%v", humanize.Bytes(uint64(r.Extra["s3_bytes_per_sec"]))), + fmt.Sprintf("%v", r.Extra["s3_objects"]), + fmt.Sprintf("%v", r.Extra["s3_objects_per_sec"]), + fmt.Sprintf("%v", r.Extra["sqs_messages"]), + fmt.Sprintf("%v", r.Extra["sqs_messages_per_sec"]), fmt.Sprintf("%v", r.Extra["sec"]), + fmt.Sprintf("%v", r.Extra["cancel_sec"]), + fmt.Sprintf("%v", r.Extra["delta_sec_from_cancel"]), + fmt.Sprintf("%v", humanize.FormatFloat("#,##", r.Extra["flushing_time_percentage"])), fmt.Sprintf("%v", runtime.GOMAXPROCS(0)), }) } @@ -311,7 +490,7 @@ func benchmarkInputS3(t *testing.T, numberOfWorkers int) testing.BenchmarkResult _ = client.Close() }() - config := makeBenchmarkConfig(t) + conf := makeBenchmarkConfig(t) b.ResetTimer() start := time.Now() @@ -347,7 +526,7 @@ func benchmarkInputS3(t *testing.T, numberOfWorkers int) testing.BenchmarkResult return } - s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), metrics, s3API, config.FileSelectors, backupConfig{}, numberOfWorkers) + s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), metrics, s3API, conf.FileSelectors, backupConfig{}, numberOfWorkers) s3Poller := newS3Poller(logp.NewLogger(inputName), metrics, s3API, client, s3EventHandlerFactory, newStates(inputCtx), store, "bucket", listPrefix, "region", "provider", numberOfWorkers, time.Second) if err := s3Poller.Poll(ctx); err != nil { diff --git a/x-pack/filebeat/input/awss3/metrics.go b/x-pack/filebeat/input/awss3/metrics.go index bef57210ca6..ecea902c332 100644 --- a/x-pack/filebeat/input/awss3/metrics.go +++ b/x-pack/filebeat/input/awss3/metrics.go @@ -54,6 +54,7 @@ type inputMetrics struct { sqsWorkerStartTimes map[uint64]time.Time // Map of SQS worker ID to the time at which the worker started. sqsMessagesReceivedTotal *monitoring.Uint // Number of SQS messages received (not necessarily processed fully). + sqsMessagesProcessedTotal *monitoring.Uint // Number of SQS messages processed fully. sqsVisibilityTimeoutExtensionsTotal *monitoring.Uint // Number of SQS visibility timeout extensions. sqsMessagesInflight *monitoring.Uint // Number of SQS messages inflight (gauge). sqsMessagesReturnedTotal *monitoring.Uint // Number of SQS message returned to queue (happens on errors implicitly after visibility timeout passes). @@ -158,6 +159,7 @@ func newInputMetrics(id string, optionalParent *monitoring.Registry, maxWorkers sqsWorkerStartTimes: map[uint64]time.Time{}, sqsWorkerUtilizationLastUpdate: currentTime(), sqsMessagesReceivedTotal: monitoring.NewUint(reg, "sqs_messages_received_total"), + sqsMessagesProcessedTotal: monitoring.NewUint(reg, "sqs_messages_processed_total"), sqsVisibilityTimeoutExtensionsTotal: monitoring.NewUint(reg, "sqs_visibility_timeout_extensions_total"), sqsMessagesInflight: monitoring.NewUint(reg, "sqs_messages_inflight_gauge"), sqsMessagesReturnedTotal: monitoring.NewUint(reg, "sqs_messages_returned_total"), diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index 32911778336..9b9b9f6f1ee 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -233,7 +233,8 @@ func (p *s3ObjectProcessor) readJSON(r io.Reader) error { dec := json.NewDecoder(r) dec.UseNumber() - for dec.More() && p.ctx.Err() == nil { + // why we check ctx.Err() here? + for dec.More() && (errors.Is(p.ctx.Err(), context.Canceled) || p.ctx.Err() == nil) { offset := dec.InputOffset() var item json.RawMessage @@ -270,7 +271,8 @@ func (p *s3ObjectProcessor) readJSONSlice(r io.Reader, evtOffset int64) (int64, } // we track each event offset separately since we are reading a slice. - for dec.More() && p.ctx.Err() == nil { + // why we check ctx.Err() here? + for dec.More() && (errors.Is(p.ctx.Err(), context.Canceled) || p.ctx.Err() == nil) { var item json.RawMessage if err := dec.Decode(&item); err != nil { return -1, fmt.Errorf("failed to decode json: %w", err) diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index f8ad084a4d6..c221834a78b 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -339,7 +339,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents int, if !expectErr { require.NoError(t, err) assert.Equal(t, numEvents, len(events)) - assert.EqualValues(t, numEvents, ack.PendingACKs) + assert.EqualValues(t, numEvents, ack.PendingACKs.Load()) } else { require.Error(t, err) } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index cf6efc97bef..7aca7214362 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -194,7 +194,7 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message } handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) - + p.metrics.sqsMessagesProcessedTotal.Inc() return receiveCount, handles, keepaliveCancel, &keepaliveWg, processingErr } diff --git a/x-pack/filebeat/input/awss3/testdata/aws-cloudtrail.json b/x-pack/filebeat/input/awss3/testdata/aws-cloudtrail.json new file mode 100644 index 00000000000..7c543cf58f2 --- /dev/null +++ b/x-pack/filebeat/input/awss3/testdata/aws-cloudtrail.json @@ -0,0 +1 @@ +{"Records":[{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}},{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}},{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}},{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}},{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}},{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}},{"eventVersion":"1.07","eventTime":"2019-11-14T00:51:00Z","awsRegion":"us-east-1","eventID":"EXAMPLE8-9621-4d00-b913-beca2EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"Start","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664}}}},"eventCategory":"Insight"},{"eventVersion":"1.07","eventTime":"2019-11-14T00:52:00Z","awsRegion":"us-east-1","eventID":"EXAMPLEc-28be-486c-8928-49ce6EXAMPLE","eventType":"AwsCloudTrailInsight","recipientAccountId":"123456789012","sharedEventID":"EXAMPLE2-1729-42f1-b735-5d8c0EXAMPLE","insightDetails":{"state":"End","eventSource":"ssm.amazonaws.com","eventName":"UpdateInstanceInformation","insightType":"ApiCallRateInsight","insightContext":{"statistics":{"baseline":{"average":85.4202380952},"insight":{"average":664},"insightDuration":1}}},"eventCategory":"Insight"},{"eventVersion":"1.04","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2016-07-14T19:15:45Z","eventSource":"cloudtrail.amazonaws.com","eventName":"UpdateTrail","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.182","userAgent":"aws-cli/1.10.32 Python/2.7.9 Windows/7 botocore/1.4.22","errorCode":"TrailNotFoundException","errorMessage":"Unknown trail: myTrail2 for the user: 123456789012","requestParameters":{"name":"myTrail2"},"responseElements":null,"requestID":"5d40662a-49f7-11e6-97e4-d9cb6ff7d6a3","eventID":"b7d4398e-b2f0-4faa-9c76-e2d316a8d67f","eventType":"AwsApiCall","recipientAccountId":"123456789012"},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-25T20:17:37Z","eventSource":"iam.amazonaws.com","eventName":"CreateRole","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"assumeRolePolicyDocument":"{\n \"Version\": \"2012-10-17\",\n \"Statement\": [\n {\n \"Sid\": \"\",\n\"Effect\": \"Allow\",\n \"Principal\": {\n \"AWS\": \"arn:aws:iam::210987654321:root\"\n },\n \"Action\": \"sts:AssumeRole\"\n }\n ]\n}","roleName":"TestRole"},"responseElements":{"role":{"assumeRolePolicyDocument":"%7B%0A%20%20%22Version%22%3A%20%222012-10-17%22%2C%0A%20%20%22Statement%22%3A%20%5B%0A%20%20%20%20%7B%0A%20%20%20%20%20%20%22Sid%22%3A%20%22%22%2C%0A%20%20%20%20%20%20%22Effect%22%3A%20%22Allow%22%2C%0A%20%20%20%20%20%20%22Principal%22%3A%20%7B%0A%20%20%20%20%20%20%20%20%22AWS%22%3A%20%22arn%3Aaws%3Aiam%3A%3A803981987763%3Aroot%22%0A%20%20%20%20%20%20%7D%2C%0A%20%20%20%20%20%20%22Action%22%3A%20%22sts%3AAssumeRole%22%0A%20%20%20%20%7D%0A%20%20%5D%0A%7D","roleName":"TestRole","roleId":"AROAIUU2EOWSWPGX2UJUO","arn":"arn:aws:iam::123456789012:role/TestRole","createDate":"Mar 25, 2014 8:17:37 PM","path":"/"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-25T18:45:11Z"}}},"eventTime":"2014-03-25T21:08:14Z","eventSource":"iam.amazonaws.com","eventName":"AddUserToGroup","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"AWSConsole","requestParameters":{"userName":"Bob","groupName":"admin"},"responseElements":null},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-24T21:11:59Z","eventSource":"iam.amazonaws.com","eventName":"CreateUser","awsRegion":"us-east-2","sourceIPAddress":"127.0.0.1","userAgent":"aws-cli/1.3.2 Python/2.7.5 Windows/7","requestParameters":{"userName":"Bob"},"responseElements":{"user":{"createDate":"Mar 24, 2014 9:11:59 PM","userName":"Bob","arn":"arn:aws:iam::123456789012:user/Bob","path":"/","userId":"EXAMPLEUSERID"}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice","sessionContext":{"attributes":{"mfaAuthenticated":"false","creationDate":"2014-03-06T15:15:06Z"}}},"eventTime":"2014-03-06T17:10:34Z","eventSource":"ec2.amazonaws.com","eventName":"CreateKeyPair","awsRegion":"us-east-2","sourceIPAddress":"72.21.198.64","userAgent":"EC2ConsoleBackend, aws-sdk-java/Linux/x.xx.fleetxen Java_HotSpot(TM)_64-Bit_Server_VM/xx","requestParameters":{"keyName":"mykeypair"},"responseElements":{"keyName":"mykeypair","keyFingerprint":"30:1d:46:d0:5b:ad:7e:1b:b6:70:62:8b:ff:38:b5:e9:ab:5d:b8:21","keyMaterial":""}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accountId":"123456789012","accessKeyId":"EXAMPLE_KEY_ID","userName":"Alice"},"eventTime":"2014-03-06T21:01:59Z","eventSource":"ec2.amazonaws.com","eventName":"StopInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]},"force":false},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":64,"name":"stopping"},"previousState":{"code":16,"name":"running"}}]}}},{"eventVersion":"1.0","userIdentity":{"type":"IAMUser","principalId":"EX_PRINCIPAL_ID","arn":"arn:aws:iam::123456789012:user/Alice","accessKeyId":"EXAMPLE_KEY_ID","accountId":"123456789012","userName":"Alice"},"eventTime":"2014-03-06T21:22:54Z","eventSource":"ec2.amazonaws.com","eventName":"StartInstances","awsRegion":"us-east-2","sourceIPAddress":"205.251.233.176","userAgent":"ec2-api-tools 1.6.12.2","requestParameters":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2"}]}},"responseElements":{"instancesSet":{"items":[{"instanceId":"i-ebeaf9e2","currentState":{"code":0,"name":"pending"},"previousState":{"code":80,"name":"stopped"}}]}}}]} diff --git a/x-pack/libbeat/common/aws/acker.go b/x-pack/libbeat/common/aws/acker.go index 95fbe14b774..1de3805e6b2 100644 --- a/x-pack/libbeat/common/aws/acker.go +++ b/x-pack/libbeat/common/aws/acker.go @@ -10,6 +10,7 @@ import ( "github.com/elastic/beats/v7/libbeat/beat" "github.com/elastic/beats/v7/libbeat/common/acker" + "github.com/elastic/beats/v7/libbeat/common/atomic" ) // EventACKTracker tracks the publishing state of S3 objects. Specifically @@ -18,54 +19,43 @@ import ( // more S3 objects. type EventACKTracker struct { sync.Mutex - PendingACKs int64 + PendingACKs *atomic.Uint64 ctx context.Context cancel context.CancelFunc } func NewEventACKTracker(ctx context.Context) *EventACKTracker { ctx, cancel := context.WithCancel(ctx) - return &EventACKTracker{ctx: ctx, cancel: cancel} + return &EventACKTracker{ctx: ctx, cancel: cancel, PendingACKs: atomic.NewUint64(0)} } // Add increments the number of pending ACKs. func (a *EventACKTracker) Add() { - a.Lock() - a.PendingACKs++ - a.Unlock() + a.PendingACKs.Inc() } // ACK decrements the number of pending ACKs. func (a *EventACKTracker) ACK() { - a.Lock() - defer a.Unlock() - - if a.PendingACKs <= 0 { + if a.PendingACKs.Load() <= 0 { panic("misuse detected: negative ACK counter") } - a.PendingACKs-- - if a.PendingACKs == 0 { - a.cancel() - } + a.PendingACKs.Dec() } // Wait waits for the number of pending ACKs to be zero. -// Wait must be called sequentially only after every expected +// Wait must be called only after every expected // `Add` calls are made. Failing to do so could reset the pendingACKs // property to 0 and would results in Wait returning after additional // calls to `Add` are made without a corresponding `ACK` call. func (a *EventACKTracker) Wait() { - // If there were never any pending ACKs then cancel the context. (This can - // happen when a document contains no events or cannot be read due to an error). - a.Lock() - if a.PendingACKs == 0 { - a.cancel() - } - a.Unlock() - // Wait. - <-a.ctx.Done() + // If there were no any pending ACKs returns. + for { + if a.PendingACKs.Load() == 0 { + return + } + } } // NewEventACKHandler returns a beat ACKer that can receive callbacks when diff --git a/x-pack/libbeat/common/aws/acker_test.go b/x-pack/libbeat/common/aws/acker_test.go index 3c470f0b922..b989089d4d8 100644 --- a/x-pack/libbeat/common/aws/acker_test.go +++ b/x-pack/libbeat/common/aws/acker_test.go @@ -21,7 +21,7 @@ func TestEventACKTracker(t *testing.T) { acker.Add() acker.ACK() - assert.EqualValues(t, 0, acker.PendingACKs) + assert.EqualValues(t, 0, acker.PendingACKs.Load()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -32,7 +32,7 @@ func TestEventACKTrackerNoACKs(t *testing.T) { acker := NewEventACKTracker(ctx) acker.Wait() - assert.EqualValues(t, 0, acker.PendingACKs) + assert.EqualValues(t, 0, acker.PendingACKs.Load()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -49,7 +49,7 @@ func TestEventACKHandler(t *testing.T) { ackHandler.AddEvent(beat.Event{Private: acker}, true) ackHandler.ACKEvents(1) - assert.EqualValues(t, 0, acker.PendingACKs) + assert.EqualValues(t, 0, acker.PendingACKs.Load()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -64,6 +64,6 @@ func TestEventACKHandlerWait(t *testing.T) { acker.Wait() acker.Add() - assert.EqualValues(t, 1, acker.PendingACKs) + assert.EqualValues(t, 1, acker.PendingACKs.Load()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } From 7120e9fd26906f94e7d7ed0f118a2dadd66cabc6 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 4 Mar 2024 16:33:02 +0900 Subject: [PATCH 05/43] Improve sqsReader.Receive --- x-pack/filebeat/input/awss3/sqs.go | 122 +++++++++++++++++++++-------- 1 file changed, 88 insertions(+), 34 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 9683ecb541c..b6aaafb05d5 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -60,54 +60,82 @@ func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessag } func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { - // This loop tries to keep the workers busy as much as possible while + // The loop tries to keep the ProcessSQS workers busy as much as possible while // honoring the max message cap as opposed to a simpler loop that receives - // N messages, waits for them all to finish, then requests N more messages. - var workerWg sync.WaitGroup - endingChan := make(chan error, 1) + // N messages, waits for them all to finish sending events to the queue, then requests N more messages. + var processingWg sync.WaitGroup + + // At the end of the loop, after a shutdown or anything else, we still need to wait for the DeleteSQS goroutines to + // complete, otherwise the messages will be sent back to the queue even if they should be deleted. + var deletionWg sync.WaitGroup + + // We send to processingChan the outcome of each ProcessSQS call. + // We don't want to buffer the processingChan, since it will prevent workers ProcessSQS goroutines to return. processingChan := make(chan processingOutcome) - go func(ctx context.Context) { - // Wait for all workers to finish. + // We use deletionChan to throttle the number of DeleteSQS goroutines. + // deletionChan := make(chan struct{}, 3200) + + go func() { for { select { - case processOutcome, ok := <-processingChan: + case outcome, ok := <-processingChan: + // processingChang is closed, no more outcomes to process, we can exit. if !ok { - if errors.Is(ctx.Err(), context.Canceled) { - // A canceled context is a normal shutdown. - close(endingChan) - return - } - - endingChan <- ctx.Err() return } - go func(processOutcome processingOutcome) { + // A ProcessSQS goroutine has sent an outcome, let's process it asynchronously in order to handle SQS message deletion. + go func(outcome processingOutcome) { + // Mark deletion wait group as done when the goroutine is done. + defer deletionWg.Done() + + r.log.Debugw("Waiting worker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) + + // We don't want to cap processingChan, since it will prevent workers ProcessSQS goroutines to return + // and in flight message would be capped as well. + // We want to cap number of goroutines for DeleteSQS + // deletionChan <- struct{}{} + + r.log.Debugw("Waited worker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) + + r.log.Debugw("Waiting acker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) + // Wait for all events to be ACKed before proceeding. - processOutcome.acker.Wait() + outcome.acker.Wait() + + r.log.Debugw("Waited acker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) - // Stop keepalive routine before deleting visibility. - processOutcome.keepaliveCancel() - processOutcome.keepaliveWg.Wait() + // Stop keepalive visibility routine before deleting. + outcome.keepaliveCancel() + outcome.keepaliveWg.Wait() - err := r.msgHandler.DeleteSQS(processOutcome.msg, processOutcome.receiveCount, processOutcome.processingErr, processOutcome.handles) + err := r.msgHandler.DeleteSQS(outcome.msg, outcome.receiveCount, outcome.processingErr, outcome.handles) if err != nil { r.log.Warnw("Failed deleting SQS message.", "error", err, - "message_id", *processOutcome.msg.MessageId, - "elapsed_time_ns", time.Since(processOutcome.start)) + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) } else { - r.log.Warnw("Success deleting SQS message.", - "message_id", *processOutcome.msg.MessageId, - "elapsed_time_ns", time.Since(processOutcome.start)) + r.log.Debugw("Success deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) } - }(processOutcome) - default: - time.Sleep(500 * time.Microsecond) + + // <-deletionChan + }(outcome) } + } - }(ctx) + }() for ctx.Err() == nil { // Determine how many SQS workers are available. @@ -133,17 +161,20 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { // Release unused workers. r.workerSem.Release(workers - len(msgs)) - // Process each SQS message asynchronously with a goroutine. r.log.Debugf("Received %v SQS messages.", len(msgs)) r.metrics.sqsMessagesReceivedTotal.Add(uint64(len(msgs))) - workerWg.Add(len(msgs)) + + // Add to processing wait group to wait for all messages to be processed. + processingWg.Add(len(msgs)) for _, msg := range msgs { + // Process each SQS message asynchronously with a goroutine. go func(msg types.Message, start time.Time) { id := r.metrics.beginSQSWorker() defer func() { + // Mark processing wait group as done. r.metrics.endSQSWorker(id) - workerWg.Done() + processingWg.Done() r.workerSem.Release(1) }() @@ -168,10 +199,22 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { defer client.Close() + r.log.Debugw("Going to process SQS message.", + "message_id", *msg.MessageId, + "elapsed_time_ns", time.Since(start)) + acker := awscommon.NewEventACKTracker(ctx) receiveCount, handles, keepaliveCancel, keepaliveWg, processingErr := r.msgHandler.ProcessSQS(ctx, &msg, client, acker) + r.log.Debugw("Success processing SQS message.", + "message_id", *msg.MessageId, + "elapsed_time_ns", time.Since(start)) + + // Add to deletion waiting group before sending to processingChan. + deletionWg.Add(1) + + // Send the outcome to the processingChan so the deletion goroutine can delete the message. processingChan <- processingOutcome{ start: start, keepaliveWg: keepaliveWg, @@ -186,10 +229,21 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { } } - workerWg.Wait() + // Wait for all processing goroutines to finish. + processingWg.Wait() + + // We need to close the processingChan to signal to the deletion goroutines that they should stop. close(processingChan) - return <-endingChan + // Wait for all deletion goroutines to finish. + deletionWg.Wait() + + if errors.Is(ctx.Err(), context.Canceled) { + // A canceled context is a normal shutdown. + return nil + } + + return ctx.Err() } func (r *sqsReader) GetApproximateMessageCount(ctx context.Context) (int, error) { From edae48fa2dd954a326515278fa30872ab2de9b32 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 4 Mar 2024 17:16:07 +0900 Subject: [PATCH 06/43] linting --- x-pack/filebeat/input/awss3/sqs.go | 89 +++++++++++++++--------------- 1 file changed, 43 insertions(+), 46 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index b6aaafb05d5..55b576ddad6 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -78,62 +78,59 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { go func() { for { - select { - case outcome, ok := <-processingChan: - // processingChang is closed, no more outcomes to process, we can exit. - if !ok { - return - } + outcome, ok := <-processingChan + // processingChang is closed, no more outcomes to process, we can exit. + if !ok { + return + } - // A ProcessSQS goroutine has sent an outcome, let's process it asynchronously in order to handle SQS message deletion. - go func(outcome processingOutcome) { - // Mark deletion wait group as done when the goroutine is done. - defer deletionWg.Done() + // A ProcessSQS goroutine has sent an outcome, let's process it asynchronously in order to handle SQS message deletion. + go func(outcome processingOutcome) { + // Mark deletion wait group as done when the goroutine is done. + defer deletionWg.Done() - r.log.Debugw("Waiting worker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) + r.log.Debugw("Waiting worker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) - // We don't want to cap processingChan, since it will prevent workers ProcessSQS goroutines to return - // and in flight message would be capped as well. - // We want to cap number of goroutines for DeleteSQS - // deletionChan <- struct{}{} + // We don't want to cap processingChan, since it will prevent workers ProcessSQS goroutines to return + // and in flight message would be capped as well. + // We want to cap number of goroutines for DeleteSQS + // deletionChan <- struct{}{} - r.log.Debugw("Waited worker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) + r.log.Debugw("Waited worker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) - r.log.Debugw("Waiting acker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) + r.log.Debugw("Waiting acker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) - // Wait for all events to be ACKed before proceeding. - outcome.acker.Wait() + // Wait for all events to be ACKed before proceeding. + outcome.acker.Wait() - r.log.Debugw("Waited acker when deleting SQS message.", + r.log.Debugw("Waited acker when deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) + + // Stop keepalive visibility routine before deleting. + outcome.keepaliveCancel() + outcome.keepaliveWg.Wait() + + err := r.msgHandler.DeleteSQS(outcome.msg, outcome.receiveCount, outcome.processingErr, outcome.handles) + if err != nil { + r.log.Warnw("Failed deleting SQS message.", + "error", err, "message_id", *outcome.msg.MessageId, "elapsed_time_ns", time.Since(outcome.start)) + } else { + r.log.Debugw("Success deleting SQS message.", + "message_id", *outcome.msg.MessageId, + "elapsed_time_ns", time.Since(outcome.start)) + } - // Stop keepalive visibility routine before deleting. - outcome.keepaliveCancel() - outcome.keepaliveWg.Wait() - - err := r.msgHandler.DeleteSQS(outcome.msg, outcome.receiveCount, outcome.processingErr, outcome.handles) - if err != nil { - r.log.Warnw("Failed deleting SQS message.", - "error", err, - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - } else { - r.log.Debugw("Success deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - } - - // <-deletionChan - }(outcome) - } - + // <-deletionChan + }(outcome) } }() From a69647375f393bc40d91faf88aad7f2063ffdf60 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 4 Mar 2024 17:56:27 +0900 Subject: [PATCH 07/43] remove ctx from awscommon-EventACKTracker --- .../filebeat/input/awscloudwatch/processor.go | 2 +- x-pack/filebeat/input/awss3/s3.go | 2 +- .../filebeat/input/awss3/s3_objects_test.go | 14 +++++------ x-pack/filebeat/input/awss3/sqs.go | 2 +- .../filebeat/input/awss3/sqs_s3_event_test.go | 12 ++++----- x-pack/libbeat/common/aws/acker.go | 8 ++---- x-pack/libbeat/common/aws/acker_test.go | 25 +++---------------- 7 files changed, 22 insertions(+), 43 deletions(-) diff --git a/x-pack/filebeat/input/awscloudwatch/processor.go b/x-pack/filebeat/input/awscloudwatch/processor.go index 999cad4d7f0..91a8c29cbf8 100644 --- a/x-pack/filebeat/input/awscloudwatch/processor.go +++ b/x-pack/filebeat/input/awscloudwatch/processor.go @@ -31,7 +31,7 @@ func newLogProcessor(log *logp.Logger, metrics *inputMetrics, publisher beat.Cli log: log, metrics: metrics, publisher: publisher, - ack: awscommon.NewEventACKTracker(ctx), + ack: awscommon.NewEventACKTracker(), } } diff --git a/x-pack/filebeat/input/awss3/s3.go b/x-pack/filebeat/input/awss3/s3.go index 96be746c160..1d7c8954d1a 100644 --- a/x-pack/filebeat/input/awss3/s3.go +++ b/x-pack/filebeat/input/awss3/s3.go @@ -128,7 +128,7 @@ func (p *s3Poller) createS3ObjectProcessor(ctx context.Context, state state) (s3 event.S3.Bucket.ARN = p.bucket event.S3.Object.Key = state.Key - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() return p.s3ObjectHandler.Create(ctx, p.log, p.client, acker, event), event } diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index c221834a78b..07b09c3fb9a 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -155,7 +155,7 @@ func TestS3ObjectProcessor(t *testing.T) { Return(nil, errFakeConnectivityFailure) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() require.Error(t, err) assert.True(t, errors.Is(err, errFakeConnectivityFailure), "expected errFakeConnectivityFailure error") @@ -177,7 +177,7 @@ func TestS3ObjectProcessor(t *testing.T) { Return(nil, nil) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() require.Error(t, err) }) @@ -204,7 +204,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() require.NoError(t, err) }) @@ -230,7 +230,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupCfg, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).FinalizeS3Object() require.NoError(t, err) }) @@ -260,7 +260,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupCfg, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).FinalizeS3Object() require.NoError(t, err) }) @@ -287,7 +287,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupCfg, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).FinalizeS3Object() require.NoError(t, err) }) @@ -333,7 +333,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents int, ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, selectors, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker(ctx) + ack := awscommon.NewEventACKTracker() err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() if !expectErr { diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 55b576ddad6..8f71ffac73e 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -200,7 +200,7 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() receiveCount, handles, keepaliveCancel, keepaliveWg, processingErr := r.msgHandler.ProcessSQS(ctx, &msg, client, acker) diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index fb190c5aed3..ef86b478469 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -49,7 +49,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) require.NoError(t, processingErr) @@ -76,7 +76,7 @@ func TestSQSS3EventProcessor(t *testing.T) { invalidBodyMsg.Body = &body mockClient := NewMockBeatClient(ctrl) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker) @@ -103,7 +103,7 @@ func TestSQSS3EventProcessor(t *testing.T) { emptyRecordsMsg := newSQSMessage([]s3EventV2{}...) mockClient := NewMockBeatClient(ctrl) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker) @@ -140,7 +140,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().ProcessS3Object().Return(nil), ) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory, 5) receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) @@ -171,7 +171,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().ProcessS3Object().Return(errors.New("fake connectivity problem")), ) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) @@ -203,7 +203,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().ProcessS3Object().Return(errors.New("fake connectivity problem")), ) - acker := awscommon.NewEventACKTracker(ctx) + acker := awscommon.NewEventACKTracker() p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) diff --git a/x-pack/libbeat/common/aws/acker.go b/x-pack/libbeat/common/aws/acker.go index 1de3805e6b2..531bb2862eb 100644 --- a/x-pack/libbeat/common/aws/acker.go +++ b/x-pack/libbeat/common/aws/acker.go @@ -5,7 +5,6 @@ package aws import ( - "context" "sync" "github.com/elastic/beats/v7/libbeat/beat" @@ -20,13 +19,10 @@ import ( type EventACKTracker struct { sync.Mutex PendingACKs *atomic.Uint64 - ctx context.Context - cancel context.CancelFunc } -func NewEventACKTracker(ctx context.Context) *EventACKTracker { - ctx, cancel := context.WithCancel(ctx) - return &EventACKTracker{ctx: ctx, cancel: cancel, PendingACKs: atomic.NewUint64(0)} +func NewEventACKTracker() *EventACKTracker { + return &EventACKTracker{PendingACKs: atomic.NewUint64(0)} } // Add increments the number of pending ACKs. diff --git a/x-pack/libbeat/common/aws/acker_test.go b/x-pack/libbeat/common/aws/acker_test.go index b989089d4d8..07a006d5dda 100644 --- a/x-pack/libbeat/common/aws/acker_test.go +++ b/x-pack/libbeat/common/aws/acker_test.go @@ -5,7 +5,6 @@ package aws import ( - "context" "testing" "github.com/stretchr/testify/assert" @@ -14,34 +13,23 @@ import ( ) func TestEventACKTracker(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - - acker := NewEventACKTracker(ctx) + acker := NewEventACKTracker() acker.Add() acker.ACK() assert.EqualValues(t, 0, acker.PendingACKs.Load()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } func TestEventACKTrackerNoACKs(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - - acker := NewEventACKTracker(ctx) + acker := NewEventACKTracker() acker.Wait() assert.EqualValues(t, 0, acker.PendingACKs.Load()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } func TestEventACKHandler(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - // Create acker. Add one pending ACK. - acker := NewEventACKTracker(ctx) + acker := NewEventACKTracker() acker.Add() // Create an ACK handler and simulate one ACKed event. @@ -50,20 +38,15 @@ func TestEventACKHandler(t *testing.T) { ackHandler.ACKEvents(1) assert.EqualValues(t, 0, acker.PendingACKs.Load()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } func TestEventACKHandlerWait(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - // Create acker. Add one pending ACK. - acker := NewEventACKTracker(ctx) + acker := NewEventACKTracker() acker.Add() acker.ACK() acker.Wait() acker.Add() assert.EqualValues(t, 1, acker.PendingACKs.Load()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } From 0636921271bf263a39f7abc3858b40795a006827 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 09:41:02 +0900 Subject: [PATCH 08/43] Revert awscommon.EventAckTracker and add awss3.EventACKTracker --- .../filebeat/input/awscloudwatch/processor.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker.go | 148 ++++++++++++++++++ x-pack/filebeat/input/awss3/sqs_acker_test.go | 73 +++++++++ x-pack/libbeat/common/aws/acker.go | 42 +++-- x-pack/libbeat/common/aws/acker_test.go | 33 +++- 5 files changed, 275 insertions(+), 23 deletions(-) create mode 100644 x-pack/filebeat/input/awss3/sqs_acker.go create mode 100644 x-pack/filebeat/input/awss3/sqs_acker_test.go diff --git a/x-pack/filebeat/input/awscloudwatch/processor.go b/x-pack/filebeat/input/awscloudwatch/processor.go index 91a8c29cbf8..999cad4d7f0 100644 --- a/x-pack/filebeat/input/awscloudwatch/processor.go +++ b/x-pack/filebeat/input/awscloudwatch/processor.go @@ -31,7 +31,7 @@ func newLogProcessor(log *logp.Logger, metrics *inputMetrics, publisher beat.Cli log: log, metrics: metrics, publisher: publisher, - ack: awscommon.NewEventACKTracker(), + ack: awscommon.NewEventACKTracker(ctx), } } diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go new file mode 100644 index 00000000000..f2ea2e271df --- /dev/null +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -0,0 +1,148 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + +package awss3 + +import ( + "context" + "github.com/aws/aws-sdk-go-v2/service/sqs/types" + "github.com/elastic/elastic-agent-libs/logp" + "sync" + "time" + + "github.com/elastic/beats/v7/libbeat/beat" + "github.com/elastic/beats/v7/libbeat/common/acker" + "github.com/elastic/beats/v7/libbeat/common/atomic" +) + +var totMessageDeleted *atomic.Uint64 + +func init() { + totMessageDeleted = atomic.NewUint64(0) +} + +// EventACKTracker tracks the publishing state of S3 objects. Specifically +// it tracks the number of message acknowledgements that are pending from the +// output. It can be used to wait until all ACKs have been received for one or +// more S3 objects. +type EventACKTracker struct { + DeletionWg *sync.WaitGroup + + EventsToBeAcked *atomic.Uint64 + TotalEventsAcked *atomic.Uint64 + + isSQSAcker bool + + ctx context.Context + cancel context.CancelFunc + + msg *types.Message + ReceiveCount int + start time.Time + processingErr error + Handles []s3ObjectHandler + keepaliveCancel context.CancelFunc + keepaliveWg *sync.WaitGroup + msgHandler sqsProcessor + log *logp.Logger +} + +func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { + ctx, cancel := context.WithCancel(ctx) + return &EventACKTracker{ + ctx: ctx, + cancel: cancel, + DeletionWg: deletionWg, + TotalEventsAcked: atomic.NewUint64(0), + EventsToBeAcked: atomic.NewUint64(0), + } +} + +func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { + a.isSQSAcker = true + + a.msg = msg + a.EventsToBeAcked = atomic.NewUint64(publishedEvent) + a.ReceiveCount = receiveCount + a.start = start + a.processingErr = processingErr + a.Handles = handles + a.keepaliveCancel = keepaliveCancel + a.keepaliveWg = keepaliveWg + a.msgHandler = msgHandler + a.log = log +} + +func (a *EventACKTracker) FullyAcked() bool { + return a.TotalEventsAcked.Load() == a.EventsToBeAcked.Load() + +} +func (a *EventACKTracker) WaitForS3() { + // If it's fully acked then cancel the context. + if a.FullyAcked() { + a.cancel() + } + + // Wait. + <-a.ctx.Done() +} + +// FlushForSQS delete related SQS message +func (a *EventACKTracker) FlushForSQS() { + if !a.isSQSAcker { + return + } + + // Stop keepalive visibility routine before deleting. + a.keepaliveCancel() + a.keepaliveWg.Wait() + + err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) + a.DeletionWg.Done() + totMessageDeleted.Inc() + + if err != nil { + a.log.Warnw("Failed deleting SQS message.", + "error", err, + "message_id", *a.msg.MessageId, + "elapsed_time_ns", time.Since(a.start)) + } else { + a.log.Debugw("Success deleting SQS message.", + "message_id", *a.msg.MessageId, + "elapsed_time_ns", time.Since(a.start)) + } +} + +// ACK decrements the number of total Events ACKed. +func (a *EventACKTracker) ACK() { + if a.FullyAcked() { + panic("misuse detected: ACK call on fully acked") + } + + a.TotalEventsAcked.Inc() + + if a.FullyAcked() { + a.cancel() + } +} + +// NewEventACKHandler returns a beat ACKer that can receive callbacks when +// an event has been ACKed an output. If the event contains a private metadata +// pointing to an eventACKTracker then it will invoke the trackers ACK() method +// to decrement the number of pending ACKs. +func NewEventACKHandler() beat.EventListener { + return acker.ConnectionOnly( + acker.EventPrivateReporter(func(_ int, privates []interface{}) { + for _, current := range privates { + if ackTracker, ok := current.(*EventACKTracker); ok { + ackTracker.ACK() + + if ackTracker.FullyAcked() { + ackTracker.FlushForSQS() + } + } + } + }), + ) +} diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go new file mode 100644 index 00000000000..050cd20d9bf --- /dev/null +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -0,0 +1,73 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + +package awss3 + +import ( + "context" + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/elastic/beats/v7/libbeat/beat" +) + +func TestEventACKTracker(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + acker := NewEventACKTracker(ctx, nil) + acker.EventsToBeAcked.Add(1) + acker.ACK() + + assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) + assert.EqualValues(t, true, acker.FullyAcked()) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} + +func TestEventACKTrackerNoACKs(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + acker := NewEventACKTracker(ctx, nil) + acker.WaitForS3() + + assert.EqualValues(t, 0, acker.EventsToBeAcked.Load()) + assert.EqualValues(t, true, acker.FullyAcked()) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} + +func TestEventACKHandler(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + // Create acker. Add one pending ACK. + acker := NewEventACKTracker(ctx, nil) + acker.EventsToBeAcked.Add(1) + + // Create an ACK handler and simulate one ACKed event. + ackHandler := NewEventACKHandler() + ackHandler.AddEvent(beat.Event{Private: acker}, true) + ackHandler.ACKEvents(1) + + assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) + assert.EqualValues(t, true, acker.FullyAcked()) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} + +func TestEventACKHandlerWaitForS3(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + // Create acker. Add one pending ACK. + acker := NewEventACKTracker(ctx, nil) + acker.EventsToBeAcked.Inc() + acker.ACK() + acker.WaitForS3() + acker.EventsToBeAcked.Inc() + + assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) + assert.EqualValues(t, false, acker.FullyAcked()) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} diff --git a/x-pack/libbeat/common/aws/acker.go b/x-pack/libbeat/common/aws/acker.go index 531bb2862eb..95fbe14b774 100644 --- a/x-pack/libbeat/common/aws/acker.go +++ b/x-pack/libbeat/common/aws/acker.go @@ -5,11 +5,11 @@ package aws import ( + "context" "sync" "github.com/elastic/beats/v7/libbeat/beat" "github.com/elastic/beats/v7/libbeat/common/acker" - "github.com/elastic/beats/v7/libbeat/common/atomic" ) // EventACKTracker tracks the publishing state of S3 objects. Specifically @@ -18,40 +18,54 @@ import ( // more S3 objects. type EventACKTracker struct { sync.Mutex - PendingACKs *atomic.Uint64 + PendingACKs int64 + ctx context.Context + cancel context.CancelFunc } -func NewEventACKTracker() *EventACKTracker { - return &EventACKTracker{PendingACKs: atomic.NewUint64(0)} +func NewEventACKTracker(ctx context.Context) *EventACKTracker { + ctx, cancel := context.WithCancel(ctx) + return &EventACKTracker{ctx: ctx, cancel: cancel} } // Add increments the number of pending ACKs. func (a *EventACKTracker) Add() { - a.PendingACKs.Inc() + a.Lock() + a.PendingACKs++ + a.Unlock() } // ACK decrements the number of pending ACKs. func (a *EventACKTracker) ACK() { - if a.PendingACKs.Load() <= 0 { + a.Lock() + defer a.Unlock() + + if a.PendingACKs <= 0 { panic("misuse detected: negative ACK counter") } - a.PendingACKs.Dec() + a.PendingACKs-- + if a.PendingACKs == 0 { + a.cancel() + } } // Wait waits for the number of pending ACKs to be zero. -// Wait must be called only after every expected +// Wait must be called sequentially only after every expected // `Add` calls are made. Failing to do so could reset the pendingACKs // property to 0 and would results in Wait returning after additional // calls to `Add` are made without a corresponding `ACK` call. func (a *EventACKTracker) Wait() { - - // If there were no any pending ACKs returns. - for { - if a.PendingACKs.Load() == 0 { - return - } + // If there were never any pending ACKs then cancel the context. (This can + // happen when a document contains no events or cannot be read due to an error). + a.Lock() + if a.PendingACKs == 0 { + a.cancel() } + a.Unlock() + + // Wait. + <-a.ctx.Done() } // NewEventACKHandler returns a beat ACKer that can receive callbacks when diff --git a/x-pack/libbeat/common/aws/acker_test.go b/x-pack/libbeat/common/aws/acker_test.go index 07a006d5dda..3c470f0b922 100644 --- a/x-pack/libbeat/common/aws/acker_test.go +++ b/x-pack/libbeat/common/aws/acker_test.go @@ -5,6 +5,7 @@ package aws import ( + "context" "testing" "github.com/stretchr/testify/assert" @@ -13,23 +14,34 @@ import ( ) func TestEventACKTracker(t *testing.T) { - acker := NewEventACKTracker() + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + acker := NewEventACKTracker(ctx) acker.Add() acker.ACK() - assert.EqualValues(t, 0, acker.PendingACKs.Load()) + assert.EqualValues(t, 0, acker.PendingACKs) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } func TestEventACKTrackerNoACKs(t *testing.T) { - acker := NewEventACKTracker() + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + acker := NewEventACKTracker(ctx) acker.Wait() - assert.EqualValues(t, 0, acker.PendingACKs.Load()) + assert.EqualValues(t, 0, acker.PendingACKs) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } func TestEventACKHandler(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + // Create acker. Add one pending ACK. - acker := NewEventACKTracker() + acker := NewEventACKTracker(ctx) acker.Add() // Create an ACK handler and simulate one ACKed event. @@ -37,16 +49,21 @@ func TestEventACKHandler(t *testing.T) { ackHandler.AddEvent(beat.Event{Private: acker}, true) ackHandler.ACKEvents(1) - assert.EqualValues(t, 0, acker.PendingACKs.Load()) + assert.EqualValues(t, 0, acker.PendingACKs) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } func TestEventACKHandlerWait(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + // Create acker. Add one pending ACK. - acker := NewEventACKTracker() + acker := NewEventACKTracker(ctx) acker.Add() acker.ACK() acker.Wait() acker.Add() - assert.EqualValues(t, 1, acker.PendingACKs.Load()) + assert.EqualValues(t, 1, acker.PendingACKs) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } From 96389b1e4741272b7cf92b37ca7be60cca5bd087 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 09:43:33 +0900 Subject: [PATCH 09/43] Implement usage of new awss3.EventACKTracker --- x-pack/filebeat/input/awss3/input.go | 4 +- x-pack/filebeat/input/awss3/interfaces.go | 18 ++-- x-pack/filebeat/input/awss3/s3.go | 5 +- x-pack/filebeat/input/awss3/s3_objects.go | 36 ++++--- x-pack/filebeat/input/awss3/sqs.go | 106 +++----------------- x-pack/filebeat/input/awss3/sqs_s3_event.go | 28 +++--- 6 files changed, 64 insertions(+), 133 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input.go b/x-pack/filebeat/input/awss3/input.go index 283fd567e10..be5445fd394 100644 --- a/x-pack/filebeat/input/awss3/input.go +++ b/x-pack/filebeat/input/awss3/input.go @@ -157,7 +157,7 @@ func (in *s3Input) Run(inputContext v2.Context, pipeline beat.Pipeline) error { // Create client for publishing events and receive notification of their ACKs. client, err := pipeline.ConnectWith(beat.ClientConfig{ CloseRef: inputContext.Cancelation, - EventListener: awscommon.NewEventACKHandler(), + EventListener: NewEventACKHandler(), Processing: beat.ProcessingConfig{ // This input only produces events with basic types so normalization // is not required. @@ -227,7 +227,7 @@ func (in *s3Input) createSQSReceiver(ctx v2.Context, pipeline beat.Pipeline) (*s } in.metrics = newInputMetrics(ctx.ID, nil, in.config.MaxNumberOfMessages) s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), in.metrics, s3API, fileSelectors, in.config.BackupConfig, in.config.MaxNumberOfMessages) - sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), in.metrics, sqsAPI, script, in.config.VisibilityTimeout, in.config.SQSMaxReceiveCount, pipeline, s3EventHandlerFactory, in.config.MaxNumberOfMessages) + sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), in.metrics, sqsAPI, script, in.config.VisibilityTimeout, in.config.SQSMaxReceiveCount, pipeline, s3EventHandlerFactory) sqsReader := newSQSReader(log.Named("sqs"), in.metrics, sqsAPI, in.config.MaxNumberOfMessages, sqsMessageHandler) return sqsReader, nil diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index 1aada072b49..0b7cab05525 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -9,20 +9,17 @@ import ( "errors" "fmt" "net/url" - "sync" "time" smithyhttp "github.com/aws/smithy-go/transport/http" "github.com/aws/smithy-go/middleware" - "github.com/elastic/beats/v7/libbeat/beat" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" - awssdk "github.com/aws/aws-sdk-go-v2/aws" "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/aws/aws-sdk-go-v2/service/sqs" "github.com/aws/aws-sdk-go-v2/service/sqs/types" + "github.com/elastic/beats/v7/libbeat/beat" "github.com/elastic/elastic-agent-libs/logp" ) @@ -66,7 +63,7 @@ type sqsProcessor interface { // given message and is responsible for updating the message's visibility // timeout while it is being processed and for deleting it when processing // completes successfully. - ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *awscommon.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) + ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) error DeleteSQS(msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error } @@ -102,16 +99,16 @@ type s3ObjectHandlerFactory interface { // Create returns a new s3ObjectHandler that can be used to process the // specified S3 object. If the handler is not configured to process the // given S3 object (based on key name) then it will return nil. - Create(ctx context.Context, log *logp.Logger, client beat.Client, acker *awscommon.EventACKTracker, obj s3EventV2) s3ObjectHandler + Create(ctx context.Context, log *logp.Logger, client beat.Client, acker *EventACKTracker, obj s3EventV2) s3ObjectHandler } type s3ObjectHandler interface { // ProcessS3Object downloads the S3 object, parses it, creates events, and // publishes them. It returns when processing finishes or when it encounters // an unrecoverable error. It does not wait for the events to be ACKed by - // the publisher before returning (use eventACKTracker's Wait() method to + // the publisher before returning (use eventACKTracker's WaitForS3() method to // determine this). - ProcessS3Object() error + ProcessS3Object() (uint64, error) // FinalizeS3Object finalizes processing of an S3 object after the current // batch is finished. @@ -119,8 +116,11 @@ type s3ObjectHandler interface { // Wait waits for every event published by ProcessS3Object() to be ACKed // by the publisher before returning. Internally it uses the - // s3ObjectHandler eventACKTracker's Wait() method + // s3ObjectHandler eventACKTracker's WaitForS3() method Wait() + + // SyncEventsToBeAcked sync the number of event published with the eventACKTracker + SyncEventsToBeAcked(s3EventsCreatedTotal uint64) } // ------ diff --git a/x-pack/filebeat/input/awss3/s3.go b/x-pack/filebeat/input/awss3/s3.go index 1d7c8954d1a..3da27203833 100644 --- a/x-pack/filebeat/input/awss3/s3.go +++ b/x-pack/filebeat/input/awss3/s3.go @@ -128,7 +128,7 @@ func (p *s3Poller) createS3ObjectProcessor(ctx context.Context, state state) (s3 event.S3.Bucket.ARN = p.bucket event.S3.Object.Key = state.Key - acker := awscommon.NewEventACKTracker() + acker := NewEventACKTracker(ctx, nil) return p.s3ObjectHandler.Create(ctx, p.log, p.client, acker, event), event } @@ -138,8 +138,9 @@ func (p *s3Poller) ProcessObject(s3ObjectPayloadChan <-chan *s3ObjectPayload) er for s3ObjectPayload := range s3ObjectPayloadChan { // Process S3 object (download, parse, create events). - err := s3ObjectPayload.s3ObjectHandler.ProcessS3Object() + s3EventsCreated, err := s3ObjectPayload.s3ObjectHandler.ProcessS3Object() + s3ObjectPayload.s3ObjectHandler.SyncEventsToBeAcked(s3EventsCreated) // Wait for all events to be ACKed before proceeding. s3ObjectPayload.s3ObjectHandler.Wait() diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index 9b9b9f6f1ee..d2fcc9875c4 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -25,7 +25,6 @@ import ( "github.com/elastic/beats/v7/libbeat/reader" "github.com/elastic/beats/v7/libbeat/reader/readfile" "github.com/elastic/beats/v7/libbeat/reader/readfile/encoding" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "github.com/elastic/elastic-agent-libs/logp" "github.com/elastic/elastic-agent-libs/mapstr" ) @@ -73,7 +72,7 @@ func (f *s3ObjectProcessorFactory) findReaderConfig(key string) *readerConfig { // Create returns a new s3ObjectProcessor. It returns nil when no file selectors // match the S3 object key. -func (f *s3ObjectProcessorFactory) Create(ctx context.Context, log *logp.Logger, client beat.Client, ack *awscommon.EventACKTracker, obj s3EventV2) s3ObjectHandler { +func (f *s3ObjectProcessorFactory) Create(ctx context.Context, log *logp.Logger, client beat.Client, ack *EventACKTracker, obj s3EventV2) s3ObjectHandler { log = log.With( "bucket_arn", obj.S3.Bucket.Name, "object_key", obj.S3.Object.Key) @@ -102,22 +101,27 @@ type s3ObjectProcessor struct { log *logp.Logger ctx context.Context publisher beat.Client - acker *awscommon.EventACKTracker // ACKer tied to the SQS message (multiple S3 readers share an ACKer when the S3 notification event contains more than one S3 object). - readerConfig *readerConfig // Config about how to process the object. - s3Obj s3EventV2 // S3 object information. + acker *EventACKTracker // ACKer tied to the SQS message (multiple S3 readers share an ACKer when the S3 notification event contains more than one S3 object). + readerConfig *readerConfig // Config about how to process the object. + s3Obj s3EventV2 // S3 object information. s3ObjHash string s3RequestURL string s3Metadata map[string]interface{} // S3 object metadata. + + s3EventsCreatedTotal uint64 } +func (p *s3ObjectProcessor) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { + p.acker.TotalEventsAcked.Add(s3EventsCreatedTotal) +} func (p *s3ObjectProcessor) Wait() { - p.acker.Wait() + p.acker.WaitForS3() } -func (p *s3ObjectProcessor) ProcessS3Object() error { +func (p *s3ObjectProcessor) ProcessS3Object() (uint64, error) { if p == nil { - return nil + return 0, nil } // Metrics and Logging @@ -135,7 +139,7 @@ func (p *s3ObjectProcessor) ProcessS3Object() error { // Request object (download). contentType, meta, body, err := p.download() if err != nil { - return fmt.Errorf("failed to get s3 object (elapsed_time_ns=%d): %w", + return 0, fmt.Errorf("failed to get s3 object (elapsed_time_ns=%d): %w", time.Since(start).Nanoseconds(), err) } defer body.Close() @@ -143,7 +147,7 @@ func (p *s3ObjectProcessor) ProcessS3Object() error { reader, err := p.addGzipDecoderIfNeeded(newMonitoredReader(body, p.metrics.s3BytesProcessedTotal)) if err != nil { - return fmt.Errorf("failed checking for gzip content: %w", err) + return 0, fmt.Errorf("failed checking for gzip content: %w", err) } // Overwrite with user configured Content-Type. @@ -154,7 +158,7 @@ func (p *s3ObjectProcessor) ProcessS3Object() error { // try to create a decoder from the using the codec config decoder, err := newDecoder(p.readerConfig.Decoding, reader) if err != nil { - return err + return 0, err } if decoder != nil { defer decoder.close() @@ -164,7 +168,7 @@ func (p *s3ObjectProcessor) ProcessS3Object() error { data, err := decoder.decode() if err != nil { if errors.Is(err, io.EOF) { - return nil + return p.s3EventsCreatedTotal, nil } break } @@ -184,11 +188,11 @@ func (p *s3ObjectProcessor) ProcessS3Object() error { } } if err != nil { - return fmt.Errorf("failed reading s3 object (elapsed_time_ns=%d): %w", + return 0, fmt.Errorf("failed reading s3 object (elapsed_time_ns=%d): %w", time.Since(start).Nanoseconds(), err) } - return nil + return p.s3EventsCreatedTotal, nil } // download requests the S3 object from AWS and returns the object's @@ -387,9 +391,9 @@ func (p *s3ObjectProcessor) readFile(r io.Reader) error { return nil } -func (p *s3ObjectProcessor) publish(ack *awscommon.EventACKTracker, event *beat.Event) { - ack.Add() +func (p *s3ObjectProcessor) publish(ack *EventACKTracker, event *beat.Event) { event.Private = ack + p.s3EventsCreatedTotal++ p.metrics.s3EventsCreatedTotal.Inc() p.publisher.Publish(*event) } diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 8f71ffac73e..ecd44fcdc50 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -37,7 +37,7 @@ type processingOutcome struct { start time.Time keepaliveWg *sync.WaitGroup keepaliveCancel context.CancelFunc - acker *awscommon.EventACKTracker + acker *EventACKTracker msg *types.Message receiveCount int handles []s3ObjectHandler @@ -65,74 +65,7 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { // N messages, waits for them all to finish sending events to the queue, then requests N more messages. var processingWg sync.WaitGroup - // At the end of the loop, after a shutdown or anything else, we still need to wait for the DeleteSQS goroutines to - // complete, otherwise the messages will be sent back to the queue even if they should be deleted. - var deletionWg sync.WaitGroup - - // We send to processingChan the outcome of each ProcessSQS call. - // We don't want to buffer the processingChan, since it will prevent workers ProcessSQS goroutines to return. - processingChan := make(chan processingOutcome) - - // We use deletionChan to throttle the number of DeleteSQS goroutines. - // deletionChan := make(chan struct{}, 3200) - - go func() { - for { - outcome, ok := <-processingChan - // processingChang is closed, no more outcomes to process, we can exit. - if !ok { - return - } - - // A ProcessSQS goroutine has sent an outcome, let's process it asynchronously in order to handle SQS message deletion. - go func(outcome processingOutcome) { - // Mark deletion wait group as done when the goroutine is done. - defer deletionWg.Done() - - r.log.Debugw("Waiting worker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - - // We don't want to cap processingChan, since it will prevent workers ProcessSQS goroutines to return - // and in flight message would be capped as well. - // We want to cap number of goroutines for DeleteSQS - // deletionChan <- struct{}{} - - r.log.Debugw("Waited worker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - - r.log.Debugw("Waiting acker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - - // Wait for all events to be ACKed before proceeding. - outcome.acker.Wait() - - r.log.Debugw("Waited acker when deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - - // Stop keepalive visibility routine before deleting. - outcome.keepaliveCancel() - outcome.keepaliveWg.Wait() - - err := r.msgHandler.DeleteSQS(outcome.msg, outcome.receiveCount, outcome.processingErr, outcome.handles) - if err != nil { - r.log.Warnw("Failed deleting SQS message.", - "error", err, - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - } else { - r.log.Debugw("Success deleting SQS message.", - "message_id", *outcome.msg.MessageId, - "elapsed_time_ns", time.Since(outcome.start)) - } - - // <-deletionChan - }(outcome) - } - }() + deletionWg := new(sync.WaitGroup) for ctx.Err() == nil { // Determine how many SQS workers are available. @@ -163,6 +96,7 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { // Add to processing wait group to wait for all messages to be processed. processingWg.Add(len(msgs)) + deletionWg.Add(len(msgs)) for _, msg := range msgs { // Process each SQS message asynchronously with a goroutine. @@ -175,13 +109,16 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { r.workerSem.Release(1) }() + acker := NewEventACKTracker(ctx, deletionWg) + // Create a pipeline client scoped to this goroutine. client, err := pipeline.ConnectWith(beat.ClientConfig{ - EventListener: awscommon.NewEventACKHandler(), + EventListener: NewEventACKHandler(), Processing: beat.ProcessingConfig{ // This input only produces events with basic types so normalization // is not required. EventNormalization: boolPtr(false), + Private: acker, }, }) @@ -200,28 +137,18 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) - acker := awscommon.NewEventACKTracker() + err = r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) + if err != nil { + r.log.Warnw("Failed processing SQS message.", + "error", err, + "message_id", *msg.MessageId, + "elapsed_time_ns", time.Since(start)) - receiveCount, handles, keepaliveCancel, keepaliveWg, processingErr := r.msgHandler.ProcessSQS(ctx, &msg, client, acker) + } r.log.Debugw("Success processing SQS message.", "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) - - // Add to deletion waiting group before sending to processingChan. - deletionWg.Add(1) - - // Send the outcome to the processingChan so the deletion goroutine can delete the message. - processingChan <- processingOutcome{ - start: start, - keepaliveWg: keepaliveWg, - keepaliveCancel: keepaliveCancel, - acker: acker, - msg: &msg, - receiveCount: receiveCount, - handles: handles, - processingErr: processingErr, - } }(msg, time.Now()) } } @@ -229,10 +156,7 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { // Wait for all processing goroutines to finish. processingWg.Wait() - // We need to close the processingChan to signal to the deletion goroutines that they should stop. - close(processingChan) - - // Wait for all deletion goroutines to finish. + // Wait for all deletion to happen. deletionWg.Wait() if errors.Is(ctx.Err(), context.Canceled) { diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 7aca7214362..23baf19cb5d 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -20,7 +20,6 @@ import ( "go.uber.org/multierr" "github.com/elastic/beats/v7/libbeat/beat" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "github.com/elastic/elastic-agent-libs/logp" ) @@ -104,7 +103,6 @@ func newSQSS3EventProcessor( maxReceiveCount int, pipeline beat.Pipeline, s3 s3ObjectHandlerFactory, - maxWorkers int, ) *sqsS3EventProcessor { if metrics == nil { // Metrics are optional. Initialize a stub. @@ -166,9 +164,9 @@ func (p *sqsS3EventProcessor) DeleteSQS(msg *types.Message, receiveCount int, pr // after maximum receives is reached. p.metrics.sqsMessagesReturnedTotal.Inc() return fmt.Errorf("failed deleting SQS message (it will return to queue after visibility timeout): %w", processingErr) - } -func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *awscommon.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) { + +func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) error { keepaliveCtx, keepaliveCancel := context.WithCancel(ctx) // Start SQS keepalive worker. @@ -193,9 +191,11 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message } } - handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) + s3EventsCreatedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) p.metrics.sqsMessagesProcessedTotal.Inc() - return receiveCount, handles, keepaliveCancel, &keepaliveWg, processingErr + acker.AddSQSDeletionData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log) + + return processingErr } func (p *sqsS3EventProcessor) keepalive(ctx context.Context, log *logp.Logger, wg *sync.WaitGroup, msg *types.Message) { @@ -293,24 +293,25 @@ func (*sqsS3EventProcessor) isObjectCreatedEvents(event s3EventV2) bool { return event.EventSource == "aws:s3" && strings.HasPrefix(event.EventName, "ObjectCreated:") } -func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Logger, body string, client beat.Client, acker *awscommon.EventACKTracker) ([]s3ObjectHandler, error) { +func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Logger, body string, client beat.Client, acker *EventACKTracker) (uint64, []s3ObjectHandler, error) { s3Events, err := p.getS3Notifications(body) if err != nil { if errors.Is(err, context.Canceled) { // Messages that are in-flight at shutdown should be returned to SQS. - return nil, err + return 0, nil, err } - return nil, &nonRetryableError{err} + return 0, nil, &nonRetryableError{err} } log.Debugf("SQS message contained %d S3 event notifications.", len(s3Events)) defer log.Debug("End processing SQS S3 event notifications.") if len(s3Events) == 0 { - return nil, nil + return 0, nil, nil } var errs []error var handles []s3ObjectHandler + var s3EventsCreatedTotal uint64 for i, event := range s3Events { s3Processor := p.s3ObjectHandler.Create(ctx, log, client, acker, event) if s3Processor == nil { @@ -318,21 +319,22 @@ func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Log } // Process S3 object (download, parse, create events). - if err := s3Processor.ProcessS3Object(); err != nil { + if s3EventsCreatedTotalByProcessor, err := s3Processor.ProcessS3Object(); err != nil { errs = append(errs, fmt.Errorf( "failed processing S3 event for object key %q in bucket %q (object record %d of %d in SQS notification): %w", event.S3.Object.Key, event.S3.Bucket.Name, i+1, len(s3Events), err)) } else { + s3EventsCreatedTotal += s3EventsCreatedTotalByProcessor handles = append(handles, s3Processor) } } // Make sure all s3 events were processed successfully if len(handles) == len(s3Events) { - return handles, multierr.Combine(errs...) + return s3EventsCreatedTotal, handles, multierr.Combine(errs...) } - return nil, multierr.Combine(errs...) + return 0, nil, multierr.Combine(errs...) } func (p *sqsS3EventProcessor) finalizeS3Objects(handles []s3ObjectHandler) error { From c6ff95f8a9af26bd0dc41b61d712fb9f2f122a02 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 09:45:48 +0900 Subject: [PATCH 10/43] fix tests --- x-pack/filebeat/input/awss3/decoding_test.go | 2 +- .../input/awss3/input_benchmark_test.go | 88 +++++++++++-------- .../input/awss3/mock_interfaces_test.go | 46 +++++----- .../input/awss3/mock_publisher_test.go | 7 +- .../filebeat/input/awss3/s3_objects_test.go | 42 +++++---- .../filebeat/input/awss3/sqs_s3_event_test.go | 64 +++++++------- x-pack/filebeat/input/awss3/sqs_test.go | 19 ++-- 7 files changed, 144 insertions(+), 124 deletions(-) diff --git a/x-pack/filebeat/input/awss3/decoding_test.go b/x-pack/filebeat/input/awss3/decoding_test.go index 81f2bbc450c..ca0c6dbb055 100644 --- a/x-pack/filebeat/input/awss3/decoding_test.go +++ b/x-pack/filebeat/input/awss3/decoding_test.go @@ -21,7 +21,7 @@ func TestParquetDecoding(t *testing.T) { name string file string contentType string - numEvents int + numEvents uint64 assertAgainst string config *readerConfig }{ diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 0b58ac76b95..51af2f11482 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -34,7 +34,6 @@ import ( "github.com/olekukonko/tablewriter" pubtest "github.com/elastic/beats/v7/libbeat/publisher/testing" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" conf "github.com/elastic/elastic-agent-libs/config" "github.com/elastic/elastic-agent-libs/logp" "github.com/elastic/elastic-agent-libs/monitoring" @@ -251,20 +250,21 @@ func (c *constantS3) ListObjectsPaginator(bucket, prefix string) s3Pager { var _ beat.Pipeline = (*fakePipeline)(nil) func newFakePipeline() *fakePipeline { - fp := &fakePipeline{flush: time.NewTicker(10 * time.Second), log: log, mutex: new(sync.Mutex)} + fp := &fakePipeline{ + mutex: new(sync.Mutex), + flush: time.NewTicker(10 * time.Second), + pendingEvents: atomic.NewUint64(0), + clients: make([]beat.Client, 0), + } + go func() { for { - fp.mutex.Lock() select { case <-fp.flush.C: - fp.ackPendings() - default: - if fp.pendingEvents > 3200 { - fp.ackPendings() - } + fp.mutex.Lock() + fp.ackEvents() + fp.mutex.Unlock() } - - fp.mutex.Unlock() } }() @@ -273,34 +273,43 @@ func newFakePipeline() *fakePipeline { // fakePipeline returns new ackClients. type fakePipeline struct { - flush *time.Ticker - mutex *sync.Mutex - pendingEvents int - publishedEvents int - events []*beat.Event - log *logp.Logger + flush *time.Ticker + mutex *sync.Mutex + pendingEvents *atomic.Uint64 + clients []beat.Client } -func (fp *fakePipeline) ackPendings() { - for _, eventToACK := range fp.events { - if eventToACK.Private.(*awscommon.EventACKTracker).PendingACKs.Load() > 0 { - fp.pendingEvents-- - eventToACK.Private.(*awscommon.EventACKTracker).ACK() +func (fp *fakePipeline) ackEvents() { + for _, client := range fp.clients { + acker := client.(*ackClient).acker + + if acker == nil || !acker.isSQSAcker { + continue } - } - events := make([]*beat.Event, 0, len(fp.events)) - for _, eventToACK := range fp.events { - if eventToACK.Private.(*awscommon.EventACKTracker).PendingACKs.Load() > 0 { - events = append(events, eventToACK) + if acker.FullyAcked() { + continue + } + + ackHandler := NewEventACKHandler() + currentEvents := acker.EventsToBeAcked.Load() - acker.TotalEventsAcked.Load() + for i := currentEvents; i > 0; i-- { + fp.pendingEvents.Dec() + ackHandler.AddEvent(beat.Event{Private: acker}, true) } - } - fp.events = events + if currentEvents > 0 { + ackHandler.ACKEvents(int(currentEvents)) + } + } } func (fp *fakePipeline) ConnectWith(clientConfig beat.ClientConfig) (beat.Client, error) { - return &ackClient{fp: fp}, nil + fp.mutex.Lock() + client := &ackClient{fp: fp} + fp.clients = append(fp.clients, client) + fp.mutex.Unlock() + return client, nil } func (fp *fakePipeline) Connect() (beat.Client, error) { @@ -311,16 +320,23 @@ var _ beat.Client = (*ackClient)(nil) // ackClient is a fake beat.Client that ACKs the published messages. type ackClient struct { - fp *fakePipeline + fp *fakePipeline + acker *EventACKTracker } func (c *ackClient) Close() error { return nil } func (c *ackClient) Publish(event beat.Event) { c.fp.mutex.Lock() - c.fp.pendingEvents++ - c.fp.publishedEvents++ - c.fp.events = append(c.fp.events, &event) + c.fp.pendingEvents.Inc() + if c.acker == nil { + c.acker = event.Private.(*EventACKTracker) + } + + if c.fp.pendingEvents.Load() > 3200 { + c.fp.ackEvents() + } + c.fp.mutex.Unlock() } @@ -364,7 +380,7 @@ func benchmarkInputSQS(t *testing.T, maxMessagesInflight int) testing.BenchmarkR conf := makeBenchmarkConfig(t) s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), metrics, s3API, conf.FileSelectors, backupConfig{}, maxMessagesInflight) - sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), metrics, sqsAPI, nil, time.Minute, 5, pipeline, s3EventHandlerFactory, maxMessagesInflight) + sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), metrics, sqsAPI, nil, time.Minute, 5, pipeline, s3EventHandlerFactory) sqsReader := newSQSReader(logSqs, metrics, sqsAPI, maxMessagesInflight, sqsMessageHandler) ctx, cancel := context.WithCancel(context.Background()) @@ -431,7 +447,6 @@ func TestBenchmarkInputSQS(t *testing.T) { benchmarkInputSQS(t, 512), benchmarkInputSQS(t, 1024), } - headers := []string{ "Max Msgs Inflight", "Events total", @@ -483,7 +498,8 @@ func benchmarkInputS3(t *testing.T, numberOfWorkers int) testing.BenchmarkResult metrics := newInputMetrics("test_id", metricRegistry, numberOfWorkers) client := pubtest.NewChanClientWithCallback(100, func(event beat.Event) { - event.Private.(*awscommon.EventACKTracker).ACK() + event.Private.(*EventACKTracker).EventsToBeAcked.Inc() + event.Private.(*EventACKTracker).ACK() }) defer func() { diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index 4a64078c898..b848b8a1a84 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -11,16 +7,13 @@ package awss3 import ( context "context" reflect "reflect" - sync "sync" time "time" s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" - aws "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" logp "github.com/elastic/elastic-agent-libs/logp" + gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. @@ -292,21 +285,17 @@ func (mr *MockSQSProcessorMockRecorder) DeleteSQS(msg, receiveCount, processingE } // ProcessSQS mocks base method. -func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *aws.EventACKTracker) (int, []s3ObjectHandler, context.CancelFunc, *sync.WaitGroup, error) { +func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg, client, acker) - ret0, _ := ret[0].(int) - ret1, _ := ret[1].([]s3ObjectHandler) - ret2, _ := ret[2].(context.CancelFunc) - ret3, _ := ret[3].(*sync.WaitGroup) - ret4, _ := ret[4].(error) - return ret0, ret1, ret2, ret3, ret4 + ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg, client, acker, start) + ret0, _ := ret[0].(error) + return ret0 } // ProcessSQS indicates an expected call of ProcessSQS. -func (mr *MockSQSProcessorMockRecorder) ProcessSQS(ctx, msg, client, acker interface{}) *gomock.Call { +func (mr *MockSQSProcessorMockRecorder) ProcessSQS(ctx, msg, client, acker, start interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessSQS", reflect.TypeOf((*MockSQSProcessor)(nil).ProcessSQS), ctx, msg, client, acker) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessSQS", reflect.TypeOf((*MockSQSProcessor)(nil).ProcessSQS), ctx, msg, client, acker, start) } // MockS3API is a mock of s3API interface. @@ -600,7 +589,7 @@ func (m *MockS3ObjectHandlerFactory) EXPECT() *MockS3ObjectHandlerFactoryMockRec } // Create mocks base method. -func (m *MockS3ObjectHandlerFactory) Create(ctx context.Context, log *logp.Logger, client beat.Client, acker *aws.EventACKTracker, obj s3EventV2) s3ObjectHandler { +func (m *MockS3ObjectHandlerFactory) Create(ctx context.Context, log *logp.Logger, client beat.Client, acker *EventACKTracker, obj s3EventV2) s3ObjectHandler { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Create", ctx, log, client, acker, obj) ret0, _ := ret[0].(s3ObjectHandler) @@ -651,11 +640,12 @@ func (mr *MockS3ObjectHandlerMockRecorder) FinalizeS3Object() *gomock.Call { } // ProcessS3Object mocks base method. -func (m *MockS3ObjectHandler) ProcessS3Object() error { +func (m *MockS3ObjectHandler) ProcessS3Object() (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ProcessS3Object") - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].(uint64) + ret1, _ := ret[1].(error) + return ret0, ret1 } // ProcessS3Object indicates an expected call of ProcessS3Object. @@ -664,6 +654,18 @@ func (mr *MockS3ObjectHandlerMockRecorder) ProcessS3Object() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessS3Object", reflect.TypeOf((*MockS3ObjectHandler)(nil).ProcessS3Object)) } +// SyncEventsToBeAcked mocks base method. +func (m *MockS3ObjectHandler) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "SyncEventsToBeAcked", s3EventsCreatedTotal) +} + +// SyncEventsToBeAcked indicates an expected call of SyncEventsToBeAcked. +func (mr *MockS3ObjectHandlerMockRecorder) SyncEventsToBeAcked(s3EventsCreatedTotal interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SyncEventsToBeAcked", reflect.TypeOf((*MockS3ObjectHandler)(nil).SyncEventsToBeAcked), s3EventsCreatedTotal) +} + // Wait mocks base method. func (m *MockS3ObjectHandler) Wait() { m.ctrl.T.Helper() diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index efbd5bcef97..9c34b58efd1 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -11,9 +7,8 @@ package awss3 import ( reflect "reflect" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" + gomock "github.com/golang/mock/gomock" ) // MockBeatClient is a mock of Client interface. diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 07b09c3fb9a..dc1f72495eb 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -22,7 +22,6 @@ import ( "github.com/stretchr/testify/require" "github.com/elastic/beats/v7/libbeat/beat" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" conf "github.com/elastic/elastic-agent-libs/config" "github.com/elastic/elastic-agent-libs/logp" ) @@ -155,8 +154,8 @@ func TestS3ObjectProcessor(t *testing.T) { Return(nil, errFakeConnectivityFailure) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker() - err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() + ack := NewEventACKTracker(ctx, nil) + _, err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() require.Error(t, err) assert.True(t, errors.Is(err, errFakeConnectivityFailure), "expected errFakeConnectivityFailure error") }) @@ -177,8 +176,8 @@ func TestS3ObjectProcessor(t *testing.T) { Return(nil, nil) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker() - err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() + ack := NewEventACKTracker(ctx, nil) + _, err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() require.Error(t, err) }) @@ -204,8 +203,8 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker() - err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() + ack := NewEventACKTracker(ctx, nil) + _, err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() require.NoError(t, err) }) @@ -230,7 +229,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupCfg, 1) - ack := awscommon.NewEventACKTracker() + ack := NewEventACKTracker(ctx, nil) err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).FinalizeS3Object() require.NoError(t, err) }) @@ -260,7 +259,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupCfg, 1) - ack := awscommon.NewEventACKTracker() + ack := NewEventACKTracker(ctx, nil) err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).FinalizeS3Object() require.NoError(t, err) }) @@ -287,7 +286,7 @@ func TestS3ObjectProcessor(t *testing.T) { ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, nil, backupCfg, 1) - ack := awscommon.NewEventACKTracker() + ack := NewEventACKTracker(ctx, nil) err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).FinalizeS3Object() require.NoError(t, err) }) @@ -301,15 +300,15 @@ func TestS3ObjectProcessor(t *testing.T) { }) } -func testProcessS3Object(t testing.TB, file, contentType string, numEvents int, selectors ...fileSelectorConfig) []beat.Event { +func testProcessS3Object(t testing.TB, file, contentType string, numEvents uint64, selectors ...fileSelectorConfig) []beat.Event { return _testProcessS3Object(t, file, contentType, numEvents, false, selectors) } -func testProcessS3ObjectError(t testing.TB, file, contentType string, numEvents int, selectors ...fileSelectorConfig) []beat.Event { +func testProcessS3ObjectError(t testing.TB, file, contentType string, numEvents uint64, selectors ...fileSelectorConfig) []beat.Event { return _testProcessS3Object(t, file, contentType, numEvents, true, selectors) } -func _testProcessS3Object(t testing.TB, file, contentType string, numEvents int, expectErr bool, selectors []fileSelectorConfig) []beat.Event { +func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint64, expectErr bool, selectors []fileSelectorConfig) []beat.Event { t.Helper() ctx, cancel := context.WithTimeout(context.Background(), testTimeout) @@ -321,6 +320,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents int, mockPublisher := NewMockBeatClient(ctrl) s3Event, s3Resp := newS3Object(t, file, contentType) + ack := NewEventACKTracker(ctx, nil) var events []beat.Event gomock.InOrder( mockS3API.EXPECT(). @@ -328,18 +328,22 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents int, Return(s3Resp, nil), mockPublisher.EXPECT(). Publish(gomock.Any()). - Do(func(event beat.Event) { events = append(events, event) }). - Times(numEvents), + Do(func(event beat.Event) { + events = append(events, event) + ack.EventsToBeAcked.Inc() + ack.ACK() + }). + Times(int(numEvents)), ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, selectors, backupConfig{}, 1) - ack := awscommon.NewEventACKTracker() - err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() + s3EventsCreatedTotal, err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() if !expectErr { require.NoError(t, err) - assert.Equal(t, numEvents, len(events)) - assert.EqualValues(t, numEvents, ack.PendingACKs.Load()) + assert.EqualValues(t, numEvents, len(events)) + assert.EqualValues(t, numEvents, ack.TotalEventsAcked.Load()) + assert.EqualValues(t, numEvents, s3EventsCreatedTotal) } else { require.Error(t, err) } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index ef86b478469..f92393c66bf 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -22,7 +22,6 @@ import ( "github.com/stretchr/testify/require" "github.com/elastic/beats/v7/libbeat/beat" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "github.com/elastic/elastic-agent-libs/logp" "github.com/elastic/go-concert/timed" ) @@ -47,17 +46,17 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) - acker := awscommon.NewEventACKTracker() - receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) + processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) require.NoError(t, processingErr) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), ) - require.NoError(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) + require.NoError(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) t.Run("invalid SQS JSON body does not retry", func(t *testing.T) { @@ -76,10 +75,10 @@ func TestSQSS3EventProcessor(t *testing.T) { invalidBodyMsg.Body = &body mockClient := NewMockBeatClient(ctrl) - acker := awscommon.NewEventACKTracker() - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) + processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker, time.Now()) t.Log(processingErr) require.Error(t, processingErr) @@ -87,7 +86,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&invalidBodyMsg)).Return(nil), ) - require.Error(t, p.DeleteSQS(&invalidBodyMsg, receiveCount, processingErr, handles)) + require.Error(t, p.DeleteSQS(&invalidBodyMsg, acker.ReceiveCount, processingErr, acker.Handles)) }) t.Run("zero S3 events in body", func(t *testing.T) { @@ -103,17 +102,17 @@ func TestSQSS3EventProcessor(t *testing.T) { emptyRecordsMsg := newSQSMessage([]s3EventV2{}...) mockClient := NewMockBeatClient(ctrl) - acker := awscommon.NewEventACKTracker() - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) + processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker, time.Now()) require.NoError(t, processingErr) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&emptyRecordsMsg)).Return(nil), ) - require.NoError(t, p.DeleteSQS(&emptyRecordsMsg, receiveCount, processingErr, handles)) + require.NoError(t, p.DeleteSQS(&emptyRecordsMsg, acker.ReceiveCount, processingErr, acker.Handles)) }) t.Run("visibility is extended after half expires", func(t *testing.T) { @@ -134,16 +133,15 @@ func TestSQSS3EventProcessor(t *testing.T) { gomock.InOrder( mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). - Do(func(ctx context.Context, _ *logp.Logger, _ beat.Client, _ *awscommon.EventACKTracker, _ s3EventV2) { + Do(func(ctx context.Context, _ *logp.Logger, _ beat.Client, _ *EventACKTracker, _ s3EventV2) { require.NoError(t, timed.Wait(ctx, 5*visibilityTimeout)) }).Return(mockS3Handler), - mockS3Handler.EXPECT().ProcessS3Object().Return(nil), + mockS3Handler.EXPECT().ProcessS3Object().Return(uint64(1), nil), ) - acker := awscommon.NewEventACKTracker() - - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory) + acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) + processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) require.NoError(t, processingErr) gomock.InOrder( @@ -151,7 +149,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().FinalizeS3Object().Return(nil), ) - require.NoError(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) + require.NoError(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) t.Run("message returns to queue on error", func(t *testing.T) { @@ -168,17 +166,16 @@ func TestSQSS3EventProcessor(t *testing.T) { gomock.InOrder( mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(mockS3Handler), - mockS3Handler.EXPECT().ProcessS3Object().Return(errors.New("fake connectivity problem")), + mockS3Handler.EXPECT().ProcessS3Object().Return(uint64(0), errors.New("fake connectivity problem")), ) - acker := awscommon.NewEventACKTracker() - - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) + processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) t.Log(processingErr) require.Error(t, processingErr) - require.Error(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) + require.Error(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) t.Run("message is deleted after multiple receives", func(t *testing.T) { @@ -200,13 +197,12 @@ func TestSQSS3EventProcessor(t *testing.T) { gomock.InOrder( mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(mockS3Handler), - mockS3Handler.EXPECT().ProcessS3Object().Return(errors.New("fake connectivity problem")), + mockS3Handler.EXPECT().ProcessS3Object().Return(uint64(0), errors.New("fake connectivity problem")), ) - acker := awscommon.NewEventACKTracker() - - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory, 5) - receiveCount, handles, _, _, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) + processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) t.Log(processingErr) require.Error(t, processingErr) @@ -214,7 +210,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), ) - require.Error(t, p.DeleteSQS(&msg, receiveCount, processingErr, handles)) + require.Error(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) } @@ -257,7 +253,7 @@ func TestSqsProcessor_keepalive(t *testing.T) { mockAPI.EXPECT().ChangeMessageVisibility(gomock.Any(), gomock.Eq(&msg), gomock.Eq(visibilityTimeout)). Times(1).Return(tc.Err) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory, 5) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory) var wg sync.WaitGroup wg.Add(1) p.keepalive(ctx, p.log, &wg, &msg) @@ -269,7 +265,7 @@ func TestSqsProcessor_keepalive(t *testing.T) { func TestSqsProcessor_getS3Notifications(t *testing.T) { logp.TestingSetup() - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, nil, nil, time.Minute, 5, nil, nil, 5) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, nil, nil, time.Minute, 5, nil, nil) t.Run("s3 key is url unescaped", func(t *testing.T) { msg := newSQSMessage(newS3Event("Happy+Face.jpg")) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 609ec05f136..e3911755a04 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -9,6 +9,7 @@ import ( "crypto/sha256" "encoding/json" "errors" + "github.com/elastic/beats/v7/libbeat/beat" "sync" "testing" "time" @@ -68,24 +69,30 @@ func TestSQSReceiver(t *testing.T) { mockClient := NewMockBeatClient(ctrl) mockBeatPipeline := NewMockBeatPipeline(ctrl) - _, keepaliveCancel := context.WithCancel(ctx) - - // Start SQS keepalive worker. - var keepaliveWg sync.WaitGroup gomock.InOrder( mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), // Expect the one message returned to have been processed. mockMsgHandler.EXPECT(). - ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any()). + ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). - Return(0, nil, keepaliveCancel, &keepaliveWg, nil), + DoAndReturn( + func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) error { + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + acker.AddSQSDeletionData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.ACK() + acker.FlushForSQS() + + return nil + }), ) // The two expected calls happen in different goroutines, we cannot enforce an oder // Expect the client to be closed mockClient.EXPECT().Close() // Expect the one message returned to have been deleted. + // We mockMsgHandler.EXPECT(). DeleteSQS(gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). From c91076b39cd1599ca27012517eeb85c08ea69071 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 09:46:58 +0900 Subject: [PATCH 11/43] make check --- x-pack/filebeat/input/awss3/interfaces.go | 1 + x-pack/filebeat/input/awss3/mock_interfaces_test.go | 7 ++++++- x-pack/filebeat/input/awss3/mock_publisher_test.go | 7 ++++++- x-pack/filebeat/input/awss3/sqs_acker.go | 6 ++++-- x-pack/filebeat/input/awss3/sqs_test.go | 3 ++- 5 files changed, 19 insertions(+), 5 deletions(-) diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index 0b7cab05525..19dcf04a019 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -19,6 +19,7 @@ import ( "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/aws/aws-sdk-go-v2/service/sqs" "github.com/aws/aws-sdk-go-v2/service/sqs/types" + "github.com/elastic/beats/v7/libbeat/beat" "github.com/elastic/elastic-agent-libs/logp" diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index b848b8a1a84..df78e0cceb2 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -11,9 +15,10 @@ import ( s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" + gomock "github.com/golang/mock/gomock" + beat "github.com/elastic/beats/v7/libbeat/beat" logp "github.com/elastic/elastic-agent-libs/logp" - gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index 9c34b58efd1..efbd5bcef97 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -7,8 +11,9 @@ package awss3 import ( reflect "reflect" - beat "github.com/elastic/beats/v7/libbeat/beat" gomock "github.com/golang/mock/gomock" + + beat "github.com/elastic/beats/v7/libbeat/beat" ) // MockBeatClient is a mock of Client interface. diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index f2ea2e271df..69a2675eb02 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -6,11 +6,13 @@ package awss3 import ( "context" - "github.com/aws/aws-sdk-go-v2/service/sqs/types" - "github.com/elastic/elastic-agent-libs/logp" "sync" "time" + "github.com/aws/aws-sdk-go-v2/service/sqs/types" + + "github.com/elastic/elastic-agent-libs/logp" + "github.com/elastic/beats/v7/libbeat/beat" "github.com/elastic/beats/v7/libbeat/common/acker" "github.com/elastic/beats/v7/libbeat/common/atomic" diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index e3911755a04..e534095b4c6 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -9,11 +9,12 @@ import ( "crypto/sha256" "encoding/json" "errors" - "github.com/elastic/beats/v7/libbeat/beat" "sync" "testing" "time" + "github.com/elastic/beats/v7/libbeat/beat" + "github.com/aws/aws-sdk-go-v2/service/sqs/types" "github.com/gofrs/uuid" "github.com/golang/mock/gomock" From e38a699bb428bcfd648b0ba4a93c055aba5318d8 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 09:59:20 +0900 Subject: [PATCH 12/43] linting --- x-pack/filebeat/input/awss3/decoding_test.go | 2 +- x-pack/filebeat/input/awss3/input_benchmark_test.go | 10 ++++------ x-pack/filebeat/input/awss3/sqs.go | 11 ----------- 3 files changed, 5 insertions(+), 18 deletions(-) diff --git a/x-pack/filebeat/input/awss3/decoding_test.go b/x-pack/filebeat/input/awss3/decoding_test.go index ca0c6dbb055..0df1f9fef81 100644 --- a/x-pack/filebeat/input/awss3/decoding_test.go +++ b/x-pack/filebeat/input/awss3/decoding_test.go @@ -121,7 +121,7 @@ func readJSONFromFile(t *testing.T, filepath string) []string { var rawMessages []json.RawMessage err = json.Unmarshal(fileBytes, &rawMessages) assert.NoError(t, err) - var data []string + data := make([]string, 0, len(rawMessages)) for _, rawMsg := range rawMessages { data = append(data, string(rawMsg)) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 51af2f11482..c42d6d10ce4 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -259,12 +259,10 @@ func newFakePipeline() *fakePipeline { go func() { for { - select { - case <-fp.flush.C: - fp.mutex.Lock() - fp.ackEvents() - fp.mutex.Unlock() - } + <-fp.flush.C + fp.mutex.Lock() + fp.ackEvents() + fp.mutex.Unlock() } }() diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index ecd44fcdc50..35d17403066 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -33,17 +33,6 @@ type sqsReader struct { metrics *inputMetrics } -type processingOutcome struct { - start time.Time - keepaliveWg *sync.WaitGroup - keepaliveCancel context.CancelFunc - acker *EventACKTracker - msg *types.Message - receiveCount int - handles []s3ObjectHandler - processingErr error -} - func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessagesInflight int, msgHandler sqsProcessor) *sqsReader { if metrics == nil { // Metrics are optional. Initialize a stub. From 1b587a1c6c46cdc1244dc4758829d4745986ac1b Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 10:16:52 +0900 Subject: [PATCH 13/43] No need to acker in beat.ClientConfig --- x-pack/filebeat/input/awss3/sqs.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 35d17403066..f5a9214f8ec 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -98,8 +98,6 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { r.workerSem.Release(1) }() - acker := NewEventACKTracker(ctx, deletionWg) - // Create a pipeline client scoped to this goroutine. client, err := pipeline.ConnectWith(beat.ClientConfig{ EventListener: NewEventACKHandler(), @@ -107,7 +105,6 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { // This input only produces events with basic types so normalization // is not required. EventNormalization: boolPtr(false), - Private: acker, }, }) @@ -126,6 +123,8 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) + acker := NewEventACKTracker(ctx, deletionWg) + err = r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) if err != nil { r.log.Warnw("Failed processing SQS message.", From 6a757ae3d9bd1f83d915d18556b08043b2eccbf5 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 14:14:02 +0900 Subject: [PATCH 14/43] Handle race condition that prevents to FlushForSQS, add ack mutex and let ACK calls before AddSQSDeletionData and SyncEventsToBeAcked to wait for it --- x-pack/filebeat/input/awss3/s3_objects.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker.go | 56 +++++++++++++++++------ 2 files changed, 44 insertions(+), 14 deletions(-) diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index d2fcc9875c4..603ce3dd1d7 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -113,7 +113,7 @@ type s3ObjectProcessor struct { } func (p *s3ObjectProcessor) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { - p.acker.TotalEventsAcked.Add(s3EventsCreatedTotal) + p.acker.SyncEventsToBeAcked(s3EventsCreatedTotal) } func (p *s3ObjectProcessor) Wait() { p.acker.WaitForS3() diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 69a2675eb02..430356ab1c2 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -18,12 +18,6 @@ import ( "github.com/elastic/beats/v7/libbeat/common/atomic" ) -var totMessageDeleted *atomic.Uint64 - -func init() { - totMessageDeleted = atomic.NewUint64(0) -} - // EventACKTracker tracks the publishing state of S3 objects. Specifically // it tracks the number of message acknowledgements that are pending from the // output. It can be used to wait until all ACKs have been received for one or @@ -34,7 +28,9 @@ type EventACKTracker struct { EventsToBeAcked *atomic.Uint64 TotalEventsAcked *atomic.Uint64 - isSQSAcker bool + ackMutex *sync.RWMutex + ackMutexLockedOnInit *atomic.Bool + isSQSAcker bool ctx context.Context cancel context.CancelFunc @@ -52,16 +48,39 @@ type EventACKTracker struct { func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { ctx, cancel := context.WithCancel(ctx) + ackMutex := new(sync.RWMutex) + // We need to lock on ack mutex, in order to know that we have passed the info to the acker about the total to be acked + // Lock it as soon as we create the acker. It will be unlocked and in either SyncEventsToBeAcked or AddSQSDeletionData + ackMutex.Lock() return &EventACKTracker{ - ctx: ctx, - cancel: cancel, - DeletionWg: deletionWg, - TotalEventsAcked: atomic.NewUint64(0), - EventsToBeAcked: atomic.NewUint64(0), + ctx: ctx, + cancel: cancel, + ackMutex: ackMutex, + ackMutexLockedOnInit: atomic.NewBool(true), + DeletionWg: deletionWg, + TotalEventsAcked: atomic.NewUint64(0), + EventsToBeAcked: atomic.NewUint64(0), + } +} + +func (a *EventACKTracker) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { + // We want to execute the logic of this call only once, when the ack mutex was locked on init + if !a.ackMutexLockedOnInit.Load() { + return } + + a.EventsToBeAcked.Add(s3EventsCreatedTotal) + + a.ackMutex.Unlock() + a.ackMutexLockedOnInit.Store(false) } func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { + // We want to execute the logic of this call only once, when the ack mutex was locked on init + if !a.ackMutexLockedOnInit.Load() { + return + } + a.isSQSAcker = true a.msg = msg @@ -74,12 +93,17 @@ func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent a.keepaliveWg = keepaliveWg a.msgHandler = msgHandler a.log = log + + a.ackMutex.Unlock() + a.ackMutexLockedOnInit.Store(false) } func (a *EventACKTracker) FullyAcked() bool { return a.TotalEventsAcked.Load() == a.EventsToBeAcked.Load() } + +// WaitForS3 must be called after SyncEventsToBeAcked func (a *EventACKTracker) WaitForS3() { // If it's fully acked then cancel the context. if a.FullyAcked() { @@ -102,7 +126,6 @@ func (a *EventACKTracker) FlushForSQS() { err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) a.DeletionWg.Done() - totMessageDeleted.Inc() if err != nil { a.log.Warnw("Failed deleting SQS message.", @@ -118,6 +141,13 @@ func (a *EventACKTracker) FlushForSQS() { // ACK decrements the number of total Events ACKed. func (a *EventACKTracker) ACK() { + // We need to lock on ack mutex, in order to know that we have passed the info to the acker about the total to be acked + // But we want to do it only before the info have been passed, once they did, no need anymore to lock on the ack mutext + if a.ackMutexLockedOnInit.Load() { + a.ackMutex.Lock() + defer a.ackMutex.Unlock() + } + if a.FullyAcked() { panic("misuse detected: ACK call on fully acked") } From 7bf18a67670c66f2f894fe54009690d156a48fbc Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 14:15:16 +0900 Subject: [PATCH 15/43] fix tests --- .../input/awss3/input_benchmark_test.go | 6 ++++- .../filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker_test.go | 26 ++++++++++++++++--- 3 files changed, 28 insertions(+), 6 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index c42d6d10ce4..7759106450f 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -496,7 +496,11 @@ func benchmarkInputS3(t *testing.T, numberOfWorkers int) testing.BenchmarkResult metrics := newInputMetrics("test_id", metricRegistry, numberOfWorkers) client := pubtest.NewChanClientWithCallback(100, func(event beat.Event) { - event.Private.(*EventACKTracker).EventsToBeAcked.Inc() + go func(acker *EventACKTracker) { + // 63 is the total number of events in a single S3 object + acker.SyncEventsToBeAcked(63) + }(event.Private.(*EventACKTracker)) + event.Private.(*EventACKTracker).ACK() }) diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index dc1f72495eb..1d8816ecc2d 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -321,6 +321,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint s3Event, s3Resp := newS3Object(t, file, contentType) ack := NewEventACKTracker(ctx, nil) + ack.SyncEventsToBeAcked(numEvents) var events []beat.Event gomock.InOrder( mockS3API.EXPECT(). @@ -330,7 +331,6 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint Publish(gomock.Any()). Do(func(event beat.Event) { events = append(events, event) - ack.EventsToBeAcked.Inc() ack.ACK() }). Times(int(numEvents)), diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index 050cd20d9bf..de6f31be980 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -18,7 +18,7 @@ func TestEventACKTracker(t *testing.T) { t.Cleanup(cancel) acker := NewEventACKTracker(ctx, nil) - acker.EventsToBeAcked.Add(1) + acker.SyncEventsToBeAcked(1) acker.ACK() assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) @@ -44,7 +44,7 @@ func TestEventACKHandler(t *testing.T) { // Create acker. Add one pending ACK. acker := NewEventACKTracker(ctx, nil) - acker.EventsToBeAcked.Add(1) + acker.SyncEventsToBeAcked(1) // Create an ACK handler and simulate one ACKed event. ackHandler := NewEventACKHandler() @@ -62,12 +62,30 @@ func TestEventACKHandlerWaitForS3(t *testing.T) { // Create acker. Add one pending ACK. acker := NewEventACKTracker(ctx, nil) - acker.EventsToBeAcked.Inc() + acker.SyncEventsToBeAcked(1) acker.ACK() acker.WaitForS3() + assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) + assert.EqualValues(t, true, acker.FullyAcked()) + + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} + +func TestEventACKHandlerFullyAcked(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + // Create acker. Add one pending ACK. + acker := NewEventACKTracker(ctx, nil) + acker.SyncEventsToBeAcked(1) + acker.ACK() + assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) + assert.EqualValues(t, true, acker.FullyAcked()) + + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) + acker.EventsToBeAcked.Inc() assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, false, acker.FullyAcked()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } From 60f7c10fc559267dbc3232edb3d0d246a6d5db61 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 16:59:30 +0900 Subject: [PATCH 16/43] Properly handle input beat.Pipeline --- x-pack/filebeat/input/awss3/input.go | 6 +++--- x-pack/filebeat/input/awss3/sqs.go | 9 ++++++--- x-pack/filebeat/input/awss3/sqs_s3_event.go | 3 --- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input.go b/x-pack/filebeat/input/awss3/input.go index be5445fd394..08c66c5c1dd 100644 --- a/x-pack/filebeat/input/awss3/input.go +++ b/x-pack/filebeat/input/awss3/input.go @@ -148,7 +148,7 @@ func (in *s3Input) Run(inputContext v2.Context, pipeline beat.Pipeline) error { // Poll metrics periodically in the background go pollSqsWaitingMetric(ctx, receiver) - if err := receiver.Receive(ctx, pipeline); err != nil { + if err := receiver.Receive(ctx); err != nil { return err } } @@ -227,8 +227,8 @@ func (in *s3Input) createSQSReceiver(ctx v2.Context, pipeline beat.Pipeline) (*s } in.metrics = newInputMetrics(ctx.ID, nil, in.config.MaxNumberOfMessages) s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), in.metrics, s3API, fileSelectors, in.config.BackupConfig, in.config.MaxNumberOfMessages) - sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), in.metrics, sqsAPI, script, in.config.VisibilityTimeout, in.config.SQSMaxReceiveCount, pipeline, s3EventHandlerFactory) - sqsReader := newSQSReader(log.Named("sqs"), in.metrics, sqsAPI, in.config.MaxNumberOfMessages, sqsMessageHandler) + sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), in.metrics, sqsAPI, script, in.config.VisibilityTimeout, in.config.SQSMaxReceiveCount, s3EventHandlerFactory) + sqsReader := newSQSReader(log.Named("sqs"), in.metrics, sqsAPI, in.config.MaxNumberOfMessages, sqsMessageHandler, pipeline) return sqsReader, nil } diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index f5a9214f8ec..8f95f4ac120 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -28,12 +28,13 @@ type sqsReader struct { maxMessagesInflight int workerSem *awscommon.Sem sqs sqsAPI + pipeline beat.Pipeline msgHandler sqsProcessor log *logp.Logger metrics *inputMetrics } -func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessagesInflight int, msgHandler sqsProcessor) *sqsReader { +func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessagesInflight int, msgHandler sqsProcessor, pipeline beat.Pipeline) *sqsReader { if metrics == nil { // Metrics are optional. Initialize a stub. metrics = newInputMetrics("", nil, 0) @@ -42,13 +43,14 @@ func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessag maxMessagesInflight: maxMessagesInflight, workerSem: awscommon.NewSem(maxMessagesInflight), sqs: sqs, + pipeline: pipeline, msgHandler: msgHandler, log: log, metrics: metrics, } } -func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { +func (r *sqsReader) Receive(ctx context.Context) error { // The loop tries to keep the ProcessSQS workers busy as much as possible while // honoring the max message cap as opposed to a simpler loop that receives // N messages, waits for them all to finish sending events to the queue, then requests N more messages. @@ -99,7 +101,7 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { }() // Create a pipeline client scoped to this goroutine. - client, err := pipeline.ConnectWith(beat.ClientConfig{ + client, err := r.pipeline.ConnectWith(beat.ClientConfig{ EventListener: NewEventACKHandler(), Processing: beat.ProcessingConfig{ // This input only produces events with basic types so normalization @@ -132,6 +134,7 @@ func (r *sqsReader) Receive(ctx context.Context, pipeline beat.Pipeline) error { "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) + return } r.log.Debugw("Success processing SQS message.", diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 23baf19cb5d..a9b03aeb37e 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -87,7 +87,6 @@ type sqsS3EventProcessor struct { sqsVisibilityTimeout time.Duration maxReceiveCount int sqs sqsAPI - pipeline beat.Pipeline // Pipeline creates clients for publishing events. log *logp.Logger warnOnce sync.Once metrics *inputMetrics @@ -101,7 +100,6 @@ func newSQSS3EventProcessor( script *script, sqsVisibilityTimeout time.Duration, maxReceiveCount int, - pipeline beat.Pipeline, s3 s3ObjectHandlerFactory, ) *sqsS3EventProcessor { if metrics == nil { @@ -113,7 +111,6 @@ func newSQSS3EventProcessor( sqsVisibilityTimeout: sqsVisibilityTimeout, maxReceiveCount: maxReceiveCount, sqs: sqs, - pipeline: pipeline, log: log, metrics: metrics, script: script, From 85279ebfff35fd548042981f9531b1c5ee1a463b Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 17:00:09 +0900 Subject: [PATCH 17/43] fix tests --- .../input/awss3/input_benchmark_test.go | 6 ++--- .../filebeat/input/awss3/sqs_s3_event_test.go | 23 +++++++------------ x-pack/filebeat/input/awss3/sqs_test.go | 14 ++++++----- 3 files changed, 19 insertions(+), 24 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 7759106450f..682b1af6c36 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -378,8 +378,8 @@ func benchmarkInputSQS(t *testing.T, maxMessagesInflight int) testing.BenchmarkR conf := makeBenchmarkConfig(t) s3EventHandlerFactory := newS3ObjectProcessorFactory(log.Named("s3"), metrics, s3API, conf.FileSelectors, backupConfig{}, maxMessagesInflight) - sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), metrics, sqsAPI, nil, time.Minute, 5, pipeline, s3EventHandlerFactory) - sqsReader := newSQSReader(logSqs, metrics, sqsAPI, maxMessagesInflight, sqsMessageHandler) + sqsMessageHandler := newSQSS3EventProcessor(log.Named("sqs_s3_event"), metrics, sqsAPI, nil, time.Minute, 5, s3EventHandlerFactory) + sqsReader := newSQSReader(logSqs, metrics, sqsAPI, maxMessagesInflight, sqsMessageHandler, pipeline) ctx, cancel := context.WithCancel(context.Background()) b.Cleanup(cancel) @@ -399,7 +399,7 @@ func benchmarkInputSQS(t *testing.T, maxMessagesInflight int) testing.BenchmarkR }() b.ResetTimer() - if err := sqsReader.Receive(ctx, pipeline); err != nil { + if err := sqsReader.Receive(ctx); err != nil { if !errors.Is(err, context.DeadlineExceeded) { t.Fatal(err) } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index f92393c66bf..7dde7de6c6d 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -40,13 +40,12 @@ func TestSQSS3EventProcessor(t *testing.T) { mockAPI := NewMockSQSAPI(ctrl) mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) mockClient := NewMockBeatClient(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) gomock.InOrder( mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) @@ -67,7 +66,6 @@ func TestSQSS3EventProcessor(t *testing.T) { defer ctrl.Finish() mockAPI := NewMockSQSAPI(ctrl) mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) invalidBodyMsg := newSQSMessage(newS3Event("log.json")) body := *invalidBodyMsg.Body @@ -76,7 +74,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockClient := NewMockBeatClient(ctrl) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker, time.Now()) t.Log(processingErr) @@ -97,13 +95,12 @@ func TestSQSS3EventProcessor(t *testing.T) { defer ctrl.Finish() mockAPI := NewMockSQSAPI(ctrl) mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) emptyRecordsMsg := newSQSMessage([]s3EventV2{}...) mockClient := NewMockBeatClient(ctrl) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker, time.Now()) require.NoError(t, processingErr) @@ -127,7 +124,6 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) mockS3Handler := NewMockS3ObjectHandler(ctrl) mockClient := NewMockBeatClient(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) mockAPI.EXPECT().ChangeMessageVisibility(gomock.Any(), gomock.Eq(&msg), gomock.Eq(visibilityTimeout)).AnyTimes().Return(nil) @@ -139,7 +135,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().ProcessS3Object().Return(uint64(1), nil), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) require.NoError(t, processingErr) @@ -162,14 +158,13 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) mockS3Handler := NewMockS3ObjectHandler(ctrl) mockClient := NewMockBeatClient(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) gomock.InOrder( mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(mockS3Handler), mockS3Handler.EXPECT().ProcessS3Object().Return(uint64(0), errors.New("fake connectivity problem")), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) t.Log(processingErr) @@ -188,7 +183,6 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) mockS3Handler := NewMockS3ObjectHandler(ctrl) mockClient := NewMockBeatClient(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) msg := msg msg.Attributes = map[string]string{ @@ -200,7 +194,7 @@ func TestSQSS3EventProcessor(t *testing.T) { mockS3Handler.EXPECT().ProcessS3Object().Return(uint64(0), errors.New("fake connectivity problem")), ) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) t.Log(processingErr) @@ -248,12 +242,11 @@ func TestSqsProcessor_keepalive(t *testing.T) { defer ctrl.Finish() mockAPI := NewMockSQSAPI(ctrl) mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) mockAPI.EXPECT().ChangeMessageVisibility(gomock.Any(), gomock.Eq(&msg), gomock.Eq(visibilityTimeout)). Times(1).Return(tc.Err) - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockBeatPipeline, mockS3HandlerFactory) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockS3HandlerFactory) var wg sync.WaitGroup wg.Add(1) p.keepalive(ctx, p.log, &wg, &msg) @@ -265,7 +258,7 @@ func TestSqsProcessor_keepalive(t *testing.T) { func TestSqsProcessor_getS3Notifications(t *testing.T) { logp.TestingSetup() - p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, nil, nil, time.Minute, 5, nil, nil) + p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, nil, nil, time.Minute, 5, nil) t.Run("s3 key is url unescaped", func(t *testing.T) { msg := newSQSMessage(newS3Event("Happy+Face.jpg")) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index e534095b4c6..0bce9a0bc23 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -100,8 +100,8 @@ func TestSQSReceiver(t *testing.T) { Return(nil) // Execute sqsReader and verify calls/state. - receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler) - require.NoError(t, receiver.Receive(ctx, mockBeatPipeline)) + receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) + require.NoError(t, receiver.Receive(ctx)) assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) @@ -134,8 +134,8 @@ func TestSQSReceiver(t *testing.T) { mockBeatPipeline := NewMockBeatPipeline(ctrl) // Execute SQSReceiver and verify calls/state. - receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler) - require.NoError(t, receiver.Receive(ctx, mockBeatPipeline)) + receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) + require.NoError(t, receiver.Receive(ctx)) assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) } @@ -156,6 +156,7 @@ func TestGetApproximateMessageCount(t *testing.T) { defer ctrl.Finish() mockAPI := NewMockSQSAPI(ctrl) mockMsgHandler := NewMockSQSProcessor(ctrl) + mockBeatPipeline := NewMockBeatPipeline(ctrl) gomock.InOrder( mockAPI.EXPECT(). @@ -166,7 +167,7 @@ func TestGetApproximateMessageCount(t *testing.T) { }), ) - receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler) + receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) receivedCount, err := receiver.GetApproximateMessageCount(ctx) assert.Equal(t, count, receivedCount) assert.Nil(t, err) @@ -181,6 +182,7 @@ func TestGetApproximateMessageCount(t *testing.T) { mockAPI := NewMockSQSAPI(ctrl) mockMsgHandler := NewMockSQSProcessor(ctrl) + mockBeatPipeline := NewMockBeatPipeline(ctrl) gomock.InOrder( mockAPI.EXPECT(). @@ -191,7 +193,7 @@ func TestGetApproximateMessageCount(t *testing.T) { }), ) - receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler) + receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) receivedCount, err := receiver.GetApproximateMessageCount(ctx) assert.Equal(t, -1, receivedCount) assert.NotNil(t, err) From a5e818d0bb52a273299b9eb5d0af7cbdd93534d5 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 17:33:02 +0900 Subject: [PATCH 18/43] Move client.Close() on deletion --- x-pack/filebeat/input/awss3/sqs.go | 2 -- x-pack/filebeat/input/awss3/sqs_acker.go | 7 ++++++- x-pack/filebeat/input/awss3/sqs_s3_event.go | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 8f95f4ac120..f6cd1838cc1 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -119,8 +119,6 @@ func (r *sqsReader) Receive(ctx context.Context) error { return } - defer client.Close() - r.log.Debugw("Going to process SQS message.", "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 430356ab1c2..19876a273dd 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -44,6 +44,7 @@ type EventACKTracker struct { keepaliveWg *sync.WaitGroup msgHandler sqsProcessor log *logp.Logger + client beat.Client } func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { @@ -75,7 +76,7 @@ func (a *EventACKTracker) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { a.ackMutexLockedOnInit.Store(false) } -func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { +func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, client beat.Client, log *logp.Logger) { // We want to execute the logic of this call only once, when the ack mutex was locked on init if !a.ackMutexLockedOnInit.Load() { return @@ -93,6 +94,7 @@ func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent a.keepaliveWg = keepaliveWg a.msgHandler = msgHandler a.log = log + a.client = client a.ackMutex.Unlock() a.ackMutexLockedOnInit.Store(false) @@ -116,6 +118,9 @@ func (a *EventACKTracker) WaitForS3() { // FlushForSQS delete related SQS message func (a *EventACKTracker) FlushForSQS() { + // This is moved here because clientOnlyACKer.ClientClosed() reset the acker listener (libbeat/common/acker/acker.go:333) + defer a.client.Close() + if !a.isSQSAcker { return } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index a9b03aeb37e..1e211a10620 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -190,7 +190,7 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message s3EventsCreatedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) p.metrics.sqsMessagesProcessedTotal.Inc() - acker.AddSQSDeletionData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log) + acker.AddSQSDeletionData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, client, p.log) return processingErr } From a005078fc179c2f5f1b040eb52188605abecc3e3 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 6 Mar 2024 17:33:30 +0900 Subject: [PATCH 19/43] fix test --- x-pack/filebeat/input/awss3/sqs_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 0bce9a0bc23..49f35bd2243 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -81,7 +81,7 @@ func TestSQSReceiver(t *testing.T) { func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) error { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") - acker.AddSQSDeletionData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.AddSQSDeletionData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, mockClient, log) acker.ACK() acker.FlushForSQS() From 0865222b4e41230b564b98431d0aa6b368ee8154 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Thu, 7 Mar 2024 10:59:06 +0900 Subject: [PATCH 20/43] Fix atomic operation on awss3.EventACKTracker.ackMutexLockedOnInit and return an error for JSON parsing interrupted by a shutdown --- x-pack/filebeat/input/awss3/s3_objects.go | 18 ++++++++---------- x-pack/filebeat/input/awss3/sqs_acker.go | 20 +++++++++++++------- x-pack/filebeat/input/awss3/sqs_s3_event.go | 2 +- 3 files changed, 22 insertions(+), 18 deletions(-) diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index 603ce3dd1d7..3bdd32d455b 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -113,7 +113,7 @@ type s3ObjectProcessor struct { } func (p *s3ObjectProcessor) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { - p.acker.SyncEventsToBeAcked(s3EventsCreatedTotal) + p.acker.MarkS3FromListingProcessedWithData(s3EventsCreatedTotal) } func (p *s3ObjectProcessor) Wait() { p.acker.WaitForS3() @@ -156,8 +156,8 @@ func (p *s3ObjectProcessor) ProcessS3Object() (uint64, error) { } // try to create a decoder from the using the codec config - decoder, err := newDecoder(p.readerConfig.Decoding, reader) - if err != nil { + decoder, decoderErr := newDecoder(p.readerConfig.Decoding, reader) + if decoderErr != nil { return 0, err } if decoder != nil { @@ -165,8 +165,8 @@ func (p *s3ObjectProcessor) ProcessS3Object() (uint64, error) { var evtOffset int64 for decoder.next() { - data, err := decoder.decode() - if err != nil { + data, decoderErr := decoder.decode() + if decoderErr != nil { if errors.Is(err, io.EOF) { return p.s3EventsCreatedTotal, nil } @@ -237,8 +237,7 @@ func (p *s3ObjectProcessor) readJSON(r io.Reader) error { dec := json.NewDecoder(r) dec.UseNumber() - // why we check ctx.Err() here? - for dec.More() && (errors.Is(p.ctx.Err(), context.Canceled) || p.ctx.Err() == nil) { + for dec.More() && p.ctx.Err() == nil { offset := dec.InputOffset() var item json.RawMessage @@ -275,8 +274,7 @@ func (p *s3ObjectProcessor) readJSONSlice(r io.Reader, evtOffset int64) (int64, } // we track each event offset separately since we are reading a slice. - // why we check ctx.Err() here? - for dec.More() && (errors.Is(p.ctx.Err(), context.Canceled) || p.ctx.Err() == nil) { + for dec.More() && p.ctx.Err() == nil { var item json.RawMessage if err := dec.Decode(&item); err != nil { return -1, fmt.Errorf("failed to decode json: %w", err) @@ -295,7 +293,7 @@ func (p *s3ObjectProcessor) readJSONSlice(r io.Reader, evtOffset int64) (int64, evtOffset++ } - return evtOffset, nil + return evtOffset, p.ctx.Err() } func (p *s3ObjectProcessor) splitEventList(key string, raw json.RawMessage, offset int64, objHash string) error { diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 19876a273dd..edc876d8dc0 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -51,7 +51,7 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA ctx, cancel := context.WithCancel(ctx) ackMutex := new(sync.RWMutex) // We need to lock on ack mutex, in order to know that we have passed the info to the acker about the total to be acked - // Lock it as soon as we create the acker. It will be unlocked and in either SyncEventsToBeAcked or AddSQSDeletionData + // Lock it as soon as we create the acker. It will be unlocked and in either MarkS3FromListingProcessedWithData or MarkSQSProcessedWithData ackMutex.Lock() return &EventACKTracker{ ctx: ctx, @@ -64,9 +64,11 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA } } -func (a *EventACKTracker) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { +// MarkS3FromListingProcessedWithData has to be used when the acker is used when the input is in s3 direct listing mode, instead of MarkSQSProcessedWithData +// Specifically we both Swap the value of EventACKTracker.ackMutexLockedOnInit initialised in NewEventACKTracker +func (a *EventACKTracker) MarkS3FromListingProcessedWithData(s3EventsCreatedTotal uint64) { // We want to execute the logic of this call only once, when the ack mutex was locked on init - if !a.ackMutexLockedOnInit.Load() { + if !a.ackMutexLockedOnInit.Swap(false) { return } @@ -76,9 +78,11 @@ func (a *EventACKTracker) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { a.ackMutexLockedOnInit.Store(false) } -func (a *EventACKTracker) AddSQSDeletionData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, client beat.Client, log *logp.Logger) { +// MarkSQSProcessedWithData has to be used when the acker is used when the input is in sqs-s3 mode, instead of MarkS3FromListingProcessedWithData +// Specifically we both Swap the value of EventACKTracker.ackMutexLockedOnInit initialised in NewEventACKTracker +func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, client beat.Client, log *logp.Logger) { // We want to execute the logic of this call only once, when the ack mutex was locked on init - if !a.ackMutexLockedOnInit.Load() { + if !a.ackMutexLockedOnInit.Swap(false) { return } @@ -105,7 +109,7 @@ func (a *EventACKTracker) FullyAcked() bool { } -// WaitForS3 must be called after SyncEventsToBeAcked +// WaitForS3 must be called after MarkS3FromListingProcessedWithData func (a *EventACKTracker) WaitForS3() { // If it's fully acked then cancel the context. if a.FullyAcked() { @@ -119,7 +123,9 @@ func (a *EventACKTracker) WaitForS3() { // FlushForSQS delete related SQS message func (a *EventACKTracker) FlushForSQS() { // This is moved here because clientOnlyACKer.ClientClosed() reset the acker listener (libbeat/common/acker/acker.go:333) - defer a.client.Close() + if a.client != nil { + defer a.client.Close() + } if !a.isSQSAcker { return diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 1e211a10620..44eb09164ea 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -190,7 +190,7 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message s3EventsCreatedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) p.metrics.sqsMessagesProcessedTotal.Inc() - acker.AddSQSDeletionData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, client, p.log) + acker.MarkSQSProcessedWithData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, client, p.log) return processingErr } From 445146a74b44adc4f87717517cf35d61fe40b4b8 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Thu, 7 Mar 2024 10:59:45 +0900 Subject: [PATCH 21/43] fix tests --- x-pack/filebeat/input/awss3/input_benchmark_test.go | 2 +- x-pack/filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker_test.go | 8 ++++---- x-pack/filebeat/input/awss3/sqs_test.go | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 682b1af6c36..705769a6fb5 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -498,7 +498,7 @@ func benchmarkInputS3(t *testing.T, numberOfWorkers int) testing.BenchmarkResult client := pubtest.NewChanClientWithCallback(100, func(event beat.Event) { go func(acker *EventACKTracker) { // 63 is the total number of events in a single S3 object - acker.SyncEventsToBeAcked(63) + acker.MarkS3FromListingProcessedWithData(63) }(event.Private.(*EventACKTracker)) event.Private.(*EventACKTracker).ACK() diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 1d8816ecc2d..57cae38c3ce 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -321,7 +321,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint s3Event, s3Resp := newS3Object(t, file, contentType) ack := NewEventACKTracker(ctx, nil) - ack.SyncEventsToBeAcked(numEvents) + ack.MarkS3FromListingProcessedWithData(numEvents) var events []beat.Event gomock.InOrder( mockS3API.EXPECT(). diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index de6f31be980..5c3e7bcb20d 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -18,7 +18,7 @@ func TestEventACKTracker(t *testing.T) { t.Cleanup(cancel) acker := NewEventACKTracker(ctx, nil) - acker.SyncEventsToBeAcked(1) + acker.MarkS3FromListingProcessedWithData(1) acker.ACK() assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) @@ -44,7 +44,7 @@ func TestEventACKHandler(t *testing.T) { // Create acker. Add one pending ACK. acker := NewEventACKTracker(ctx, nil) - acker.SyncEventsToBeAcked(1) + acker.MarkS3FromListingProcessedWithData(1) // Create an ACK handler and simulate one ACKed event. ackHandler := NewEventACKHandler() @@ -62,7 +62,7 @@ func TestEventACKHandlerWaitForS3(t *testing.T) { // Create acker. Add one pending ACK. acker := NewEventACKTracker(ctx, nil) - acker.SyncEventsToBeAcked(1) + acker.MarkS3FromListingProcessedWithData(1) acker.ACK() acker.WaitForS3() assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) @@ -77,7 +77,7 @@ func TestEventACKHandlerFullyAcked(t *testing.T) { // Create acker. Add one pending ACK. acker := NewEventACKTracker(ctx, nil) - acker.SyncEventsToBeAcked(1) + acker.MarkS3FromListingProcessedWithData(1) acker.ACK() assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, true, acker.FullyAcked()) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 49f35bd2243..675946bbc09 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -81,7 +81,7 @@ func TestSQSReceiver(t *testing.T) { func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) error { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") - acker.AddSQSDeletionData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, mockClient, log) + acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, mockClient, log) acker.ACK() acker.FlushForSQS() From 46e92a5ed664dfbf913d65d9f538804e4e569d51 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Fri, 8 Mar 2024 20:54:56 +0900 Subject: [PATCH 22/43] Introduce capped level of workers --- x-pack/filebeat/input/awss3/sqs.go | 158 ++++++++++++-------- x-pack/filebeat/input/awss3/sqs_acker.go | 41 +++-- x-pack/filebeat/input/awss3/sqs_s3_event.go | 2 +- 3 files changed, 118 insertions(+), 83 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index f6cd1838cc1..5e1cdee7406 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -7,6 +7,8 @@ package awss3 import ( "context" "errors" + "github.com/elastic/beats/v7/libbeat/common/atomic" + "github.com/elastic/go-concert/timed" "strconv" "sync" "time" @@ -16,7 +18,6 @@ import ( "github.com/elastic/beats/v7/libbeat/beat" awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "github.com/elastic/elastic-agent-libs/logp" - "github.com/elastic/go-concert/timed" ) const ( @@ -50,76 +51,69 @@ func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessag } } +type processingData struct { + msg types.Message + start time.Time +} + func (r *sqsReader) Receive(ctx context.Context) error { - // The loop tries to keep the ProcessSQS workers busy as much as possible while - // honoring the max message cap as opposed to a simpler loop that receives - // N messages, waits for them all to finish sending events to the queue, then requests N more messages. - var processingWg sync.WaitGroup + workersWg := new(sync.WaitGroup) + workersWg.Add(r.maxMessagesInflight) + workersChan := make(chan processingData, r.maxMessagesInflight) deletionWg := new(sync.WaitGroup) + deletionWaiter := atomic.NewBool(true) - for ctx.Err() == nil { - // Determine how many SQS workers are available. - workers, err := r.workerSem.AcquireContext(r.maxMessagesInflight, ctx) - if err != nil { - break - } - - // Receive (at most) as many SQS messages as there are workers. - msgs, err := r.sqs.ReceiveMessage(ctx, workers) - if err != nil { - r.workerSem.Release(workers) - - if ctx.Err() == nil { - r.log.Warnw("SQS ReceiveMessage returned an error. Will retry after a short delay.", "error", err) - - // Throttle retries. - _ = timed.Wait(ctx, sqsRetryDelay) - } - continue - } + var clientsMutex sync.Mutex + clients := make(map[uint64]beat.Client, r.maxMessagesInflight) - // Release unused workers. - r.workerSem.Release(workers - len(msgs)) + // Start a fixed amount of goroutines that will process all the SQS messages sent to the workersChan asynchronously. + for i := 0; i < r.maxMessagesInflight; i++ { + id := r.metrics.beginSQSWorker() - r.log.Debugf("Received %v SQS messages.", len(msgs)) - r.metrics.sqsMessagesReceivedTotal.Add(uint64(len(msgs))) + // Create a pipeline client scoped to this goroutine. + client, err := r.pipeline.ConnectWith(beat.ClientConfig{ + EventListener: NewEventACKHandler(), + Processing: beat.ProcessingConfig{ + // This input only produces events with basic types so normalization + // is not required. + EventNormalization: boolPtr(false), + }, + }) - // Add to processing wait group to wait for all messages to be processed. - processingWg.Add(len(msgs)) - deletionWg.Add(len(msgs)) + clientsMutex.Lock() + clients[id] = client + clientsMutex.Unlock() - for _, msg := range msgs { - // Process each SQS message asynchronously with a goroutine. - go func(msg types.Message, start time.Time) { - id := r.metrics.beginSQSWorker() - defer func() { - // Mark processing wait group as done. - r.metrics.endSQSWorker(id) - processingWg.Done() - r.workerSem.Release(1) - }() - - // Create a pipeline client scoped to this goroutine. - client, err := r.pipeline.ConnectWith(beat.ClientConfig{ - EventListener: NewEventACKHandler(), - Processing: beat.ProcessingConfig{ - // This input only produces events with basic types so normalization - // is not required. - EventNormalization: boolPtr(false), - }, - }) + if err != nil { + r.log.Warnw("Failed setting up worker.", + "worker_id", id, + "error", err) - if err != nil { - r.log.Warnw("Failed processing SQS message.", - "error", err, - "message_id", *msg.MessageId, - "elapsed_time_ns", time.Since(start)) + r.metrics.endSQSWorker(id) + workersWg.Done() + } + go func(id uint64, client beat.Client) { + defer func() { + // Mark processing wait group as done. + r.metrics.endSQSWorker(id) + workersWg.Done() + }() + for { + incomingData, ok := <-workersChan + if !ok { return } + deletionWg.Add(1) + deletionWaiter.Swap(false) + + msg := incomingData.msg + start := incomingData.start + r.log.Debugw("Going to process SQS message.", + "worker_id", id, "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) @@ -128,6 +122,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { err = r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) if err != nil { r.log.Warnw("Failed processing SQS message.", + "worker_id", id, "error", err, "message_id", *msg.MessageId, "elapsed_time_ns", time.Since(start)) @@ -136,18 +131,55 @@ func (r *sqsReader) Receive(ctx context.Context) error { } r.log.Debugw("Success processing SQS message.", - "message_id", *msg.MessageId, + "worker_id", id, + "message_id", msg.MessageId, "elapsed_time_ns", time.Since(start)) + } + }(id, client) + } + + // The loop tries to keep a fixed amount of goroutines that process SQS message busy as much as possible while + // honoring the max message cap as opposed to a simpler loop that receives N messages, waits for them all to finish + // sending events to the queue, then requests N more messages. + for ctx.Err() == nil { + // Receive (at most) as many SQS messages as there are workers. + msgs, err := r.sqs.ReceiveMessage(ctx, r.maxMessagesInflight) + if err != nil { + if ctx.Err() == nil { + r.log.Warnw("SQS ReceiveMessage returned an error. Will retry after a short delay.", "error", err) + + // Throttle retries. + _ = timed.Wait(ctx, sqsRetryDelay) + } + continue + } + + r.log.Debugf("Received %v SQS messages.", len(msgs)) + r.metrics.sqsMessagesReceivedTotal.Add(uint64(len(msgs))) + + for _, msg := range msgs { + // Send each SQS message to a channel where a fixed amount of goroutines will process all of them asynchronously. + go func(msg types.Message, start time.Time) { + workersChan <- processingData{msg: msg, start: start} }(msg, time.Now()) } } - // Wait for all processing goroutines to finish. - processingWg.Wait() + // Let's stop the workers + close(workersChan) + + // Wait for all processing to happen. + workersWg.Wait() // Wait for all deletion to happen. + for deletionWaiter.Load() { + _ = timed.Wait(ctx, 500*time.Millisecond) + } + deletionWg.Wait() + closeClients(clients) + if errors.Is(ctx.Err(), context.Canceled) { // A canceled context is a normal shutdown. return nil @@ -156,6 +188,12 @@ func (r *sqsReader) Receive(ctx context.Context) error { return ctx.Err() } +func closeClients(clients map[uint64]beat.Client) { + for _, client := range clients { + client.Close() + } +} + func (r *sqsReader) GetApproximateMessageCount(ctx context.Context) (int, error) { attributes, err := r.sqs.GetQueueAttributes(ctx, []types.QueueAttributeName{sqsApproximateNumberOfMessages}) if err == nil { diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index edc876d8dc0..17cd84cfe34 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -18,23 +18,29 @@ import ( "github.com/elastic/beats/v7/libbeat/common/atomic" ) +var ackerIDCounter *atomic.Uint64 + +func init() { + ackerIDCounter = atomic.NewUint64(0) +} + // EventACKTracker tracks the publishing state of S3 objects. Specifically // it tracks the number of message acknowledgements that are pending from the // output. It can be used to wait until all ACKs have been received for one or // more S3 objects. type EventACKTracker struct { - DeletionWg *sync.WaitGroup + ID uint64 + + EventsToBeAcked *atomic.Uint64 - EventsToBeAcked *atomic.Uint64 - TotalEventsAcked *atomic.Uint64 + ctx context.Context + cancel context.CancelFunc + deletionWg *sync.WaitGroup ackMutex *sync.RWMutex ackMutexLockedOnInit *atomic.Bool - isSQSAcker bool - - ctx context.Context - cancel context.CancelFunc + isSQSAcker bool msg *types.Message ReceiveCount int start time.Time @@ -44,7 +50,6 @@ type EventACKTracker struct { keepaliveWg *sync.WaitGroup msgHandler sqsProcessor log *logp.Logger - client beat.Client } func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { @@ -54,12 +59,12 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA // Lock it as soon as we create the acker. It will be unlocked and in either MarkS3FromListingProcessedWithData or MarkSQSProcessedWithData ackMutex.Lock() return &EventACKTracker{ + ID: ackerIDCounter.Inc(), ctx: ctx, cancel: cancel, + deletionWg: deletionWg, ackMutex: ackMutex, ackMutexLockedOnInit: atomic.NewBool(true), - DeletionWg: deletionWg, - TotalEventsAcked: atomic.NewUint64(0), EventsToBeAcked: atomic.NewUint64(0), } } @@ -75,12 +80,11 @@ func (a *EventACKTracker) MarkS3FromListingProcessedWithData(s3EventsCreatedTota a.EventsToBeAcked.Add(s3EventsCreatedTotal) a.ackMutex.Unlock() - a.ackMutexLockedOnInit.Store(false) } // MarkSQSProcessedWithData has to be used when the acker is used when the input is in sqs-s3 mode, instead of MarkS3FromListingProcessedWithData // Specifically we both Swap the value of EventACKTracker.ackMutexLockedOnInit initialised in NewEventACKTracker -func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, client beat.Client, log *logp.Logger) { +func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { // We want to execute the logic of this call only once, when the ack mutex was locked on init if !a.ackMutexLockedOnInit.Swap(false) { return @@ -98,14 +102,12 @@ func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, published a.keepaliveWg = keepaliveWg a.msgHandler = msgHandler a.log = log - a.client = client a.ackMutex.Unlock() - a.ackMutexLockedOnInit.Store(false) } func (a *EventACKTracker) FullyAcked() bool { - return a.TotalEventsAcked.Load() == a.EventsToBeAcked.Load() + return a.EventsToBeAcked.Load() == 0 } @@ -122,11 +124,6 @@ func (a *EventACKTracker) WaitForS3() { // FlushForSQS delete related SQS message func (a *EventACKTracker) FlushForSQS() { - // This is moved here because clientOnlyACKer.ClientClosed() reset the acker listener (libbeat/common/acker/acker.go:333) - if a.client != nil { - defer a.client.Close() - } - if !a.isSQSAcker { return } @@ -136,7 +133,7 @@ func (a *EventACKTracker) FlushForSQS() { a.keepaliveWg.Wait() err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) - a.DeletionWg.Done() + a.deletionWg.Done() if err != nil { a.log.Warnw("Failed deleting SQS message.", @@ -163,7 +160,7 @@ func (a *EventACKTracker) ACK() { panic("misuse detected: ACK call on fully acked") } - a.TotalEventsAcked.Inc() + a.EventsToBeAcked.Dec() if a.FullyAcked() { a.cancel() diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 44eb09164ea..b87fbc79704 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -190,7 +190,7 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message s3EventsCreatedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) p.metrics.sqsMessagesProcessedTotal.Inc() - acker.MarkSQSProcessedWithData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, client, p.log) + acker.MarkSQSProcessedWithData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log) return processingErr } From 8af443504f34531e46d625b076778046e322eae3 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Fri, 8 Mar 2024 21:18:29 +0900 Subject: [PATCH 23/43] fix tests --- .../input/awss3/input_benchmark_test.go | 47 +++++++++---------- .../filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs.go | 11 ++--- x-pack/filebeat/input/awss3/sqs_acker_test.go | 5 -- x-pack/filebeat/input/awss3/sqs_test.go | 47 ++++++++++--------- 5 files changed, 53 insertions(+), 59 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 705769a6fb5..bcf92289dc8 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -254,7 +254,7 @@ func newFakePipeline() *fakePipeline { mutex: new(sync.Mutex), flush: time.NewTicker(10 * time.Second), pendingEvents: atomic.NewUint64(0), - clients: make([]beat.Client, 0), + clients: make([]*ackClient, 0), } go func() { @@ -274,37 +274,33 @@ type fakePipeline struct { flush *time.Ticker mutex *sync.Mutex pendingEvents *atomic.Uint64 - clients []beat.Client + clients []*ackClient } func (fp *fakePipeline) ackEvents() { for _, client := range fp.clients { - acker := client.(*ackClient).acker - - if acker == nil || !acker.isSQSAcker { - continue - } - - if acker.FullyAcked() { - continue - } + for _, acker := range client.ackers { + if acker.FullyAcked() { + continue + } - ackHandler := NewEventACKHandler() - currentEvents := acker.EventsToBeAcked.Load() - acker.TotalEventsAcked.Load() - for i := currentEvents; i > 0; i-- { - fp.pendingEvents.Dec() - ackHandler.AddEvent(beat.Event{Private: acker}, true) - } + addedEvents := 0 + for acker.EventsToBeAcked.Load() > 0 && uint64(addedEvents) < acker.EventsToBeAcked.Load() { + addedEvents++ + fp.pendingEvents.Dec() + client.eventListener.AddEvent(beat.Event{Private: acker}, true) + } - if currentEvents > 0 { - ackHandler.ACKEvents(int(currentEvents)) + if addedEvents > 0 { + client.eventListener.ACKEvents(addedEvents) + } } } } func (fp *fakePipeline) ConnectWith(clientConfig beat.ClientConfig) (beat.Client, error) { fp.mutex.Lock() - client := &ackClient{fp: fp} + client := &ackClient{fp: fp, ackers: make(map[uint64]*EventACKTracker), eventListener: NewEventACKHandler()} fp.clients = append(fp.clients, client) fp.mutex.Unlock() return client, nil @@ -318,8 +314,9 @@ var _ beat.Client = (*ackClient)(nil) // ackClient is a fake beat.Client that ACKs the published messages. type ackClient struct { - fp *fakePipeline - acker *EventACKTracker + fp *fakePipeline + ackers map[uint64]*EventACKTracker + eventListener beat.EventListener } func (c *ackClient) Close() error { return nil } @@ -327,10 +324,8 @@ func (c *ackClient) Close() error { return nil } func (c *ackClient) Publish(event beat.Event) { c.fp.mutex.Lock() c.fp.pendingEvents.Inc() - if c.acker == nil { - c.acker = event.Private.(*EventACKTracker) - } - + acker := event.Private.(*EventACKTracker) + c.ackers[acker.ID] = acker if c.fp.pendingEvents.Load() > 3200 { c.fp.ackEvents() } diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 57cae38c3ce..3508754296b 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -342,8 +342,8 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint if !expectErr { require.NoError(t, err) assert.EqualValues(t, numEvents, len(events)) - assert.EqualValues(t, numEvents, ack.TotalEventsAcked.Load()) assert.EqualValues(t, numEvents, s3EventsCreatedTotal) + assert.EqualValues(t, 0, ack.EventsToBeAcked.Load()) } else { require.Error(t, err) } diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 5e1cdee7406..e9751c3e667 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -158,10 +158,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { r.metrics.sqsMessagesReceivedTotal.Add(uint64(len(msgs))) for _, msg := range msgs { - // Send each SQS message to a channel where a fixed amount of goroutines will process all of them asynchronously. - go func(msg types.Message, start time.Time) { - workersChan <- processingData{msg: msg, start: start} - }(msg, time.Now()) + workersChan <- processingData{msg: msg, start: time.Now()} } } @@ -172,8 +169,10 @@ func (r *sqsReader) Receive(ctx context.Context) error { workersWg.Wait() // Wait for all deletion to happen. - for deletionWaiter.Load() { - _ = timed.Wait(ctx, 500*time.Millisecond) + if r.metrics.sqsMessagesReceivedTotal.Get() > 0 { + for deletionWaiter.Load() { + _ = timed.Wait(ctx, 500*time.Millisecond) + } } deletionWg.Wait() diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index 5c3e7bcb20d..35383577954 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -21,7 +21,6 @@ func TestEventACKTracker(t *testing.T) { acker.MarkS3FromListingProcessedWithData(1) acker.ACK() - assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -51,7 +50,6 @@ func TestEventACKHandler(t *testing.T) { ackHandler.AddEvent(beat.Event{Private: acker}, true) ackHandler.ACKEvents(1) - assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -65,7 +63,6 @@ func TestEventACKHandlerWaitForS3(t *testing.T) { acker.MarkS3FromListingProcessedWithData(1) acker.ACK() acker.WaitForS3() - assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) @@ -79,13 +76,11 @@ func TestEventACKHandlerFullyAcked(t *testing.T) { acker := NewEventACKTracker(ctx, nil) acker.MarkS3FromListingProcessedWithData(1) acker.ACK() - assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) acker.EventsToBeAcked.Inc() - assert.EqualValues(t, 1, acker.TotalEventsAcked.Load()) assert.EqualValues(t, false, acker.FullyAcked()) } diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 675946bbc09..d5c2fad8fcb 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -71,29 +71,27 @@ func TestSQSReceiver(t *testing.T) { mockClient := NewMockBeatClient(ctrl) mockBeatPipeline := NewMockBeatPipeline(ctrl) - gomock.InOrder( - mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil), - // Expect the one message returned to have been processed. - mockMsgHandler.EXPECT(). - ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). - Times(1). - DoAndReturn( - func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) error { - _, keepaliveCancel := context.WithCancel(ctx) - log := log.Named("sqs_s3_event") - acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, mockClient, log) - acker.ACK() - acker.FlushForSQS() - - return nil - }), - ) + mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil).Times(maxMessages) + + // Expect the one message returned to have been processed. + mockMsgHandler.EXPECT(). + ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). + Times(1). + DoAndReturn( + func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) error { + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.ACK() + acker.FlushForSQS() + + return nil + }) - // The two expected calls happen in different goroutines, we cannot enforce an oder // Expect the client to be closed - mockClient.EXPECT().Close() + mockClient.EXPECT().Close().Times(maxMessages) + // Expect the one message returned to have been deleted. - // We mockMsgHandler.EXPECT(). DeleteSQS(gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). @@ -114,6 +112,11 @@ func TestSQSReceiver(t *testing.T) { mockAPI := NewMockSQSAPI(ctrl) mockMsgHandler := NewMockSQSProcessor(ctrl) + mockClient := NewMockBeatClient(ctrl) + mockBeatPipeline := NewMockBeatPipeline(ctrl) + + mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil).Times(maxMessages) + gomock.InOrder( // Initial ReceiveMessage gets an error. mockAPI.EXPECT(). @@ -132,7 +135,9 @@ func TestSQSReceiver(t *testing.T) { }), ) - mockBeatPipeline := NewMockBeatPipeline(ctrl) + // Expect the client to be closed + mockClient.EXPECT().Close().Times(maxMessages) + // Execute SQSReceiver and verify calls/state. receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) require.NoError(t, receiver.Receive(ctx)) From 8d26b37d07454f6c03ad3ab19e20c26fa06b66d7 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Fri, 8 Mar 2024 21:48:55 +0900 Subject: [PATCH 24/43] lint --- x-pack/filebeat/input/awss3/sqs.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index e9751c3e667..28749fdabb2 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -7,12 +7,13 @@ package awss3 import ( "context" "errors" - "github.com/elastic/beats/v7/libbeat/common/atomic" - "github.com/elastic/go-concert/timed" "strconv" "sync" "time" + "github.com/elastic/beats/v7/libbeat/common/atomic" + "github.com/elastic/go-concert/timed" + "github.com/aws/aws-sdk-go-v2/service/sqs/types" "github.com/elastic/beats/v7/libbeat/beat" From 3d0b7d1d02c2f0b8d98252a537d931fbebeaab54 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 11 Mar 2024 12:52:03 +0900 Subject: [PATCH 25/43] add benchmarks markdown --- .../input/awss3/benchmarks-TO-BE-DELETED.md | 237 ++++++++++++++++++ 1 file changed, 237 insertions(+) create mode 100644 x-pack/filebeat/input/awss3/benchmarks-TO-BE-DELETED.md diff --git a/x-pack/filebeat/input/awss3/benchmarks-TO-BE-DELETED.md b/x-pack/filebeat/input/awss3/benchmarks-TO-BE-DELETED.md new file mode 100644 index 00000000000..92cbe2bab9a --- /dev/null +++ b/x-pack/filebeat/input/awss3/benchmarks-TO-BE-DELETED.md @@ -0,0 +1,237 @@ +## DELETION ON ACKER + +### DYNAMIC NUMBER OF MESSAGES (124.20s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 534 | 49517.80415430267 | 386 kB | 36 MB | 8 | 741.839762611276 | 2 | 0.19998500028504163 | 10.000750042 | 0.010784 | 9.989966042 | 99,89 | 12 | +| 2 | 1096 | 66849.9877126551 | 806 kB | 49 MB | 16 | 975.9122293818261 | 3 | 0.29995790216825385 | 10.001403458 | 0.016394917 | 9.985008541000001 | 99,84 | 12 | +| 4 | 1569 | 94558.18546638881 | 1.2 MB | 70 MB | 21 | 1265.5971286132344 | 5 | 0.49993063257501463 | 10.001387541 | 0.016592958 | 9.984794583 | 99,83 | 12 | +| 8 | 3394 | 125587.03334972182 | 2.5 MB | 91 MB | 41 | 1517.1091241421905 | 9 | 0.8999142269252461 | 10.000953125 | 0.027025083 | 9.973928042 | 99,73 | 12 | +| 16 | 5956 | 272461.62721531175 | 4.3 MB | 199 MB | 72 | 3293.6932772838227 | 18 | 1.7997335643835817 | 10.001480417 | 0.021859959 | 9.979620458 | 99,78 | 12 | +| 32 | 12800 | 266282.4986867904 | 9.3 MB | 194 MB | 161 | 3349.3345537947857 | 36 | 3.5995028334290944 | 10.001381209 | 0.04806925 | 9.953311959 | 99,52 | 12 | +| 64 | 27343 | 186554.56881472556 | 20 MB | 135 MB | 344 | 2347.027453910163 | 71 | 7.099137336967879 | 10.001215166 | 0.146568375 | 9.854646791 | 98,53 | 12 | +| 128 | 55831 | 129989.33761940138 | 41 MB | 94 MB | 681 | 1585.5481527970544 | 141 | 14.098866039483537 | 10.000804292 | 0.429504458 | 9.571299834 | 95,71 | 12 | +| 256 | 109313 | 83960.9816045163 | 79 MB | 61 MB | 1357 | 1042.2827297515266 | 282 | 28.19473134641538 | 10.001868666 | 1.30195 | 8.699918666 | 86,98 | 12 | +| 512 | 219611 | 46694.95705637275 | 160 MB | 34 MB | 2712 | 576.6410768899686 | 564 | 56.389861570925405 | 10.001797917 | 4.703098875 | 5.298699041999999 | 52,98 | 12 | +| 1024 | 448582 | 23887.37921413506 | 326 MB | 17 MB | 5565 | 296.34105988796165 | 1127 | 56.34529821869664 | 20.001668917 | 18.779037917 | 1.2226309999999998 | 6,11 | 12 | + +### 71 MESSAGES (111.50s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 26140 | 73290.30170185407 | 19 MB | 53 MB | 329 | 922.4372325902827 | 71 | 7.098849690328135 | 10.001620417 | 0.356663834 | 9.644956582999999 | 96,43 | 12 | +| 2 | 26140 | 141016.09063185065 | 19 MB | 102 MB | 329 | 1774.8390902019457 | 71 | 7.0995195101991655 | 10.000676792 | 0.185368917 | 9.815307875 | 98,15 | 12 | +| 4 | 26140 | 253372.72881857055 | 19 MB | 184 MB | 329 | 3188.9681630187342 | 71 | 7.099005488252842 | 10.001400917 | 0.103168167 | 9.89823275 | 98,97 | 12 | +| 8 | 26140 | 379671.6865344571 | 19 MB | 276 MB | 329 | 4778.576314836892 | 71 | 7.098718711087268 | 10.001804958 | 0.068848958 | 9.932955999999999 | 99,31 | 12 | +| 16 | 26140 | 294557.1993402324 | 19 MB | 214 MB | 329 | 3707.3189970518924 | 71 | 7.0989061472990285 | 10.001540875 | 0.088743375 | 9.9127975 | 99,11 | 12 | +| 32 | 26140 | 227728.69946491945 | 19 MB | 165 MB | 329 | 2866.2104867619933 | 71 | 7.098394106876501 | 10.002262333 | 0.114785708 | 9.887476625 | 98,85 | 12 | +| 64 | 26140 | 192476.25771489978 | 19 MB | 140 MB | 329 | 2422.5206116374147 | 71 | 7.098927944038247 | 10.001510166 | 0.135808958 | 9.865701207999999 | 98,64 | 12 | +| 128 | 26140 | 177702.24337185588 | 19 MB | 129 MB | 329 | 2236.5737593473827 | 71 | 7.099228284778609 | 10.001087041 | 0.1471 | 9.853987041 | 98,53 | 12 | +| 256 | 26140 | 136009.97141443985 | 19 MB | 99 MB | 329 | 1711.8316983684283 | 71 | 7.099239198672492 | 10.001071666 | 0.192191791 | 9.808879874999999 | 98,08 | 12 | +| 512 | 26140 | 116130.85213918249 | 19 MB | 84 MB | 329 | 1461.6316126163367 | 71 | 7.0980791412464574 | 10.002706167 | 0.225090917 | 9.77761525 | 97,75 | 12 | +| 1024 | 26140 | 87955.27538177604 | 19 MB | 64 MB | 329 | 1107.0116909182982 | 71 | 7.098742930466147 | 10.001770834 | 0.2971965 | 9.704574334 | 97,03 | 12 | + +### 1 MESSAGE : 1 OBJECT : 1 EVENT (110.12s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 2 | 357.9311365548448 | 1.2 kB | 212 kB | 2 | 357.9311365548448 | 2 | 0.1999996550005951 | 10.00001725 | 0.005587667 | 9.994429583 | 99,94 | 12 | +| 2 | 3 | 528.5334974840924 | 2.6 kB | 457 kB | 3 | 528.5334974840924 | 3 | 0.2999928589399865 | 10.000238041 | 0.005676083 | 9.994561957999998 | 99,94 | 12 | +| 4 | 5 | 889.6797153024911 | 3.8 kB | 678 kB | 5 | 889.6797153024911 | 5 | 0.5000020625085079 | 9.99995875 | 0.00562 | 9.994338749999999 | 99,94 | 12 | +| 8 | 9 | 1387.5238480661387 | 7.4 kB | 1.1 MB | 9 | 1387.5238480661387 | 9 | 0.9000190428729149 | 9.999788417 | 0.006486375 | 9.993302042 | 99,94 | 12 | +| 16 | 18 | 3181.7579212514916 | 16 kB | 2.8 MB | 18 | 3181.7579212514916 | 18 | 1.7999130942560961 | 10.000482833 | 0.00565725 | 9.994825582999999 | 99,94 | 12 | +| 32 | 36 | 6485.805184788696 | 28 kB | 5.1 MB | 36 | 6485.805184788696 | 36 | 3.5998180144401157 | 10.000505541 | 0.005550583 | 9.994954958000001 | 99,94 | 12 | +| 64 | 71 | 6472.688990211288 | 54 kB | 4.9 MB | 71 | 6472.688990211288 | 71 | 7.099864422408963 | 10.000190958 | 0.010969166 | 9.989221791999999 | 99,89 | 12 | +| 128 | 141 | 17653.503788554593 | 108 kB | 14 MB | 141 | 17653.503788554593 | 141 | 14.099971330528293 | 10.000020333 | 0.007987083 | 9.99203325 | 99,92 | 12 | +| 256 | 282 | 17596.772577189615 | 216 kB | 14 MB | 282 | 17596.772577189615 | 282 | 28.19464947881269 | 10.001897708 | 0.016025666 | 9.985872041999999 | 99,84 | 12 | +| 512 | 564 | 29950.81216353462 | 426 kB | 23 MB | 564 | 29950.81216353462 | 564 | 56.39437794859186 | 10.000996917 | 0.018830875 | 9.982166042 | 99,81 | 12 | +| 1024 | 1127 | 52982.10257993222 | 843 kB | 40 MB | 1127 | 52982.10257993222 | 1127 | 112.65294204978225 | 10.00417725 | 0.021271334 | 9.982905916 | 99,79 | 12 | + + +## UNCAPPED DELETION + +### DYNAMIC NUMBER OF MESSAGES (236.53s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|---------------------:|------------------:|-----:| +| 1 | 534 | 33476.651853352705 | 386 kB | 24 MB | 8 | 501.5228742075312 | 2 | 0.19999470432022484 | 10.000264791 | 0.015951416 | 9.984313375 | 99,84 | 10 | +| 2 | 1096 | 104412.43521618043 | 806 kB | 77 MB | 16 | 1524.2691272435097 | 3 | 0.29997075660086586 | 10.000974875 | 0.010496834 | 9.990478041000001 | 99,90 | 10 | +| 4 | 1569 | 30350.754911676788 | 1.2 MB | 22 MB | 21 | 406.224253119957 | 5 | 0.49994750756149087 | 10.001049959 | 0.051695584 | 9.949354374999999 | 99,48 | 10 | +| 8 | 3394 | 10774.6003355815 | 2.5 MB | 7.8 MB | 41 | 130.15869586294681 | 9 | 0.8999113850059893 | 10.000984708 | 0.315000083 | 9.685984625000001 | 96,85 | 10 | +| 16 | 5956 | 52104.57380964997 | 4.3 MB | 38 MB | 72 | 629.8739614329747 | 18 | 1.79256181708282 | 10.041494708 | 0.114308583 | 9.927186125 | 98,86 | 10 | +| 32 | 12800 | 41893.11600502637 | 9.3 MB | 31 MB | 161 | 526.9368497507223 | 36 | 3.580617015831019 | 10.054133084 | 0.305539459 | 9.748593625 | 96,96 | 10 | +| 64 | 27343 | 24680.776188829113 | 20 MB | 18 MB | 344 | 310.50678451366764 | 71 | 7.069355492709372 | 10.043348375 | 1.107866292 | 8.935482083 | 88,97 | 10 | +| 128 | 55831 | 41824.388657950345 | 41 MB | 30 MB | 681 | 510.1540125748095 | 141 | 14.051082386142353 | 10.034814125 | 1.334891 | 8.699923125 | 86,70 | 10 | +| 256 | 109313 | 22552.537371923707 | 79 MB | 16 MB | 1357 | 279.9648094343808 | 282 | 28.10006119662218 | 10.035565333 | 4.84703775 | 5.188527582999999 | 51,70 | 10 | +| 512 | 219611 | 63324.72009374242 | 160 MB | 46 MB | 2712 | 782.0038199098836 | 564 | 56.15162754927949 | 10.044232458 | 3.46801375 | 6.576218708 | 65,47 | 10 | +| 1024 | 448582 | 55551.67442589024 | 326 MB | 40 MB | 5565 | 689.1606622202389 | 1127 | 138.58965301839567 | 8.131920208 | 8.075040125 | 0.056880083000001136 | 0,70 | 10 | + +### 71 MESSAGES (111.73s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 26140 | 91748.08364349176 | 19 MB | 67 MB | 329 | 1154.7482600883238 | 71 | 7.0997052436673105 | 10.000415167 | 0.284910583 | 9.715504584 | 97,15 | 10 | +| 2 | 26140 | 61259.79814029408 | 19 MB | 44 MB | 329 | 771.0204127068382 | 71 | 7.086361323258707 | 10.019246375 | 0.42670725 | 9.592539125 | 95,74 | 10 | +| 4 | 26140 | 28801.399292295628 | 19 MB | 21 MB | 329 | 362.49657104687304 | 71 | 7.099345381431265 | 10.000922083 | 0.907594792 | 9.093327291000001 | 90,92 | 10 | +| 8 | 26140 | 51415.166866842264 | 19 MB | 37 MB | 329 | 647.1151453401341 | 71 | 7.096252912352794 | 10.005280375 | 0.508410292 | 9.496870083 | 94,92 | 10 | +| 16 | 26140 | 43575.091184295634 | 19 MB | 32 MB | 329 | 548.4393649438891 | 71 | 7.083790399028503 | 10.022882666 | 0.599884 | 9.422998666 | 94,01 | 10 | +| 32 | 26140 | 32072.511182303715 | 19 MB | 23 MB | 329 | 403.6670305653375 | 71 | 7.099737930923625 | 10.000369125 | 0.815028167 | 9.185340958000001 | 91,85 | 10 | +| 64 | 26140 | 27670.26601536393 | 19 MB | 20 MB | 329 | 348.260042810051 | 71 | 7.099000579271748 | 10.001407833 | 0.944696375 | 9.056711458 | 90,55 | 10 | +| 128 | 26140 | 21984.360450447064 | 19 MB | 16 MB | 329 | 276.6968090358487 | 71 | 7.099141299227072 | 10.001209584 | 1.1890270840000001 | 8.812182499999999 | 88,11 | 10 | +| 256 | 26140 | 23343.357775469092 | 19 MB | 17 MB | 329 | 293.8012512673807 | 71 | 7.095628413136305 | 10.006160958 | 1.119804625 | 8.886356333 | 88,81 | 10 | +| 512 | 26140 | 20567.43918692126 | 19 MB | 15 MB | 329 | 258.86333177112067 | 71 | 7.057589123978741 | 10.060092583 | 1.270940916 | 8.789151666999999 | 87,37 | 10 | +| 1024 | 26140 | 47420.73058881012 | 19 MB | 34 MB | 329 | 596.8408708385052 | 71 | 7.075070928859076 | 10.035235083 | 0.551235708 | 9.483999375 | 94,51 | 10 | + +### 1 MESSAGE : 1 OBJECT : 1 EVENT (112.92s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 2 | 395.3024627145776 | 1.2 kB | 234 kB | 2 | 395.3024627145776 | 2 | 0.2000003366605667 | 9.999983167 | 0.005059417 | 9.99492375 | 99,95 | 10 | +| 2 | 3 | 572.5828985520524 | 2.6 kB | 495 kB | 3 | 572.5828985520524 | 3 | 0.29999898252345086 | 10.000033916 | 0.005239416 | 9.9947945 | 99,95 | 10 | +| 4 | 5 | 436.0227712020215 | 3.8 kB | 332 kB | 5 | 436.0227712020215 | 5 | 0.4999889960921773 | 10.000220083 | 0.011467291 | 9.988752792 | 99,89 | 10 | +| 8 | 9 | 609.3467288507261 | 7.4 kB | 498 kB | 9 | 609.3467288507261 | 9 | 0.8996953406652672 | 10.00338625 | 0.014769916 | 9.988616334 | 99,85 | 10 | +| 16 | 18 | 96.57043097741648 | 16 kB | 84 kB | 18 | 96.57043097741648 | 18 | 1.7894709097457604 | 10.058839125 | 0.186392458 | 9.872446667 | 98,15 | 10 | +| 32 | 36 | 94.81145422867225 | 28 kB | 75 kB | 36 | 94.81145422867225 | 36 | 3.5984909278482706 | 10.004193625 | 0.379700958 | 9.624492666999998 | 96,20 | 10 | +| 64 | 71 | 32.996492205653425 | 54 kB | 25 kB | 71 | 32.996492205653425 | 71 | 6.894487619016878 | 10.298082167 | 2.151743875 | 8.146338292 | 79,11 | 10 | +| 128 | 141 | 90.3410929260777 | 108 kB | 70 kB | 141 | 90.3410929260777 | 141 | 13.614254133733517 | 10.356792125 | 1.5607515410000001 | 8.796040584 | 84,93 | 10 | +| 256 | 282 | 84.55657598731302 | 216 kB | 65 kB | 282 | 84.55657598731302 | 282 | 26.45563037584542 | 10.659356666 | 3.335045166 | 7.3243115 | 68,71 | 10 | +| 512 | 564 | 117.41881487011658 | 426 kB | 89 kB | 564 | 117.41881487011658 | 564 | 54.02426795979435 | 10.43975275 | 4.803318792 | 5.6364339580000005 | 53,99 | 10 | +| 1024 | 1127 | 173.7517371943818 | 843 kB | 130 kB | 1127 | 173.7517371943818 | 1127 | 102.50634920938927 | 10.994440917 | 6.486266084 | 4.508174833 | 41,00 | 10 | + +## CAPPED DELETION + +### DYNAMIC NUMBER OF MESSAGES (129.09s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 534 | 34795.169179455625 | 386 kB | 25 MB | 8 | 521.2759427633802 | 2 | 0.19999374353572283 | 10.000312833 | 0.015346958 | 9.984965875 | 99,85 | 10 | +| 2 | 1096 | 69790.21164263744 | 806 kB | 51 MB | 16 | 1018.8352064618604 | 3 | 0.29998010882894977 | 10.000663083 | 0.015704208 | 9.984958874999998 | 99,84 | 10 | +| 4 | 1569 | 79595.34517898757 | 1.2 MB | 59 MB | 21 | 1065.3296677875965 | 5 | 0.499988644007923 | 10.000227125 | 0.019712208 | 9.980514917 | 99,80 | 10 | +| 8 | 3394 | 30285.38667630374 | 2.5 MB | 22 MB | 41 | 365.85175419223725 | 9 | 0.8999114412449165 | 10.000984083 | 0.11206725 | 9.888916833 | 98,88 | 10 | +| 16 | 5956 | 20495.25031176799 | 4.3 MB | 15 MB | 72 | 247.75990974602007 | 18 | 1.7947778397347496 | 10.029096416 | 0.290603916 | 9.7384925 | 97,10 | 10 | +| 32 | 12800 | 17914.975587444715 | 9.3 MB | 13 MB | 161 | 225.33680231082806 | 36 | 3.5967396156040063 | 10.009064833 | 0.714486042 | 9.294578791 | 92,86 | 10 | +| 64 | 27343 | 23536.6331404344 | 20 MB | 17 MB | 344 | 296.11241635187923 | 71 | 7.096650351937619 | 10.004720041 | 1.161720958 | 8.842999083 | 88,39 | 10 | +| 128 | 55831 | 30150.36774297852 | 41 MB | 22 MB | 681 | 367.7598544351413 | 141 | 13.982909330758774 | 10.083738417 | 1.8517518750000002 | 8.231986542 | 81,64 | 10 | +| 256 | 109313 | 42706.806371763625 | 79 MB | 31 MB | 1357 | 530.1577694005584 | 282 | 28.126771012235494 | 10.026035334 | 2.559615417 | 7.466419917 | 74,47 | 10 | +| 512 | 219611 | 27256.798510843055 | 160 MB | 20 MB | 2712 | 336.5971538830312 | 564 | 56.21467240679431 | 10.032967833 | 8.057109125 | 1.9758587080000005 | 19,69 | 10 | +| 1024 | 448582 | 40054.32150041598 | 326 MB | 29 MB | 5565 | 496.9042430365349 | 1127 | 99.99467493309047 | 11.270600167 | 11.199340875 | 0.07125929199999881 | 0,63 | 10 | + +### 71 MESSAGES : 329 OBJECTS : 26140 EVENTS (105.90s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 26140 | 109625.84677526618 | 19 MB | 80 MB | 329 | 1379.75912735511 | 71 | 7.099808630338217 | 10.000269542 | 0.238447417 | 9.761822125 | 97,62 | 10 | +| 2 | 26140 | 72137.42531654994 | 19 MB | 52 MB | 329 | 907.927043961168 | 71 | 7.099324055608355 | 10.000952125 | 0.362363917 | 9.638588208 | 96,38 | 10 | +| 4 | 26140 | 47008.762487278 | 19 MB | 34 MB | 329 | 591.6558094228945 | 71 | 7.099316424569769 | 10.000962875 | 0.556066542 | 9.444896333 | 94,44 | 10 | +| 8 | 26140 | 48051.90120040138 | 19 MB | 35 MB | 329 | 604.784831481716 | 71 | 7.099304356914327 | 10.000979875 | 0.543995125 | 9.45698475 | 94,56 | 10 | +| 16 | 26140 | 79621.05373773864 | 19 MB | 58 MB | 329 | 1002.1165523992354 | 71 | 7.099496231816533 | 10.000709583 | 0.328305125 | 9.672404458 | 96,72 | 10 | +| 32 | 26140 | 43767.73080754615 | 19 MB | 32 MB | 329 | 550.8639416864071 | 71 | 7.091037578748608 | 10.012639083 | 0.597243666 | 9.415395417 | 94,04 | 10 | +| 64 | 26140 | 28775.26650170689 | 19 MB | 21 MB | 329 | 362.1676617850638 | 71 | 7.096071378585638 | 10.005536333 | 0.908419041 | 9.097117292 | 90,92 | 10 | +| 128 | 26140 | 26747.535715972164 | 19 MB | 19 MB | 329 | 336.64649007478357 | 71 | 7.08464309100885 | 10.021676334 | 0.977286292 | 9.044390042 | 90,25 | 10 | +| 256 | 26140 | 48014.57333039672 | 19 MB | 35 MB | 329 | 604.315020110961 | 71 | 7.068498969274716 | 10.044565375 | 0.544418042 | 9.500147333 | 94,58 | 10 | +| 512 | 26140 | 26388.236329374038 | 19 MB | 19 MB | 329 | 332.1243210544781 | 71 | 68.50246301522354 | 1.036459083 | 0.990592917 | 0.04586616600000004 | 4,43 | 10 | +| 1024 | 26140 | 24969.348500847864 | 19 MB | 18 MB | 329 | 314.2660924552007 | 71 | 7.077267345233379 | 10.032120667 | 1.046883542 | 8.985237125 | 89,56 | 10 | + +### 1 MESSAGE : 1 OBJECT : 1 EVENT (123.52s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 2 | 395.83055839618953 | 1.2 kB | 234 kB | 2 | 395.83055839618953 | 2 | 0.2000018583572673 | 9.999907083 | 0.005052667 | 9.994854416 | 99,95 | 10 | +| 2 | 3 | 536.1371639023137 | 2.6 kB | 464 kB | 3 | 536.1371639023137 | 3 | 0.29999378511875185 | 10.000207167 | 0.005595583 | 9.994611584 | 99,94 | 10 | +| 4 | 5 | 491.4427530623027 | 3.8 kB | 375 kB | 5 | 491.4427530623027 | 5 | 0.5000005354505734 | 9.999989291 | 0.010174125 | 9.989815166 | 99,90 | 10 | +| 8 | 9 | 127.73076126724754 | 7.4 kB | 104 kB | 9 | 127.73076126724754 | 9 | 0.8959297054285792 | 10.045430959 | 0.070460709 | 9.97497025 | 99,30 | 10 | +| 16 | 18 | 90.40842003751949 | 16 kB | 79 kB | 18 | 90.40842003751949 | 18 | 1.795549797160854 | 10.024784625 | 0.1990965 | 9.825688125000001 | 98,01 | 10 | +| 32 | 36 | 41.16429983834505 | 28 kB | 32 kB | 36 | 41.16429983834505 | 36 | 3.5983315587274998 | 10.004636708 | 0.874544208 | 9.1300925 | 91,26 | 10 | +| 64 | 71 | 52.876211464847614 | 54 kB | 40 kB | 71 | 52.876211464847614 | 71 | 6.92309304923333 | 10.255531667 | 1.342758833 | 8.912772834 | 86,91 | 10 | +| 128 | 141 | 55.888284984832495 | 108 kB | 43 kB | 141 | 55.888284984832495 | 141 | 13.692525170393644 | 10.29758925 | 2.522890084 | 7.7746991659999996 | 75,50 | 10 | +| 256 | 282 | 52.68909681601697 | 216 kB | 40 kB | 282 | 52.68909681601697 | 282 | 26.954578632512252 | 10.462044458 | 5.352150958 | 5.109893499999999 | 48,84 | 10 | +| 512 | 564 | 91.11824318695749 | 426 kB | 69 kB | 564 | 91.11824318695749 | 564 | 46.264973271827536 | 12.190647916 | 6.189759375 | 6.000888540999999 | 49,23 | 10 | +| 1024 | 1127 | 78.80667488058174 | 843 kB | 59 kB | 1127 | 78.80667488058174 | 1127 | 55.92493299201936 | 20.152013417 | 14.300819083 | 5.851194333999999 | 29,04 | 10 | + + +## CURRENT IMPLEMENTATION (acker with mutex) + +### DYNAMIC NUMBER OF MESSAGES (420.15s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 534 | 53.377307971948426 | 386 kB | 39 kB | 8 | 0.799660044523572 | 2 | 0.09999887376268425 | 20.00022525 | 10.00425125 | 9.995974 | 49,98 | 10 | +| 2 | 1096 | 109.54472373240463 | 806 kB | 80 kB | 16 | 1.5991930471883886 | 3 | 0.1499952010910351 | 20.000639875 | 10.005046 | 9.995593875 | 49,98 | 10 | +| 4 | 1569 | 156.58283429763634 | 1.2 MB | 115 kB | 21 | 2.0957549523584214 | 5 | 0.2499882995101315 | 20.000936083 | 10.020255458 | 9.980680625 | 49,90 | 10 | +| 8 | 3394 | 327.7503948980639 | 2.5 MB | 238 kB | 41 | 3.9592711228110256 | 9 | 0.4499758991108533 | 20.001071208 | 10.355441375 | 9.645629833 | 48,23 | 10 | +| 16 | 5956 | 6995.8233255091345 | 4.3 MB | 5.1 MB | 72 | 84.5700603486665 | 18 | 1.68781488955836 | 10.664676625 | 0.851365125 | 9.813311500000001 | 92,02 | 10 | +| 32 | 12800 | 9577.580684203449 | 9.3 MB | 7.0 MB | 161 | 120.4680070434965 | 36 | 3.5207708469266525 | 10.225033541 | 1.336454416 | 8.888579125 | 86,93 | 10 | +| 64 | 27343 | 5969.565759122024 | 20 MB | 4.3 MB | 344 | 75.10260838744748 | 71 | 7.09735159344403 | 10.003731542 | 4.580400167 | 5.423331375000001 | 54,21 | 10 | +| 128 | 55831 | 6528.538457378312 | 41 MB | 4.7 MB | 681 | 79.63200891036576 | 141 | 13.948629932462296 | 10.108519667 | 8.5518375 | 1.556682167 | 15,40 | 10 | +| 256 | 109313 | 2418.262154430809 | 79 MB | 1.8 MB | 1357 | 30.0200501638653 | 282 | 5.63602730525321 | 50.03524375 | 45.203122333 | 4.832121416999996 | 9,66 | 10 | +| 512 | 219611 | 3242.754954766131 | 160 MB | 2.4 MB | 2712 | 40.04513178905313 | 564 | 7.980402861454806 | 70.673123875 | 67.723587833 | 2.9495360420000054 | 4,17 | 10 | +| 1024 | 448582 | 2735.7629115360114 | 326 MB | 2.0 MB | 5565 | 33.939214241092834 | 1127 | 6.612725338373049 | 170.428974792 | 163.969618167 | 6.459356624999998 | 3,79 | 10 | + +### 71 MESSAGES : 329 OBJECTS : 26140 EVENTS (113.31s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|---------------------:|------------------:|-----:| +| 1 | 26140 | 9877.562063511137 | 19 MB | 7.2 MB | 329 | 124.31973675957016 | 71 | 7.098897157221147 | 10.001553541 | 2.646402 | 7.355151541 | 73,54 | 10 | +| 2 | 26140 | 12573.780468960163 | 19 MB | 9.1 MB | 329 | 158.25454377535937 | 71 | 18.855849203387365 | 3.765409833 | 2.07892925 | 1.6864805830000003 | 44,79 | 10 | +| 4 | 26140 | 7938.4983023008135 | 19 MB | 5.8 MB | 329 | 99.91453486828492 | 71 | 6.85461611092427 | 10.357983416 | 3.2928142080000002 | 7.0651692079999995 | 68,21 | 10 | +| 8 | 26140 | 5360.695072382419 | 19 MB | 3.9 MB | 329 | 67.47011013059739 | 71 | 7.098512566319234 | 10.002095416 | 4.876233333 | 5.1258620829999995 | 51,25 | 10 | +| 16 | 26140 | 2351.529242916884 | 19 MB | 1.7 MB | 329 | 29.5965233710656 | 71 | 3.5123938673326833 | 20.214133916 | 11.1161705 | 9.097963416 | 45,01 | 10 | +| 32 | 26140 | 9500.726928457989 | 19 MB | 6.9 MB | 329 | 119.57686149436412 | 71 | 6.861171234604384 | 10.348087458 | 2.751368416 | 7.596719042 | 73,41 | 10 | +| 64 | 26140 | 6361.638976257994 | 19 MB | 4.6 MB | 329 | 80.06806515642234 | 71 | 6.87722113211347 | 10.323937334 | 4.109004 | 6.214933334 | 60,20 | 10 | +| 128 | 26140 | 6578.852782344765 | 19 MB | 4.8 MB | 329 | 82.80193440671108 | 71 | 17.592678026519042 | 4.035769875 | 3.973337125 | 0.062432749999999704 | 1,55 | 10 | +| 256 | 26140 | 7012.815402100417 | 19 MB | 5.1 MB | 329 | 88.26382047785145 | 71 | 7.000925642568938 | 10.141516083 | 3.727461583 | 6.414054500000001 | 63,25 | 10 | +| 512 | 26140 | 6679.284014737735 | 19 MB | 4.8 MB | 329 | 84.0659694280304 | 71 | 6.996431130334545 | 10.148031 | 3.9135931250000002 | 6.234437874999999 | 61,43 | 10 | +| 1024 | 26140 | 3193.499741765352 | 19 MB | 2.3 MB | 329 | 40.19362720125481 | 71 | 6.531022966383471 | 10.871191292 | 8.185377208 | 2.6858140840000004 | 24,71 | 10 | + +### 1 MESSAGE : 1 OBJECT : 1 EVENT (900.27s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|--------------------:|---------------:|-----------------:|-----------------:|--------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 2 | 0.19998794072717416 | 1.2 kB | 118 B | 2 | 0.19998794072717416 | 2 | 0.09999999729500009 | 20.000000541 | 10.000603 | 9.999397540999999 | 50,00 | 10 | +| 2 | 3 | 0.2999579983812767 | 2.6 kB | 259 B | 3 | 0.2999579983812767 | 3 | 0.14999602135553355 | 20.0005305 | 10.00140025 | 9.99913025 | 49,99 | 10 | +| 4 | 5 | 0.49973281994669333 | 3.8 kB | 380 B | 5 | 0.49973281994669333 | 5 | 0.2500007073020011 | 19.999943416 | 10.005346458 | 9.994596958 | 49,97 | 10 | +| 8 | 9 | 0.8990750353797166 | 7.4 kB | 734 B | 9 | 0.8990750353797166 | 9 | 0.4499962050245043 | 20.000168667 | 10.010287958 | 9.989880709000001 | 49,95 | 10 | +| 16 | 18 | 1.7925517681481675 | 16 kB | 1.6 kB | 18 | 1.7925517681481675 | 18 | 0.8999971687439067 | 20.000062917 | 10.041551 | 9.958511917000001 | 49,79 | 10 | +| 32 | 36 | 3.521597946166357 | 28 kB | 2.8 kB | 36 | 3.521597946166357 | 36 | 1.7986028827815255 | 20.015535583 | 10.222632041 | 9.792903541999998 | 48,93 | 10 | +| 64 | 71 | 6.530635221320813 | 54 kB | 5.0 kB | 71 | 6.530635221320813 | 71 | 3.5422948887154386 | 20.0435035 | 10.87183675 | 9.17166675 | 45,76 | 10 | +| 128 | 141 | 11.55643631263828 | 108 kB | 8.9 kB | 141 | 11.55643631263828 | 141 | 6.994094252240433 | 20.159865583 | 12.200993125 | 7.958872457999998 | 39,48 | 10 | +| 256 | 282 | 14.330504553562285 | 216 kB | 11 kB | 282 | 14.330504553562285 | 282 | 5.574490878106893 | 50.587579416 | 19.67830225 | 30.909277165999995 | 61,10 | 10 | +| 512 | 564 | 14.132980354791805 | 426 kB | 11 kB | 564 | 14.132980354791805 | 564 | 3.96003592248513 | 142.422950458 | 39.906657042 | 102.516293416 | 71,98 | 10 | +| 1024 | 1127 | 10.888147574358264 | 843 kB | 8.1 kB | 1127 | 10.888147574358264 | 1127 | 2.060683835679075 | 546.905828292 | 103.50704675 | 443.39878154200005 | 81,07 | 10 | + + +## CURRENT IMPLEMENTATION (acker with atomic) + +### DYNAMIC NUMBER OF MESSAGES (164.32s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 534 | 53.36880126588998 | 386 kB | 39 kB | 8 | 0.7995326032343067 | 2 | 0.0999987339710285 | 20.000253209 | 10.005845875 | 9.994407334 | 49,97 | 10 | +| 2 | 1096 | 109.5183622748013 | 806 kB | 80 kB | 16 | 1.5988082083912598 | 3 | 0.14999513391036398 | 20.000648833 | 10.00745425 | 9.993194583 | 49,96 | 10 | +| 4 | 1569 | 156.72384371614336 | 1.2 MB | 115 kB | 21 | 2.097642267711288 | 5 | 0.24983862454793027 | 20.012918375 | 10.011239916 | 10.001678459000003 | 49,98 | 10 | +| 8 | 3394 | 338.5693441187711 | 2.5 MB | 246 kB | 41 | 4.089965559478378 | 9 | 0.449976796204042 | 20.001031333 | 10.024534291 | 9.976497042 | 49,88 | 10 | +| 16 | 5956 | 10475.731016772343 | 4.3 MB | 7.6 MB | 72 | 126.63744681121703 | 18 | 1.799447509692922 | 10.003070333 | 0.568552208 | 9.434518125 | 94,32 | 10 | +| 32 | 12800 | 17234.142316989062 | 9.3 MB | 13 MB | 161 | 216.77319633087802 | 36 | 3.5888302145609554 | 10.03112375 | 0.74271175 | 9.288412000000001 | 92,60 | 10 | +| 64 | 27343 | 19069.442066866573 | 20 MB | 14 MB | 344 | 239.9110584428227 | 71 | 7.096113340218469 | 10.005477167 | 1.433864709 | 8.571612458 | 85,67 | 10 | +| 128 | 55831 | 35664.64780353814 | 41 MB | 26 MB | 681 | 435.0204215258453 | 141 | 14.081119155366537 | 10.013408625 | 1.56544375 | 8.447964875 | 84,37 | 10 | +| 256 | 109313 | 25977.965417155614 | 79 MB | 19 MB | 1357 | 322.4877102547745 | 282 | 28.14029976872242 | 10.021215208 | 4.207912292 | 5.813302916 | 58,01 | 10 | +| 512 | 219611 | 29818.010625950345 | 160 MB | 22 MB | 2712 | 368.22583940502676 | 564 | 56.35987951526856 | 10.007118625 | 7.365045333 | 2.642073292 | 26,40 | 10 | +| 1024 | 448582 | 37269.46254020562 | 326 MB | 27 MB | 5565 | 462.35595506784546 | 1127 | 92.49486483283363 | 12.184460208 | 12.036181083 | 0.14827912500000018 | 1,22 | 10 | + +### 71 MESSAGES : 329 OBJECTS : 26140 EVENTS (113.02s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|-------------------:|---------------:|-----------------:|-----------------:|-------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 26140 | 66013.45586060404 | 19 MB | 48 MB | 329 | 830.8503052080614 | 71 | 7.097534700421701 | 10.003473459 | 0.395979875 | 9.607493584 | 96,04 | 10 | +| 2 | 26140 | 81726.08569439051 | 19 MB | 59 MB | 329 | 1028.610642442788 | 71 | 7.099410541622258 | 10.000830292 | 0.319848917 | 9.680981375 | 96,80 | 10 | +| 4 | 26140 | 67420.14255630635 | 19 MB | 49 MB | 329 | 848.5549694347663 | 71 | 7.099278210704895 | 10.001016708 | 0.387717958 | 9.61329875 | 96,12 | 10 | +| 8 | 26140 | 49733.531297140966 | 19 MB | 36 MB | 329 | 625.9499539693717 | 71 | 7.099305332973168 | 10.0009785 | 0.525601125 | 9.475377375 | 94,74 | 10 | +| 16 | 26140 | 37976.62825897801 | 19 MB | 28 MB | 329 | 477.97669078820826 | 71 | 7.0829980599942655 | 10.024003875 | 0.688318084 | 9.335685791 | 93,13 | 10 | +| 32 | 26140 | 30417.46339417117 | 19 MB | 22 MB | 329 | 382.8364750069746 | 71 | 7.096205718410493 | 10.005346916 | 0.85937475 | 9.145972166 | 91,41 | 10 | +| 64 | 26140 | 23190.059236007757 | 19 MB | 17 MB | 329 | 291.8718243552621 | 71 | 7.091070009187293 | 10.012593291 | 1.127207125 | 8.885386166 | 88,74 | 10 | +| 128 | 26140 | 22119.799277562448 | 19 MB | 16 MB | 329 | 278.40145226924426 | 71 | 7.080700459054835 | 10.027256542 | 1.181746709 | 8.845509833 | 88,21 | 10 | +| 256 | 26140 | 28731.619614081766 | 19 MB | 21 MB | 329 | 361.61831878473225 | 71 | 7.087304628867566 | 10.017912834 | 0.909799042 | 9.108113792000001 | 90,92 | 10 | +| 512 | 26140 | 27872.444273062203 | 19 MB | 20 MB | 329 | 350.80467352094354 | 71 | 7.081151420182479 | 10.026617959 | 0.937843834 | 9.088774124999999 | 90,65 | 10 | +| 1024 | 26140 | 19964.594159382417 | 19 MB | 14 MB | 329 | 251.27587905267083 | 71 | 7.0626779549516066 | 10.052844042 | 1.309317875 | 8.743526167 | 86,98 | 10 | + +### 1 MESSAGE : 1 OBJECT : 1 EVENT (220.35s) +| MAX MSGS INFLIGHT | EVENTS TOTAL | EVENTS PER SEC | S3 BYTES TOTAL | S3 BYTES PER SEC | S3 OBJECTS TOTAL | S3 OBJECTS PER SEC | SQS MESSAGES TOTAL | SQS MESSAGES PER SEC | FULL TIME (SEC) | PROCESSING TIME (SEC) | FLUSHING TIME (SEC) | FLUSHING TIME (%) | CPUS | +|------------------:|-------------:|--------------------:|---------------:|-----------------:|-----------------:|--------------------:|-------------------:|---------------------:|----------------:|----------------------:|--------------------:|------------------:|-----:| +| 1 | 2 | 0.19990798735112197 | 1.2 kB | 118 B | 2 | 0.19990798735112197 | 2 | 0.10000075792574448 | 19.999848416 | 10.00460275 | 9.995245665999999 | 49,98 | 10 | +| 2 | 3 | 0.2999677996865716 | 2.6 kB | 259 B | 3 | 0.2999677996865716 | 3 | 0.14999982531020342 | 20.000023292 | 10.001073459 | 9.998949833000001 | 49,99 | 10 | +| 4 | 5 | 0.4997554779905357 | 3.8 kB | 380 B | 5 | 0.4997554779905357 | 5 | 0.24999950312598754 | 20.00003975 | 10.004892833 | 9.995146917 | 49,98 | 10 | +| 8 | 9 | 0.894460453636782 | 7.4 kB | 730 B | 9 | 0.894460453636782 | 9 | 0.4499972034698792 | 20.000124291 | 10.061931708 | 9.938192583 | 49,69 | 10 | +| 16 | 18 | 1.7925670684868178 | 16 kB | 1.6 kB | 18 | 1.7925670684868178 | 18 | 0.9000027825386028 | 19.999938166 | 10.041465291 | 9.958472875 | 49,79 | 10 | +| 32 | 36 | 3.5774289276666726 | 28 kB | 2.8 kB | 36 | 3.5774289276666726 | 36 | 1.7957864302908053 | 20.046927292 | 10.063093 | 9.983834292 | 49,80 | 10 | +| 64 | 71 | 6.927311449043817 | 54 kB | 5.3 kB | 71 | 6.927311449043817 | 71 | 3.5460116087271363 | 20.022495083 | 10.249286541 | 9.773208541999999 | 48,81 | 10 | +| 128 | 141 | 13.838002220126679 | 108 kB | 11 kB | 141 | 13.838002220126679 | 141 | 7.048862151877301 | 20.003228459 | 10.189332084 | 9.813896374999999 | 49,06 | 10 | +| 256 | 282 | 27.005488400390938 | 216 kB | 21 kB | 282 | 27.005488400390938 | 282 | 14.057648405968855 | 20.060254166 | 10.442321791 | 9.617932375 | 47,95 | 10 | +| 512 | 564 | 50.91963451508912 | 426 kB | 38 kB | 564 | 50.91963451508912 | 564 | 28.157323059061508 | 20.030313209 | 11.076277459 | 8.95403575 | 44,70 | 10 | +| 1024 | 1127 | 79.02205520334414 | 843 kB | 59 kB | 1127 | 79.02205520334414 | 1127 | 56.14676718350413 | 20.072393417 | 14.261841167 | 5.810552250000001 | 28,95 | 10 | From 6cb6221572829690e0ccbcae61de76ac245a3275 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 11 Mar 2024 16:39:37 +0900 Subject: [PATCH 26/43] Handle deletion WaitGroup when no events to be acked --- x-pack/filebeat/input/awss3/interfaces.go | 2 +- x-pack/filebeat/input/awss3/s3.go | 4 ++-- x-pack/filebeat/input/awss3/s3_objects.go | 12 ++++++------ x-pack/filebeat/input/awss3/sqs.go | 20 +++++++++++--------- x-pack/filebeat/input/awss3/sqs_acker.go | 4 ++-- x-pack/filebeat/input/awss3/sqs_s3_event.go | 16 ++++++++-------- 6 files changed, 30 insertions(+), 28 deletions(-) diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index 19dcf04a019..c93ad12a69c 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -64,7 +64,7 @@ type sqsProcessor interface { // given message and is responsible for updating the message's visibility // timeout while it is being processed and for deleting it when processing // completes successfully. - ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) error + ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) (uint64, error) DeleteSQS(msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error } diff --git a/x-pack/filebeat/input/awss3/s3.go b/x-pack/filebeat/input/awss3/s3.go index d10110e3ead..b3dedb562dd 100644 --- a/x-pack/filebeat/input/awss3/s3.go +++ b/x-pack/filebeat/input/awss3/s3.go @@ -137,9 +137,9 @@ func (p *s3Poller) ProcessObject(s3ObjectPayloadChan <-chan *s3ObjectPayload) er for s3ObjectPayload := range s3ObjectPayloadChan { // Process S3 object (download, parse, create events). - s3EventsCreated, err := s3ObjectPayload.s3ObjectHandler.ProcessS3Object() + eventsPublished, err := s3ObjectPayload.s3ObjectHandler.ProcessS3Object() - s3ObjectPayload.s3ObjectHandler.SyncEventsToBeAcked(s3EventsCreated) + s3ObjectPayload.s3ObjectHandler.SyncEventsToBeAcked(eventsPublished) // Wait for all events to be ACKed before proceeding. s3ObjectPayload.s3ObjectHandler.Wait() diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index 3bdd32d455b..b152b4a460a 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -109,11 +109,11 @@ type s3ObjectProcessor struct { s3Metadata map[string]interface{} // S3 object metadata. - s3EventsCreatedTotal uint64 + eventsPublishedTotal uint64 } -func (p *s3ObjectProcessor) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { - p.acker.MarkS3FromListingProcessedWithData(s3EventsCreatedTotal) +func (p *s3ObjectProcessor) SyncEventsToBeAcked(eventsPublished uint64) { + p.acker.MarkS3FromListingProcessedWithData(eventsPublished) } func (p *s3ObjectProcessor) Wait() { p.acker.WaitForS3() @@ -168,7 +168,7 @@ func (p *s3ObjectProcessor) ProcessS3Object() (uint64, error) { data, decoderErr := decoder.decode() if decoderErr != nil { if errors.Is(err, io.EOF) { - return p.s3EventsCreatedTotal, nil + return p.eventsPublishedTotal, nil } break } @@ -192,7 +192,7 @@ func (p *s3ObjectProcessor) ProcessS3Object() (uint64, error) { time.Since(start).Nanoseconds(), err) } - return p.s3EventsCreatedTotal, nil + return p.eventsPublishedTotal, nil } // download requests the S3 object from AWS and returns the object's @@ -391,7 +391,7 @@ func (p *s3ObjectProcessor) readFile(r io.Reader) error { func (p *s3ObjectProcessor) publish(ack *EventACKTracker, event *beat.Event) { event.Private = ack - p.s3EventsCreatedTotal++ + p.eventsPublishedTotal++ p.metrics.s3EventsCreatedTotal.Inc() p.publisher.Publish(*event) } diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 28749fdabb2..4fe858c3992 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -11,7 +11,6 @@ import ( "sync" "time" - "github.com/elastic/beats/v7/libbeat/common/atomic" "github.com/elastic/go-concert/timed" "github.com/aws/aws-sdk-go-v2/service/sqs/types" @@ -63,7 +62,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { workersChan := make(chan processingData, r.maxMessagesInflight) deletionWg := new(sync.WaitGroup) - deletionWaiter := atomic.NewBool(true) + deletionWaiter := make(chan struct{}) var clientsMutex sync.Mutex clients := make(map[uint64]beat.Client, r.maxMessagesInflight) @@ -107,9 +106,6 @@ func (r *sqsReader) Receive(ctx context.Context) error { return } - deletionWg.Add(1) - deletionWaiter.Swap(false) - msg := incomingData.msg start := incomingData.start @@ -120,7 +116,15 @@ func (r *sqsReader) Receive(ctx context.Context) error { acker := NewEventACKTracker(ctx, deletionWg) - err = r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) + deletionWg.Add(1) + deletionWaiter <- struct{}{} + + eventsCreatedTotal, err := r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) + // No ACK will be invoked by the client event listener, deletionWg.Done() has to be called here + if eventsCreatedTotal == 0 { + deletionWg.Done() + } + if err != nil { r.log.Warnw("Failed processing SQS message.", "worker_id", id, @@ -171,9 +175,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { // Wait for all deletion to happen. if r.metrics.sqsMessagesReceivedTotal.Get() > 0 { - for deletionWaiter.Load() { - _ = timed.Wait(ctx, 500*time.Millisecond) - } + <-deletionWaiter } deletionWg.Wait() diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 17cd84cfe34..ae8330be49d 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -71,13 +71,13 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA // MarkS3FromListingProcessedWithData has to be used when the acker is used when the input is in s3 direct listing mode, instead of MarkSQSProcessedWithData // Specifically we both Swap the value of EventACKTracker.ackMutexLockedOnInit initialised in NewEventACKTracker -func (a *EventACKTracker) MarkS3FromListingProcessedWithData(s3EventsCreatedTotal uint64) { +func (a *EventACKTracker) MarkS3FromListingProcessedWithData(eventsPublished uint64) { // We want to execute the logic of this call only once, when the ack mutex was locked on init if !a.ackMutexLockedOnInit.Swap(false) { return } - a.EventsToBeAcked.Add(s3EventsCreatedTotal) + a.EventsToBeAcked.Add(eventsPublished) a.ackMutex.Unlock() } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index b87fbc79704..04f22b13319 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -163,7 +163,7 @@ func (p *sqsS3EventProcessor) DeleteSQS(msg *types.Message, receiveCount int, pr return fmt.Errorf("failed deleting SQS message (it will return to queue after visibility timeout): %w", processingErr) } -func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) error { +func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) (uint64, error) { keepaliveCtx, keepaliveCancel := context.WithCancel(ctx) // Start SQS keepalive worker. @@ -188,11 +188,11 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message } } - s3EventsCreatedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) + eventsPublishedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) p.metrics.sqsMessagesProcessedTotal.Inc() - acker.MarkSQSProcessedWithData(msg, s3EventsCreatedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log) + acker.MarkSQSProcessedWithData(msg, eventsPublishedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log) - return processingErr + return eventsPublishedTotal, processingErr } func (p *sqsS3EventProcessor) keepalive(ctx context.Context, log *logp.Logger, wg *sync.WaitGroup, msg *types.Message) { @@ -308,7 +308,7 @@ func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Log var errs []error var handles []s3ObjectHandler - var s3EventsCreatedTotal uint64 + var eventsPublishedTotal uint64 for i, event := range s3Events { s3Processor := p.s3ObjectHandler.Create(ctx, log, client, acker, event) if s3Processor == nil { @@ -316,19 +316,19 @@ func (p *sqsS3EventProcessor) processS3Events(ctx context.Context, log *logp.Log } // Process S3 object (download, parse, create events). - if s3EventsCreatedTotalByProcessor, err := s3Processor.ProcessS3Object(); err != nil { + if eventsPublished, err := s3Processor.ProcessS3Object(); err != nil { errs = append(errs, fmt.Errorf( "failed processing S3 event for object key %q in bucket %q (object record %d of %d in SQS notification): %w", event.S3.Object.Key, event.S3.Bucket.Name, i+1, len(s3Events), err)) } else { - s3EventsCreatedTotal += s3EventsCreatedTotalByProcessor + eventsPublishedTotal += eventsPublished handles = append(handles, s3Processor) } } // Make sure all s3 events were processed successfully if len(handles) == len(s3Events) { - return s3EventsCreatedTotal, handles, multierr.Combine(errs...) + return eventsPublishedTotal, handles, multierr.Combine(errs...) } return 0, nil, multierr.Combine(errs...) From 0b75813a9bdc85c689aee6ca1b3057c47bc0ffe1 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 11 Mar 2024 16:40:02 +0900 Subject: [PATCH 27/43] fix tests --- .../input/awss3/input_benchmark_test.go | 9 ++++--- .../input/awss3/mock_interfaces_test.go | 14 ++++------- .../input/awss3/mock_publisher_test.go | 7 +----- .../filebeat/input/awss3/sqs_s3_event_test.go | 24 +++++++++++++------ x-pack/filebeat/input/awss3/sqs_test.go | 4 ++-- 5 files changed, 31 insertions(+), 27 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index bcf92289dc8..f4f3c1ed1bd 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -279,6 +279,7 @@ type fakePipeline struct { func (fp *fakePipeline) ackEvents() { for _, client := range fp.clients { + addedEventsForAllAcker := 0 for _, acker := range client.ackers { if acker.FullyAcked() { continue @@ -291,9 +292,11 @@ func (fp *fakePipeline) ackEvents() { client.eventListener.AddEvent(beat.Event{Private: acker}, true) } - if addedEvents > 0 { - client.eventListener.ACKEvents(addedEvents) - } + addedEventsForAllAcker += addedEvents + } + + if addedEventsForAllAcker > 0 { + client.eventListener.ACKEvents(addedEventsForAllAcker) } } } diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index df78e0cceb2..d5cd68d1eac 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -15,10 +11,9 @@ import ( s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" logp "github.com/elastic/elastic-agent-libs/logp" + gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. @@ -290,11 +285,12 @@ func (mr *MockSQSProcessorMockRecorder) DeleteSQS(msg, receiveCount, processingE } // ProcessSQS mocks base method. -func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) error { +func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) (uint64, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg, client, acker, start) - ret0, _ := ret[0].(error) - return ret0 + ret0, _ := ret[0].(uint64) + ret1, _ := ret[1].(error) + return ret0, ret1 } // ProcessSQS indicates an expected call of ProcessSQS. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index efbd5bcef97..9c34b58efd1 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -11,9 +7,8 @@ package awss3 import ( reflect "reflect" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" + gomock "github.com/golang/mock/gomock" ) // MockBeatClient is a mock of Client interface. diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index 7dde7de6c6d..07b22c4655b 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -38,18 +38,23 @@ func TestSQSS3EventProcessor(t *testing.T) { ctrl, ctx := gomock.WithContext(ctx, t) defer ctrl.Finish() mockAPI := NewMockSQSAPI(ctrl) + mockS3Handler := NewMockS3ObjectHandler(ctrl) mockS3HandlerFactory := NewMockS3ObjectHandlerFactory(ctrl) mockClient := NewMockBeatClient(ctrl) + expectedEventsPublishedTotal := uint64(10) gomock.InOrder( - mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil), + mockS3HandlerFactory.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(mockS3Handler), + mockS3Handler.EXPECT().ProcessS3Object().Return(expectedEventsPublishedTotal, nil), + mockS3Handler.EXPECT().FinalizeS3Object().Return(nil), ) p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) require.NoError(t, processingErr) + require.Equal(t, expectedEventsPublishedTotal, eventsPublishedTotal) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), @@ -76,9 +81,10 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker, time.Now()) t.Log(processingErr) require.Error(t, processingErr) + require.Equal(t, uint64(0), eventsPublishedTotal) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&invalidBodyMsg)).Return(nil), @@ -102,8 +108,9 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker, time.Now()) require.NoError(t, processingErr) + require.Equal(t, uint64(0), eventsPublishedTotal) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&emptyRecordsMsg)).Return(nil), @@ -137,8 +144,9 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) require.NoError(t, processingErr) + require.Equal(t, uint64(1), eventsPublishedTotal) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), @@ -166,9 +174,10 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) t.Log(processingErr) require.Error(t, processingErr) + require.Equal(t, uint64(0), eventsPublishedTotal) require.Error(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) @@ -196,9 +205,10 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) t.Log(processingErr) require.Error(t, processingErr) + require.Equal(t, uint64(0), eventsPublishedTotal) gomock.InOrder( mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index d5c2fad8fcb..33933c583e1 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -78,14 +78,14 @@ func TestSQSReceiver(t *testing.T) { ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). DoAndReturn( - func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) error { + func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) (uint64, error) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) acker.ACK() acker.FlushForSQS() - return nil + return 1, nil }) // Expect the client to be closed From 826a2983bf373e3a445b7a1aa2159255296639e2 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Mon, 11 Mar 2024 16:41:55 +0900 Subject: [PATCH 28/43] lint --- x-pack/filebeat/input/awss3/mock_interfaces_test.go | 7 ++++++- x-pack/filebeat/input/awss3/mock_publisher_test.go | 7 ++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index d5cd68d1eac..2e9cd67eb4c 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -11,9 +15,10 @@ import ( s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" + gomock "github.com/golang/mock/gomock" + beat "github.com/elastic/beats/v7/libbeat/beat" logp "github.com/elastic/elastic-agent-libs/logp" - gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index 9c34b58efd1..efbd5bcef97 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -7,8 +11,9 @@ package awss3 import ( reflect "reflect" - beat "github.com/elastic/beats/v7/libbeat/beat" gomock "github.com/golang/mock/gomock" + + beat "github.com/elastic/beats/v7/libbeat/beat" ) // MockBeatClient is a mock of Client interface. From 7413873408d7de7bfce5c02edc5b4126cb443cb6 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:16:25 +0900 Subject: [PATCH 29/43] revert awscommon.EventACKTracker in S3 polling --- x-pack/filebeat/input/awss3/interfaces.go | 10 ++-- x-pack/filebeat/input/awss3/s3.go | 7 ++- x-pack/filebeat/input/awss3/s3_objects.go | 66 ++++++++++++++++------- 3 files changed, 55 insertions(+), 28 deletions(-) diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index c93ad12a69c..9b07568adc0 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -8,6 +8,7 @@ import ( "context" "errors" "fmt" + awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "net/url" "time" @@ -101,14 +102,14 @@ type s3ObjectHandlerFactory interface { // specified S3 object. If the handler is not configured to process the // given S3 object (based on key name) then it will return nil. Create(ctx context.Context, log *logp.Logger, client beat.Client, acker *EventACKTracker, obj s3EventV2) s3ObjectHandler + CreateForS3Polling(ctx context.Context, log *logp.Logger, client beat.Client, acker *awscommon.EventACKTracker, obj s3EventV2) s3ObjectHandler } type s3ObjectHandler interface { // ProcessS3Object downloads the S3 object, parses it, creates events, and // publishes them. It returns when processing finishes or when it encounters // an unrecoverable error. It does not wait for the events to be ACKed by - // the publisher before returning (use eventACKTracker's WaitForS3() method to - // determine this). + // the publisher before returning. ProcessS3Object() (uint64, error) // FinalizeS3Object finalizes processing of an S3 object after the current @@ -117,11 +118,8 @@ type s3ObjectHandler interface { // Wait waits for every event published by ProcessS3Object() to be ACKed // by the publisher before returning. Internally it uses the - // s3ObjectHandler eventACKTracker's WaitForS3() method + // s3ObjectHandler ackerForPolling's Wait() method Wait() - - // SyncEventsToBeAcked sync the number of event published with the eventACKTracker - SyncEventsToBeAcked(s3EventsCreatedTotal uint64) } // ------ diff --git a/x-pack/filebeat/input/awss3/s3.go b/x-pack/filebeat/input/awss3/s3.go index b3dedb562dd..b04f50bc071 100644 --- a/x-pack/filebeat/input/awss3/s3.go +++ b/x-pack/filebeat/input/awss3/s3.go @@ -127,9 +127,9 @@ func (p *s3Poller) createS3ObjectProcessor(ctx context.Context, state state) (s3 event.S3.Bucket.ARN = p.bucket event.S3.Object.Key = state.Key - acker := NewEventACKTracker(ctx, nil) + acker := awscommon.NewEventACKTracker(ctx) - return p.s3ObjectHandler.Create(ctx, p.log, p.client, acker, event), event + return p.s3ObjectHandler.CreateForS3Polling(ctx, p.log, p.client, acker, event), event } func (p *s3Poller) ProcessObject(s3ObjectPayloadChan <-chan *s3ObjectPayload) error { @@ -137,9 +137,8 @@ func (p *s3Poller) ProcessObject(s3ObjectPayloadChan <-chan *s3ObjectPayload) er for s3ObjectPayload := range s3ObjectPayloadChan { // Process S3 object (download, parse, create events). - eventsPublished, err := s3ObjectPayload.s3ObjectHandler.ProcessS3Object() + _, err := s3ObjectPayload.s3ObjectHandler.ProcessS3Object() - s3ObjectPayload.s3ObjectHandler.SyncEventsToBeAcked(eventsPublished) // Wait for all events to be ACKed before proceeding. s3ObjectPayload.s3ObjectHandler.Wait() diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index b152b4a460a..ac011117576 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -14,6 +14,7 @@ import ( "encoding/json" "errors" "fmt" + awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "io" "reflect" "strings" @@ -95,28 +96,51 @@ func (f *s3ObjectProcessorFactory) Create(ctx context.Context, log *logp.Logger, } } +// CreateForS3Polling returns a new s3ObjectProcessor. It returns nil when no file selectors +// match the S3 object key. +func (f *s3ObjectProcessorFactory) CreateForS3Polling(ctx context.Context, log *logp.Logger, client beat.Client, ack *awscommon.EventACKTracker, obj s3EventV2) s3ObjectHandler { + log = log.With( + "bucket_arn", obj.S3.Bucket.Name, + "object_key", obj.S3.Object.Key) + + readerConfig := f.findReaderConfig(obj.S3.Object.Key) + if readerConfig == nil { + log.Debug("Skipping S3 object processing. No file_selectors are a match.") + return nil + } + + return &s3ObjectProcessor{ + s3ObjectProcessorFactory: f, + log: log, + ctx: ctx, + publisher: client, + ackerForPolling: ack, + readerConfig: readerConfig, + s3Obj: obj, + s3ObjHash: s3ObjectHash(obj), + } +} + type s3ObjectProcessor struct { *s3ObjectProcessorFactory - log *logp.Logger - ctx context.Context - publisher beat.Client - acker *EventACKTracker // ACKer tied to the SQS message (multiple S3 readers share an ACKer when the S3 notification event contains more than one S3 object). - readerConfig *readerConfig // Config about how to process the object. - s3Obj s3EventV2 // S3 object information. - s3ObjHash string - s3RequestURL string + log *logp.Logger + ctx context.Context + publisher beat.Client + acker *EventACKTracker // ACKer tied to the SQS message (multiple S3 readers share an ACKer when the S3 notification event contains more than one S3 object). + ackerForPolling *awscommon.EventACKTracker // ACKer tied to the S3 object (multiple S3 readers share an ACKer when the S3 notification event contains more than one S3 object). + readerConfig *readerConfig // Config about how to process the object. + s3Obj s3EventV2 // S3 object information. + s3ObjHash string + s3RequestURL string s3Metadata map[string]interface{} // S3 object metadata. eventsPublishedTotal uint64 } -func (p *s3ObjectProcessor) SyncEventsToBeAcked(eventsPublished uint64) { - p.acker.MarkS3FromListingProcessedWithData(eventsPublished) -} func (p *s3ObjectProcessor) Wait() { - p.acker.WaitForS3() + p.ackerForPolling.Wait() } func (p *s3ObjectProcessor) ProcessS3Object() (uint64, error) { @@ -254,7 +278,7 @@ func (p *s3ObjectProcessor) readJSON(r io.Reader) error { data, _ := item.MarshalJSON() evt := p.createEvent(string(data), offset) - p.publish(p.acker, &evt) + p.publish(&evt) } return nil @@ -289,7 +313,7 @@ func (p *s3ObjectProcessor) readJSONSlice(r io.Reader, evtOffset int64) (int64, data, _ := item.MarshalJSON() evt := p.createEvent(string(data), evtOffset) - p.publish(p.acker, &evt) + p.publish(&evt) evtOffset++ } @@ -334,7 +358,7 @@ func (p *s3ObjectProcessor) splitEventList(key string, raw json.RawMessage, offs data, _ := item.MarshalJSON() p.s3ObjHash = objHash evt := p.createEvent(string(data), offset+arrayOffset) - p.publish(p.acker, &evt) + p.publish(&evt) } return nil @@ -374,7 +398,7 @@ func (p *s3ObjectProcessor) readFile(r io.Reader) error { event := p.createEvent(string(message.Content), offset) event.Fields.DeepUpdate(message.Fields) offset += int64(message.Bytes) - p.publish(p.acker, &event) + p.publish(&event) } if errors.Is(err, io.EOF) { @@ -389,8 +413,14 @@ func (p *s3ObjectProcessor) readFile(r io.Reader) error { return nil } -func (p *s3ObjectProcessor) publish(ack *EventACKTracker, event *beat.Event) { - event.Private = ack +func (p *s3ObjectProcessor) publish(event *beat.Event) { + if p.acker != nil { + event.Private = p.acker + } else if p.ackerForPolling != nil { + p.ackerForPolling.Add() + event.Private = p.ackerForPolling + } + p.eventsPublishedTotal++ p.metrics.s3EventsCreatedTotal.Inc() p.publisher.Publish(*event) From 2aedb51bc0b9c78c22be33f4f114a3adcd449711 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:17:12 +0900 Subject: [PATCH 30/43] SQS acker with tracker listener and no blocking --- x-pack/filebeat/input/awss3/sqs_acker.go | 138 ++++++++++------------- 1 file changed, 59 insertions(+), 79 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index ae8330be49d..b26a5bcf073 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -31,16 +31,13 @@ func init() { type EventACKTracker struct { ID uint64 - EventsToBeAcked *atomic.Uint64 + EventsToBeTracked *atomic.Uint64 + EventsTracked *atomic.Uint64 ctx context.Context cancel context.CancelFunc deletionWg *sync.WaitGroup - ackMutex *sync.RWMutex - ackMutexLockedOnInit *atomic.Bool - - isSQSAcker bool msg *types.Message ReceiveCount int start time.Time @@ -54,46 +51,50 @@ type EventACKTracker struct { func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { ctx, cancel := context.WithCancel(ctx) - ackMutex := new(sync.RWMutex) - // We need to lock on ack mutex, in order to know that we have passed the info to the acker about the total to be acked - // Lock it as soon as we create the acker. It will be unlocked and in either MarkS3FromListingProcessedWithData or MarkSQSProcessedWithData - ackMutex.Lock() - return &EventACKTracker{ - ID: ackerIDCounter.Inc(), - ctx: ctx, - cancel: cancel, - deletionWg: deletionWg, - ackMutex: ackMutex, - ackMutexLockedOnInit: atomic.NewBool(true), - EventsToBeAcked: atomic.NewUint64(0), + acker := &EventACKTracker{ + ID: ackerIDCounter.Inc(), + ctx: ctx, + cancel: cancel, + deletionWg: deletionWg, + EventsToBeTracked: atomic.NewUint64(0), + EventsTracked: atomic.NewUint64(0), } + + go func() { + t := time.NewTicker(500 * time.Microsecond) + defer t.Stop() + + for { + <-t.C + if acker.checkForCancel() { + return + } + } + }() + + return acker } -// MarkS3FromListingProcessedWithData has to be used when the acker is used when the input is in s3 direct listing mode, instead of MarkSQSProcessedWithData -// Specifically we both Swap the value of EventACKTracker.ackMutexLockedOnInit initialised in NewEventACKTracker -func (a *EventACKTracker) MarkS3FromListingProcessedWithData(eventsPublished uint64) { - // We want to execute the logic of this call only once, when the ack mutex was locked on init - if !a.ackMutexLockedOnInit.Swap(false) { - return +func (a *EventACKTracker) checkForCancel() bool { + if !a.FullyAcked() { + return false } - a.EventsToBeAcked.Add(eventsPublished) + a.cancel() + a.FlushForSQS() - a.ackMutex.Unlock() + return true } -// MarkSQSProcessedWithData has to be used when the acker is used when the input is in sqs-s3 mode, instead of MarkS3FromListingProcessedWithData -// Specifically we both Swap the value of EventACKTracker.ackMutexLockedOnInit initialised in NewEventACKTracker +// MarkSQSProcessedWithData Every call after the first one is a no-op func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { // We want to execute the logic of this call only once, when the ack mutex was locked on init - if !a.ackMutexLockedOnInit.Swap(false) { + if a.EventsToBeTracked.Load() > 0 { return } - a.isSQSAcker = true - a.msg = msg - a.EventsToBeAcked = atomic.NewUint64(publishedEvent) + a.EventsToBeTracked = atomic.NewUint64(publishedEvent) a.ReceiveCount = receiveCount a.start = start a.processingErr = processingErr @@ -102,32 +103,19 @@ func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, published a.keepaliveWg = keepaliveWg a.msgHandler = msgHandler a.log = log - - a.ackMutex.Unlock() } func (a *EventACKTracker) FullyAcked() bool { - return a.EventsToBeAcked.Load() == 0 - -} - -// WaitForS3 must be called after MarkS3FromListingProcessedWithData -func (a *EventACKTracker) WaitForS3() { - // If it's fully acked then cancel the context. - if a.FullyAcked() { - a.cancel() + eventsToBeTracked := a.EventsToBeTracked.Load() + if eventsToBeTracked == 0 { + return false } - // Wait. - <-a.ctx.Done() + return a.EventsTracked.Load() == eventsToBeTracked } // FlushForSQS delete related SQS message func (a *EventACKTracker) FlushForSQS() { - if !a.isSQSAcker { - return - } - // Stop keepalive visibility routine before deleting. a.keepaliveCancel() a.keepaliveWg.Wait() @@ -147,24 +135,9 @@ func (a *EventACKTracker) FlushForSQS() { } } -// ACK decrements the number of total Events ACKed. -func (a *EventACKTracker) ACK() { - // We need to lock on ack mutex, in order to know that we have passed the info to the acker about the total to be acked - // But we want to do it only before the info have been passed, once they did, no need anymore to lock on the ack mutext - if a.ackMutexLockedOnInit.Load() { - a.ackMutex.Lock() - defer a.ackMutex.Unlock() - } - - if a.FullyAcked() { - panic("misuse detected: ACK call on fully acked") - } - - a.EventsToBeAcked.Dec() - - if a.FullyAcked() { - a.cancel() - } +// Track decrements the number of total Events. +func (a *EventACKTracker) Track(_ int, total int) { + a.EventsTracked.Add(uint64(total)) } // NewEventACKHandler returns a beat ACKer that can receive callbacks when @@ -172,17 +145,24 @@ func (a *EventACKTracker) ACK() { // pointing to an eventACKTracker then it will invoke the trackers ACK() method // to decrement the number of pending ACKs. func NewEventACKHandler() beat.EventListener { - return acker.ConnectionOnly( - acker.EventPrivateReporter(func(_ int, privates []interface{}) { - for _, current := range privates { - if ackTracker, ok := current.(*EventACKTracker); ok { - ackTracker.ACK() - - if ackTracker.FullyAcked() { - ackTracker.FlushForSQS() - } - } - } - }), - ) + return acker.ConnectionOnly(newEventListener()) +} + +func newEventListener() *eventListener { + return &eventListener{} +} + +type eventListener struct{} + +func (a *eventListener) ACKEvents(n int) {} + +func (a *eventListener) ClientClosed() {} + +func (a *eventListener) AddEvent(event beat.Event, published bool) { + acker, ok := event.Private.(*EventACKTracker) + if !ok { + return + } + + acker.Track(0, 1) } From 7fb85cbdeb6ea89d3c9195eb474550406a09d68f Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:17:50 +0900 Subject: [PATCH 31/43] fix deletionWaiter --- x-pack/filebeat/input/awss3/sqs.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 4fe858c3992..5334465cdfd 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -7,6 +7,7 @@ package awss3 import ( "context" "errors" + "github.com/elastic/beats/v7/libbeat/common/atomic" "strconv" "sync" "time" @@ -62,7 +63,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { workersChan := make(chan processingData, r.maxMessagesInflight) deletionWg := new(sync.WaitGroup) - deletionWaiter := make(chan struct{}) + deletionWaiter := new(atomic.Bool) var clientsMutex sync.Mutex clients := make(map[uint64]beat.Client, r.maxMessagesInflight) @@ -117,10 +118,10 @@ func (r *sqsReader) Receive(ctx context.Context) error { acker := NewEventACKTracker(ctx, deletionWg) deletionWg.Add(1) - deletionWaiter <- struct{}{} + deletionWaiter.Swap(false) eventsCreatedTotal, err := r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) - // No ACK will be invoked by the client event listener, deletionWg.Done() has to be called here + // No Track will be invoked by the client event listener, deletionWg.Done() has to be called here if eventsCreatedTotal == 0 { deletionWg.Done() } @@ -175,7 +176,9 @@ func (r *sqsReader) Receive(ctx context.Context) error { // Wait for all deletion to happen. if r.metrics.sqsMessagesReceivedTotal.Get() > 0 { - <-deletionWaiter + for deletionWaiter.Load() { + time.Sleep(500 * time.Millisecond) + } } deletionWg.Wait() From 732c97ad5b4c39ca9488df88724ae4ee254e1b87 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:18:13 +0900 Subject: [PATCH 32/43] fix test --- .../input/awss3/input_benchmark_test.go | 29 ++----- .../input/awss3/mock_interfaces_test.go | 34 ++++---- .../input/awss3/mock_publisher_test.go | 7 +- .../filebeat/input/awss3/s3_objects_test.go | 28 ++++--- x-pack/filebeat/input/awss3/sqs_acker_test.go | 82 ++++++++++++------- .../filebeat/input/awss3/sqs_s3_event_test.go | 16 +--- x-pack/filebeat/input/awss3/sqs_test.go | 4 +- 7 files changed, 99 insertions(+), 101 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index f4f3c1ed1bd..a146e8ddff9 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -9,6 +9,8 @@ import ( "encoding/json" "errors" "fmt" + pubtest "github.com/elastic/beats/v7/libbeat/publisher/testing" + awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "math" "math/rand" "os" @@ -33,7 +35,6 @@ import ( "github.com/dustin/go-humanize" "github.com/olekukonko/tablewriter" - pubtest "github.com/elastic/beats/v7/libbeat/publisher/testing" conf "github.com/elastic/elastic-agent-libs/config" "github.com/elastic/elastic-agent-libs/logp" "github.com/elastic/elastic-agent-libs/monitoring" @@ -133,6 +134,7 @@ func (c *constantSQS) GetQueueAttributes(ctx context.Context, attr []sqsTypes.Qu } type s3PagerConstant struct { + listPrefix string mutex *sync.Mutex objects []s3Types.Object currentIndex int @@ -161,13 +163,13 @@ func (c *s3PagerConstant) NextPage(ctx context.Context, optFns ...func(*s3.Optio ret.Contents = c.objects[c.currentIndex : c.currentIndex+pageSize] c.currentIndex = c.currentIndex + pageSize - return ret, nil } func newS3PagerConstant(listPrefix string) *s3PagerConstant { lastModified := time.Now() ret := &s3PagerConstant{ + listPrefix: listPrefix, mutex: new(sync.Mutex), currentIndex: 0, } @@ -279,24 +281,12 @@ type fakePipeline struct { func (fp *fakePipeline) ackEvents() { for _, client := range fp.clients { - addedEventsForAllAcker := 0 for _, acker := range client.ackers { - if acker.FullyAcked() { - continue - } - - addedEvents := 0 - for acker.EventsToBeAcked.Load() > 0 && uint64(addedEvents) < acker.EventsToBeAcked.Load() { - addedEvents++ + addedEvents := acker.EventsToBeTracked.Load() + for addedEvents > 0 && acker.EventsTracked.Load() != addedEvents { fp.pendingEvents.Dec() client.eventListener.AddEvent(beat.Event{Private: acker}, true) } - - addedEventsForAllAcker += addedEvents - } - - if addedEventsForAllAcker > 0 { - client.eventListener.ACKEvents(addedEventsForAllAcker) } } } @@ -494,12 +484,7 @@ func benchmarkInputS3(t *testing.T, numberOfWorkers int) testing.BenchmarkResult metrics := newInputMetrics("test_id", metricRegistry, numberOfWorkers) client := pubtest.NewChanClientWithCallback(100, func(event beat.Event) { - go func(acker *EventACKTracker) { - // 63 is the total number of events in a single S3 object - acker.MarkS3FromListingProcessedWithData(63) - }(event.Private.(*EventACKTracker)) - - event.Private.(*EventACKTracker).ACK() + event.Private.(*awscommon.EventACKTracker).ACK() }) defer func() { diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index 2e9cd67eb4c..b210ace53c7 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -15,10 +11,10 @@ import ( s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" + aws "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" logp "github.com/elastic/elastic-agent-libs/logp" + gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. @@ -608,6 +604,20 @@ func (mr *MockS3ObjectHandlerFactoryMockRecorder) Create(ctx, log, client, acker return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Create", reflect.TypeOf((*MockS3ObjectHandlerFactory)(nil).Create), ctx, log, client, acker, obj) } +// CreateForS3Polling mocks base method. +func (m *MockS3ObjectHandlerFactory) CreateForS3Polling(ctx context.Context, log *logp.Logger, client beat.Client, acker *aws.EventACKTracker, obj s3EventV2) s3ObjectHandler { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CreateForS3Polling", ctx, log, client, acker, obj) + ret0, _ := ret[0].(s3ObjectHandler) + return ret0 +} + +// CreateForS3Polling indicates an expected call of CreateForS3Polling. +func (mr *MockS3ObjectHandlerFactoryMockRecorder) CreateForS3Polling(ctx, log, client, acker, obj interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateForS3Polling", reflect.TypeOf((*MockS3ObjectHandlerFactory)(nil).CreateForS3Polling), ctx, log, client, acker, obj) +} + // MockS3ObjectHandler is a mock of s3ObjectHandler interface. type MockS3ObjectHandler struct { ctrl *gomock.Controller @@ -660,18 +670,6 @@ func (mr *MockS3ObjectHandlerMockRecorder) ProcessS3Object() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessS3Object", reflect.TypeOf((*MockS3ObjectHandler)(nil).ProcessS3Object)) } -// SyncEventsToBeAcked mocks base method. -func (m *MockS3ObjectHandler) SyncEventsToBeAcked(s3EventsCreatedTotal uint64) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "SyncEventsToBeAcked", s3EventsCreatedTotal) -} - -// SyncEventsToBeAcked indicates an expected call of SyncEventsToBeAcked. -func (mr *MockS3ObjectHandlerMockRecorder) SyncEventsToBeAcked(s3EventsCreatedTotal interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SyncEventsToBeAcked", reflect.TypeOf((*MockS3ObjectHandler)(nil).SyncEventsToBeAcked), s3EventsCreatedTotal) -} - // Wait mocks base method. func (m *MockS3ObjectHandler) Wait() { m.ctrl.T.Helper() diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index efbd5bcef97..9c34b58efd1 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,7 +1,3 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License; -// you may not use this file except in compliance with the Elastic License. - // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -11,9 +7,8 @@ package awss3 import ( reflect "reflect" - gomock "github.com/golang/mock/gomock" - beat "github.com/elastic/beats/v7/libbeat/beat" + gomock "github.com/golang/mock/gomock" ) // MockBeatClient is a mock of Client interface. diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 3508754296b..b0911f9ccb1 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -8,18 +8,19 @@ import ( "bytes" "context" "errors" - "io" - "os" - "path/filepath" - "strings" - "testing" - awssdk "github.com/aws/aws-sdk-go-v2/aws" "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/aws/aws-sdk-go-v2/service/s3/types" "github.com/golang/mock/gomock" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "io" + "os" + "path/filepath" + "strings" + "sync" + "testing" + "time" "github.com/elastic/beats/v7/libbeat/beat" conf "github.com/elastic/elastic-agent-libs/config" @@ -320,8 +321,14 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint mockPublisher := NewMockBeatClient(ctrl) s3Event, s3Resp := newS3Object(t, file, contentType) - ack := NewEventACKTracker(ctx, nil) - ack.MarkS3FromListingProcessedWithData(numEvents) + acker := NewEventACKTracker(ctx, nil) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + acker.MarkSQSProcessedWithData(&msg, numEvents, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.EventsToBeTracked.Add(numEvents) + var events []beat.Event gomock.InOrder( mockS3API.EXPECT(). @@ -331,19 +338,18 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint Publish(gomock.Any()). Do(func(event beat.Event) { events = append(events, event) - ack.ACK() + acker.Track(0, 1) }). Times(int(numEvents)), ) s3ObjProc := newS3ObjectProcessorFactory(logp.NewLogger(inputName), nil, mockS3API, selectors, backupConfig{}, 1) - s3EventsCreatedTotal, err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, ack, s3Event).ProcessS3Object() + s3EventsCreatedTotal, err := s3ObjProc.Create(ctx, logp.NewLogger(inputName), mockPublisher, acker, s3Event).ProcessS3Object() if !expectErr { require.NoError(t, err) assert.EqualValues(t, numEvents, len(events)) assert.EqualValues(t, numEvents, s3EventsCreatedTotal) - assert.EqualValues(t, 0, ack.EventsToBeAcked.Load()) } else { require.Error(t, err) } diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index 35383577954..4689896db4a 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -6,7 +6,10 @@ package awss3 import ( "context" + "github.com/golang/mock/gomock" + "sync" "testing" + "time" "github.com/stretchr/testify/assert" @@ -17,23 +20,35 @@ func TestEventACKTracker(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) - acker := NewEventACKTracker(ctx, nil) - acker.MarkS3FromListingProcessedWithData(1) - acker.ACK() + deletionWg := new(sync.WaitGroup) + deletionWg.Add(1) + + acker := NewEventACKTracker(ctx, deletionWg) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + ctrl, ctx := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.Track(0, 1) + acker.checkForCancel() assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKTrackerNoACKs(t *testing.T) { +func TestEventACKTrackerNoTrack(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) - acker := NewEventACKTracker(ctx, nil) - acker.WaitForS3() + deletionWg := new(sync.WaitGroup) + acker := NewEventACKTracker(ctx, deletionWg) + cancel() + <-acker.ctx.Done() - assert.EqualValues(t, 0, acker.EventsToBeAcked.Load()) - assert.EqualValues(t, true, acker.FullyAcked()) + assert.EqualValues(t, false, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -41,46 +56,53 @@ func TestEventACKHandler(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) - // Create acker. Add one pending ACK. - acker := NewEventACKTracker(ctx, nil) - acker.MarkS3FromListingProcessedWithData(1) + // Create acker. Add one ACK to event listener. + deletionWg := new(sync.WaitGroup) + deletionWg.Add(1) + acker := NewEventACKTracker(ctx, deletionWg) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + ctrl, ctx := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) // Create an ACK handler and simulate one ACKed event. ackHandler := NewEventACKHandler() ackHandler.AddEvent(beat.Event{Private: acker}, true) ackHandler.ACKEvents(1) + acker.checkForCancel() assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKHandlerWaitForS3(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - - // Create acker. Add one pending ACK. - acker := NewEventACKTracker(ctx, nil) - acker.MarkS3FromListingProcessedWithData(1) - acker.ACK() - acker.WaitForS3() - assert.EqualValues(t, true, acker.FullyAcked()) - - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) -} - func TestEventACKHandlerFullyAcked(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) - // Create acker. Add one pending ACK. - acker := NewEventACKTracker(ctx, nil) - acker.MarkS3FromListingProcessedWithData(1) - acker.ACK() + // Create acker. Add one Track. + deletionWg := new(sync.WaitGroup) + deletionWg.Add(1) + + acker := NewEventACKTracker(ctx, deletionWg) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + ctrl, ctx := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.Track(0, 1) + acker.checkForCancel() assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) - acker.EventsToBeAcked.Inc() + acker.EventsToBeTracked.Inc() assert.EqualValues(t, false, acker.FullyAcked()) } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index 07b22c4655b..0e162227512 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -56,9 +56,7 @@ func TestSQSS3EventProcessor(t *testing.T) { require.NoError(t, processingErr) require.Equal(t, expectedEventsPublishedTotal, eventsPublishedTotal) - gomock.InOrder( - mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), - ) + mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil) require.NoError(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) @@ -86,9 +84,7 @@ func TestSQSS3EventProcessor(t *testing.T) { require.Error(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) - gomock.InOrder( - mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&invalidBodyMsg)).Return(nil), - ) + mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&invalidBodyMsg)).Return(nil) require.Error(t, p.DeleteSQS(&invalidBodyMsg, acker.ReceiveCount, processingErr, acker.Handles)) }) @@ -112,9 +108,7 @@ func TestSQSS3EventProcessor(t *testing.T) { require.NoError(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) - gomock.InOrder( - mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&emptyRecordsMsg)).Return(nil), - ) + mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&emptyRecordsMsg)).Return(nil) require.NoError(t, p.DeleteSQS(&emptyRecordsMsg, acker.ReceiveCount, processingErr, acker.Handles)) }) @@ -210,9 +204,7 @@ func TestSQSS3EventProcessor(t *testing.T) { require.Error(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) - gomock.InOrder( - mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil), - ) + mockAPI.EXPECT().DeleteMessage(gomock.Any(), gomock.Eq(&msg)).Return(nil) require.Error(t, p.DeleteSQS(&msg, acker.ReceiveCount, processingErr, acker.Handles)) }) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 33933c583e1..b681b9f3245 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -82,8 +82,8 @@ func TestSQSReceiver(t *testing.T) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.ACK() - acker.FlushForSQS() + acker.Track(0, 1) + acker.checkForCancel() return 1, nil }) From 136f026ccf01563a0c233741d089e394d3ac388b Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:19:10 +0900 Subject: [PATCH 33/43] lint --- x-pack/filebeat/input/awss3/input_benchmark_test.go | 5 +++-- x-pack/filebeat/input/awss3/interfaces.go | 3 ++- x-pack/filebeat/input/awss3/mock_interfaces_test.go | 7 ++++++- x-pack/filebeat/input/awss3/mock_publisher_test.go | 7 ++++++- x-pack/filebeat/input/awss3/s3_objects.go | 3 ++- x-pack/filebeat/input/awss3/s3_objects_test.go | 13 +++++++------ x-pack/filebeat/input/awss3/sqs.go | 3 ++- x-pack/filebeat/input/awss3/sqs_acker_test.go | 3 ++- 8 files changed, 30 insertions(+), 14 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index a146e8ddff9..184b32c5bb9 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -9,8 +9,6 @@ import ( "encoding/json" "errors" "fmt" - pubtest "github.com/elastic/beats/v7/libbeat/publisher/testing" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "math" "math/rand" "os" @@ -22,6 +20,9 @@ import ( "testing" "time" + pubtest "github.com/elastic/beats/v7/libbeat/publisher/testing" + awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" + "github.com/elastic/beats/v7/libbeat/common/atomic" "github.com/elastic/beats/v7/libbeat/statestore" "github.com/elastic/beats/v7/libbeat/statestore/storetest" diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index 9b07568adc0..a5c732806a3 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -8,10 +8,11 @@ import ( "context" "errors" "fmt" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "net/url" "time" + awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" + smithyhttp "github.com/aws/smithy-go/transport/http" "github.com/aws/smithy-go/middleware" diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index b210ace53c7..7131d736825 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: interfaces.go @@ -11,10 +15,11 @@ import ( s3 "github.com/aws/aws-sdk-go-v2/service/s3" types "github.com/aws/aws-sdk-go-v2/service/sqs/types" + gomock "github.com/golang/mock/gomock" + beat "github.com/elastic/beats/v7/libbeat/beat" aws "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" logp "github.com/elastic/elastic-agent-libs/logp" - gomock "github.com/golang/mock/gomock" ) // MockSQSAPI is a mock of sqsAPI interface. diff --git a/x-pack/filebeat/input/awss3/mock_publisher_test.go b/x-pack/filebeat/input/awss3/mock_publisher_test.go index 9c34b58efd1..efbd5bcef97 100644 --- a/x-pack/filebeat/input/awss3/mock_publisher_test.go +++ b/x-pack/filebeat/input/awss3/mock_publisher_test.go @@ -1,3 +1,7 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License; +// you may not use this file except in compliance with the Elastic License. + // Code generated by MockGen. DO NOT EDIT. // Source: github.com/elastic/beats/v7/libbeat/beat (interfaces: Client,Pipeline) @@ -7,8 +11,9 @@ package awss3 import ( reflect "reflect" - beat "github.com/elastic/beats/v7/libbeat/beat" gomock "github.com/golang/mock/gomock" + + beat "github.com/elastic/beats/v7/libbeat/beat" ) // MockBeatClient is a mock of Client interface. diff --git a/x-pack/filebeat/input/awss3/s3_objects.go b/x-pack/filebeat/input/awss3/s3_objects.go index ac011117576..47cf17947ce 100644 --- a/x-pack/filebeat/input/awss3/s3_objects.go +++ b/x-pack/filebeat/input/awss3/s3_objects.go @@ -14,12 +14,13 @@ import ( "encoding/json" "errors" "fmt" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "io" "reflect" "strings" "time" + awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/elastic/beats/v7/libbeat/beat" diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index b0911f9ccb1..50c85810017 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -8,12 +8,6 @@ import ( "bytes" "context" "errors" - awssdk "github.com/aws/aws-sdk-go-v2/aws" - "github.com/aws/aws-sdk-go-v2/service/s3" - "github.com/aws/aws-sdk-go-v2/service/s3/types" - "github.com/golang/mock/gomock" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" "io" "os" "path/filepath" @@ -22,6 +16,13 @@ import ( "testing" "time" + awssdk "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" + "github.com/aws/aws-sdk-go-v2/service/s3/types" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/elastic/beats/v7/libbeat/beat" conf "github.com/elastic/elastic-agent-libs/config" "github.com/elastic/elastic-agent-libs/logp" diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index 5334465cdfd..d041c11140a 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -7,11 +7,12 @@ package awss3 import ( "context" "errors" - "github.com/elastic/beats/v7/libbeat/common/atomic" "strconv" "sync" "time" + "github.com/elastic/beats/v7/libbeat/common/atomic" + "github.com/elastic/go-concert/timed" "github.com/aws/aws-sdk-go-v2/service/sqs/types" diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index 4689896db4a..50390c78bdb 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -6,11 +6,12 @@ package awss3 import ( "context" - "github.com/golang/mock/gomock" "sync" "testing" "time" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/assert" "github.com/elastic/beats/v7/libbeat/beat" From d06021e8f5bb7aab5d3a127217f5269c4d0bacb6 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:25:12 +0900 Subject: [PATCH 34/43] fix race in acker goroutine --- x-pack/filebeat/input/awss3/sqs_acker.go | 15 ++++++--------- x-pack/filebeat/input/awss3/sqs_acker_test.go | 6 +++--- x-pack/filebeat/input/awss3/sqs_test.go | 2 +- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index b26a5bcf073..1e3e77caeaa 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -66,24 +66,21 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA for { <-t.C - if acker.checkForCancel() { - return + if !acker.FullyAcked() { + continue } + + acker.cancelAndFlush() + return } }() return acker } -func (a *EventACKTracker) checkForCancel() bool { - if !a.FullyAcked() { - return false - } - +func (a *EventACKTracker) cancelAndFlush() { a.cancel() a.FlushForSQS() - - return true } // MarkSQSProcessedWithData Every call after the first one is a no-op diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index 50390c78bdb..d6a9ea7b4bc 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -34,7 +34,7 @@ func TestEventACKTracker(t *testing.T) { mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) acker.Track(0, 1) - acker.checkForCancel() + acker.cancelAndFlush() assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) @@ -74,7 +74,7 @@ func TestEventACKHandler(t *testing.T) { ackHandler := NewEventACKHandler() ackHandler.AddEvent(beat.Event{Private: acker}, true) ackHandler.ACKEvents(1) - acker.checkForCancel() + acker.cancelAndFlush() assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) @@ -98,7 +98,7 @@ func TestEventACKHandlerFullyAcked(t *testing.T) { mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) acker.Track(0, 1) - acker.checkForCancel() + acker.cancelAndFlush() assert.EqualValues(t, true, acker.FullyAcked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index b681b9f3245..be7e8d7f55f 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -83,7 +83,7 @@ func TestSQSReceiver(t *testing.T) { log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) acker.Track(0, 1) - acker.checkForCancel() + acker.cancelAndFlush() return 1, nil }) From d1be46cd11b773b3eeb3563eb48f3a488e9e2e3d Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 15:28:34 +0900 Subject: [PATCH 35/43] lint --- x-pack/filebeat/input/awss3/sqs_acker_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index d6a9ea7b4bc..fb771813b6c 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -27,7 +27,7 @@ func TestEventACKTracker(t *testing.T) { acker := NewEventACKTracker(ctx, deletionWg) _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") - ctrl, ctx := gomock.WithContext(ctx, t) + ctrl, _ := gomock.WithContext(ctx, t) defer ctrl.Finish() mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) @@ -63,7 +63,7 @@ func TestEventACKHandler(t *testing.T) { acker := NewEventACKTracker(ctx, deletionWg) _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") - ctrl, ctx := gomock.WithContext(ctx, t) + ctrl, _ := gomock.WithContext(ctx, t) defer ctrl.Finish() mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) @@ -91,7 +91,7 @@ func TestEventACKHandlerFullyAcked(t *testing.T) { acker := NewEventACKTracker(ctx, deletionWg) _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") - ctrl, ctx := gomock.WithContext(ctx, t) + ctrl, _ := gomock.WithContext(ctx, t) defer ctrl.Finish() mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) From 32bf3da76d4ae665cb6607b348a108a6422b83e7 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 17:05:46 +0900 Subject: [PATCH 36/43] Delete only on fully acked --- x-pack/filebeat/input/awss3/sqs_acker.go | 50 +++++++++++++++++------- 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 1e3e77caeaa..20470b9fd13 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -31,13 +31,16 @@ func init() { type EventACKTracker struct { ID uint64 - EventsToBeTracked *atomic.Uint64 + EventsAcked *atomic.Uint64 EventsTracked *atomic.Uint64 + EventsToBeTracked *atomic.Uint64 ctx context.Context cancel context.CancelFunc deletionWg *sync.WaitGroup + mutex *sync.Mutex + msg *types.Message ReceiveCount int start time.Time @@ -56,8 +59,10 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA ctx: ctx, cancel: cancel, deletionWg: deletionWg, - EventsToBeTracked: atomic.NewUint64(0), + mutex: new(sync.Mutex), + EventsAcked: atomic.NewUint64(0), EventsTracked: atomic.NewUint64(0), + EventsToBeTracked: atomic.NewUint64(0), } go func() { @@ -66,7 +71,7 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA for { <-t.C - if !acker.FullyAcked() { + if !acker.FullyTracked() { continue } @@ -102,7 +107,7 @@ func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, published a.log = log } -func (a *EventACKTracker) FullyAcked() bool { +func (a *EventACKTracker) FullyTracked() bool { eventsToBeTracked := a.EventsToBeTracked.Load() if eventsToBeTracked == 0 { return false @@ -117,23 +122,34 @@ func (a *EventACKTracker) FlushForSQS() { a.keepaliveCancel() a.keepaliveWg.Wait() - err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) - a.deletionWg.Done() - - if err != nil { - a.log.Warnw("Failed deleting SQS message.", - "error", err, - "message_id", *a.msg.MessageId, - "elapsed_time_ns", time.Since(a.start)) + if a.EventsAcked.Load() == a.EventsTracked.Load() { + err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) + if err != nil { + a.log.Warnw("Failed deleting SQS message.", + "error", err, + "message_id", *a.msg.MessageId, + "elapsed_time_ns", time.Since(a.start)) + } else { + a.log.Debugw("Success deleting SQS message.", + "message_id", *a.msg.MessageId, + "elapsed_time_ns", time.Since(a.start)) + } } else { - a.log.Debugw("Success deleting SQS message.", + a.log.Infow("Skipping deleting SQS message, not all events acked.", + "events_published", a.EventsTracked.Load(), + "events_acked", a.EventsAcked.Load(), "message_id", *a.msg.MessageId, "elapsed_time_ns", time.Since(a.start)) + } + + a.deletionWg.Done() + } // Track decrements the number of total Events. -func (a *EventACKTracker) Track(_ int, total int) { +func (a *EventACKTracker) Track(acked int, total int) { + a.EventsAcked.Add(uint64(acked)) a.EventsTracked.Add(uint64(total)) } @@ -161,5 +177,9 @@ func (a *eventListener) AddEvent(event beat.Event, published bool) { return } - acker.Track(0, 1) + var acked int + if published { + acked = 1 + } + acker.Track(acked, 1) } From 01ee8d18fcb523586883cf946914643902b01631 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 12 Mar 2024 17:07:52 +0900 Subject: [PATCH 37/43] fix tests --- .../filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker_test.go | 96 ++++++++++++++++--- x-pack/filebeat/input/awss3/sqs_test.go | 67 ++++++++++++- 3 files changed, 150 insertions(+), 15 deletions(-) diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 50c85810017..4ac3f69ec76 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -339,7 +339,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint Publish(gomock.Any()). Do(func(event beat.Event) { events = append(events, event) - acker.Track(0, 1) + acker.Track(1, 1) }). Times(int(numEvents)), ) diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index fb771813b6c..0bcd2e190ec 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -17,7 +17,7 @@ import ( "github.com/elastic/beats/v7/libbeat/beat" ) -func TestEventACKTracker(t *testing.T) { +func TestEventACKTrackerWithDelete(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -33,10 +33,32 @@ func TestEventACKTracker(t *testing.T) { msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.Track(1, 1) + <-acker.ctx.Done() + + assert.EqualValues(t, true, acker.FullyTracked()) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} + +func TestEventACKTrackerNoDelete(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + deletionWg := new(sync.WaitGroup) + deletionWg.Add(1) + + acker := NewEventACKTracker(ctx, deletionWg) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + ctrl, _ := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) acker.Track(0, 1) - acker.cancelAndFlush() + <-acker.ctx.Done() - assert.EqualValues(t, true, acker.FullyAcked()) + assert.EqualValues(t, true, acker.FullyTracked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } @@ -49,11 +71,37 @@ func TestEventACKTrackerNoTrack(t *testing.T) { cancel() <-acker.ctx.Done() - assert.EqualValues(t, false, acker.FullyAcked()) + assert.EqualValues(t, false, acker.FullyTracked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKHandler(t *testing.T) { +func TestEventACKHandlerNoDelete(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + // Create acker. Add one ACK to event listener. + deletionWg := new(sync.WaitGroup) + deletionWg.Add(1) + acker := NewEventACKTracker(ctx, deletionWg) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + ctrl, _ := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + + // Create an ACK handler and simulate one ACKed event. + ackHandler := NewEventACKHandler() + ackHandler.AddEvent(beat.Event{Private: acker}, false) + ackHandler.ACKEvents(1) + <-acker.ctx.Done() + + assert.EqualValues(t, true, acker.FullyTracked()) + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) +} + +func TestEventACKHandlerWithDelete(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -74,13 +122,13 @@ func TestEventACKHandler(t *testing.T) { ackHandler := NewEventACKHandler() ackHandler.AddEvent(beat.Event{Private: acker}, true) ackHandler.ACKEvents(1) - acker.cancelAndFlush() + <-acker.ctx.Done() - assert.EqualValues(t, true, acker.FullyAcked()) + assert.EqualValues(t, true, acker.FullyTracked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKHandlerFullyAcked(t *testing.T) { +func TestEventACKHandlerFullyTrackedWithDelete(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -97,13 +145,39 @@ func TestEventACKHandlerFullyAcked(t *testing.T) { msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.Track(1, 1) + <-acker.ctx.Done() + assert.EqualValues(t, true, acker.FullyTracked()) + + assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) + + acker.EventsToBeTracked.Inc() + + assert.EqualValues(t, false, acker.FullyTracked()) +} +func TestEventACKHandlerFullyTrackedNoDelete(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + // Create acker. Add one Track. + deletionWg := new(sync.WaitGroup) + deletionWg.Add(1) + + acker := NewEventACKTracker(ctx, deletionWg) + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + ctrl, _ := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) acker.Track(0, 1) - acker.cancelAndFlush() - assert.EqualValues(t, true, acker.FullyAcked()) + <-acker.ctx.Done() + assert.EqualValues(t, true, acker.FullyTracked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) acker.EventsToBeTracked.Inc() - assert.EqualValues(t, false, acker.FullyAcked()) + assert.EqualValues(t, false, acker.FullyTracked()) } diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index be7e8d7f55f..3aa8e30fcba 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -36,7 +36,7 @@ func TestSQSReceiver(t *testing.T) { const maxMessages = 5 - t.Run("ReceiveMessage success", func(t *testing.T) { + t.Run("ReceiveMessage success with delete", func(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), testTimeout) defer cancel() @@ -82,8 +82,8 @@ func TestSQSReceiver(t *testing.T) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(0, 1) - acker.cancelAndFlush() + acker.Track(1, 1) + <-acker.ctx.Done() return 1, nil }) @@ -103,6 +103,67 @@ func TestSQSReceiver(t *testing.T) { assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) + t.Run("ReceiveMessage success no delete", func(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), testTimeout) + defer cancel() + + ctrl, ctx := gomock.WithContext(ctx, t) + defer ctrl.Finish() + mockAPI := NewMockSQSAPI(ctrl) + mockMsgHandler := NewMockSQSProcessor(ctrl) + msg := newSQSMessage(newS3Event("log.json")) + + gomock.InOrder( + // Initial ReceiveMessage for maxMessages. + mockAPI.EXPECT(). + ReceiveMessage(gomock.Any(), gomock.Eq(maxMessages)). + Times(1). + DoAndReturn(func(_ context.Context, _ int) ([]types.Message, error) { + // Return single message. + return []types.Message{msg}, nil + }), + + // Follow up ReceiveMessages for either maxMessages-1 or maxMessages + // depending on how long processing of previous message takes. + mockAPI.EXPECT(). + ReceiveMessage(gomock.Any(), gomock.Any()). + Times(1). + DoAndReturn(func(_ context.Context, _ int) ([]types.Message, error) { + // Stop the test. + cancel() + return nil, nil + }), + ) + + mockClient := NewMockBeatClient(ctrl) + mockBeatPipeline := NewMockBeatPipeline(ctrl) + + mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil).Times(maxMessages) + + // Expect the one message returned to have been processed. + mockMsgHandler.EXPECT(). + ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). + Times(1). + DoAndReturn( + func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) (uint64, error) { + _, keepaliveCancel := context.WithCancel(ctx) + log := log.Named("sqs_s3_event") + acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.Track(0, 1) + <-acker.ctx.Done() + + return 1, nil + }) + + // Expect the client to be closed + mockClient.EXPECT().Close().Times(maxMessages) + + // Execute sqsReader and verify calls/state. + receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) + require.NoError(t, receiver.Receive(ctx)) + assert.Equal(t, maxMessages, receiver.workerSem.Available()) + }) + t.Run("retry after ReceiveMessage error", func(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), sqsRetryDelay+testTimeout) defer cancel() From f7dda957b3e595e1808b8edb333ca8dc7ffcf1ff Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 13 Mar 2024 08:31:18 +0900 Subject: [PATCH 38/43] track published, dropped and acked events --- x-pack/filebeat/input/awss3/sqs_acker.go | 53 +++++++++++++++++------- 1 file changed, 39 insertions(+), 14 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 20470b9fd13..fe3f8e1edf9 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -32,7 +32,8 @@ type EventACKTracker struct { ID uint64 EventsAcked *atomic.Uint64 - EventsTracked *atomic.Uint64 + EventsDropped *atomic.Uint64 + EventsPublished *atomic.Uint64 EventsToBeTracked *atomic.Uint64 ctx context.Context @@ -61,7 +62,8 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA deletionWg: deletionWg, mutex: new(sync.Mutex), EventsAcked: atomic.NewUint64(0), - EventsTracked: atomic.NewUint64(0), + EventsDropped: atomic.NewUint64(0), + EventsPublished: atomic.NewUint64(0), EventsToBeTracked: atomic.NewUint64(0), } @@ -113,7 +115,7 @@ func (a *EventACKTracker) FullyTracked() bool { return false } - return a.EventsTracked.Load() == eventsToBeTracked + return a.EventsDropped.Load()+a.EventsPublished.Load() == eventsToBeTracked } // FlushForSQS delete related SQS message @@ -122,7 +124,8 @@ func (a *EventACKTracker) FlushForSQS() { a.keepaliveCancel() a.keepaliveWg.Wait() - if a.EventsAcked.Load() == a.EventsTracked.Load() { + eventsPublished := a.EventsPublished.Load() + if eventsPublished > 0 && eventsPublished == a.EventsAcked.Load() { err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) if err != nil { a.log.Warnw("Failed deleting SQS message.", @@ -136,8 +139,10 @@ func (a *EventACKTracker) FlushForSQS() { } } else { a.log.Infow("Skipping deleting SQS message, not all events acked.", - "events_published", a.EventsTracked.Load(), "events_acked", a.EventsAcked.Load(), + "events_dropped", a.EventsDropped.Load(), + "events_published", a.EventsPublished.Load(), + "events_tracked", a.EventsToBeTracked.Load(), "message_id", *a.msg.MessageId, "elapsed_time_ns", time.Since(a.start)) @@ -147,10 +152,19 @@ func (a *EventACKTracker) FlushForSQS() { } -// Track decrements the number of total Events. -func (a *EventACKTracker) Track(acked int, total int) { - a.EventsAcked.Add(uint64(acked)) - a.EventsTracked.Add(uint64(total)) +// ACK increments the number of EventsAcked. +func (a *EventACKTracker) ACK() { + a.EventsAcked.Inc() +} + +// Drop increments the number of EventsDropped. +func (a *EventACKTracker) Drop() { + a.EventsDropped.Inc() +} + +// Published increments the number of EventsPublished. +func (a *EventACKTracker) Published() { + a.EventsPublished.Inc() } // NewEventACKHandler returns a beat ACKer that can receive callbacks when @@ -158,7 +172,17 @@ func (a *EventACKTracker) Track(acked int, total int) { // pointing to an eventACKTracker then it will invoke the trackers ACK() method // to decrement the number of pending ACKs. func NewEventACKHandler() beat.EventListener { - return acker.ConnectionOnly(newEventListener()) + return acker.ConnectionOnly( + acker.Combine( + newEventListener(), + acker.EventPrivateReporter(func(_ int, privates []interface{}) { + for _, private := range privates { + if acker, ok := private.(*EventACKTracker); ok { + acker.ACK() + } + } + }), + )) } func newEventListener() *eventListener { @@ -177,9 +201,10 @@ func (a *eventListener) AddEvent(event beat.Event, published bool) { return } - var acked int - if published { - acked = 1 + if !published { + acker.Drop() + } else { + acker.Published() } - acker.Track(acked, 1) + } From c9699f3ebddda7bd208c2b72d3d47a52da67295e Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 13 Mar 2024 08:31:26 +0900 Subject: [PATCH 39/43] fix tests --- x-pack/filebeat/input/awss3/input_benchmark_test.go | 3 ++- x-pack/filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker_test.go | 13 +++++++------ x-pack/filebeat/input/awss3/sqs_test.go | 4 ++-- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 184b32c5bb9..1e42b71d175 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -284,9 +284,10 @@ func (fp *fakePipeline) ackEvents() { for _, client := range fp.clients { for _, acker := range client.ackers { addedEvents := acker.EventsToBeTracked.Load() - for addedEvents > 0 && acker.EventsTracked.Load() != addedEvents { + for addedEvents > 0 && acker.EventsAcked.Load() != addedEvents { fp.pendingEvents.Dec() client.eventListener.AddEvent(beat.Event{Private: acker}, true) + client.eventListener.ACKEvents(1) } } } diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 4ac3f69ec76..62b663b2d62 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -339,7 +339,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint Publish(gomock.Any()). Do(func(event beat.Event) { events = append(events, event) - acker.Track(1, 1) + acker.ACK() }). Times(int(numEvents)), ) diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index 0bcd2e190ec..b96110b3795 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -33,7 +33,8 @@ func TestEventACKTrackerWithDelete(t *testing.T) { msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(1, 1) + acker.Published() + acker.ACK() <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) @@ -55,14 +56,14 @@ func TestEventACKTrackerNoDelete(t *testing.T) { mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(0, 1) + acker.Drop() <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKTrackerNoTrack(t *testing.T) { +func TestEventACKTrackerCancel(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -94,7 +95,6 @@ func TestEventACKHandlerNoDelete(t *testing.T) { // Create an ACK handler and simulate one ACKed event. ackHandler := NewEventACKHandler() ackHandler.AddEvent(beat.Event{Private: acker}, false) - ackHandler.ACKEvents(1) <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) @@ -145,7 +145,8 @@ func TestEventACKHandlerFullyTrackedWithDelete(t *testing.T) { msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(1, 1) + acker.Published() + acker.ACK() <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) @@ -171,7 +172,7 @@ func TestEventACKHandlerFullyTrackedNoDelete(t *testing.T) { mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(0, 1) + acker.Drop() <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 3aa8e30fcba..fecb7cccbfa 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -82,7 +82,7 @@ func TestSQSReceiver(t *testing.T) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(1, 1) + acker.ACK() <-acker.ctx.Done() return 1, nil @@ -149,7 +149,7 @@ func TestSQSReceiver(t *testing.T) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Track(0, 1) + acker.Drop() <-acker.ctx.Done() return 1, nil From 895a9d72e47f0a9ef89b2c12ef50fa2170d00a81 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Wed, 13 Mar 2024 11:18:31 +0900 Subject: [PATCH 40/43] wait for acked --- x-pack/filebeat/input/awss3/sqs_acker.go | 4 +++- x-pack/filebeat/input/awss3/sqs_test.go | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index fe3f8e1edf9..cdb4e74f260 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -115,7 +115,9 @@ func (a *EventACKTracker) FullyTracked() bool { return false } - return a.EventsDropped.Load()+a.EventsPublished.Load() == eventsToBeTracked + // This is eating its own tail: we should check for dropped+published, but then we won't wait for acked. + // Acked might not be equal to published? + return a.EventsDropped.Load()+a.EventsAcked.Load() == eventsToBeTracked } // FlushForSQS delete related SQS message diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index fecb7cccbfa..efcc300e96a 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -82,6 +82,7 @@ func TestSQSReceiver(t *testing.T) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.Published() acker.ACK() <-acker.ctx.Done() From 4dbd25b66fa7862ac40779ea93c94d609d1f2197 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Thu, 14 Mar 2024 16:49:45 +0900 Subject: [PATCH 41/43] No need to track dropped and published --- x-pack/filebeat/input/awss3/sqs_acker.go | 114 ++++++----------------- 1 file changed, 28 insertions(+), 86 deletions(-) diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index cdb4e74f260..49d256469b5 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -31,17 +31,13 @@ func init() { type EventACKTracker struct { ID uint64 - EventsAcked *atomic.Uint64 - EventsDropped *atomic.Uint64 - EventsPublished *atomic.Uint64 - EventsToBeTracked *atomic.Uint64 + eventsAcked atomic.Uint64 + eventsToBeAcked atomic.Uint64 ctx context.Context cancel context.CancelFunc deletionWg *sync.WaitGroup - mutex *sync.Mutex - msg *types.Message ReceiveCount int start time.Time @@ -56,15 +52,10 @@ type EventACKTracker struct { func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { ctx, cancel := context.WithCancel(ctx) acker := &EventACKTracker{ - ID: ackerIDCounter.Inc(), - ctx: ctx, - cancel: cancel, - deletionWg: deletionWg, - mutex: new(sync.Mutex), - EventsAcked: atomic.NewUint64(0), - EventsDropped: atomic.NewUint64(0), - EventsPublished: atomic.NewUint64(0), - EventsToBeTracked: atomic.NewUint64(0), + ID: ackerIDCounter.Inc(), + ctx: ctx, + cancel: cancel, + deletionWg: deletionWg, } go func() { @@ -86,19 +77,19 @@ func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventA } func (a *EventACKTracker) cancelAndFlush() { + a.deleteSQS() a.cancel() - a.FlushForSQS() } // MarkSQSProcessedWithData Every call after the first one is a no-op func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { // We want to execute the logic of this call only once, when the ack mutex was locked on init - if a.EventsToBeTracked.Load() > 0 { + if a.eventsToBeAcked.Load() > 0 { return } a.msg = msg - a.EventsToBeTracked = atomic.NewUint64(publishedEvent) + a.eventsToBeAcked.Store(publishedEvent) a.ReceiveCount = receiveCount a.start = start a.processingErr = processingErr @@ -110,63 +101,41 @@ func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, published } func (a *EventACKTracker) FullyTracked() bool { - eventsToBeTracked := a.EventsToBeTracked.Load() + eventsToBeTracked := a.eventsToBeAcked.Load() if eventsToBeTracked == 0 { return false } // This is eating its own tail: we should check for dropped+published, but then we won't wait for acked. // Acked might not be equal to published? - return a.EventsDropped.Load()+a.EventsAcked.Load() == eventsToBeTracked + return a.eventsAcked.Load() == eventsToBeTracked } -// FlushForSQS delete related SQS message -func (a *EventACKTracker) FlushForSQS() { +// deleteSQS delete related SQS message +func (a *EventACKTracker) deleteSQS() { // Stop keepalive visibility routine before deleting. a.keepaliveCancel() a.keepaliveWg.Wait() - eventsPublished := a.EventsPublished.Load() - if eventsPublished > 0 && eventsPublished == a.EventsAcked.Load() { - err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) - if err != nil { - a.log.Warnw("Failed deleting SQS message.", - "error", err, - "message_id", *a.msg.MessageId, - "elapsed_time_ns", time.Since(a.start)) - } else { - a.log.Debugw("Success deleting SQS message.", - "message_id", *a.msg.MessageId, - "elapsed_time_ns", time.Since(a.start)) - } + err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) + if err != nil { + a.log.Warnw("Failed deleting SQS message.", + "error", err, + "message_id", *a.msg.MessageId, + "elapsed_time_ns", time.Since(a.start)) } else { - a.log.Infow("Skipping deleting SQS message, not all events acked.", - "events_acked", a.EventsAcked.Load(), - "events_dropped", a.EventsDropped.Load(), - "events_published", a.EventsPublished.Load(), - "events_tracked", a.EventsToBeTracked.Load(), + a.log.Debugw("Success deleting SQS message.", "message_id", *a.msg.MessageId, "elapsed_time_ns", time.Since(a.start)) - } a.deletionWg.Done() } -// ACK increments the number of EventsAcked. +// ACK increments the number of eventsAcked. func (a *EventACKTracker) ACK() { - a.EventsAcked.Inc() -} - -// Drop increments the number of EventsDropped. -func (a *EventACKTracker) Drop() { - a.EventsDropped.Inc() -} - -// Published increments the number of EventsPublished. -func (a *EventACKTracker) Published() { - a.EventsPublished.Inc() + a.eventsAcked.Inc() } // NewEventACKHandler returns a beat ACKer that can receive callbacks when @@ -175,38 +144,11 @@ func (a *EventACKTracker) Published() { // to decrement the number of pending ACKs. func NewEventACKHandler() beat.EventListener { return acker.ConnectionOnly( - acker.Combine( - newEventListener(), - acker.EventPrivateReporter(func(_ int, privates []interface{}) { - for _, private := range privates { - if acker, ok := private.(*EventACKTracker); ok { - acker.ACK() - } + acker.EventPrivateReporter(func(_ int, privates []interface{}) { + for _, private := range privates { + if acker, ok := private.(*EventACKTracker); ok { + acker.ACK() } - }), - )) -} - -func newEventListener() *eventListener { - return &eventListener{} -} - -type eventListener struct{} - -func (a *eventListener) ACKEvents(n int) {} - -func (a *eventListener) ClientClosed() {} - -func (a *eventListener) AddEvent(event beat.Event, published bool) { - acker, ok := event.Private.(*EventACKTracker) - if !ok { - return - } - - if !published { - acker.Drop() - } else { - acker.Published() - } - + } + })) } From 8c3c39a69efe506763aaad50efc5e8280cc0f7c5 Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Thu, 14 Mar 2024 16:50:17 +0900 Subject: [PATCH 42/43] fix tests --- .../input/awss3/input_benchmark_test.go | 4 +- .../filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs_acker_test.go | 83 +------------------ x-pack/filebeat/input/awss3/sqs_test.go | 65 +-------------- 4 files changed, 8 insertions(+), 146 deletions(-) diff --git a/x-pack/filebeat/input/awss3/input_benchmark_test.go b/x-pack/filebeat/input/awss3/input_benchmark_test.go index 1e42b71d175..2a7c21ace0b 100644 --- a/x-pack/filebeat/input/awss3/input_benchmark_test.go +++ b/x-pack/filebeat/input/awss3/input_benchmark_test.go @@ -283,8 +283,8 @@ type fakePipeline struct { func (fp *fakePipeline) ackEvents() { for _, client := range fp.clients { for _, acker := range client.ackers { - addedEvents := acker.EventsToBeTracked.Load() - for addedEvents > 0 && acker.EventsAcked.Load() != addedEvents { + addedEvents := acker.eventsToBeAcked.Load() + for addedEvents > 0 && acker.eventsAcked.Load() != addedEvents { fp.pendingEvents.Dec() client.eventListener.AddEvent(beat.Event{Private: acker}, true) client.eventListener.ACKEvents(1) diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 62b663b2d62..31d5b179707 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -328,7 +328,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) acker.MarkSQSProcessedWithData(&msg, numEvents, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.EventsToBeTracked.Add(numEvents) + acker.eventsToBeAcked.Add(numEvents) var events []beat.Event gomock.InOrder( diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index b96110b3795..cbbb8799bfc 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -17,7 +17,7 @@ import ( "github.com/elastic/beats/v7/libbeat/beat" ) -func TestEventACKTrackerWithDelete(t *testing.T) { +func TestEventACKTracker(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -33,7 +33,6 @@ func TestEventACKTrackerWithDelete(t *testing.T) { msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Published() acker.ACK() <-acker.ctx.Done() @@ -41,28 +40,6 @@ func TestEventACKTrackerWithDelete(t *testing.T) { assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKTrackerNoDelete(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - - deletionWg := new(sync.WaitGroup) - deletionWg.Add(1) - - acker := NewEventACKTracker(ctx, deletionWg) - _, keepaliveCancel := context.WithCancel(ctx) - log := log.Named("sqs_s3_event") - ctrl, _ := gomock.WithContext(ctx, t) - defer ctrl.Finish() - mockMsgHandler := NewMockSQSProcessor(ctrl) - msg := newSQSMessage(newS3Event("log.json")) - acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Drop() - <-acker.ctx.Done() - - assert.EqualValues(t, true, acker.FullyTracked()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) -} - func TestEventACKTrackerCancel(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -76,32 +53,7 @@ func TestEventACKTrackerCancel(t *testing.T) { assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKHandlerNoDelete(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - - // Create acker. Add one ACK to event listener. - deletionWg := new(sync.WaitGroup) - deletionWg.Add(1) - acker := NewEventACKTracker(ctx, deletionWg) - _, keepaliveCancel := context.WithCancel(ctx) - log := log.Named("sqs_s3_event") - ctrl, _ := gomock.WithContext(ctx, t) - defer ctrl.Finish() - mockMsgHandler := NewMockSQSProcessor(ctrl) - msg := newSQSMessage(newS3Event("log.json")) - acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - - // Create an ACK handler and simulate one ACKed event. - ackHandler := NewEventACKHandler() - ackHandler.AddEvent(beat.Event{Private: acker}, false) - <-acker.ctx.Done() - - assert.EqualValues(t, true, acker.FullyTracked()) - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) -} - -func TestEventACKHandlerWithDelete(t *testing.T) { +func TestEventACKHandler(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -128,7 +80,7 @@ func TestEventACKHandlerWithDelete(t *testing.T) { assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) } -func TestEventACKHandlerFullyTrackedWithDelete(t *testing.T) { +func TestEventACKHandlerFullyAcked(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) @@ -145,40 +97,13 @@ func TestEventACKHandlerFullyTrackedWithDelete(t *testing.T) { msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Published() acker.ACK() <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) - acker.EventsToBeTracked.Inc() - - assert.EqualValues(t, false, acker.FullyTracked()) -} -func TestEventACKHandlerFullyTrackedNoDelete(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - t.Cleanup(cancel) - - // Create acker. Add one Track. - deletionWg := new(sync.WaitGroup) - deletionWg.Add(1) - - acker := NewEventACKTracker(ctx, deletionWg) - _, keepaliveCancel := context.WithCancel(ctx) - log := log.Named("sqs_s3_event") - ctrl, _ := gomock.WithContext(ctx, t) - defer ctrl.Finish() - mockMsgHandler := NewMockSQSProcessor(ctrl) - msg := newSQSMessage(newS3Event("log.json")) - acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Drop() - <-acker.ctx.Done() - assert.EqualValues(t, true, acker.FullyTracked()) - - assert.ErrorIs(t, acker.ctx.Err(), context.Canceled) - - acker.EventsToBeTracked.Inc() + acker.eventsToBeAcked.Inc() assert.EqualValues(t, false, acker.FullyTracked()) } diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index efcc300e96a..3725794c10b 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -36,7 +36,7 @@ func TestSQSReceiver(t *testing.T) { const maxMessages = 5 - t.Run("ReceiveMessage success with delete", func(t *testing.T) { + t.Run("ReceiveMessage success", func(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), testTimeout) defer cancel() @@ -82,10 +82,8 @@ func TestSQSReceiver(t *testing.T) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Published() acker.ACK() <-acker.ctx.Done() - return 1, nil }) @@ -104,67 +102,6 @@ func TestSQSReceiver(t *testing.T) { assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) - t.Run("ReceiveMessage success no delete", func(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), testTimeout) - defer cancel() - - ctrl, ctx := gomock.WithContext(ctx, t) - defer ctrl.Finish() - mockAPI := NewMockSQSAPI(ctrl) - mockMsgHandler := NewMockSQSProcessor(ctrl) - msg := newSQSMessage(newS3Event("log.json")) - - gomock.InOrder( - // Initial ReceiveMessage for maxMessages. - mockAPI.EXPECT(). - ReceiveMessage(gomock.Any(), gomock.Eq(maxMessages)). - Times(1). - DoAndReturn(func(_ context.Context, _ int) ([]types.Message, error) { - // Return single message. - return []types.Message{msg}, nil - }), - - // Follow up ReceiveMessages for either maxMessages-1 or maxMessages - // depending on how long processing of previous message takes. - mockAPI.EXPECT(). - ReceiveMessage(gomock.Any(), gomock.Any()). - Times(1). - DoAndReturn(func(_ context.Context, _ int) ([]types.Message, error) { - // Stop the test. - cancel() - return nil, nil - }), - ) - - mockClient := NewMockBeatClient(ctrl) - mockBeatPipeline := NewMockBeatPipeline(ctrl) - - mockBeatPipeline.EXPECT().ConnectWith(gomock.Any()).Return(mockClient, nil).Times(maxMessages) - - // Expect the one message returned to have been processed. - mockMsgHandler.EXPECT(). - ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). - Times(1). - DoAndReturn( - func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) (uint64, error) { - _, keepaliveCancel := context.WithCancel(ctx) - log := log.Named("sqs_s3_event") - acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) - acker.Drop() - <-acker.ctx.Done() - - return 1, nil - }) - - // Expect the client to be closed - mockClient.EXPECT().Close().Times(maxMessages) - - // Execute sqsReader and verify calls/state. - receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) - require.NoError(t, receiver.Receive(ctx)) - assert.Equal(t, maxMessages, receiver.workerSem.Available()) - }) - t.Run("retry after ReceiveMessage error", func(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), sqsRetryDelay+testTimeout) defer cancel() From fe71445448bbc97f8c220afa8b5edc297ee2912e Mon Sep 17 00:00:00 2001 From: Andrea Spacca Date: Tue, 19 Mar 2024 08:17:06 +0900 Subject: [PATCH 43/43] cr fixes and other cleaning --- .../filebeat/docs/inputs/input-aws-s3.asciidoc | 17 +++++++++-------- x-pack/filebeat/input/awss3/interfaces.go | 2 +- x-pack/filebeat/input/awss3/metrics.go | 4 ---- .../input/awss3/mock_interfaces_test.go | 8 ++++---- x-pack/filebeat/input/awss3/s3_objects_test.go | 2 +- x-pack/filebeat/input/awss3/sqs.go | 13 ++++--------- x-pack/filebeat/input/awss3/sqs_acker.go | 6 +++++- x-pack/filebeat/input/awss3/sqs_acker_test.go | 6 +++--- x-pack/filebeat/input/awss3/sqs_s3_event.go | 4 ++-- .../filebeat/input/awss3/sqs_s3_event_test.go | 12 ++++++------ x-pack/filebeat/input/awss3/sqs_test.go | 8 +++----- 11 files changed, 38 insertions(+), 44 deletions(-) diff --git a/x-pack/filebeat/docs/inputs/input-aws-s3.asciidoc b/x-pack/filebeat/docs/inputs/input-aws-s3.asciidoc index 794a51de081..fd4fb472739 100644 --- a/x-pack/filebeat/docs/inputs/input-aws-s3.asciidoc +++ b/x-pack/filebeat/docs/inputs/input-aws-s3.asciidoc @@ -120,7 +120,7 @@ characters. This only applies to non-JSON logs. See <<_encoding_3>>. ==== `decoding` The file decoding option is used to specify a codec that will be used to -decode the file contents. This can apply to any file stream data. +decode the file contents. This can apply to any file stream data. An example config is shown below: [source,yaml] @@ -131,17 +131,17 @@ An example config is shown below: Currently supported codecs are given below:- 1. <>: This codec decodes parquet compressed data streams. - + [id="attrib-decoding-parquet"] [float] ==== `the parquet codec` The `parquet` codec is used to decode parquet compressed data streams. Only enabling the codec will use the default codec options. The parquet codec supports -two sub attributes which can make parquet decoding more efficient. The `batch_size` attribute and +two sub attributes which can make parquet decoding more efficient. The `batch_size` attribute and the `process_parallel` attribute. The `batch_size` attribute can be used to specify the number of -records to read from the parquet stream at a time. By default the `batch size` is set to `1` and -`process_parallel` is set to `false`. If the `process_parallel` attribute is set to `true` then functions -which read multiple columns will read those columns in parallel from the parquet stream with a +records to read from the parquet stream at a time. By default the `batch size` is set to `1` and +`process_parallel` is set to `false`. If the `process_parallel` attribute is set to `true` then functions +which read multiple columns will read those columns in parallel from the parquet stream with a number of readers equal to the number of columns. Setting `process_parallel` to `true` will greatly increase the rate of processing at the cost of increased memory usage. Having a larger `batch_size` also helps to increase the rate of processing. An example config is shown below: @@ -265,8 +265,9 @@ The default is `10 MiB`. [float] ==== `max_number_of_messages` -The maximum number of SQS messages that can be inflight at any time. Defaults -to 5. +The maximum number of SQS messages to receive during a single `ReceiveMessage` call. +As many goroutine workers will be started to process the SQS messages received. +Defaults to 5. [id="input-{type}-parsers"] [float] diff --git a/x-pack/filebeat/input/awss3/interfaces.go b/x-pack/filebeat/input/awss3/interfaces.go index a5c732806a3..762af280cfa 100644 --- a/x-pack/filebeat/input/awss3/interfaces.go +++ b/x-pack/filebeat/input/awss3/interfaces.go @@ -66,7 +66,7 @@ type sqsProcessor interface { // given message and is responsible for updating the message's visibility // timeout while it is being processed and for deleting it when processing // completes successfully. - ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) (uint64, error) + ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time, metrics *inputMetrics) (uint64, error) DeleteSQS(msg *types.Message, receiveCount int, processingErr error, handles []s3ObjectHandler) error } diff --git a/x-pack/filebeat/input/awss3/metrics.go b/x-pack/filebeat/input/awss3/metrics.go index ecea902c332..c697a0339fa 100644 --- a/x-pack/filebeat/input/awss3/metrics.go +++ b/x-pack/filebeat/input/awss3/metrics.go @@ -84,8 +84,6 @@ func (m *inputMetrics) Close() { // must be used to call endSQSWorker when the worker finishes. It also // increments the sqsMessagesInflight counter. func (m *inputMetrics) beginSQSWorker() (id uint64) { - m.sqsMessagesInflight.Inc() - m.sqsWorkerUtilizationMutex.Lock() defer m.sqsWorkerUtilizationMutex.Unlock() m.sqsWorkerIDCounter++ @@ -98,8 +96,6 @@ func (m *inputMetrics) beginSQSWorker() (id uint64) { // It also decrements the sqsMessagesInflight counter and // sqsMessageProcessingTime histogram. func (m *inputMetrics) endSQSWorker(id uint64) { - m.sqsMessagesInflight.Dec() - m.sqsWorkerUtilizationMutex.Lock() defer m.sqsWorkerUtilizationMutex.Unlock() now := currentTime() diff --git a/x-pack/filebeat/input/awss3/mock_interfaces_test.go b/x-pack/filebeat/input/awss3/mock_interfaces_test.go index 7131d736825..d47a557567f 100644 --- a/x-pack/filebeat/input/awss3/mock_interfaces_test.go +++ b/x-pack/filebeat/input/awss3/mock_interfaces_test.go @@ -291,18 +291,18 @@ func (mr *MockSQSProcessorMockRecorder) DeleteSQS(msg, receiveCount, processingE } // ProcessSQS mocks base method. -func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) (uint64, error) { +func (m *MockSQSProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time, metrics *inputMetrics) (uint64, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg, client, acker, start) + ret := m.ctrl.Call(m, "ProcessSQS", ctx, msg, client, acker, start, metrics) ret0, _ := ret[0].(uint64) ret1, _ := ret[1].(error) return ret0, ret1 } // ProcessSQS indicates an expected call of ProcessSQS. -func (mr *MockSQSProcessorMockRecorder) ProcessSQS(ctx, msg, client, acker, start interface{}) *gomock.Call { +func (mr *MockSQSProcessorMockRecorder) ProcessSQS(ctx, msg, client, acker, start, metrics interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessSQS", reflect.TypeOf((*MockSQSProcessor)(nil).ProcessSQS), ctx, msg, client, acker, start) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ProcessSQS", reflect.TypeOf((*MockSQSProcessor)(nil).ProcessSQS), ctx, msg, client, acker, start, metrics) } // MockS3API is a mock of s3API interface. diff --git a/x-pack/filebeat/input/awss3/s3_objects_test.go b/x-pack/filebeat/input/awss3/s3_objects_test.go index 31d5b179707..fa08b0a5b97 100644 --- a/x-pack/filebeat/input/awss3/s3_objects_test.go +++ b/x-pack/filebeat/input/awss3/s3_objects_test.go @@ -327,7 +327,7 @@ func _testProcessS3Object(t testing.TB, file, contentType string, numEvents uint log := log.Named("sqs_s3_event") mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) - acker.MarkSQSProcessedWithData(&msg, numEvents, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.MarkSQSProcessedWithData(&msg, numEvents, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log, newInputMetrics("", nil, 0)) acker.eventsToBeAcked.Add(numEvents) var events []beat.Event diff --git a/x-pack/filebeat/input/awss3/sqs.go b/x-pack/filebeat/input/awss3/sqs.go index d041c11140a..18f9917fcd3 100644 --- a/x-pack/filebeat/input/awss3/sqs.go +++ b/x-pack/filebeat/input/awss3/sqs.go @@ -18,7 +18,6 @@ import ( "github.com/aws/aws-sdk-go-v2/service/sqs/types" "github.com/elastic/beats/v7/libbeat/beat" - awscommon "github.com/elastic/beats/v7/x-pack/libbeat/common/aws" "github.com/elastic/elastic-agent-libs/logp" ) @@ -29,7 +28,6 @@ const ( type sqsReader struct { maxMessagesInflight int - workerSem *awscommon.Sem sqs sqsAPI pipeline beat.Pipeline msgHandler sqsProcessor @@ -44,7 +42,6 @@ func newSQSReader(log *logp.Logger, metrics *inputMetrics, sqs sqsAPI, maxMessag } return &sqsReader{ maxMessagesInflight: maxMessagesInflight, - workerSem: awscommon.NewSem(maxMessagesInflight), sqs: sqs, pipeline: pipeline, msgHandler: msgHandler, @@ -64,9 +61,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { workersChan := make(chan processingData, r.maxMessagesInflight) deletionWg := new(sync.WaitGroup) - deletionWaiter := new(atomic.Bool) - - var clientsMutex sync.Mutex + deletionWaiter := atomic.MakeBool(true) clients := make(map[uint64]beat.Client, r.maxMessagesInflight) // Start a fixed amount of goroutines that will process all the SQS messages sent to the workersChan asynchronously. @@ -83,9 +78,7 @@ func (r *sqsReader) Receive(ctx context.Context) error { }, }) - clientsMutex.Lock() clients[id] = client - clientsMutex.Unlock() if err != nil { r.log.Warnw("Failed setting up worker.", @@ -121,7 +114,9 @@ func (r *sqsReader) Receive(ctx context.Context) error { deletionWg.Add(1) deletionWaiter.Swap(false) - eventsCreatedTotal, err := r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start) + r.metrics.sqsMessagesInflight.Inc() + + eventsCreatedTotal, err := r.msgHandler.ProcessSQS(ctx, &msg, client, acker, start, r.metrics) // No Track will be invoked by the client event listener, deletionWg.Done() has to be called here if eventsCreatedTotal == 0 { deletionWg.Done() diff --git a/x-pack/filebeat/input/awss3/sqs_acker.go b/x-pack/filebeat/input/awss3/sqs_acker.go index 49d256469b5..bd53905a9c0 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker.go +++ b/x-pack/filebeat/input/awss3/sqs_acker.go @@ -47,6 +47,7 @@ type EventACKTracker struct { keepaliveWg *sync.WaitGroup msgHandler sqsProcessor log *logp.Logger + metrics *inputMetrics } func NewEventACKTracker(ctx context.Context, deletionWg *sync.WaitGroup) *EventACKTracker { @@ -82,7 +83,7 @@ func (a *EventACKTracker) cancelAndFlush() { } // MarkSQSProcessedWithData Every call after the first one is a no-op -func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger) { +func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, publishedEvent uint64, receiveCount int, start time.Time, processingErr error, handles []s3ObjectHandler, keepaliveCancel context.CancelFunc, keepaliveWg *sync.WaitGroup, msgHandler sqsProcessor, log *logp.Logger, metrics *inputMetrics) { // We want to execute the logic of this call only once, when the ack mutex was locked on init if a.eventsToBeAcked.Load() > 0 { return @@ -98,6 +99,7 @@ func (a *EventACKTracker) MarkSQSProcessedWithData(msg *types.Message, published a.keepaliveWg = keepaliveWg a.msgHandler = msgHandler a.log = log + a.metrics = metrics } func (a *EventACKTracker) FullyTracked() bool { @@ -117,6 +119,8 @@ func (a *EventACKTracker) deleteSQS() { a.keepaliveCancel() a.keepaliveWg.Wait() + a.metrics.sqsMessagesInflight.Dec() + err := a.msgHandler.DeleteSQS(a.msg, a.ReceiveCount, a.processingErr, a.Handles) if err != nil { a.log.Warnw("Failed deleting SQS message.", diff --git a/x-pack/filebeat/input/awss3/sqs_acker_test.go b/x-pack/filebeat/input/awss3/sqs_acker_test.go index cbbb8799bfc..c14138f26b2 100644 --- a/x-pack/filebeat/input/awss3/sqs_acker_test.go +++ b/x-pack/filebeat/input/awss3/sqs_acker_test.go @@ -32,7 +32,7 @@ func TestEventACKTracker(t *testing.T) { mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) - acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log, newInputMetrics("", nil, 0)) acker.ACK() <-acker.ctx.Done() @@ -68,7 +68,7 @@ func TestEventACKHandler(t *testing.T) { mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) - acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log, newInputMetrics("", nil, 0)) // Create an ACK handler and simulate one ACKed event. ackHandler := NewEventACKHandler() @@ -96,7 +96,7 @@ func TestEventACKHandlerFullyAcked(t *testing.T) { mockMsgHandler := NewMockSQSProcessor(ctrl) msg := newSQSMessage(newS3Event("log.json")) mockMsgHandler.EXPECT().DeleteSQS(gomock.Eq(&msg), gomock.Eq(-1), gomock.Nil(), gomock.Nil()).Return(nil) - acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.MarkSQSProcessedWithData(&msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log, newInputMetrics("", nil, 0)) acker.ACK() <-acker.ctx.Done() assert.EqualValues(t, true, acker.FullyTracked()) diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event.go b/x-pack/filebeat/input/awss3/sqs_s3_event.go index 04f22b13319..87dc4d88b16 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event.go @@ -163,7 +163,7 @@ func (p *sqsS3EventProcessor) DeleteSQS(msg *types.Message, receiveCount int, pr return fmt.Errorf("failed deleting SQS message (it will return to queue after visibility timeout): %w", processingErr) } -func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time) (uint64, error) { +func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message, client beat.Client, acker *EventACKTracker, start time.Time, metrics *inputMetrics) (uint64, error) { keepaliveCtx, keepaliveCancel := context.WithCancel(ctx) // Start SQS keepalive worker. @@ -190,7 +190,7 @@ func (p *sqsS3EventProcessor) ProcessSQS(ctx context.Context, msg *types.Message eventsPublishedTotal, handles, processingErr := p.processS3Events(ctx, log, *msg.Body, client, acker) p.metrics.sqsMessagesProcessedTotal.Inc() - acker.MarkSQSProcessedWithData(msg, eventsPublishedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log) + acker.MarkSQSProcessedWithData(msg, eventsPublishedTotal, receiveCount, start, processingErr, handles, keepaliveCancel, &keepaliveWg, p, p.log, metrics) return eventsPublishedTotal, processingErr } diff --git a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go index 0e162227512..953a50cb7f7 100644 --- a/x-pack/filebeat/input/awss3/sqs_s3_event_test.go +++ b/x-pack/filebeat/input/awss3/sqs_s3_event_test.go @@ -52,7 +52,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now(), newInputMetrics("", nil, 0)) require.NoError(t, processingErr) require.Equal(t, expectedEventsPublishedTotal, eventsPublishedTotal) @@ -79,7 +79,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &invalidBodyMsg, mockClient, acker, time.Now(), newInputMetrics("", nil, 0)) t.Log(processingErr) require.Error(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) @@ -104,7 +104,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &emptyRecordsMsg, mockClient, acker, time.Now(), newInputMetrics("", nil, 0)) require.NoError(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) @@ -138,7 +138,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, visibilityTimeout, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now(), newInputMetrics("", nil, 0)) require.NoError(t, processingErr) require.Equal(t, uint64(1), eventsPublishedTotal) @@ -168,7 +168,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now(), newInputMetrics("", nil, 0)) t.Log(processingErr) require.Error(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) @@ -199,7 +199,7 @@ func TestSQSS3EventProcessor(t *testing.T) { p := newSQSS3EventProcessor(logp.NewLogger(inputName), nil, mockAPI, nil, time.Minute, 5, mockS3HandlerFactory) acker := NewEventACKTracker(ctx, new(sync.WaitGroup)) - eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now()) + eventsPublishedTotal, processingErr := p.ProcessSQS(ctx, &msg, mockClient, acker, time.Now(), newInputMetrics("", nil, 0)) t.Log(processingErr) require.Error(t, processingErr) require.Equal(t, uint64(0), eventsPublishedTotal) diff --git a/x-pack/filebeat/input/awss3/sqs_test.go b/x-pack/filebeat/input/awss3/sqs_test.go index 3725794c10b..ab2be37f7e1 100644 --- a/x-pack/filebeat/input/awss3/sqs_test.go +++ b/x-pack/filebeat/input/awss3/sqs_test.go @@ -75,13 +75,13 @@ func TestSQSReceiver(t *testing.T) { // Expect the one message returned to have been processed. mockMsgHandler.EXPECT(). - ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any()). + ProcessSQS(gomock.Any(), gomock.Eq(&msg), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). Times(1). DoAndReturn( - func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time) (uint64, error) { + func(ctx context.Context, msg *types.Message, _ beat.Client, acker *EventACKTracker, _ time.Time, metrics *inputMetrics) (uint64, error) { _, keepaliveCancel := context.WithCancel(ctx) log := log.Named("sqs_s3_event") - acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log) + acker.MarkSQSProcessedWithData(msg, 1, -1, time.Now(), nil, nil, keepaliveCancel, new(sync.WaitGroup), mockMsgHandler, log, metrics) acker.ACK() <-acker.ctx.Done() return 1, nil @@ -99,7 +99,6 @@ func TestSQSReceiver(t *testing.T) { // Execute sqsReader and verify calls/state. receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) require.NoError(t, receiver.Receive(ctx)) - assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) t.Run("retry after ReceiveMessage error", func(t *testing.T) { @@ -140,7 +139,6 @@ func TestSQSReceiver(t *testing.T) { // Execute SQSReceiver and verify calls/state. receiver := newSQSReader(logp.NewLogger(inputName), nil, mockAPI, maxMessages, mockMsgHandler, mockBeatPipeline) require.NoError(t, receiver.Receive(ctx)) - assert.Equal(t, maxMessages, receiver.workerSem.Available()) }) }