From 4cae96d838a329e8bde447d1a6310dfa784a050c Mon Sep 17 00:00:00 2001 From: "siddarth.msv" <82795818+Sidddddarth@users.noreply.github.com> Date: Tue, 3 Sep 2024 00:22:38 +0530 Subject: [PATCH 01/29] chore: send connection config to router transform (#4903) --- backend-config/backend_config_test.go | 73 +++++++++++++++++++ backend-config/types.go | 9 +++ .../testdata/mtGatewayTest02.json | 59 ++++++++++++++- processor/processor.go | 24 +++++- processor/trackingplan.go | 3 +- processor/transformer/transformer.go | 1 + router/handle.go | 1 + router/handle_lifecycle.go | 12 +++ router/router_test.go | 52 ++++++++----- router/transformer/transformer.go | 19 ++++- router/types/types.go | 11 +++ router/worker.go | 15 +++- 12 files changed, 252 insertions(+), 27 deletions(-) diff --git a/backend-config/backend_config_test.go b/backend-config/backend_config_test.go index bb299935ff..7b84523702 100644 --- a/backend-config/backend_config_test.go +++ b/backend-config/backend_config_test.go @@ -61,6 +61,48 @@ var sampleBackendConfig = ConfigT{ }, } +var sampleConfigWithConnection = ConfigT{ + WorkspaceID: sampleWorkspaceID, + Sources: []SourceT{ + { + ID: "1", + WriteKey: "d", + Enabled: false, + }, { + ID: "2", + WriteKey: "d2", + Enabled: false, + Destinations: []DestinationT{ + { + ID: "d1", + Name: "processor Disabled", + IsProcessorEnabled: false, + }, { + ID: "d2", + Name: "processor Enabled", + IsProcessorEnabled: true, + }, + }, + }, + }, + Connections: map[string]Connection{ + "1": { + SourceID: "2", + DestinationID: "d1", + Enabled: true, + Config: map[string]interface{}{"key": "value"}, + ProcessorEnabled: false, + }, + "2": { + SourceID: "2", + DestinationID: "d2", + Enabled: true, + Config: map[string]interface{}{"key2": "value2"}, + ProcessorEnabled: true, + }, + }, +} + // This configuration is assumed by all gateway tests and, is returned on Subscribe of mocked backend config var sampleFilteredSources = ConfigT{ Sources: []SourceT{ @@ -264,6 +306,37 @@ func TestConfigUpdate(t *testing.T) { require.Equal(t, (<-chProcess).Data, map[string]ConfigT{workspaces: sampleFilteredSources}) require.Equal(t, (<-chBackend).Data, map[string]ConfigT{workspaces: sampleBackendConfig}) }) + + t.Run("new config with connections", func(t *testing.T) { + var ( + ctrl = gomock.NewController(t) + ctx, cancel = context.WithCancel(context.Background()) + workspaces = "foo" + cacheStore = cache.NewMockCache(ctrl) + ) + defer ctrl.Finish() + defer cancel() + + wc := NewMockworkspaceConfig(ctrl) + wc.EXPECT().Get(gomock.Eq(ctx)).Return(map[string]ConfigT{workspaces: sampleConfigWithConnection}, nil).Times(1) + + var pubSub pubsub.PublishSubscriber + bc := &backendConfigImpl{ + eb: &pubSub, + workspaceConfig: wc, + cache: cacheStore, + } + bc.curSourceJSON = map[string]ConfigT{workspaces: sampleBackendConfig2} + + chProcess := pubSub.Subscribe(ctx, string(TopicProcessConfig)) + chBackend := pubSub.Subscribe(ctx, string(TopicBackendConfig)) + + bc.configUpdate(ctx) + require.True(t, bc.initialized) + require.Equal(t, (<-chProcess).Data, map[string]ConfigT{workspaces: sampleFilteredSources}) + require.Equal(t, (<-chBackend).Data, map[string]ConfigT{workspaces: sampleConfigWithConnection}) + require.Equal(t, bc.curSourceJSON[workspaces].Connections, sampleConfigWithConnection.Connections) + }) } func TestFilterProcessorEnabledDestinations(t *testing.T) { diff --git a/backend-config/types.go b/backend-config/types.go index b764b30875..ad708c27aa 100644 --- a/backend-config/types.go +++ b/backend-config/types.go @@ -94,6 +94,15 @@ type ConfigT struct { Settings Settings `json:"settings"` UpdatedAt time.Time `json:"updatedAt"` Credentials map[string]Credential `json:"credentials"` + Connections map[string]Connection `json:"connections"` +} + +type Connection struct { + SourceID string `json:"sourceId"` + DestinationID string `json:"destinationId"` + Enabled bool `json:"enabled"` + Config map[string]interface{} `json:"config"` + ProcessorEnabled bool `json:"processorEnabled"` } func (c *ConfigT) SourcesMap() map[string]*SourceT { diff --git a/integration_test/multi_tenant_test/testdata/mtGatewayTest02.json b/integration_test/multi_tenant_test/testdata/mtGatewayTest02.json index 1ef6fe27fa..3fcd793da0 100644 --- a/integration_test/multi_tenant_test/testdata/mtGatewayTest02.json +++ b/integration_test/multi_tenant_test/testdata/mtGatewayTest02.json @@ -53,5 +53,62 @@ } ] } - ] + ], + "connections": { + "connedctionID": { + "sourceId": "xxxyyyzzEaEurW247ad9WYZLUyk", + "destinationId": "xxxyyyzzP9kQfzOoKd1tuxchYAG", + "enabled": true, + "config": { + "mapping": { + "enabled": true, + "mapping": { + "alias": { + "enabled": true, + "mapping": { + "email": "email", + "name": "name" + } + }, + "group": { + "enabled": true, + "mapping": { + "groupId": "groupId", + "groupName": "groupName" + } + }, + "identify": { + "enabled": true, + "mapping": { + "email": "email", + "name": "name" + } + }, + "page": { + "enabled": true, + "mapping": { + "pageName": "pageName", + "pageType": "pageType" + } + }, + "screen": { + "enabled": true, + "mapping": { + "screenName": "screenName", + "screenType": "screenType" + } + }, + "track": { + "enabled": true, + "mapping": { + "eventName": "eventName", + "eventValue": "eventValue" + } + } + } + } + }, + "processorEnabled": true + } + } } diff --git a/processor/processor.go b/processor/processor.go index 78e0de3265..08b43fcf6b 100644 --- a/processor/processor.go +++ b/processor/processor.go @@ -139,6 +139,7 @@ type Handle struct { sourceIdSourceMap map[string]backendconfig.SourceT workspaceLibrariesMap map[string]backendconfig.LibrariesT oneTrustConsentCategoriesMap map[string][]string + connectionConfigMap map[connection]backendconfig.Connection ketchConsentCategoriesMap map[string][]string destGenericConsentManagementMap map[string]map[string]GenericConsentManagementProviderData batchDestinations []string @@ -816,6 +817,10 @@ func (proc *Handle) loadReloadableConfig(defaultPayloadLimit int64, defaultMaxEv proc.config.captureEventNameStats = config.GetReloadableBoolVar(false, "Processor.Stats.captureEventName") } +type connection struct { + sourceID, destinationID string +} + func (proc *Handle) backendConfigSubscriber(ctx context.Context) { var initDone bool ch := proc.backendConfig.Subscribe(ctx, backendconfig.TopicProcessConfig) @@ -831,8 +836,12 @@ func (proc *Handle) backendConfigSubscriber(ctx context.Context) { eventAuditEnabled = make(map[string]bool) credentialsMap = make(map[string][]transformer.Credential) nonEventStreamSources = make(map[string]bool) + connectionConfigMap = make(map[connection]backendconfig.Connection) ) for workspaceID, wConfig := range config { + for _, conn := range wConfig.Connections { + connectionConfigMap[connection{sourceID: conn.SourceID, destinationID: conn.DestinationID}] = conn + } for i := range wConfig.Sources { source := &wConfig.Sources[i] sourceIdSourceMap[source.ID] = *source @@ -866,6 +875,7 @@ func (proc *Handle) backendConfigSubscriber(ctx context.Context) { }) } proc.config.configSubscriberLock.Lock() + proc.config.connectionConfigMap = connectionConfigMap proc.config.oneTrustConsentCategoriesMap = oneTrustConsentCategoriesMap proc.config.ketchConsentCategoriesMap = ketchConsentCategoriesMap proc.config.destGenericConsentManagementMap = destGenericConsentManagementMap @@ -889,6 +899,12 @@ func (proc *Handle) getWorkspaceLibraries(workspaceID string) backendconfig.Libr return proc.config.workspaceLibrariesMap[workspaceID] } +func (proc *Handle) getConnectionConfig(conn connection) backendconfig.Connection { + proc.config.configSubscriberLock.RLock() + defer proc.config.configSubscriberLock.RUnlock() + return proc.config.connectionConfigMap[conn] +} + func (proc *Handle) getSourceBySourceID(sourceId string) (*backendconfig.SourceT, error) { var err error proc.config.configSubscriberLock.RLock() @@ -1091,6 +1107,7 @@ func (proc *Handle) getTransformerEvents( commonMetaData *transformer.Metadata, eventsByMessageID map[string]types.SingularEventWithReceivedAt, destination *backendconfig.DestinationT, + connection backendconfig.Connection, inPU, pu string, ) ( []transformer.TransformerEvent, @@ -1153,6 +1170,7 @@ func (proc *Handle) getTransformerEvents( Message: userTransformedEvent.Output, Metadata: *eventMetadata, Destination: *destination, + Connection: connection, Credentials: proc.config.credentialsMap[commonMetaData.WorkspaceID], } eventsToTransform = append(eventsToTransform, updatedEvent) @@ -2005,6 +2023,7 @@ func (proc *Handle) processJobsForDest(partition string, subJobs subJob) *transf for idx := range enabledDestinationsList { destination := &enabledDestinationsList[idx] shallowEventCopy := transformer.TransformerEvent{} + shallowEventCopy.Connection = proc.getConnectionConfig(connection{sourceID: sourceId, destinationID: destination.ID}) shallowEventCopy.Message = singularEvent shallowEventCopy.Destination = *destination shallowEventCopy.Libraries = workspaceLibraries @@ -2499,6 +2518,7 @@ func (proc *Handle) transformSrcDest( sourceID, destID := getSourceAndDestIDsFromKey(srcAndDestKey) sourceName := eventList[0].Metadata.SourceName destination := &eventList[0].Destination + connection := eventList[0].Connection workspaceID := eventList[0].Metadata.WorkspaceID destType := destination.DestinationDefinition.Name commonMetaData := &transformer.Metadata{ @@ -2592,7 +2612,7 @@ func (proc *Handle) transformSrcDest( var successMetrics []*types.PUReportedMetric var successCountMap map[string]int64 var successCountMetadataMap map[string]MetricMetadata - eventsToTransform, successMetrics, successCountMap, successCountMetadataMap = proc.getTransformerEvents(response, commonMetaData, eventsByMessageID, destination, inPU, types.USER_TRANSFORMER) + eventsToTransform, successMetrics, successCountMap, successCountMetadataMap = proc.getTransformerEvents(response, commonMetaData, eventsByMessageID, destination, connection, inPU, types.USER_TRANSFORMER) nonSuccessMetrics := proc.getNonSuccessfulMetrics(response, commonMetaData, eventsByMessageID, inPU, types.USER_TRANSFORMER) droppedJobs = append(droppedJobs, append(proc.getDroppedJobs(response, eventList), append(nonSuccessMetrics.failedJobs, nonSuccessMetrics.filteredJobs...)...)...) if _, ok := procErrorJobsByDestID[destID]; !ok { @@ -2689,7 +2709,7 @@ func (proc *Handle) transformSrcDest( procErrorJobsByDestID[destID] = make([]*jobsdb.JobT, 0) } procErrorJobsByDestID[destID] = append(procErrorJobsByDestID[destID], nonSuccessMetrics.failedJobs...) - eventsToTransform, successMetrics, successCountMap, successCountMetadataMap = proc.getTransformerEvents(response, commonMetaData, eventsByMessageID, destination, inPU, types.EVENT_FILTER) + eventsToTransform, successMetrics, successCountMap, successCountMetadataMap = proc.getTransformerEvents(response, commonMetaData, eventsByMessageID, destination, connection, inPU, types.EVENT_FILTER) proc.logger.Debug("Supported messages filtering output size", len(eventsToTransform)) // REPORTING - START diff --git a/processor/trackingplan.go b/processor/trackingplan.go index 3c778c35b7..87abcf4276 100644 --- a/processor/trackingplan.go +++ b/processor/trackingplan.go @@ -7,6 +7,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/processor/transformer" "github.com/rudderlabs/rudder-server/utils/misc" @@ -113,7 +114,7 @@ func (proc *Handle) validateEvents(groupedEventsBySourceId map[SourceIDT][]trans trackingPlanEnabledMap[SourceIDT(sourceID)] = true var successMetrics []*types.PUReportedMetric - eventsToTransform, successMetrics, _, _ := proc.getTransformerEvents(response, commonMetaData, eventsByMessageID, destination, types.DESTINATION_FILTER, types.TRACKINGPLAN_VALIDATOR) // Note: Sending false for usertransformation enabled is safe because this stage is before user transformation. + eventsToTransform, successMetrics, _, _ := proc.getTransformerEvents(response, commonMetaData, eventsByMessageID, destination, backendconfig.Connection{}, types.DESTINATION_FILTER, types.TRACKINGPLAN_VALIDATOR) // Note: Sending false for usertransformation enabled is safe because this stage is before user transformation. nonSuccessMetrics := proc.getNonSuccessfulMetrics(response, commonMetaData, eventsByMessageID, types.DESTINATION_FILTER, types.TRACKINGPLAN_VALIDATOR) validationStat.numValidationSuccessEvents.Count(len(eventsToTransform)) diff --git a/processor/transformer/transformer.go b/processor/transformer/transformer.go index 5514dace67..640de608f1 100644 --- a/processor/transformer/transformer.go +++ b/processor/transformer/transformer.go @@ -92,6 +92,7 @@ type TransformerEvent struct { Message types.SingularEventT `json:"message"` Metadata Metadata `json:"metadata"` Destination backendconfig.DestinationT `json:"destination"` + Connection backendconfig.Connection `json:"connection"` Libraries []backendconfig.LibraryT `json:"libraries"` Credentials []Credential `json:"credentials"` } diff --git a/router/handle.go b/router/handle.go index ccd7f66d62..d95211ed6f 100644 --- a/router/handle.go +++ b/router/handle.go @@ -90,6 +90,7 @@ type Handle struct { oauth oauth.Authorizer destinationsMapMu sync.RWMutex destinationsMap map[string]*routerutils.DestinationWithSources // destinationID -> destination + connectionsMap map[types.SourceDest]types.ConnectionWithID isBackendConfigInitialized bool backendConfigInitialized chan bool responseQ chan workerJobStatus diff --git a/router/handle_lifecycle.go b/router/handle_lifecycle.go index a601f8d67a..261f93d71d 100644 --- a/router/handle_lifecycle.go +++ b/router/handle_lifecycle.go @@ -414,8 +414,19 @@ func (rt *Handle) backendConfigSubscriber() { ch := rt.backendConfig.Subscribe(context.TODO(), backendconfig.TopicBackendConfig) for configEvent := range ch { destinationsMap := map[string]*routerutils.DestinationWithSources{} + connectionsMap := map[types.SourceDest]types.ConnectionWithID{} configData := configEvent.Data.(map[string]backendconfig.ConfigT) for _, wConfig := range configData { + for connectionID := range wConfig.Connections { + connection := wConfig.Connections[connectionID] + connectionsMap[types.SourceDest{ + SourceID: connection.SourceID, + DestinationID: connection.DestinationID, + }] = types.ConnectionWithID{ + ConnectionID: connectionID, + Connection: connection, + } + } for i := range wConfig.Sources { source := &wConfig.Sources[i] for i := range source.Destinations { @@ -446,6 +457,7 @@ func (rt *Handle) backendConfigSubscriber() { } } rt.destinationsMapMu.Lock() + rt.connectionsMap = connectionsMap rt.destinationsMap = destinationsMap rt.destinationsMapMu.Unlock() if !rt.isBackendConfigInitialized { diff --git a/router/router_test.go b/router/router_test.go index 94b0f54774..82e2130606 100644 --- a/router/router_test.go +++ b/router/router_test.go @@ -103,6 +103,15 @@ var ( }, }, }, + Connections: map[string]backendconfig.Connection{ + "connection1": { + Enabled: true, + SourceID: sourceIDEnabled, + DestinationID: gaDestinationID, + ProcessorEnabled: true, + Config: map[string]interface{}{"key": "value"}, + }, + }, } ) @@ -528,7 +537,7 @@ var _ = Describe("router", func() { router.Setup(gaDestinationDefinition, logger.NOP, conf, c.mockBackendConfig, c.mockRouterJobsDB, c.mockProcErrorsDB, transientsource.NewEmptyService(), rsources.NewNoOpService(), transformerFeaturesService.NewNoOpService(), destinationdebugger.NewNoOpService(), throttler.NewNoOpThrottlerFactory()) gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { UUID: uuid.New(), @@ -622,7 +631,7 @@ var _ = Describe("router", func() { router.netHandle = mockNetHandle gaPayload := `{}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 unprocessedJobsList := []*jobsdb.JobT{ { @@ -708,7 +717,7 @@ var _ = Describe("router", func() { router.netHandle = mockNetHandle gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 unprocessedJobsList := []*jobsdb.JobT{ { @@ -786,7 +795,7 @@ var _ = Describe("router", func() { router.netHandle = mockNetHandle gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_job_run_id": "someJobRunId", "source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_job_run_id": "someJobRunId", "source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 unprocessedJobsList := []*jobsdb.JobT{ { @@ -880,12 +889,12 @@ var _ = Describe("router", func() { ErrorResponse: []byte(fmt.Sprintf(`{"firstAttemptedAt": %q}`, firstAttemptedAt.Format(misc.RFC3339Milli))), }, Parameters: []byte(fmt.Sprintf(`{ - "source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", + "source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "%s", "transform_at": "processor" - }`, gaDestinationID, firstAttemptedAt.Add(-time.Minute).Format(misc.RFC3339Milli))), + }`, sourceIDEnabled, gaDestinationID, firstAttemptedAt.Add(-time.Minute).Format(misc.RFC3339Milli))), WorkspaceId: workspaceID, }, } @@ -978,22 +987,22 @@ var _ = Describe("router", func() { ErrorCode: "500", ErrorResponse: []byte(fmt.Sprintf(`{"firstAttemptedAt": %q}`, firstAttemptedAt.Format(misc.RFC3339Milli))), JobParameters: []byte(fmt.Sprintf(`{ - "source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", + "source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "%s", "transform_at": "processor", "source_job_run_id": "someJobRunId" - }`, gaDestinationID, firstAttemptedAt.Add(-time.Minute).Format(misc.RFC3339Milli))), + }`, sourceIDEnabled, gaDestinationID, firstAttemptedAt.Add(-time.Minute).Format(misc.RFC3339Milli))), }, Parameters: []byte(fmt.Sprintf(`{ - "source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", + "source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "%s", "transform_at": "processor", "source_job_run_id": "someJobRunId" - }`, gaDestinationID, firstAttemptedAt.Add(-time.Minute).Format(misc.RFC3339Milli))), + }`, sourceIDEnabled, gaDestinationID, firstAttemptedAt.Add(-time.Minute).Format(misc.RFC3339Milli))), WorkspaceId: workspaceID, }, } @@ -1072,7 +1081,7 @@ var _ = Describe("router", func() { router.reloadableConfig.noOfJobsToBatchInAWorker = config.SingleValueLoader(5) gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, nonexistentDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, nonexistentDestinationID) // skipcq: GO-R4002 unprocessedJobsList := []*jobsdb.JobT{ { @@ -1173,7 +1182,7 @@ var _ = Describe("router", func() { router.noOfWorkers = 1 gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -1318,7 +1327,7 @@ var _ = Describe("router", func() { router.enableBatching = true gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -1491,7 +1500,7 @@ var _ = Describe("router", func() { router.reloadableConfig.noOfJobsToBatchInAWorker = config.SingleValueLoader(5) gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -1700,7 +1709,7 @@ var _ = Describe("router", func() { router.reloadableConfig.noOfJobsToBatchInAWorker = config.SingleValueLoader(3) gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -1869,7 +1878,7 @@ var _ = Describe("router", func() { router.noOfWorkers = 1 gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -2029,7 +2038,7 @@ var _ = Describe("router", func() { router.noOfWorkers = 1 gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -2175,7 +2184,7 @@ var _ = Describe("router", func() { router.noOfWorkers = 1 gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` - parameters := fmt.Sprintf(`{"source_id": "1fMCVYZboDlYlauh4GFsEo2JU77", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, gaDestinationID) // skipcq: GO-R4002 + parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "router"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 toRetryJobsList := []*jobsdb.JobT{ { @@ -2311,6 +2320,13 @@ var _ = Describe("router", func() { func assertRouterJobs(routerJob *types.RouterJobT, job *jobsdb.JobT) { Expect(routerJob.JobMetadata.JobID).To(Equal(job.JobID)) Expect(routerJob.JobMetadata.UserID).To(Equal(job.UserID)) + Expect(routerJob.Connection).To(Equal(backendconfig.Connection{ + Enabled: true, + SourceID: sourceIDEnabled, + DestinationID: gaDestinationID, + ProcessorEnabled: true, + Config: map[string]interface{}{"key": "value"}, + })) } func assertJobStatus(job *jobsdb.JobT, status *jobsdb.JobStatusT, expectedState, errorCode, errorResponse string, attemptNum int) { diff --git a/router/transformer/transformer.go b/router/transformer/transformer.go index be4a70d1c3..6753ece88e 100644 --- a/router/transformer/transformer.go +++ b/router/transformer/transformer.go @@ -92,6 +92,7 @@ type ProxyRequestParams struct { DestName string Adapter transformerProxyAdapter DestInfo *oauthv2.DestinationInfo + Connection backendconfig.Connection `json:"connection"` } type ProxyRequestResponse struct { @@ -299,7 +300,14 @@ func (trans *handle) Transform(transformType string, transformMessage *types.Tra destinationJobs = []types.DestinationJobT{} for i := range transformMessage.Data { routerJob := &transformMessage.Data[i] - resp := types.DestinationJobT{Message: routerJob.Message, JobMetadataArray: []types.JobMetadataT{routerJob.JobMetadata}, Destination: routerJob.Destination, StatusCode: statusCode, Error: invalidResponseError} + resp := types.DestinationJobT{ + Message: routerJob.Message, + JobMetadataArray: []types.JobMetadataT{routerJob.JobMetadata}, + Destination: routerJob.Destination, + Connection: routerJob.Connection, + StatusCode: statusCode, + Error: invalidResponseError, + } destinationJobs = append(destinationJobs, resp) } } @@ -310,7 +318,14 @@ func (trans *handle) Transform(transformType string, transformMessage *types.Tra } for i := range transformMessage.Data { routerJob := &transformMessage.Data[i] - resp := types.DestinationJobT{Message: routerJob.Message, JobMetadataArray: []types.JobMetadataT{routerJob.JobMetadata}, Destination: routerJob.Destination, StatusCode: statusCode, Error: string(respData)} + resp := types.DestinationJobT{ + Message: routerJob.Message, + JobMetadataArray: []types.JobMetadataT{routerJob.JobMetadata}, + Destination: routerJob.Destination, + Connection: routerJob.Connection, + StatusCode: statusCode, + Error: string(respData), + } destinationJobs = append(destinationJobs, resp) } } diff --git a/router/types/types.go b/router/types/types.go index f0a649b91b..b90ddd3d47 100644 --- a/router/types/types.go +++ b/router/types/types.go @@ -18,6 +18,16 @@ type RouterJobT struct { Message json.RawMessage `json:"message"` JobMetadata JobMetadataT `json:"metadata"` Destination backendconfig.DestinationT `json:"destination"` + Connection backendconfig.Connection `json:"connection"` +} + +type SourceDest struct { + SourceID, DestinationID string +} + +type ConnectionWithID struct { + ConnectionID string + Connection backendconfig.Connection } type DestinationJobs []DestinationJobT @@ -39,6 +49,7 @@ type DestinationJobT struct { Message json.RawMessage `json:"batchedRequest"` JobMetadataArray []JobMetadataT `json:"metadata"` // multiple jobs may be batched in a single message Destination backendconfig.DestinationT `json:"destination"` + Connection backendconfig.Connection `json:"connection"` Batched bool `json:"batched"` StatusCode int `json:"statusCode"` Error string `json:"error"` diff --git a/router/worker.go b/router/worker.go index e09d3962be..22738ab920 100644 --- a/router/worker.go +++ b/router/worker.go @@ -181,12 +181,17 @@ func (w *worker) workLoop() { } w.rt.destinationsMapMu.RLock() - batchDestination, ok := w.rt.destinationsMap[parameters.DestinationID] + batchDestination, destOK := w.rt.destinationsMap[parameters.DestinationID] + conn, connOK := w.rt.connectionsMap[types.SourceDest{ + SourceID: parameters.SourceID, + DestinationID: parameters.DestinationID, + }] w.rt.destinationsMapMu.RUnlock() - if !ok { + if !destOK || (parameters.SourceJobRunID != "" && !connOK) { continue } destination := batchDestination.Destination + connection := conn.Connection oauthV2Enabled := w.rt.reloadableConfig.oauthV2Enabled.Load() // TODO: Remove later w.logger.Debugn("[router worker]", @@ -218,6 +223,7 @@ func (w *worker) workLoop() { Message: job.EventPayload, JobMetadata: jobMetadata, Destination: destination, + Connection: connection, }) if len(w.routerJobs) >= w.rt.reloadableConfig.noOfJobsToBatchInAWorker.Load() { @@ -229,6 +235,7 @@ func (w *worker) workLoop() { Message: job.EventPayload, JobMetadata: jobMetadata, Destination: destination, + Connection: connection, }) if len(w.routerJobs) >= w.rt.reloadableConfig.noOfJobsToBatchInAWorker.Load() { @@ -239,6 +246,7 @@ func (w *worker) workLoop() { w.destinationJobs = append(w.destinationJobs, types.DestinationJobT{ Message: job.EventPayload, Destination: destination, + Connection: connection, JobMetadataArray: []types.JobMetadataT{jobMetadata}, }) w.processDestinationJobs() @@ -773,7 +781,8 @@ func (w *worker) proxyRequest(ctx context.Context, destinationJob types.Destinat DefinitionName: destinationJob.Destination.DestinationDefinition.Name, ID: destinationJob.Destination.ID, }, - Adapter: transformer.NewTransformerProxyAdapter(w.rt.transformerFeaturesService.TransformerProxyVersion(), w.rt.logger), + Connection: destinationJob.Connection, + Adapter: transformer.NewTransformerProxyAdapter(w.rt.transformerFeaturesService.TransformerProxyVersion(), w.rt.logger), } rtlTime := time.Now() oauthV2Enabled := w.rt.reloadableConfig.oauthV2Enabled.Load() From 8e3be2940a3e6ec466d4c4da8d6bd13c84599c0e Mon Sep 17 00:00:00 2001 From: Dilip Kola <33080863+koladilip@users.noreply.github.com> Date: Tue, 3 Sep 2024 14:56:05 +0530 Subject: [PATCH 02/29] chore: refine error extractor (#5049) * chore: refine error extractor Add support for html error extraction. * chore: refine error extractor * chore: run go mod tidy --- enterprise/reporting/error_extractor.go | 13 ++++++++++++- enterprise/reporting/reporting_test.go | 8 ++++++++ go.mod | 1 + go.sum | 11 +++++++++++ 4 files changed, 32 insertions(+), 1 deletion(-) diff --git a/enterprise/reporting/error_extractor.go b/enterprise/reporting/error_extractor.go index 82a4c078ec..db26317b36 100644 --- a/enterprise/reporting/error_extractor.go +++ b/enterprise/reporting/error_extractor.go @@ -9,6 +9,8 @@ import ( "github.com/samber/lo" "github.com/tidwall/gjson" + "github.com/k3a/html2text" + "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" ) @@ -30,7 +32,7 @@ var ( spaceRegex = regexp.MustCompile(`\s+`) whitespacesRegex = regexp.MustCompile("[ \t\n\r]*") // used in checking if string is a valid json to remove extra-spaces - defaultErrorMessageKeys = []string{"message", "description", "detail", "title", errorKey, "error_message"} + defaultErrorMessageKeys = []string{"message", "description", "detail", errorKey, "title", "error_message"} deprecationKeywords = map[string]int{ "deprecated": 2, "deprecation": 2, @@ -128,6 +130,11 @@ func (ext *ExtractorHandle) getSimpleMessage(jsonStr string) string { } return getErrorMessageFromResponse(unmarshalledJson, ext.ErrorMessageKeys) } + lowerErResStr := strings.ToLower(erResStr) + if strings.Contains(lowerErResStr, "") { + return getHTMLErrorMessage(erResStr) + } + if len(erResStr) == 0 { return "" } @@ -146,6 +153,10 @@ func (ext *ExtractorHandle) getSimpleMessage(jsonStr string) string { return "" } +func getHTMLErrorMessage(erResStr string) string { + return html2text.HTML2Text(erResStr) +} + func (ext *ExtractorHandle) GetErrorMessage(sampleResponse string) string { return ext.getSimpleMessage(sampleResponse) } diff --git a/enterprise/reporting/reporting_test.go b/enterprise/reporting/reporting_test.go index a5c6369741..f1c3a6457e 100644 --- a/enterprise/reporting/reporting_test.go +++ b/enterprise/reporting/reporting_test.go @@ -342,6 +342,14 @@ var tcs = []getValTc{ inputStr: `{"response":"{\"status\":400,\"destinationResponse\":{\"response\":[{\"duplicateResut\":{\"allowSave\":true,\"duplicateRule\":\"Contacts_DR\",\"duplicateRuleEntityType\":\"Contact\",\"errorMessage\":\"You're creating a duplicate record. We recommend you use an existing record instead.\",\"matchResults\":[{\"entityType\":\"Contact\",\"errors\":[],\"matchEngine\":\"ExactMatchEngine\",\"matchRecords\":[{\"additionalInformation\":[],\"fieldDiffs\":[],\"matchConfidence\":100,\"record\":{\"attributes\":{\"type\":\"Contact\",\"url\":\"/services/data/v50.0/sobjects/Contact/0031i000013x2TEAAY\"},\"Id\":\"0031i000013x2TEAAY\"}}],\"rule\":\"Contact_MR\",\"size\":1,\"success\":true}]},\"errorCode\":\"DUPLICATES_DETECTED\",\"message\":\"You're creating a duplicate record. We recommend you use an existing record instead.\"}],\"status\":400,\"rudderJobMetadata\":{\"jobId\":1466739020,\"attemptNum\":0,\"userId\":\"\",\"sourceId\":\"2JF2LaBUedeOfAyt9EoIVJylkKS\",\"destinationId\":\"2JJDsqHbkuIZ89ldOBMaBjPi9L7\",\"workspaceId\":\"26TTcz2tQucRs2xZiGThQzGRk2l\",\"secret\":null,\"destInfo\":{\"authKey\":\"2JJDsqHbkuIZ89ldOBMaBjPi9L7\"}}},\"message\":\"Salesforce Request Failed: \\\"400\\\" due to \\\"You're creating a duplicate record. We recommend you use an existing record instead.\\\", (Aborted) during Salesforce Response Handling\",\"statTags\":{\"errorCategory\":\"network\",\"errorType\":\"aborted\",\"destType\":\"SALESFORCE\",\"module\":\"destination\",\"implementation\":\"native\",\"feature\":\"dataDelivery\",\"destinationId\":\"2JJDsqHbkuIZ89ldOBMaBjPi9L7\",\"workspaceId\":\"26TTcz2tQucRs2xZiGThQzGRk2l\"}}","firstAttemptedAt":"2023-03-30T17:07:52.359Z","content-type":"application/json"}`, expected: `You're creating a duplicate record. We recommend you use an existing record instead.`, }, + { + inputStr: `{"response":"\u003c!--\n ~ Copyright (C) 2010-2021 Evergage, Inc.\n ~ All rights reserved.\n --\u003e\n\n\u003c!DOCTYPE html\u003e\n\u003chtml lang=\"en\"\u003e\n\u003chead\u003e\n \u003cmeta charset=\"UTF-8\"\u003e\n \u003ctitle\u003eSalesforce Personalization\u003c/title\u003e\n \u003clink rel=\"icon\" type=\"image/x-icon\" href=\"https://www.salesforce.com/etc/designs/sfdc-www/en_us/favicon.ico\"/\u003e\n \u003clink rel=\"shortcut icon\" type=\"image/x-icon\" href=\"https://www.salesforce.com/etc/designs/sfdc-www/en_us/favicon.ico\"/\u003e\n \u003cstyle\u003e\n body { font-family: Salesforce Sans,Arial,sans-serif; text-align: center; padding: 50px; background-color:#fff; }\n h1 { font-size: 1.25rem; color: #080707; text-align: center; margin-top: -0.5rem; }\n p { font-size: 0.8125rem; color: #3E3E3C; text-align:center; }\n \u003c/style\u003e\n\u003c/head\u003e\n\u003cbody\u003e\n \u003cdiv align=”center”\u003e\n \u003cimg src=\"/PageNotAvailable.svg\" /\u003e\n \u003c/div\u003e\n \u003cdiv align=”center”\u003e\n \u003ch1\u003eThe page you want isn’t available.\u003c/h1\u003e\n \u003cp\u003eTo find the page you want, use the main navigation.\u003c/p\u003e\n \u003c/div\u003e\n\u003c/body\u003e\n\u003c/html\u003e","firstAttemptedAt":"2024-09-02T06:57:13.829Z","content-type":"text/html"}`, + expected: "The page you want isn’t available.\r\n\r\n To find the page you want, use the main navigation.\r\n\r\n ", + }, + { + inputStr: `{"response":"{\"status\":\"fail\",\"processed\":0,\"unprocessed\":[{\"status\":\"fail\",\"code\":513,\"error\":\"Event Name is incorrect. ErrorCode: 513 - Trying to raise a restricted system event. Skipped record number : 1\",\"record\":{\"evtData\":{\"initial_referrer\":\"https://www.google.com/\",\"initial_referring_domain\":\"www.google.com\",\"path\":\"/busca/\",\"referrer\":\"https://www.extrabom.com.br/busca/?q=Bombril\u0026anonymous=347f65ea66096fd7db4e1bd88211a83dfbe263b78da6a5de0261d160c54100ba\",\"referring_domain\":\"www.extrabom.com.br\",\"search\":\"?q=X14\u0026anonymous=347f65ea66096fd7db4e1bd88211a83dfbe263b78da6a5de0261d160c54100ba\",\"tab_url\":\"https://www.extrabom.com.br/busca/?q=X14\u0026anonymous=347f65ea66096fd7db4e1bd88211a83dfbe263b78da6a5de0261d160c54100ba\",\"title\":\"X14 - Busca - Extrabom\",\"url\":\"https://www.extrabom.com.br/busca/?q=X14\"},\"evtName\":\"Web Page Viewed\",\"identity\":\"69298\",\"type\":\"event\"}}]}","firstAttemptedAt":"2024-09-02T00:40:06.451Z","content-type":"application/json"}`, + expected: "Event Name is incorrect. ErrorCode: 513 - Trying to raise a restricted system event. Skipped record number : 1", + }, { inputStr: `{"response":"{\"destinationResponse\":\"\u003c!DOCTYPE html\u003e\\n\u003chtml lang=\\\"en\\\" id=\\\"facebook\\\"\u003e\\n \u003chead\u003e\\n \u003ctitle\u003eFacebook | Error\u003c/title\u003e\\n \u003cmeta charset=\\\"utf-8\\\"\u003e\\n \u003cmeta http-equiv=\\\"cache-control\\\" content=\\\"no-cache\\\"\u003e\\n \u003cmeta http-equiv=\\\"cache-control\\\" content=\\\"no-store\\\"\u003e\\n \u003cmeta http-equiv=\\\"cache-control\\\" content=\\\"max-age=0\\\"\u003e\\n \u003cmeta http-equiv=\\\"expires\\\" content=\\\"-1\\\"\u003e\\n \u003cmeta http-equiv=\\\"pragma\\\" content=\\\"no-cache\\\"\u003e\\n \u003cmeta name=\\\"robots\\\" content=\\\"noindex,nofollow\\\"\u003e\\n \u003cstyle\u003e\\n html, body {\\n color: #141823;\\n background-color: #e9eaed;\\n font-family: Helvetica, Lucida Grande, Arial,\\n Tahoma, Verdana, sans-serif;\\n margin: 0;\\n padding: 0;\\n text-align: center;\\n }\\n\\n #header {\\n height: 30px;\\n padding-bottom: 10px;\\n padding-top: 10px;\\n text-align: center;\\n }\\n\\n #icon {\\n width: 30px;\\n }\\n\\n h1 {\\n font-size: 18px;\\n }\\n\\n p {\\n font-size: 13px;\\n }\\n\\n #footer {\\n border-top: 1px solid #ddd;\\n color: #9197a3;\\n font-size: 12px;\\n padding: 5px 8px 6px 0;\\n }\\n \u003c/style\u003e\\n \u003c/head\u003e\\n \u003cbody\u003e\\n \u003cdiv id=\\\"header\\\"\u003e\\n \u003ca href=\\\"//www.facebook.com/\\\"\u003e\\n \u003cimg id=\\\"icon\\\" src=\\\"//static.facebook.com/images/logos/facebook_2x.png\\\" /\u003e\\n \u003c/a\u003e\\n \u003c/div\u003e\\n \u003cdiv id=\\\"core\\\"\u003e\\n \u003ch1 id=\\\"sorry\\\"\u003eSorry, something went wrong.\u003c/h1\u003e\\n \u003cp id=\\\"promise\\\"\u003e\\n We're working on it and we'll get it fixed as soon as we can.\\n \u003c/p\u003e\\n \u003cp id=\\\"back-link\\\"\u003e\\n \u003ca id=\\\"back\\\" href=\\\"//www.facebook.com/\\\"\u003eGo Back\u003c/a\u003e\\n \u003c/p\u003e\\n \u003cdiv id=\\\"footer\\\"\u003e\\n Facebook\\n \u003cspan id=\\\"copyright\\\"\u003e\\n \u0026copy; 2022\\n \u003c/span\u003e\\n \u003cspan id=\\\"help-link\\\"\u003e\\n \u0026#183;\\n \u003ca id=\\\"help\\\" href=\\\"//www.facebook.com/help/\\\"\u003eHelp Center\u003c/a\u003e\\n \u003c/span\u003e\\n \u003c/div\u003e\\n \u003c/div\u003e\\n \u003cscript\u003e\\n document.getElementById('back').onclick = function() {\\n if (history.length \u003e 1) {\\n history.back();\\n return false;\\n }\\n };\\n\\n // Adjust the display based on the window size\\n if (window.innerHeight \u003c 80 || window.innerWidth \u003c 80) {\\n // Blank if window is too small\\n document.body.style.display = 'none';\\n };\\n if (window.innerWidth \u003c 200 || window.innerHeight \u003c 150) {\\n document.getElementById('back-link').style.display = 'none';\\n document.getElementById('help-link').style.display = 'none';\\n };\\n if (window.innerWidth \u003c 200) {\\n document.getElementById('sorry').style.fontSize = '16px';\\n };\\n if (window.innerWidth \u003c 150) {\\n document.getElementById('promise').style.display = 'none';\\n };\\n if (window.innerHeight \u003c 150) {\\n document.getElementById('sorry').style.margin = '4px 0 0 0';\\n document.getElementById('sorry').style.fontSize = '14px';\\n document.getElementById('promise').style.display = 'none';\\n };\\n \u003c/script\u003e\\n \u003c/body\u003e\\n\u003c/html\u003e\\n\",\"message\":\"Request Processed Successfully\",\"status\":502}","firstAttemptedAt":"2023-03-30T17:31:41.884Z","content-type":"application/json"}`, expected: "Request Processed Successfully", diff --git a/go.mod b/go.mod index 6322503170..e3aec7b9e6 100644 --- a/go.mod +++ b/go.mod @@ -58,6 +58,7 @@ require ( github.com/jeremywohl/flatten v1.0.1 github.com/joho/godotenv v1.5.1 github.com/json-iterator/go v1.1.12 + github.com/k3a/html2text v1.2.1 github.com/lensesio/tableprinter v0.0.0-20201125135848-89e81fc956e7 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.0 diff --git a/go.sum b/go.sum index b1b5631f28..4aa2db7296 100644 --- a/go.sum +++ b/go.sum @@ -739,6 +739,8 @@ github.com/googleapis/gax-go/v2 v2.1.1/go.mod h1:hddJymUZASv3XPyGkUpKj8pPO47Rmb0 github.com/googleapis/gax-go/v2 v2.2.0/go.mod h1:as02EH8zWkzwUoLbBaFeQ+arQaj/OthfcblKl4IGNaM= github.com/googleapis/gax-go/v2 v2.13.0 h1:yitjD5f7jQHhyDsnhKEBU52NdvvdSeGzlAnDPT0hH1s= github.com/googleapis/gax-go/v2 v2.13.0/go.mod h1:Z/fvTZXF8/uw7Xu5GuslPw+bplx6SS338j1Is2S+B7A= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/handlers v1.5.2 h1:cLTUSsNkgcwhgRqvCNmdbRWG0A3N4F+M2nWKdScwyEE= github.com/gorilla/handlers v1.5.2/go.mod h1:dX+xVpaxdSw+q0Qek8SSsl3dfMk3jNddUkMzo0GtH0w= github.com/gorilla/mux v1.8.1 h1:TuBL49tXwgrFYWhqrNgrUNEY92u81SPhu7sTdzQEiWY= @@ -864,8 +866,12 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= +github.com/k3a/html2text v1.2.1 h1:nvnKgBvBR/myqrwfLuiqecUtaK1lB9hGziIJKatNFVY= +github.com/k3a/html2text v1.2.1/go.mod h1:ieEXykM67iT8lTvEWBh6fhpH4B23kB9OMKPdIBmgUqA= github.com/kataras/tablewriter v0.0.0-20180708051242-e063d29b7c23 h1:M8exrBzuhWcU6aoHJlHWPe4qFjVKzkMGRal78f5jRRU= github.com/kataras/tablewriter v0.0.0-20180708051242-e063d29b7c23/go.mod h1:kBSna6b0/RzsOcOZf515vAXwSsXYusl2U7SA0XP09yI= github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 h1:Z9n2FFNUXsshfwJMBgNA0RU6/i7WVaAegv3PtuIHPMs= @@ -1184,6 +1190,10 @@ github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/skratchdot/open-golang v0.0.0-20200116055534-eef842397966 h1:JIAuq3EEf9cgbU6AtGPK4CTG3Zf6CKMNqf0MHTggAUA= github.com/skratchdot/open-golang v0.0.0-20200116055534-eef842397966/go.mod h1:sUM3LWHvSMaG192sy56D9F7CNvL7jUJVXoqM1QKLnog= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/snowflakedb/gosnowflake v1.11.0 h1:qyqunGCVyq/Qyx40KQT+6sJ1CAGuuG2qv3WiCTLTctI= github.com/snowflakedb/gosnowflake v1.11.0/go.mod h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A= github.com/sony/gobreaker v1.0.0 h1:feX5fGGXSl3dYd4aHZItw+FpHLvvoaqkawKjVNiFMNQ= @@ -1594,6 +1604,7 @@ golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3 golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190422233926-fe54fb35175b/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190425163242-31fd60d6bfdc/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= From 66edcfb67185a754e5c0724c48d761542ad1afe3 Mon Sep 17 00:00:00 2001 From: Leonidas Vrachnis Date: Tue, 3 Sep 2024 12:58:59 +0200 Subject: [PATCH 03/29] chore: update webhook test cases (#5053) --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index e3aec7b9e6..bc2a77e8d3 100644 --- a/go.mod +++ b/go.mod @@ -81,7 +81,7 @@ require ( github.com/rudderlabs/rudder-go-kit v0.38.0 github.com/rudderlabs/rudder-observability-kit v0.0.3 github.com/rudderlabs/rudder-schemas v0.5.1 - github.com/rudderlabs/rudder-transformer/go v0.0.0-20240812044419-23196ec42acf + github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32 github.com/rudderlabs/sql-tunnels v0.1.7 github.com/rudderlabs/sqlconnect-go v1.9.0 github.com/samber/lo v1.47.0 diff --git a/go.sum b/go.sum index 4aa2db7296..25e3faebb0 100644 --- a/go.sum +++ b/go.sum @@ -1140,8 +1140,8 @@ github.com/rudderlabs/rudder-observability-kit v0.0.3 h1:vZtuZRkGX+6rjaeKtxxFE2Y github.com/rudderlabs/rudder-observability-kit v0.0.3/go.mod h1:6UjAh3H6rkE0fFLh7z8ZGQEQbKtUkRfhWOf/OUhfqW8= github.com/rudderlabs/rudder-schemas v0.5.1 h1:g4I5wp2yA6ZWQZ1MjSNn4zby3XctG/TOgbYUW3dS4z4= github.com/rudderlabs/rudder-schemas v0.5.1/go.mod h1:JoDTB9nCDXwRz+G+aYwP3Fj42HLssKARxsFFm+qqgb4= -github.com/rudderlabs/rudder-transformer/go v0.0.0-20240812044419-23196ec42acf h1:nsU2tKjPV/sbmOoIk39ncFT8D5HBDVppmrCWO0v9HsU= -github.com/rudderlabs/rudder-transformer/go v0.0.0-20240812044419-23196ec42acf/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= +github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32 h1:hsvbOvwfA8wr9GjNBWuh3lZnpqMgIzLU7HNESB87wSg= +github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= github.com/rudderlabs/sql-tunnels v0.1.7 h1:wDCRl6zY4M5gfWazf7XkSTGQS3yjBzUiUgEMBIfHNDA= github.com/rudderlabs/sql-tunnels v0.1.7/go.mod h1:5f7+YL49JHYgteP4rAgqKnr4K2OadB0oIpUS+Tt3sPM= github.com/rudderlabs/sqlconnect-go v1.9.0 h1:icLgqvVQ15Vh+oP7epA0b0yK6sIzxRVwPlRzOoDNVRA= From 8da3748c6b8483e6caf6fea176d20d6505f57351 Mon Sep 17 00:00:00 2001 From: Rohith BCS Date: Tue, 3 Sep 2024 17:00:22 +0530 Subject: [PATCH 04/29] chore: upgrade go to 1.23.0 (#5052) --- .github/workflows/verify.yml | 2 +- Dockerfile | 2 +- Makefile | 2 +- cmd/rudder-cli/status/status.go | 12 ++++++------ go.mod | 2 +- regulation-worker/internal/client/client_test.go | 2 +- regulation-worker/internal/delete/api/api.go | 6 +++--- regulation-worker/internal/delete/api/api_test.go | 5 +++-- .../bing-ads/offline-conversions/bingads_test.go | 13 +++++++------ .../klaviyobulkupload/klaviyobulkupload.go | 3 ++- .../yandexmetrica/yandexmetrica.go | 3 ++- router/transformer/transformer.go | 3 ++- services/streammanager/bqstream/bqstreammanager.go | 2 +- suppression-backup-service/Dockerfile | 2 +- warehouse/integrations/postgres/postgres.go | 2 +- warehouse/internal/loadfiles/loadfiles.go | 3 ++- warehouse/utils/utils_test.go | 2 +- warehouse/validations/validate.go | 6 ++++-- 18 files changed, 40 insertions(+), 32 deletions(-) diff --git a/.github/workflows/verify.yml b/.github/workflows/verify.yml index 72ffe66dff..49807de307 100644 --- a/.github/workflows/verify.yml +++ b/.github/workflows/verify.yml @@ -77,5 +77,5 @@ jobs: - name: golangci-lint uses: golangci/golangci-lint-action@v6 with: - version: v1.57.1 + version: v1.60.3 args: -v diff --git a/Dockerfile b/Dockerfile index 6cb3219ac6..749277686c 100644 --- a/Dockerfile +++ b/Dockerfile @@ -2,7 +2,7 @@ # syntax=docker/dockerfile:1 # GO_VERSION is updated automatically to match go.mod, see Makefile -ARG GO_VERSION=1.22.5 +ARG GO_VERSION=1.23.0 ARG ALPINE_VERSION=3.20 FROM golang:${GO_VERSION}-alpine${ALPINE_VERSION} AS builder ARG VERSION diff --git a/Makefile b/Makefile index f3f1197122..00a8c668d2 100644 --- a/Makefile +++ b/Makefile @@ -6,7 +6,7 @@ TESTFILE=_testok MOUNT_PATH=/local # go tools versions -GOLANGCI=github.com/golangci/golangci-lint/cmd/golangci-lint@v1.57.1 +GOLANGCI=github.com/golangci/golangci-lint/cmd/golangci-lint@v1.60.3 gofumpt=mvdan.cc/gofumpt@latest govulncheck=golang.org/x/vuln/cmd/govulncheck@latest goimports=golang.org/x/tools/cmd/goimports@latest diff --git a/cmd/rudder-cli/status/status.go b/cmd/rudder-cli/status/status.go index 11ded250c5..807937b9c9 100644 --- a/cmd/rudder-cli/status/status.go +++ b/cmd/rudder-cli/status/status.go @@ -100,16 +100,16 @@ func GetArgs(request string) error { if request == "Jobs between JobID's of a User" { fmt.Printf("Enter DS Type:") var table string - fmt.Scanf("%s", &table) + _, _ = fmt.Scanf("%s", &table) fmt.Print("Enter JobID 1: ") var input string - fmt.Scanf("%s", &input) + _, _ = fmt.Scanf("%s", &input) fmt.Print("Enter JobID 2: ") var input2 string - fmt.Scanf("%s", &input2) + _, _ = fmt.Scanf("%s", &input2) fmt.Printf("Enter UserID:") var input3 string - fmt.Scanf("%s", &input3) + _, _ = fmt.Scanf("%s", &input3) argString := table + ":" + request + ":" + input + ":" + input2 + ":" + input3 var reply string err := client.GetUDSClient().Call("JobsdbUtilsHandler.RunSQLQuery", argString, &reply) @@ -120,10 +120,10 @@ func GetArgs(request string) error { } else if request == "Error Code Count By Destination" { fmt.Printf("Enter DS Type:") var table string - fmt.Scanf("%s", &table) + _, _ = fmt.Scanf("%s", &table) fmt.Print("Enter DS Number: ") var dsNum string - fmt.Scanf("%s", &dsNum) + _, _ = fmt.Scanf("%s", &dsNum) argString := table + ":" + request + ":" + dsNum var reply string err := client.GetUDSClient().Call("JobsdbUtilsHandler.RunSQLQuery", argString, &reply) diff --git a/go.mod b/go.mod index bc2a77e8d3..974c3101af 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/rudderlabs/rudder-server -go 1.22.5 +go 1.23.0 // Addressing snyk vulnerabilities in indirect dependencies // When upgrading a dependency, please make sure that diff --git a/regulation-worker/internal/client/client_test.go b/regulation-worker/internal/client/client_test.go index 16b1dc7c08..255d2d2537 100644 --- a/regulation-worker/internal/client/client_test.go +++ b/regulation-worker/internal/client/client_test.go @@ -106,7 +106,7 @@ func TestGet(t *testing.T) { path := r.URL.Path require.Equal(t, tt.expectedPath, path) time.Sleep(time.Duration(tt.serverDelay) * time.Millisecond) - fmt.Fprintf(w, tt.respBody) + _, _ = w.Write([]byte(tt.respBody)) })) defer svr.Close() httpClient := &http.Client{} diff --git a/regulation-worker/internal/delete/api/api.go b/regulation-worker/internal/delete/api/api.go index efa331172c..1daa7c2295 100644 --- a/regulation-worker/internal/delete/api/api.go +++ b/regulation-worker/internal/delete/api/api.go @@ -273,7 +273,7 @@ func (m *APIManager) inactivateAuthStatus(destination *model.Destination, job mo AuthStatus: oauth.AuthStatusInactive, }) jobStatus.Status = model.JobStatusAborted - jobStatus.Error = fmt.Errorf(resp) + jobStatus.Error = errors.New(resp) return jobStatus } @@ -289,7 +289,7 @@ func (m *APIManager) refreshOAuthToken(destination *model.Destination, job model if refreshResponse.Err == oauth.REF_TOKEN_INVALID_GRANT { // authStatus should be made inactive m.inactivateAuthStatus(destination, job, oAuthDetail) - return fmt.Errorf(refreshResponse.ErrorMessage) + return errors.New(refreshResponse.ErrorMessage) } var refreshRespErr string @@ -350,7 +350,7 @@ func (m *APIManager) PostResponse(ctx context.Context, params PostResponseParams } if authErrorCategory == common.CategoryAuthStatusInactive { // Abort the regulation request - return model.JobStatus{Status: model.JobStatusAborted, Error: fmt.Errorf(string(params.responseBodyBytes))} + return model.JobStatus{Status: model.JobStatusAborted, Error: errors.New(string(params.responseBodyBytes))} } } return jobStatus diff --git a/regulation-worker/internal/delete/api/api_test.go b/regulation-worker/internal/delete/api/api_test.go index 7f1f441ae6..b24c349b51 100644 --- a/regulation-worker/internal/delete/api/api_test.go +++ b/regulation-worker/internal/delete/api/api_test.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "encoding/json" + "errors" "fmt" "net/http" "net/http/httptest" @@ -711,8 +712,8 @@ var oauthTests = []oauthTestCases{ Response: `{"message": "AuthStatus toggle skipped as already request in-progress: (1234, 1001)"}`, }, }, - expectedDeleteStatus: model.JobStatus{Status: model.JobStatusAborted, Error: fmt.Errorf("problem with user permission or access/refresh token have been revoked")}, - expectedDeleteStatus_OAuthV2: model.JobStatus{Status: model.JobStatusAborted, Error: fmt.Errorf(fmt.Sprintf(`[{"status":"failed","authErrorCategory": "%v", "error": "User does not have sufficient permissions"}]`, common.CategoryAuthStatusInactive))}, + expectedDeleteStatus: model.JobStatus{Status: model.JobStatusAborted, Error: errors.New("problem with user permission or access/refresh token have been revoked")}, + expectedDeleteStatus_OAuthV2: model.JobStatus{Status: model.JobStatusAborted, Error: fmt.Errorf(`[{"status":"failed","authErrorCategory": "%v", "error": "User does not have sufficient permissions"}]`, common.CategoryAuthStatusInactive)}, expectedPayload: `[{"jobId":"16","destType":"ga","config":{"authStatus":"active","rudderDeleteAccountId":"xyz"},"userAttributes":[{"email":"greymore@gmail.com","phone":"8463633841","userId":"203984798476"}]}]`, }, { diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go index 1a2cf0c36f..425809eb07 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go @@ -3,6 +3,7 @@ package offline_conversions import ( "archive/zip" stdjson "encoding/json" + "errors" "fmt" "io" "net/http" @@ -137,9 +138,9 @@ var _ = Describe("Bing ads Offline Conversions", func() { clientI := Client{} bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) errorMsg := "Error in getting bulk upload url" - bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf(errorMsg)) - bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf(errorMsg)) - bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf(errorMsg)) + bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errorMsg)) + bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errorMsg)) + bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errorMsg)) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1, 2, 3, 4, 5, 6}, @@ -180,11 +181,11 @@ var _ = Describe("Bing ads Offline Conversions", func() { ClientI := Client{} bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &ClientI) errMsg := "unable to get bulk upload url, check your credentials" - bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf(errMsg)) + bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errMsg)) - bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf(errMsg)) + bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errMsg)) - bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf(errMsg)) + bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errMsg)) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1, 2, 3, 4, 5, 6}, diff --git a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go index 3de7ad8d3b..5f977facf9 100644 --- a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go +++ b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go @@ -3,6 +3,7 @@ package klaviyobulkupload import ( "bufio" "bytes" + "errors" "fmt" "io" "net/http" @@ -375,7 +376,7 @@ func (kbu *KlaviyoBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationS if resp.StatusCode != 202 { failedJobs = append(failedJobs, importingJobIDs[idx]) - kbu.logger.Error("Got non 202 as statusCode.", fmt.Errorf(string(bodyBytes))) + kbu.logger.Error("Got non 202 as statusCode.", errors.New(string(bodyBytes))) } var uploadresp UploadResp uploadRespErr := json.Unmarshal((bodyBytes), &uploadresp) diff --git a/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go b/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go index 78614ee465..9a9153bd02 100644 --- a/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go +++ b/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go @@ -3,6 +3,7 @@ package yandexmetrica import ( "bytes" "encoding/csv" + "errors" "fmt" "io" "mime/multipart" @@ -361,7 +362,7 @@ func (ym *YandexMetricaBulkUploader) Upload(asyncDestStruct *common.AsyncDestina uploadTimeStat.Since(startTime) if resp.StatusCode != http.StatusOK { // error scenario - return ym.generateErrorOutput("got non 200 response from the destination", fmt.Errorf(string(bodyBytes)), importingJobIDs) + return ym.generateErrorOutput("got non 200 response from the destination", errors.New(string(bodyBytes)), importingJobIDs) } eventsSuccessStat.Count(len(asyncDestStruct.ImportingJobIDs)) return common.AsyncUploadOutput{ diff --git a/router/transformer/transformer.go b/router/transformer/transformer.go index 6753ece88e..19ff652b70 100644 --- a/router/transformer/transformer.go +++ b/router/transformer/transformer.go @@ -6,6 +6,7 @@ import ( "bytes" "context" "encoding/json" + "errors" "fmt" "io" "net/http" @@ -597,7 +598,7 @@ func (trans *handle) doProxyRequest(ctx context.Context, proxyUrl string, proxyR return httpProxyResponse{ respData: []byte{}, statusCode: http.StatusInternalServerError, - err: fmt.Errorf(errStr), + err: errors.New(errStr), } } diff --git a/services/streammanager/bqstream/bqstreammanager.go b/services/streammanager/bqstream/bqstreammanager.go index 26a13e8afd..cd742e3d2e 100644 --- a/services/streammanager/bqstream/bqstreammanager.go +++ b/services/streammanager/bqstream/bqstreammanager.go @@ -156,5 +156,5 @@ func (producer *BQStreamProducer) Close() error { func createErr(err error, msg string) error { fmtMsg := fmt.Errorf("[BQStream] error :: %v:: %w", msg, err).Error() pkgLogger.Errorf(fmtMsg) - return fmt.Errorf(fmtMsg) + return errors.New(fmtMsg) } diff --git a/suppression-backup-service/Dockerfile b/suppression-backup-service/Dockerfile index 84ddadb543..47a942816c 100644 --- a/suppression-backup-service/Dockerfile +++ b/suppression-backup-service/Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1 # GO_VERSION is updated automatically to match go.mod, see Makefile -ARG GO_VERSION=1.22.5 +ARG GO_VERSION=1.23.0 ARG ALPINE_VERSION=3.20 FROM golang:${GO_VERSION}-alpine${ALPINE_VERSION} AS builder RUN mkdir /app diff --git a/warehouse/integrations/postgres/postgres.go b/warehouse/integrations/postgres/postgres.go index 3554f0fdb3..5c6b41ee7c 100644 --- a/warehouse/integrations/postgres/postgres.go +++ b/warehouse/integrations/postgres/postgres.go @@ -496,7 +496,7 @@ func (pg *Postgres) Connect(_ context.Context, warehouse model.Warehouse) (clien if warehouse.Destination.Config["sslMode"] == "verify-ca" { if err := warehouseutils.WriteSSLKeys(warehouse.Destination); err.IsError() { pg.logger.Error(err.Error()) - return client.Client{}, fmt.Errorf(err.Error()) + return client.Client{}, errors.New(err.Error()) } } pg.Warehouse = warehouse diff --git a/warehouse/internal/loadfiles/loadfiles.go b/warehouse/internal/loadfiles/loadfiles.go index 0a4d2d10ff..a0b6b39ff8 100644 --- a/warehouse/internal/loadfiles/loadfiles.go +++ b/warehouse/internal/loadfiles/loadfiles.go @@ -3,6 +3,7 @@ package loadfiles import ( "context" stdjson "encoding/json" + "errors" "fmt" "slices" "strings" @@ -291,7 +292,7 @@ func (lf *LoadFileGenerator) createFromStaging(ctx context.Context, job *model.U if resp.Status == notifier.Aborted && resp.Error != nil { lf.Logger.Errorf("[WH]: Error in generating load files: %v", resp.Error) - sampleError = fmt.Errorf(resp.Error.Error()) + sampleError = errors.New(resp.Error.Error()) err = lf.StageRepo.SetErrorStatus(ctx, jobResponse.StagingFileID, sampleError) if err != nil { return fmt.Errorf("set staging file error status: %w", err) diff --git a/warehouse/utils/utils_test.go b/warehouse/utils/utils_test.go index f728f23dcd..e9a305c1a6 100644 --- a/warehouse/utils/utils_test.go +++ b/warehouse/utils/utils_test.go @@ -817,7 +817,7 @@ func TestObjectStorageType(t *testing.T) { } func TestGetTablePathInObjectStorage(t *testing.T) { - require.NoError(t, os.Setenv("WAREHOUSE_DATALAKE_FOLDER_NAME", "rudder-test-payload")) + t.Setenv("WAREHOUSE_DATALAKE_FOLDER_NAME", "rudder-test-payload") inputs := []struct { namespace string tableName string diff --git a/warehouse/validations/validate.go b/warehouse/validations/validate.go index 64da553076..026cc5cac8 100644 --- a/warehouse/validations/validate.go +++ b/warehouse/validations/validate.go @@ -552,8 +552,10 @@ type dummyUploader struct { dest *backendconfig.DestinationT } -func (*dummyUploader) IsWarehouseSchemaEmpty() bool { return true } -func (*dummyUploader) GetLocalSchema(context.Context) (model.Schema, error) { return nil, nil } +func (*dummyUploader) IsWarehouseSchemaEmpty() bool { return true } +func (*dummyUploader) GetLocalSchema(context.Context) (model.Schema, error) { + return model.Schema{}, nil +} func (*dummyUploader) UpdateLocalSchema(context.Context, model.Schema) error { return nil } func (*dummyUploader) ShouldOnDedupUseNewRecord() bool { return false } func (*dummyUploader) GetFirstLastEvent() (time.Time, time.Time) { return time.Time{}, time.Time{} } From 443428048fda7202624e9cb2febcae9ba8a32dac Mon Sep 17 00:00:00 2001 From: Rohith BCS Date: Tue, 3 Sep 2024 18:16:19 +0530 Subject: [PATCH 05/29] feat: enable retention period for warehouse uploads (#5045) --- jobsdb/jobsdb.go | 7 ++-- warehouse/app.go | 6 +-- warehouse/archive/archiver.go | 59 ++++++++++++++++++++++++++-- warehouse/archive/archiver_test.go | 62 +++++++++++++++++++++++++++++- warehouse/archive/cron.go | 10 ++++- 5 files changed, 131 insertions(+), 13 deletions(-) diff --git a/jobsdb/jobsdb.go b/jobsdb/jobsdb.go index 6f92f3b1b7..382964f346 100644 --- a/jobsdb/jobsdb.go +++ b/jobsdb/jobsdb.go @@ -1483,8 +1483,9 @@ func (jd *Handle) prepareAndExecStmtInTxAllowMissing(tx *sql.Tx, sqlStatement st _, err = stmt.Exec() if err != nil { - pqError, ok := err.(*pq.Error) - if ok && pqError.Code == pq.ErrorCode("42P01") { + var pqError *pq.Error + ok := errors.As(err, &pqError) + if ok && pqError.Code == ("42P01") { jd.logger.Infof("[%s] sql statement(%s) exec failed because table doesn't exist", jd.tablePrefix, sqlStatement) _, err = tx.Exec(rollbackSql) jd.assertError(err) @@ -3178,7 +3179,7 @@ func (jd *Handle) GetLastJob(ctx context.Context) *JobT { var job JobT sqlStatement := fmt.Sprintf(`SELECT %[1]s.job_id, %[1]s.uuid, %[1]s.user_id, %[1]s.parameters, %[1]s.custom_val, %[1]s.event_payload, %[1]s.created_at, %[1]s.expire_at FROM %[1]s WHERE %[1]s.job_id = %[2]d`, dsList[len(dsList)-1].JobTable, maxID) err := jd.dbHandle.QueryRow(sqlStatement).Scan(&job.JobID, &job.UUID, &job.UserID, &job.Parameters, &job.CustomVal, &job.EventPayload, &job.CreatedAt, &job.ExpireAt) - if err != nil && err != sql.ErrNoRows { + if err != nil && !errors.Is(err, sql.ErrNoRows) { jd.assertError(err) } return &job diff --git a/warehouse/app.go b/warehouse/app.go index 6bd6a1d460..2f6bdfb3ef 100644 --- a/warehouse/app.go +++ b/warehouse/app.go @@ -11,15 +11,13 @@ import ( "sync/atomic" "time" - "golang.org/x/sync/errgroup" - - "github.com/rudderlabs/rudder-go-kit/sqlutil" - "github.com/cenkalti/backoff/v4" + "golang.org/x/sync/errgroup" "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/filemanager" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/sqlutil" "github.com/rudderlabs/rudder-go-kit/stats" "github.com/rudderlabs/rudder-server/admin" diff --git a/warehouse/archive/archiver.go b/warehouse/archive/archiver.go index f546d5ea94..f8e623b04e 100644 --- a/warehouse/archive/archiver.go +++ b/warehouse/archive/archiver.go @@ -9,10 +9,6 @@ import ( "net/url" "time" - sqlmw "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" - - "github.com/rudderlabs/rudder-go-kit/stats" - "github.com/iancoleman/strcase" "github.com/lib/pq" "github.com/tidwall/gjson" @@ -21,11 +17,13 @@ import ( "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/filemanager" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" "github.com/rudderlabs/rudder-server/services/archiver/tablearchiver" "github.com/rudderlabs/rudder-server/utils/filemanagerutil" "github.com/rudderlabs/rudder-server/utils/misc" "github.com/rudderlabs/rudder-server/utils/timeutil" + sqlmw "github.com/rudderlabs/rudder-server/warehouse/integrations/middleware/sqlquerywrapper" "github.com/rudderlabs/rudder-server/warehouse/internal/model" "github.com/rudderlabs/rudder-server/warehouse/multitenant" warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" @@ -61,7 +59,9 @@ type Archiver struct { config struct { archiveUploadRelatedRecords config.ValueLoader[bool] + canDeleteUploads config.ValueLoader[bool] uploadsArchivalTimeInDays config.ValueLoader[int] + uploadRetentionTimeInDays config.ValueLoader[int] archiverTickerTime config.ValueLoader[time.Duration] backupRowsBatchSize config.ValueLoader[int] maxLimit config.ValueLoader[int] @@ -88,7 +88,9 @@ func New( } a.config.archiveUploadRelatedRecords = a.conf.GetReloadableBoolVar(true, "Warehouse.archiveUploadRelatedRecords") + a.config.canDeleteUploads = a.conf.GetReloadableBoolVar(false, "Warehouse.canDeleteUploads") a.config.uploadsArchivalTimeInDays = a.conf.GetReloadableIntVar(5, 1, "Warehouse.uploadsArchivalTimeInDays") + a.config.uploadRetentionTimeInDays = a.conf.GetReloadableIntVar(90, 1, "Warehouse.uploadRetentionTimeInDays") a.config.backupRowsBatchSize = a.conf.GetReloadableIntVar(100, 1, "Warehouse.Archiver.backupRowsBatchSize") a.config.archiverTickerTime = a.conf.GetReloadableDurationVar(360, time.Minute, "Warehouse.archiverTickerTime", "Warehouse.archiverTickerTimeInMin") // default 6 hours a.config.maxLimit = a.conf.GetReloadableIntVar(10000, 1, "Warehouse.Archiver.maxLimit") @@ -510,3 +512,52 @@ func (a *Archiver) deleteLoadFileRecords( return nil } + +func (a *Archiver) Delete(ctx context.Context) error { + a.log.Infon(`Started deleting for warehouse`) + + maxLimit := a.config.maxLimit.Load() + + for { + count, err := a.deleteUploads(ctx, maxLimit) + if err != nil { + return fmt.Errorf("deleting uploads: %w", err) + } + if count == 0 { + break + } + } + return nil +} + +func (a *Archiver) deleteUploads(ctx context.Context, limit int) (int64, error) { + skipWorkspaceIDs := []string{""} + skipWorkspaceIDs = append(skipWorkspaceIDs, a.tenantManager.DegradedWorkspaces()...) + + sqlStatement := fmt.Sprintf(` + WITH rows_to_delete AS ( + SELECT ctid + FROM %[1]s + WHERE created_at < NOW() - $1::interval + AND status = $2 + AND NOT workspace_id = ANY ($3) + LIMIT $4 + ) + DELETE FROM %[1]s + WHERE ctid IN (SELECT ctid FROM rows_to_delete); +`, + pq.QuoteIdentifier(warehouseutils.WarehouseUploadsTable)) + + result, err := a.db.ExecContext( + ctx, + sqlStatement, + fmt.Sprintf("%d DAY", a.config.uploadRetentionTimeInDays.Load()), + model.ExportedData, + pq.Array(skipWorkspaceIDs), + limit, + ) + if err != nil { + return 0, fmt.Errorf("error deleting uploads: %w", err) + } + return result.RowsAffected() +} diff --git a/warehouse/archive/archiver_test.go b/warehouse/archive/archiver_test.go index 188e3d1b4f..62731bf216 100644 --- a/warehouse/archive/archiver_test.go +++ b/warehouse/archive/archiver_test.go @@ -155,7 +155,7 @@ func TestArchiver(t *testing.T) { db := sqlmw.New(pgResource.DB) archiver := archive.New( - config.New(), + c, logger.NOP, mockStats, db, @@ -214,6 +214,66 @@ func TestArchiver(t *testing.T) { } } +func TestArchiver_Delete(t *testing.T) { + var pgResource *postgres.Resource + + pool, err := dockertest.NewPool("") + require.NoError(t, err) + pgResource, err = postgres.Setup(pool, t) + require.NoError(t, err) + + t.Log("db:", pgResource.DBDsn) + + err = (&migrator.Migrator{ + Handle: pgResource.DB, + MigrationsTable: "wh_schema_migrations", + }).Migrate("warehouse") + require.NoError(t, err) + + sqlStatement, err := os.ReadFile("testdata/dump.sql") + require.NoError(t, err) + + _, err = pgResource.DB.Exec(string(sqlStatement)) + require.NoError(t, err) + + ctrl := gomock.NewController(t) + mockStats := mock_stats.NewMockStats(ctrl) + mockStats.EXPECT().NewStat(gomock.Any(), gomock.Any()).Times(1) + + status := model.ExportedData + workspaceID := "1" + _, err = pgResource.DB.Exec(` + UPDATE wh_uploads SET workspace_id = $1, status = $2 + `, workspaceID, status) + require.NoError(t, err) + + c := config.New() + c.Set("Warehouse.uploadRetentionTimeInDays", 0) + tenantManager := multitenant.New(c, backendConfig.DefaultBackendConfig) + + db := sqlmw.New(pgResource.DB) + + archiver := archive.New( + c, + logger.NOP, + mockStats, + db, + filemanager.New, + tenantManager, + ) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + err = archiver.Delete(ctx) + require.NoError(t, err) + + var count int + err = pgResource.DB.QueryRow(fmt.Sprintf(`SELECT COUNT(*) FROM %q`, warehouseutils.WarehouseUploadsTable)).Scan(&count) + require.NoError(t, err) + require.Zero(t, count, "wh_uploads rows should be deleted") +} + func jsonTestData(t require.TestingT, file string, value any) { f, err := os.Open(file) require.NoError(t, err) diff --git a/warehouse/archive/cron.go b/warehouse/archive/cron.go index d01b22dcc5..13e5c3a199 100644 --- a/warehouse/archive/cron.go +++ b/warehouse/archive/cron.go @@ -3,6 +3,8 @@ package archive import ( "context" "time" + + obskit "github.com/rudderlabs/rudder-observability-kit/go/labels" ) func CronArchiver(ctx context.Context, a *Archiver) { @@ -15,7 +17,13 @@ func CronArchiver(ctx context.Context, a *Archiver) { if a.config.archiveUploadRelatedRecords.Load() { err := a.Do(ctx) if err != nil { - a.log.Errorf(`Error archiving uploads: %v`, err) + a.log.Errorn(`Error archiving uploads`, obskit.Error(err)) + } + } + if a.config.canDeleteUploads.Load() { + err := a.Delete(ctx) + if err != nil { + a.log.Errorn(`Error deleting uploads`, obskit.Error(err)) } } } From 907c97cf8f265d1ba9e58c348aea68d9817cf173 Mon Sep 17 00:00:00 2001 From: Leonidas Vrachnis Date: Tue, 3 Sep 2024 16:49:18 +0200 Subject: [PATCH 06/29] chore: add trufflehog scan step before image push (#5035) --- .github/workflows/builds.yml | 81 +++++++++++++++++++++++++++++++++++- Makefile | 10 ++++- build/scan_docker.sh | 73 ++++++++++++++++++++++++++++++++ 3 files changed, 161 insertions(+), 3 deletions(-) create mode 100755 build/scan_docker.sh diff --git a/.github/workflows/builds.yml b/.github/workflows/builds.yml index b98cbeec8c..34f43ef26b 100644 --- a/.github/workflows/builds.yml +++ b/.github/workflows/builds.yml @@ -107,7 +107,32 @@ jobs: password: ${{ secrets.DOCKERHUB_TOKEN }} - name: Set up Docker Buildx uses: docker/setup-buildx-action@v3 + - name: Build image (no-push) + uses: docker/build-push-action@v6 + with: + context: . + platforms: ${{ matrix.build-config.platform }} + push: false + tags: ${{ matrix.build-config.tags }} + labels: ${{ matrix.build-config.labels }} + build-args: | + BUILD_DATE=${{ needs.docker-oss-meta.outputs.build-date }} + VERSION=${{ needs.docker-oss-meta.outputs.version }} + COMMIT_HASH=${{ github.sha }} + REVISION=${{ needs.docker-oss-meta.outputs.revision }} + outputs: type=docker,dest=${{ runner.temp }}/local-rudder-server.tar + cache-to: type=local,dest=${{ runner.temp }}/.buildx-cache,mode=max,tag=temp + - name: Run TruffleHog + run: | + docker pull trufflesecurity/trufflehog:latest + docker run --rm -v ${{ runner.temp }}:/tmp \ + trufflesecurity/trufflehog:latest \ + docker --image file:///tmp/local-rudder-server.tar \ + --github-actions \ + --no-verification \ + --fail - name: Build and push + if: success() uses: docker/build-push-action@v6 with: context: . @@ -120,7 +145,7 @@ jobs: VERSION=${{ needs.docker-oss-meta.outputs.version }} COMMIT_HASH=${{ github.sha }} REVISION=${{ needs.docker-oss-meta.outputs.revision }} - + cache-from: type=local,src=${{ runner.temp }}/.buildx-cache,tag=temp docker-ent-meta: runs-on: ubuntu-latest outputs: @@ -181,7 +206,33 @@ jobs: with: username: rudderlabs password: ${{ secrets.DOCKERHUB_TOKEN }} + - name: Build image (no-push) + uses: docker/build-push-action@v6 + with: + context: . + platforms: ${{ matrix.build-config.platform }} + push: false + tags: ${{ matrix.build-config.tags }} + labels: ${{ matrix.build-config.labels }} + build-args: | + BUILD_DATE=${{ needs.docker-ent-meta.outputs.build-date }} + VERSION=${{ needs.docker-ent-meta.outputs.version }} + COMMIT_HASH=${{ github.sha }} + REVISION=${{ needs.docker-ent-meta.outputs.revision }} + ENTERPRISE_TOKEN=${{ secrets.ENTERPRISE_TOKEN }} + outputs: type=docker,dest=${{ runner.temp }}/local-rudder-server.tar + cache-to: type=local,dest=${{ runner.temp }}/.buildx-cache,mode=max,tag=temp + - name: Run TruffleHog + run: | + docker pull trufflesecurity/trufflehog:latest + docker run --rm -v ${{ runner.temp }}:/tmp \ + trufflesecurity/trufflehog:latest \ + docker --image file:///tmp/local-rudder-server.tar \ + --github-actions \ + --no-verification \ + --fail - name: Build and push + if: success() uses: docker/build-push-action@v6 with: context: . @@ -195,6 +246,7 @@ jobs: COMMIT_HASH=${{ github.sha }} REVISION=${{ needs.docker-ent-meta.outputs.revision }} ENTERPRISE_TOKEN=${{ secrets.ENTERPRISE_TOKEN }} + cache-from: type=local,src=${{ runner.temp }}/.buildx-cache,tag=temp docker-sbsvc-meta: runs-on: ubuntu-latest outputs: @@ -255,7 +307,33 @@ jobs: with: username: rudderlabs password: ${{ secrets.DOCKERHUB_TOKEN }} + - name: Build image (no-push) + uses: docker/build-push-action@v6 + with: + context: . + platforms: ${{ matrix.build-config.platform }} + file: ./suppression-backup-service/Dockerfile + push: false + tags: ${{ matrix.build-config.tags }} + labels: ${{ matrix.build-config.labels }} + build-args: | + BUILD_DATE=${{ needs.docker-sbsvc-meta.outputs.build-date }} + VERSION=${{ needs.docker-sbsvc-meta.outputs.version }} + COMMIT_HASH=${{ github.sha }} + REVISION=${{ needs.docker-sbsvc-meta.outputs.revision }} + outputs: type=docker,dest=${{ runner.temp }}/suppression-backup-service.tar + cache-to: type=local,dest=${{ runner.temp }}/.buildx-cache,mode=max,tag=temp + - name: Run TruffleHog + run: | + docker pull trufflesecurity/trufflehog:latest + docker run --rm -v ${{ runner.temp }}:/tmp \ + trufflesecurity/trufflehog:latest \ + docker --image file:///tmp/suppression-backup-service.tar \ + --github-actions \ + --no-verification \ + --fail - name: Build and push + if: success() uses: docker/build-push-action@v6 with: context: . @@ -269,6 +347,7 @@ jobs: VERSION=${{ needs.docker-sbsvc-meta.outputs.version }} COMMIT_HASH=${{ github.sha }} REVISION=${{ needs.docker-sbsvc-meta.outputs.revision }} + cache-from: type=local,src=${{ runner.temp }}/.buildx-cache,tag=temp create-manifest-docker-oss: runs-on: ubuntu-latest needs: [ docker-oss, docker-oss-meta ] diff --git a/Makefile b/Makefile index 00a8c668d2..b689ade3c1 100644 --- a/Makefile +++ b/Makefile @@ -96,11 +96,12 @@ install-tools: $(GO) install $(protoc-gen-go) $(GO) install $(protoc-gen-go-grpc) $(GO) install $(gotestsum) + .PHONY: lint -lint: fmt ## Run linters on all go files +lint: fmt ## Run linters on all go files $(GO) run $(GOLANGCI) run -v - $(GO) run $(govulncheck) ./... $(GO) run $(actionlint) + @$(MAKE) sec .PHONY: fmt fmt: install-tools ## Formats all go files @@ -126,3 +127,8 @@ generate-openapi-spec: install-tools -g html2 \ -o ${MOUNT_PATH}/gateway/openapi +.PHONY: sec +sec: ## Run security checks + $(GO) run $(govulncheck) ./... + ./build/scan_docker.sh + ./build/scan_docker.sh -f ./suppression-backup-service/Dockerfile \ No newline at end of file diff --git a/build/scan_docker.sh b/build/scan_docker.sh new file mode 100755 index 0000000000..cd37b1bd47 --- /dev/null +++ b/build/scan_docker.sh @@ -0,0 +1,73 @@ +#!/bin/bash + +# Exit immediately if a command exits with a non-zero status +set -e + +# Function to display usage information +usage() { + echo "Usage: $0 [context_path] [-f ]" + echo " context_path: Path to the build context (default: current directory)" + echo " -f: Path to the Dockerfile (default: ./Dockerfile)" + exit 1 +} + +# Function to clean up resources +cleanup() { + local exit_code=$? + echo "Cleaning up..." + [ -n "$TAR_FILENAME" ] && rm -f "$TAR_FILENAME" + [ -n "$IMAGE_SHA" ] && docker rmi "$IMAGE_SHA" > /dev/null 2>&1 + exit $exit_code +} + +# Set trap to ensure cleanup happens even if the script exits unexpectedly +trap cleanup EXIT + +# Default values +CONTEXT="." +DOCKERFILE="./Dockerfile" + +# Parse command line arguments +if [ $# -gt 0 ] && [ "${1:0:1}" != "-" ]; then + CONTEXT="$1" + shift +fi + +while getopts "f:h" opt; do + case $opt in + f) DOCKERFILE="$OPTARG";; + h) usage;; + *) usage;; + esac +done + +# Build the Docker image and capture the SHA +echo "Building Docker image..." +IMAGE_SHA=$(docker build -q -f "$DOCKERFILE" "$CONTEXT") + +if [ $? -ne 0 ] || [ -z "$IMAGE_SHA" ]; then + echo "Error: Failed to build Docker image or capture its SHA." + exit 1 +fi + +echo "Image built with SHA: $IMAGE_SHA" + +# Create a unique filename for the tar +TAR_FILENAME="/tmp/image-${IMAGE_SHA#sha256:}-$(date +%s).tar" + +# Save the image to a tar file +echo "Saving image to a tar file..." +docker save "$IMAGE_SHA" > "$TAR_FILENAME" + +trivy image --input "$TAR_FILENAME" --scanners vuln,secret + +# Run TruffleHog scan +echo "Running TruffleHog scan..." +docker run --rm -v /tmp:/tmp trufflesecurity/trufflehog:latest \ + docker --image "file://$TAR_FILENAME" \ + --no-verification \ + --fail + + +echo "Build and scan completed. Image SHA: $IMAGE_SHA" + From bb0478131aa40196dc360a5d0010ddd45c01232c Mon Sep 17 00:00:00 2001 From: Leonidas Vrachnis Date: Tue, 3 Sep 2024 17:05:19 +0200 Subject: [PATCH 07/29] chore: include transformationId in transformer client metrics (#5055) --- processor/transformer/transformer.go | 13 +++- processor/transformer/transformer_test.go | 81 ++++++++++++++++------- 2 files changed, 70 insertions(+), 24 deletions(-) diff --git a/processor/transformer/transformer.go b/processor/transformer/transformer.go index 640de608f1..30991c21c8 100644 --- a/processor/transformer/transformer.go +++ b/processor/transformer/transformer.go @@ -375,11 +375,22 @@ func (trans *handle) request(ctx context.Context, url, stage string, data []Tran // endless backoff loop, only nil error or panics inside _ = backoff.RetryNotify( func() error { + transformationID := "" + if len(data[0].Destination.Transformations) > 0 { + transformationID = data[0].Destination.Transformations[0].ID + } + respData, statusCode = trans.doPost(ctx, rawJSON, url, stage, stats.Tags{ + "destinationType": data[0].Destination.DestinationDefinition.Name, + "destinationId": data[0].Destination.ID, + "sourceId": data[0].Metadata.SourceID, + "transformationId": transformationID, + "stage": stage, + + // Legacy tags: to be removed "dest_type": data[0].Destination.DestinationDefinition.Name, "dest_id": data[0].Destination.ID, "src_id": data[0].Metadata.SourceID, - "stage": stage, }) if statusCode == StatusCPDown { trans.cpDownGauge.Gauge(1) diff --git a/processor/transformer/transformer_test.go b/processor/transformer/transformer_test.go index 86ca09247e..ceef9540c9 100644 --- a/processor/transformer/transformer_test.go +++ b/processor/transformer/transformer_test.go @@ -15,6 +15,8 @@ import ( "go.uber.org/mock/gomock" + "github.com/rudderlabs/rudder-go-kit/testhelper/rand" + "github.com/rudderlabs/rudder-server/testhelper/backendconfigtest" warehouseutils "github.com/rudderlabs/rudder-server/warehouse/utils" "github.com/rudderlabs/rudder-server/utils/types" @@ -23,6 +25,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/logger/mock_logger" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-go-kit/stats/memstats" backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/gateway/response" @@ -147,23 +150,6 @@ func TestTransformer(t *testing.T) { srv := httptest.NewServer(ft) defer srv.Close() - tr := handle{} - tr.stat = stats.Default - tr.logger = logger.NOP - tr.conf = config.Default - tr.client = srv.Client() - tr.guardConcurrency = make(chan struct{}, 200) - tr.sentStat = tr.stat.NewStat("transformer_sent", stats.CountType) - tr.receivedStat = tr.stat.NewStat("transformer_received", stats.CountType) - tr.cpDownGauge = tr.stat.NewStat("control_plane_down", stats.GaugeType) - tr.config.timeoutDuration = 1 * time.Second - tr.config.failOnUserTransformTimeout = config.SingleValueLoader(true) - tr.config.failOnError = config.SingleValueLoader(true) - - tr.config.maxRetryBackoffInterval = config.SingleValueLoader(1 * time.Second) - - tr.config.maxRetry = config.SingleValueLoader(1) - tc := []struct { batchSize int eventsCount int @@ -180,6 +166,24 @@ func TestTransformer(t *testing.T) { } for _, tt := range tc { + statsStore, err := memstats.New() + require.NoError(t, err) + + tr := handle{} + tr.stat = statsStore + tr.logger = logger.NOP + tr.conf = config.Default + tr.client = srv.Client() + tr.guardConcurrency = make(chan struct{}, 200) + tr.sentStat = tr.stat.NewStat("transformer_sent", stats.CountType) + tr.receivedStat = tr.stat.NewStat("transformer_received", stats.CountType) + tr.cpDownGauge = tr.stat.NewStat("control_plane_down", stats.GaugeType) + tr.config.timeoutDuration = 1 * time.Second + tr.config.failOnUserTransformTimeout = config.SingleValueLoader(true) + tr.config.failOnError = config.SingleValueLoader(true) + tr.config.maxRetryBackoffInterval = config.SingleValueLoader(1 * time.Second) + tr.config.maxRetry = config.SingleValueLoader(1) + batchSize := tt.batchSize eventsCount := tt.eventsCount failEvery := tt.failEvery @@ -187,6 +191,18 @@ func TestTransformer(t *testing.T) { events := make([]TransformerEvent, eventsCount) expectedResponse := Response{} + transformationID := rand.String(10) + + destinationConfig := backendconfigtest.NewDestinationBuilder("WEBHOOK"). + WithUserTransformation(transformationID, rand.String(10)).Build() + + metadata := Metadata{ + DestinationType: destinationConfig.DestinationDefinition.Name, + SourceID: rand.String(10), + DestinationID: destinationConfig.ID, + TransformationID: destinationConfig.Transformations[0].ID, + } + for i := range events { msgID := fmt.Sprintf("messageID-%d", i) statusCode := http.StatusOK @@ -195,14 +211,16 @@ func TestTransformer(t *testing.T) { statusCode = http.StatusBadRequest } + metadata := metadata + metadata.MessageID = msgID + events[i] = TransformerEvent{ - Metadata: Metadata{ - MessageID: msgID, - }, + Metadata: metadata, Message: map[string]interface{}{ "src-key-1": msgID, "forceStatusCode": statusCode, }, + Destination: destinationConfig, Credentials: []Credential{ { ID: "test-credential", @@ -214,9 +232,7 @@ func TestTransformer(t *testing.T) { } tResp := TransformerResponse{ - Metadata: Metadata{ - MessageID: msgID, - }, + Metadata: metadata, StatusCode: statusCode, Output: map[string]interface{}{ "src-key-1": msgID, @@ -234,6 +250,25 @@ func TestTransformer(t *testing.T) { rsp := tr.transform(context.TODO(), events, srv.URL, batchSize, "test-stage") require.Equal(t, expectedResponse, rsp) + + metrics := statsStore.GetByName("processor.transformer_request_time") + if tt.eventsCount > 0 { + require.NotEmpty(t, metrics) + for _, m := range metrics { + require.Equal(t, stats.Tags{ + "stage": "test-stage", + "sourceId": metadata.SourceID, + "destinationType": destinationConfig.DestinationDefinition.Name, + "destinationId": destinationConfig.ID, + "transformationId": destinationConfig.Transformations[0].ID, + + // Legacy tags: to be removed + "dest_type": destinationConfig.DestinationDefinition.Name, + "dest_id": destinationConfig.ID, + "src_id": metadata.SourceID, + }, m.Tags) + } + } } }) From 59e1d3525602bb66c3c5d4d54d30b4984a219823 Mon Sep 17 00:00:00 2001 From: Rohith BCS Date: Wed, 4 Sep 2024 17:02:23 +0530 Subject: [PATCH 08/29] chore: remove workspace table (#4952) --- app/apphandlers/embeddedAppHandler.go | 3 - app/apphandlers/gatewayAppHandler.go | 3 - app/apphandlers/processorAppHandler.go | 3 - app/apphandlers/setup.go | 4 - services/validators/envValidator.go | 150 -------------------- warehouse/integrations/bigquery/bigquery.go | 3 +- warehouse/router/tracker_test.go | 15 +- 7 files changed, 8 insertions(+), 173 deletions(-) diff --git a/app/apphandlers/embeddedAppHandler.go b/app/apphandlers/embeddedAppHandler.go index e38dadd3cd..9cdb6c96ff 100644 --- a/app/apphandlers/embeddedAppHandler.go +++ b/app/apphandlers/embeddedAppHandler.go @@ -66,9 +66,6 @@ func (a *embeddedApp) Setup() error { if err := rudderCoreDBValidator(); err != nil { return err } - if err := rudderCoreWorkSpaceTableSetup(); err != nil { - return err - } if err := rudderCoreNodeSetup(); err != nil { return err } diff --git a/app/apphandlers/gatewayAppHandler.go b/app/apphandlers/gatewayAppHandler.go index 8cac82ef2b..4c85f5908a 100644 --- a/app/apphandlers/gatewayAppHandler.go +++ b/app/apphandlers/gatewayAppHandler.go @@ -42,9 +42,6 @@ func (a *gatewayApp) Setup() error { if err := rudderCoreDBValidator(); err != nil { return err } - if err := rudderCoreWorkSpaceTableSetup(); err != nil { - return err - } a.setupDone = true return nil } diff --git a/app/apphandlers/processorAppHandler.go b/app/apphandlers/processorAppHandler.go index 373761546f..81604621f2 100644 --- a/app/apphandlers/processorAppHandler.go +++ b/app/apphandlers/processorAppHandler.go @@ -76,9 +76,6 @@ func (a *processorApp) Setup() error { if err := rudderCoreDBValidator(); err != nil { return err } - if err := rudderCoreWorkSpaceTableSetup(); err != nil { - return err - } if err := rudderCoreNodeSetup(); err != nil { return err } diff --git a/app/apphandlers/setup.go b/app/apphandlers/setup.go index c5e963ffdf..7218809149 100644 --- a/app/apphandlers/setup.go +++ b/app/apphandlers/setup.go @@ -52,10 +52,6 @@ func rudderCoreNodeSetup() error { return validators.InitializeNodeMigrations() } -func rudderCoreWorkSpaceTableSetup() error { - return validators.CheckAndValidateWorkspaceToken() -} - // NewRsourcesService produces a rsources.JobService through environment configuration (env variables & config file) func NewRsourcesService(deploymentType deployment.Type, shouldSetupSharedDB bool) (rsources.JobService, error) { var rsourcesConfig rsources.JobServiceConfig diff --git a/services/validators/envValidator.go b/services/validators/envValidator.go index 88f1b37b0a..bb6dfdfcfd 100644 --- a/services/validators/envValidator.go +++ b/services/validators/envValidator.go @@ -3,11 +3,8 @@ package validators import ( "context" "database/sql" - "encoding/json" "errors" "fmt" - "strconv" - "time" "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" @@ -28,101 +25,6 @@ func init() { pkgLogger = logger.NewLogger().Child("validators").Child("envValidator") } -func createWorkspaceTable(dbHandle *sql.DB) error { - // Create table to store workspace token - sqlStatement := `CREATE TABLE IF NOT EXISTS workspace ( - token TEXT PRIMARY KEY, - created_at TIMESTAMP NOT NULL, - parameters JSONB);` - - _, err := dbHandle.Exec(sqlStatement) - if err != nil { - return fmt.Errorf("error creating workspace table: %w", err) - } - return nil -} - -func insertTokenIfNotExists(dbHandle *sql.DB) error { - // Read entries, if there are no entries insert hashed current workspace token - var totalCount int - sqlStatement := `SELECT COUNT(*) FROM workspace` - row := dbHandle.QueryRow(sqlStatement) - err := row.Scan(&totalCount) - if err != nil { - return fmt.Errorf("error reading workspace table: %w", err) - } - - if totalCount > 0 { - return nil - } - - // There are no entries in the table, hash current workspace token and insert - if _, err := dbHandle.Exec(`INSERT INTO workspace (token, created_at) - VALUES ($1, $2)`, - misc.GetMD5Hash(config.GetWorkspaceToken()), - time.Now()); err != nil { - return fmt.Errorf("error inserting workspace token: %w", err) - } - return nil -} - -func setWHSchemaVersionIfNotExists(dbHandle *sql.DB) error { - hashedToken := misc.GetMD5Hash(config.GetWorkspaceToken()) - whSchemaVersion := config.GetString("Warehouse.schemaVersion", "v1") - - var parameters sql.NullString - sqlStatement := fmt.Sprintf(`SELECT parameters FROM workspace WHERE token = '%s'`, hashedToken) - row := dbHandle.QueryRow(sqlStatement) - err := row.Scan(¶meters) - if err == sql.ErrNoRows { - return nil - } - if err != nil { - return fmt.Errorf("error reading workspace table: %w", err) - } - - if !parameters.Valid { - // insert current version - sqlStatement = fmt.Sprintf(`UPDATE workspace SET parameters = '{"wh_schema_version":%q}' WHERE token = '%s'`, whSchemaVersion, hashedToken) - _, err := dbHandle.Exec(sqlStatement) - if err != nil { - return fmt.Errorf("error updating workspace table: %w", err) - } - } else { - var parametersMap map[string]interface{} - err = json.Unmarshal([]byte(parameters.String), ¶metersMap) - if err != nil { - return fmt.Errorf("error unmarshalling parameters: %w", err) - } - if version, ok := parametersMap["wh_schema_version"]; ok { - whSchemaVersion = version.(string) - config.Set("Warehouse.schemaVersion", whSchemaVersion) - return nil - } - parametersMap["wh_schema_version"] = whSchemaVersion - marshalledParameters, err := json.Marshal(parametersMap) - if err != nil { - return fmt.Errorf("error marshalling parameters: %w", err) - } - sqlStatement = fmt.Sprintf(`UPDATE workspace SET parameters = '%s' WHERE token = '%s'`, marshalledParameters, hashedToken) - _, err = dbHandle.Exec(sqlStatement) - if err != nil { - return fmt.Errorf("error updating workspace table: %w", err) - } - } - return nil -} - -func getWorkspaceFromDB(dbHandle *sql.DB) (string, error) { - sqlStatement := `SELECT token FROM workspace order by created_at desc limit 1` - var token string - row := dbHandle.QueryRow(sqlStatement) - if err := row.Scan(&token); err != nil { - return token, fmt.Errorf("error reading workspace table: %w", err) - } - return token, nil -} - func createDBConnection() (*sql.DB, error) { psqlInfo := misc.GetConnectionString(config.Default, "env-validator") var err error @@ -241,55 +143,3 @@ func InitializeNodeMigrations() error { } return nil } - -func CheckAndValidateWorkspaceToken() error { - dbHandle, err := createDBConnection() - if err != nil { - return err - } - defer func() { _ = closeDBConnection(dbHandle) }() - - if err := createWorkspaceTable(dbHandle); err != nil { - return err - } - if err := insertTokenIfNotExists(dbHandle); err != nil { - return err - } - if err := setWHSchemaVersionIfNotExists(dbHandle); err != nil { - return err - } - - workspaceTokenHashInDB, err := getWorkspaceFromDB(dbHandle) - if err != nil { - return err - } - if workspaceTokenHashInDB == misc.GetMD5Hash(config.GetWorkspaceToken()) { - return nil - } - - // db connection should be closed. Else alter db fails. - // A new connection will be created again below, which will be closed on returning of this function (due to defer statement). - _ = closeDBConnection(dbHandle) - - pkgLogger.Warn("Previous workspace token is not same as the current workspace token. Parking current jobsdb aside and creating a new one") - - dbName := config.GetString("DB.name", "ubuntu") - misc.ReplaceDB(dbName, dbName+"_"+strconv.FormatInt(time.Now().Unix(), 10)+"_"+workspaceTokenHashInDB, config.Default) - - dbHandle, err = createDBConnection() - if err != nil { - return err - } - - // create workspace table and insert hashed token - if err := createWorkspaceTable(dbHandle); err != nil { - return err - } - if err := insertTokenIfNotExists(dbHandle); err != nil { - return err - } - if err := setWHSchemaVersionIfNotExists(dbHandle); err != nil { - return err - } - return nil -} diff --git a/warehouse/integrations/bigquery/bigquery.go b/warehouse/integrations/bigquery/bigquery.go index 4413c6a146..bad268a437 100644 --- a/warehouse/integrations/bigquery/bigquery.go +++ b/warehouse/integrations/bigquery/bigquery.go @@ -17,11 +17,10 @@ import ( "google.golang.org/api/iterator" "google.golang.org/api/option" - "github.com/rudderlabs/rudder-go-kit/stats" - "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/googleutil" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" "github.com/rudderlabs/rudder-server/utils/misc" "github.com/rudderlabs/rudder-server/warehouse/client" diff --git a/warehouse/router/tracker_test.go b/warehouse/router/tracker_test.go index 3c7a774504..716a67d062 100644 --- a/warehouse/router/tracker_test.go +++ b/warehouse/router/tracker_test.go @@ -219,13 +219,12 @@ func TestRouter_CronTracker(t *testing.T) { require.NoError(t, err) m := statsStore.GetByName("warehouse_cron_tracker_tick") - require.Equal(t, []memstats.Metric{{ - Name: "warehouse_cron_tracker_tick", - Tags: stats.Tags{ - "module": moduleName, - "destType": warehouseutils.POSTGRES, - }, - Value: 1, - }}, m) + require.Equal(t, len(m), 1) + require.Equal(t, m[0].Name, "warehouse_cron_tracker_tick") + require.Equal(t, m[0].Tags, stats.Tags{ + "module": moduleName, + "destType": warehouseutils.POSTGRES, + }) + require.GreaterOrEqual(t, m[0].Value, 1.0) }) } From cf6cb7a06d806641c7139c7aa6485f671ed5d530 Mon Sep 17 00:00:00 2001 From: "siddarth.msv" <82795818+Sidddddarth@users.noreply.github.com> Date: Wed, 4 Sep 2024 19:07:15 +0530 Subject: [PATCH 09/29] chore: sample events based on stage of failure (#4999) Co-authored-by: Rohith BCS Co-authored-by: Francesco Casula --- router/handle.go | 2 +- router/router_test.go | 100 +++++++++++++++++++++++------------------ router/types.go | 9 ++-- router/worker.go | 101 ++++++++++++++++++++++++++---------------- 4 files changed, 127 insertions(+), 85 deletions(-) diff --git a/router/handle.go b/router/handle.go index d95211ed6f..d0f26bd112 100644 --- a/router/handle.go +++ b/router/handle.go @@ -358,7 +358,7 @@ func (rt *Handle) commitStatusList(workerJobStatuses *[]workerJobStatus) { } sd, ok := statusDetailsMap[key] if !ok { - sampleEvent := workerJobStatus.job.EventPayload + sampleEvent := workerJobStatus.payload if rt.transientSources.Apply(parameters.SourceID) { sampleEvent = routerutils.EmptyPayload } diff --git a/router/router_test.go b/router/router_test.go index 82e2130606..74ac5dd9db 100644 --- a/router/router_test.go +++ b/router/router_test.go @@ -34,6 +34,7 @@ import ( mocksJobsDB "github.com/rudderlabs/rudder-server/mocks/jobsdb" mocksRouter "github.com/rudderlabs/rudder-server/mocks/router" mocksTransformer "github.com/rudderlabs/rudder-server/mocks/router/transformer" + mockutils "github.com/rudderlabs/rudder-server/mocks/utils/types" "github.com/rudderlabs/rudder-server/router/internal/eventorder" "github.com/rudderlabs/rudder-server/router/throttler" "github.com/rudderlabs/rudder-server/router/transformer" @@ -46,6 +47,8 @@ import ( "github.com/rudderlabs/rudder-server/utils/misc" "github.com/rudderlabs/rudder-server/utils/pubsub" testutils "github.com/rudderlabs/rudder-server/utils/tests" + txutils "github.com/rudderlabs/rudder-server/utils/tx" + utilTypes "github.com/rudderlabs/rudder-server/utils/types" ) const ( @@ -123,6 +126,7 @@ type testContext struct { mockRouterJobsDB *mocksJobsDB.MockJobsDB mockProcErrorsDB *mocksJobsDB.MockJobsDB mockBackendConfig *mocksBackendConfig.MockBackendConfig + mockReporting *mockutils.MockReporting } // Initialize mocks and common expectations @@ -132,6 +136,7 @@ func (c *testContext) Setup() { c.mockRouterJobsDB = mocksJobsDB.NewMockJobsDB(c.mockCtrl) c.mockProcErrorsDB = mocksJobsDB.NewMockJobsDB(c.mockCtrl) c.mockBackendConfig = mocksBackendConfig.NewMockBackendConfig(c.mockCtrl) + c.mockReporting = mockutils.NewMockReporting(c.mockCtrl) tFunc := c.asyncHelper.ExpectAndNotifyCallbackWithName("backend_config") @@ -1169,7 +1174,7 @@ var _ = Describe("router", func() { mockNetHandle := mocksRouter.NewMockNetHandle(c.mockCtrl) mockTransformer := mocksTransformer.NewMockTransformer(c.mockCtrl) router := &Handle{ - Reporting: &reporting.NOOP{}, + Reporting: c.mockReporting, netHandle: mockNetHandle, } c.mockBackendConfig.EXPECT().AccessToken().AnyTimes() @@ -1293,6 +1298,15 @@ var _ = Describe("router", func() { assertTransformJobStatuses(unprocessedJobsList[0], statuses[1], jobsdb.Succeeded.State, "200", 1) assertTransformJobStatuses(unprocessedJobsList[1], statuses[2], jobsdb.Succeeded.State, "200", 1) }) + c.mockReporting.EXPECT().Report(gomock.Any(), gomock.Any(), gomock.Any()).Times(1).DoAndReturn( + func(_ context.Context, metrics []*utilTypes.PUReportedMetric, _ *txutils.Tx) error { + Expect(metrics).To(HaveLen(1)) + Expect(metrics[0].StatusDetail.StatusCode).To(Equal(200)) + Expect(metrics[0].StatusDetail.Status).To(Equal(jobsdb.Succeeded.State)) + Expect(metrics[0].StatusDetail.SampleEvent).To(Equal(json.RawMessage(`{"message": "some transformed message"}`))) + return nil + }, + ) <-router.backendConfigInitialized worker := newPartitionWorker(context.Background(), router, gaDestinationID) @@ -1309,18 +1323,16 @@ var _ = Describe("router", func() { }, 20*time.Second, 100*time.Millisecond).Should(Equal(true)) }) - It("aborts jobs if batching fails for few of the jobs", func() { + It("fails jobs if batching fails for few of the jobs", func() { mockNetHandle := mocksRouter.NewMockNetHandle(c.mockCtrl) mockTransformer := mocksTransformer.NewMockTransformer(c.mockCtrl) router := &Handle{ - Reporting: &reporting.NOOP{}, + Reporting: c.mockReporting, netHandle: mockNetHandle, } c.mockBackendConfig.EXPECT().AccessToken().AnyTimes() router.Setup(gaDestinationDefinition, logger.NOP, conf, c.mockBackendConfig, c.mockRouterJobsDB, c.mockProcErrorsDB, transientsource.NewEmptyService(), rsources.NewNoOpService(), transformerFeaturesService.NewNoOpService(), destinationdebugger.NewNoOpService(), throttler.NewNoOpThrottlerFactory()) - // we have a job that has failed once(toRetryJobsList), it should abort when picked up next - // Because we only allow one failure per job with this router.transformer = mockTransformer router.reloadableConfig.noOfJobsToBatchInAWorker = config.SingleValueLoader(3) router.reloadableConfig.maxFailedCountForJob = config.SingleValueLoader(5) @@ -1329,23 +1341,18 @@ var _ = Describe("router", func() { gaPayload := `{"body": {"XML": {}, "FORM": {}, "JSON": {}}, "type": "REST", "files": {}, "method": "POST", "params": {"t": "event", "v": "1", "an": "RudderAndroidClient", "av": "1.0", "ds": "android-sdk", "ea": "Demo Track", "ec": "Demo Category", "el": "Demo Label", "ni": 0, "qt": 59268380964, "ul": "en-US", "cid": "anon_id", "tid": "UA-185645846-1", "uip": "[::1]", "aiid": "com.rudderlabs.android.sdk"}, "userId": "anon_id", "headers": {}, "version": "1", "endpoint": "https://www.google-analytics.com/collect"}` parameters := fmt.Sprintf(`{"source_id": "%s", "destination_id": "%s", "message_id": "2f548e6d-60f6-44af-a1f4-62b3272445c3", "received_at": "2021-06-28T10:04:48.527+05:30", "transform_at": "processor"}`, sourceIDEnabled, gaDestinationID) // skipcq: GO-R4002 - toRetryJobsList := []*jobsdb.JobT{ + unprocessedJobsList := []*jobsdb.JobT{ { - UUID: uuid.New(), - UserID: "u1", - JobID: 2009, - CreatedAt: time.Date(2020, 0o4, 28, 13, 26, 0o0, 0o0, time.UTC), - ExpireAt: time.Date(2020, 0o4, 28, 13, 26, 0o0, 0o0, time.UTC), - CustomVal: customVal["GA"], - EventPayload: []byte(gaPayload), - LastJobStatus: jobsdb.JobStatusT{ - AttemptNum: 1, - ErrorResponse: []byte(`{"firstAttemptedAt": "2021-06-28T15:57:30.742+05:30"}`), - }, - Parameters: []byte(parameters), + UUID: uuid.New(), + UserID: "u1", + JobID: 2009, + CreatedAt: time.Date(2020, 0o4, 28, 13, 26, 0o0, 0o0, time.UTC), + ExpireAt: time.Date(2020, 0o4, 28, 13, 26, 0o0, 0o0, time.UTC), + CustomVal: customVal["GA"], + EventPayload: []byte(gaPayload), + LastJobStatus: jobsdb.JobStatusT{}, + Parameters: []byte(parameters), }, - } - unprocessedJobsList := []*jobsdb.JobT{ { UUID: uuid.New(), UserID: "u1", @@ -1373,7 +1380,7 @@ var _ = Describe("router", func() { Parameters: []byte(parameters), }, } - allJobs := append(toRetryJobsList, unprocessedJobsList...) + allJobs := unprocessedJobsList payloadLimit := router.reloadableConfig.payloadLimit callAllJobs := c.mockRouterJobsDB.EXPECT().GetToProcess(gomock.Any(), @@ -1382,38 +1389,34 @@ var _ = Describe("router", func() { ParameterFilters: []jobsdb.ParameterFilterT{{Name: "destination_id", Value: gaDestinationID}}, PayloadSizeLimit: payloadLimit.Load(), JobsLimit: 10000, - }, nil).Return(&jobsdb.MoreJobsResult{JobsResult: jobsdb.JobsResult{Jobs: toRetryJobsList}}, nil).Times( - 1).Return(&jobsdb.MoreJobsResult{JobsResult: jobsdb.JobsResult{Jobs: allJobs}}, nil) + }, nil).Times(1).Return(&jobsdb.MoreJobsResult{JobsResult: jobsdb.JobsResult{Jobs: allJobs}}, nil) c.mockRouterJobsDB.EXPECT().UpdateJobStatus(gomock.Any(), gomock.Any(), []string{customVal["GA"]}, nil).Times(1). Do(func(ctx context.Context, statuses []*jobsdb.JobStatusT, _, _ interface{}) { - assertJobStatus(toRetryJobsList[0], statuses[0], jobsdb.Executing.State, "", `{}`, 1) - assertJobStatus(unprocessedJobsList[0], statuses[1], jobsdb.Executing.State, "", `{}`, 0) - assertJobStatus(unprocessedJobsList[1], statuses[2], jobsdb.Executing.State, "", `{}`, 0) + assertJobStatus(unprocessedJobsList[0], statuses[0], jobsdb.Executing.State, "", `{}`, 0) + assertJobStatus(unprocessedJobsList[1], statuses[1], jobsdb.Executing.State, "", `{}`, 0) + assertJobStatus(unprocessedJobsList[2], statuses[2], jobsdb.Executing.State, "", `{}`, 0) }).Return(nil).After(callAllJobs) mockTransformer.EXPECT().Transform("BATCH", gomock.Any()).After(callAllJobs).Times(1).DoAndReturn( func(_ string, transformMessage *types.TransformMessageT) []types.DestinationJobT { - assertRouterJobs(&transformMessage.Data[0], toRetryJobsList[0]) - assertRouterJobs(&transformMessage.Data[1], unprocessedJobsList[0]) - assertRouterJobs(&transformMessage.Data[2], unprocessedJobsList[1]) + assertRouterJobs(&transformMessage.Data[0], unprocessedJobsList[0]) + assertRouterJobs(&transformMessage.Data[1], unprocessedJobsList[1]) + assertRouterJobs(&transformMessage.Data[2], unprocessedJobsList[2]) return []types.DestinationJobT{ { Message: []byte(`{"message": "some transformed message"}`), JobMetadataArray: []types.JobMetadataT{ { - UserID: "u1", - JobID: 2009, - JobT: toRetryJobsList[0], - FirstAttemptedAt: "2021-06-28T15:57:30.742+05:30", - AttemptNum: 1, + UserID: "u1", + JobID: 2009, + JobT: unprocessedJobsList[0], }, { - UserID: "u1", - JobID: 2010, - JobT: unprocessedJobsList[0], - FirstAttemptedAt: "2021-06-28T15:57:30.742+05:30", + UserID: "u1", + JobID: 2010, + JobT: unprocessedJobsList[1], }, }, Batched: true, @@ -1421,12 +1424,12 @@ var _ = Describe("router", func() { StatusCode: 500, }, { - Message: []byte(`{"message": "some transformed message"}`), + Message: []byte(`{"message": "some other transformed message"}`), JobMetadataArray: []types.JobMetadataT{ { UserID: "u1", JobID: 2011, - JobT: unprocessedJobsList[1], + JobT: unprocessedJobsList[2], }, }, Batched: true, @@ -1445,10 +1448,21 @@ var _ = Describe("router", func() { }).Return(nil) c.mockRouterJobsDB.EXPECT().UpdateJobStatusInTx(gomock.Any(), gomock.Any(), gomock.Any(), []string{customVal["GA"]}, nil).Times(1). Do(func(ctx context.Context, _ interface{}, statuses []*jobsdb.JobStatusT, _, _ interface{}) { - assertTransformJobStatuses(toRetryJobsList[0], statuses[0], jobsdb.Failed.State, "500", 2) - assertTransformJobStatuses(unprocessedJobsList[0], statuses[1], jobsdb.Waiting.State, "", 0) - assertTransformJobStatuses(unprocessedJobsList[1], statuses[2], jobsdb.Waiting.State, "", 0) + assertTransformJobStatuses(unprocessedJobsList[0], statuses[0], jobsdb.Failed.State, "500", 1) + assertTransformJobStatuses(unprocessedJobsList[1], statuses[1], jobsdb.Waiting.State, "", 0) + assertTransformJobStatuses(unprocessedJobsList[2], statuses[2], jobsdb.Waiting.State, "", 0) }) + c.mockReporting.EXPECT().Report(gomock.Any(), gomock.Any(), gomock.Any()).Times(1).DoAndReturn( + func(_ context.Context, metrics []*utilTypes.PUReportedMetric, _ *txutils.Tx) error { + Expect(metrics).To(HaveLen(1)) + Expect(metrics[0].StatusDetail.StatusCode).To(Equal(500)) + Expect(metrics[0].StatusDetail.Status).To(Equal(jobsdb.Failed.State)) + Expect(metrics[0].StatusDetail.SampleEvent).To(Equal(json.RawMessage(gaPayload))) + Expect(metrics[0].StatusDetail.SampleResponse).To(ContainSubstring(`failureStage":"RudderStack Transformation Error"`)) + + return nil + }, + ) <-router.backendConfigInitialized worker := newPartitionWorker(context.Background(), router, gaDestinationID) diff --git a/router/types.go b/router/types.go index 16f6e48b68..71609953ee 100644 --- a/router/types.go +++ b/router/types.go @@ -13,10 +13,11 @@ import ( ) type workerJobStatus struct { - userID string - worker *worker - job *jobsdb.JobT - status *jobsdb.JobStatusT + userID string + worker *worker + job *jobsdb.JobT + status *jobsdb.JobStatusT + payload json.RawMessage } type HandleDestOAuthRespParams struct { diff --git a/router/worker.go b/router/worker.go index 22738ab920..e4bbf8cc71 100644 --- a/router/worker.go +++ b/router/worker.go @@ -112,7 +112,13 @@ func (w *worker) workLoop() { // Enhancing job parameter with the drain reason. job.Parameters = routerutils.EnhanceJSON(job.Parameters, "stage", "router") job.Parameters = routerutils.EnhanceJSON(job.Parameters, "reason", abortReason) - w.rt.responseQ <- workerJobStatus{userID: userID, worker: w, job: job, status: &status} + w.rt.responseQ <- workerJobStatus{ + userID: userID, + worker: w, + job: job, + status: &status, + payload: job.EventPayload, + } stats.Default.NewTaggedStat(`drained_events`, stats.CountType, stats.Tags{ "destType": w.rt.destType, "destId": parameters.DestinationID, @@ -597,7 +603,7 @@ func (w *worker) processDestinationJobs() { respStatusCode := http.StatusInternalServerError var respBody string if !w.rt.enableBatching { - respBody = "skipping sending to destination because previous job (of user) in batch is failed." + respBody = "skipping sending to destination because previous job (of user) in batch failed." } respStatusCodes, respBodys = w.prepareResponsesForJobs(&destinationJob, respStatusCode, respBody) errorAt = routerutils.ERROR_AT_TF @@ -979,6 +985,15 @@ func (w *worker) postStatusOnResponseQ(respStatusCode int, payload json.RawMessa } } + inputPayload := payload + switch errorAt { + case routerutils.ERROR_AT_TF: + inputPayload = destinationJobMetadata.JobT.EventPayload + status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "failureStage", "RudderStack Transformation Error") + default: // includes ERROR_AT_DEL, ERROR_AT_CUST + status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "failureStage", "Destination Error") + } + status.ErrorResponse = routerutils.EnhanceJSON(status.ErrorResponse, "firstAttemptedAt", firstAttemptedAtTime.Format(misc.RFC3339Milli)) status.ErrorResponse = routerutils.EnhanceJSON(status.ErrorResponse, "content-type", respContentType) @@ -988,47 +1003,59 @@ func (w *worker) postStatusOnResponseQ(respStatusCode int, payload json.RawMessa status.JobState = jobsdb.Filtered.State } w.logger.Debugf("sending success status to response") - w.rt.responseQ <- workerJobStatus{userID: destinationJobMetadata.UserID, worker: w, job: destinationJobMetadata.JobT, status: status} - } else { - // Saving payload to DB only - // 1. if job failed and - // 2. if router job undergoes batching or dest transform. - if payload != nil && (w.rt.enableBatching || destinationJobMetadata.TransformAt == "router") { - if w.rt.reloadableConfig.savePayloadOnError.Load() { - status.ErrorResponse = routerutils.EnhanceJSON(status.ErrorResponse, "payload", string(payload)) - } + w.rt.responseQ <- workerJobStatus{ + userID: destinationJobMetadata.UserID, + worker: w, + job: destinationJobMetadata.JobT, + status: status, + payload: inputPayload, } - // the job failed - w.logger.Debugf("Job failed to send, analyzing...") - - if isJobTerminated(respStatusCode) { - status.JobState = jobsdb.Aborted.State - w.updateAbortedMetrics(destinationJobMetadata.DestinationID, status.WorkspaceId, status.ErrorCode, errorAt) - destinationJobMetadata.JobT.Parameters = misc.UpdateJSONWithNewKeyVal(destinationJobMetadata.JobT.Parameters, "stage", "router") - destinationJobMetadata.JobT.Parameters = misc.UpdateJSONWithNewKeyVal(destinationJobMetadata.JobT.Parameters, "reason", status.ErrorResponse) // NOTE: Old key used was "error_response" - } else { - status.JobState = jobsdb.Failed.State - if !w.rt.retryLimitReached(status) { // don't delay retry time if retry limit is reached, so that the job can be aborted immediately on the next loop - status.RetryTime = status.ExecTime.Add(nextAttemptAfter(status.AttemptNum, w.rt.reloadableConfig.minRetryBackoff.Load(), w.rt.reloadableConfig.maxRetryBackoff.Load())) - } + return + } + // Saving payload to DB only + // 1. if job failed and + // 2. if router job undergoes batching or dest transform. + if payload != nil && (w.rt.enableBatching || destinationJobMetadata.TransformAt == "router") { + if w.rt.reloadableConfig.savePayloadOnError.Load() { + status.ErrorResponse = routerutils.EnhanceJSON(status.ErrorResponse, "payload", string(payload)) } + } + // the job failed + w.logger.Debugn("Job failed to send, analyzing...") + + if isJobTerminated(respStatusCode) { + status.JobState = jobsdb.Aborted.State + w.updateAbortedMetrics(destinationJobMetadata.DestinationID, status.WorkspaceId, status.ErrorCode, errorAt) + destinationJobMetadata.JobT.Parameters = misc.UpdateJSONWithNewKeyVal(destinationJobMetadata.JobT.Parameters, "stage", "router") + destinationJobMetadata.JobT.Parameters = misc.UpdateJSONWithNewKeyVal(destinationJobMetadata.JobT.Parameters, "reason", status.ErrorResponse) // NOTE: Old key used was "error_response" + } else { + status.JobState = jobsdb.Failed.State + if !w.rt.retryLimitReached(status) { // don't delay retry time if retry limit is reached, so that the job can be aborted immediately on the next loop + status.RetryTime = status.ExecTime.Add(nextAttemptAfter(status.AttemptNum, w.rt.reloadableConfig.minRetryBackoff.Load(), w.rt.reloadableConfig.maxRetryBackoff.Load())) + } + } - if w.rt.guaranteeUserEventOrder { - if status.JobState == jobsdb.Failed.State { + if w.rt.guaranteeUserEventOrder { + if status.JobState == jobsdb.Failed.State { - orderKey := eventorder.BarrierKey{ - UserID: destinationJobMetadata.UserID, - DestinationID: destinationJobMetadata.DestinationID, - WorkspaceID: destinationJobMetadata.WorkspaceID, - } - w.logger.Debugf("EventOrder: [%d] job %d for key %s failed", w.id, status.JobID, orderKey) - if err := w.barrier.StateChanged(orderKey, destinationJobMetadata.JobID, status.JobState); err != nil { - panic(err) - } + orderKey := eventorder.BarrierKey{ + UserID: destinationJobMetadata.UserID, + DestinationID: destinationJobMetadata.DestinationID, + WorkspaceID: destinationJobMetadata.WorkspaceID, + } + w.logger.Debugf("EventOrder: [%d] job %d for key %s failed", w.id, status.JobID, orderKey) + if err := w.barrier.StateChanged(orderKey, destinationJobMetadata.JobID, status.JobState); err != nil { + panic(err) } } - w.logger.Debugf("sending failed/aborted state as response") - w.rt.responseQ <- workerJobStatus{userID: destinationJobMetadata.UserID, worker: w, job: destinationJobMetadata.JobT, status: status} + } + w.logger.Debugf("sending failed/aborted state as response") + w.rt.responseQ <- workerJobStatus{ + userID: destinationJobMetadata.UserID, + worker: w, + job: destinationJobMetadata.JobT, + status: status, + payload: inputPayload, } } From bbbd07f05a975517bab46f067618d76fa9fe9531 Mon Sep 17 00:00:00 2001 From: "siddarth.msv" <82795818+Sidddddarth@users.noreply.github.com> Date: Wed, 4 Sep 2024 21:53:58 +0530 Subject: [PATCH 10/29] chore: remove multiGzipWriter (#5016) Co-authored-by: Leonidas Vrachnis Co-authored-by: Rohith BCS Co-authored-by: Francesco Casula --- archiver/worker.go | 13 +++- cmd/backupfilemigrator/file_migrator.go | 19 ++++-- processor/stash/stash.go | 86 ++++++++++++++++--------- router/batchrouter/handle.go | 12 +++- services/fileuploader/gzwriter.go | 62 ------------------ services/fileuploader/gzwriter_test.go | 33 ---------- 6 files changed, 86 insertions(+), 139 deletions(-) delete mode 100644 services/fileuploader/gzwriter.go delete mode 100644 services/fileuploader/gzwriter_test.go diff --git a/archiver/worker.go b/archiver/worker.go index a02abde756..4c4046a4a1 100644 --- a/archiver/worker.go +++ b/archiver/worker.go @@ -7,6 +7,7 @@ import ( "fmt" "os" "path" + "path/filepath" "time" "github.com/google/uuid" @@ -155,13 +156,20 @@ func (w *worker) uploadJobs(ctx context.Context, jobs []*jobsdb.JobT) (string, e lastJobCreatedAt := jobs[len(jobs)-1].CreatedAt.UTC() workspaceID := jobs[0].WorkspaceId - gzWriter := fileuploader.NewGzMultiFileWriter() filePath := path.Join( lo.Must(misc.CreateTMPDIR()), "rudder-backups", w.sourceID, fmt.Sprintf("%d_%d_%s_%s.json.gz", firstJobCreatedAt.Unix(), lastJobCreatedAt.Unix(), workspaceID, uuid.NewString()), ) + if err := os.MkdirAll(filepath.Dir(filePath), os.ModePerm); err != nil { + return "", fmt.Errorf("creating gz file %q: mkdir error: %w", filePath, err) + } + gzWriter, err := misc.CreateGZ(filePath) + if err != nil { + return "", fmt.Errorf("create gz writer: %w", err) + } + defer func() { _ = os.Remove(filePath) }() for _, job := range jobs { j, err := marshalJob(job) @@ -169,7 +177,7 @@ func (w *worker) uploadJobs(ctx context.Context, jobs []*jobsdb.JobT) (string, e _ = gzWriter.Close() return "", fmt.Errorf("marshal job: %w", err) } - if _, err := gzWriter.Write(filePath, append(j, '\n')); err != nil { + if _, err := gzWriter.Write(append(j, '\n')); err != nil { _ = gzWriter.Close() return "", fmt.Errorf("write to file: %w", err) } @@ -177,7 +185,6 @@ func (w *worker) uploadJobs(ctx context.Context, jobs []*jobsdb.JobT) (string, e if err := gzWriter.Close(); err != nil { return "", fmt.Errorf("close writer: %w", err) } - defer func() { _ = os.Remove(filePath) }() fileUploader, err := w.storageProvider.GetFileManager(w.lifecycle.ctx, workspaceID) if err != nil { diff --git a/cmd/backupfilemigrator/file_migrator.go b/cmd/backupfilemigrator/file_migrator.go index 209bd63fcc..a2d808e96c 100644 --- a/cmd/backupfilemigrator/file_migrator.go +++ b/cmd/backupfilemigrator/file_migrator.go @@ -25,7 +25,6 @@ import ( "github.com/samber/lo" - "github.com/rudderlabs/rudder-server/services/fileuploader" "github.com/rudderlabs/rudder-server/utils/misc" "github.com/tidwall/gjson" @@ -129,29 +128,35 @@ func (m *fileMigrator) uploadFile(ctx context.Context, jobs []*newFileFormat, so firstJobCreatedAt := jobs[0].CreatedAt.UTC() lastJobCreatedAt := jobs[len(jobs)-1].CreatedAt.UTC() - gzWriter := fileuploader.NewGzMultiFileWriter() localFilePath := path.Join( lo.Must(misc.CreateTMPDIR()), "rudder-backups", sourceId, fmt.Sprintf("%d_%d_%s.json.gz", firstJobCreatedAt.Unix(), lastJobCreatedAt.Unix(), workspaceId), ) + if err := os.MkdirAll(filepath.Dir(localFilePath), os.ModePerm); err != nil { + return fmt.Errorf("creating gz file %q: mkdir error: %w", localFilePath, err) + } + writer, err := misc.CreateGZ(localFilePath) + if err != nil { + return fmt.Errorf("failed to create gz writer: %w", err) + } + defer func() { _ = os.Remove(localFilePath) }() for _, job := range jobs { jobBytes, err := json.Marshal(job) if err != nil { - _ = gzWriter.Close() + _ = writer.Close() return fmt.Errorf("failed to marshal job: %w", err) } - if _, err := gzWriter.Write(localFilePath, append(jobBytes, '\n')); err != nil { - _ = gzWriter.Close() + if _, err := writer.Write(append(jobBytes, '\n')); err != nil { + _ = writer.Close() return fmt.Errorf("write to local file failed: %w", err) } } - if err := gzWriter.Close(); err != nil { + if err := writer.Close(); err != nil { return fmt.Errorf("failed to close writer: %w", err) } - defer func() { _ = os.Remove(localFilePath) }() localFile, err := os.Open(localFilePath) if err != nil { diff --git a/processor/stash/stash.go b/processor/stash/stash.go index 68d700d9f2..e0d533b46f 100644 --- a/processor/stash/stash.go +++ b/processor/stash/stash.go @@ -3,8 +3,10 @@ package stash import ( "context" "encoding/json" + "errors" "fmt" "os" + "path/filepath" "sync" "time" @@ -17,6 +19,8 @@ import ( "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + kitsync "github.com/rudderlabs/rudder-go-kit/sync" + obskit "github.com/rudderlabs/rudder-observability-kit/go/labels" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/services/fileuploader" "github.com/rudderlabs/rudder-server/services/transientsource" @@ -144,7 +148,7 @@ func (st *HandleT) storeErrorsToObjectStorage(jobs []*jobsdb.JobT) (errorJob []E localTmpDirName := "/rudder-processor-errors/" uuid := uuid.New().String() - st.logger.Debug("[Processor: storeErrorsToObjectStorage]: Starting logging to object storage") + st.logger.Debugn("[Processor: storeErrorsToObjectStorage]: Starting logging to object storage") tmpDirPath, err := misc.CreateTMPDIR() if err != nil { @@ -154,8 +158,7 @@ func (st *HandleT) storeErrorsToObjectStorage(jobs []*jobsdb.JobT) (errorJob []E jobsPerWorkspace := lo.GroupBy(jobs, func(job *jobsdb.JobT) string { return job.WorkspaceId }) - gzWriter := fileuploader.NewGzMultiFileWriter() - dumps := make(map[string]string) + writerMap := make(map[string]string) errorJobs := make([]ErrorJob, 0) @@ -163,7 +166,7 @@ func (st *HandleT) storeErrorsToObjectStorage(jobs []*jobsdb.JobT) (errorJob []E for workspaceID, jobsForWorkspace := range jobsPerWorkspace { preferences, err := st.fileuploader.GetStoragePreferences(ctx, workspaceID) if err != nil { - st.logger.Errorf("Skipping Storing errors for workspace: %s since no storage preferences are found", workspaceID) + st.logger.Errorn("Skipping Storing errors for workspace since no storage preferences are found", obskit.WorkspaceID(workspaceID), obskit.Error(err)) errorJobs = append(errorJobs, ErrorJob{ jobs: jobsForWorkspace, errorOutput: StoreErrorOutputT{ @@ -174,7 +177,7 @@ func (st *HandleT) storeErrorsToObjectStorage(jobs []*jobsdb.JobT) (errorJob []E continue } if !preferences.ProcErrors { - st.logger.Infof("Skipping Storing errors for workspace: %s since ProcErrors is set to false", workspaceID) + st.logger.Infon("Skipping Storing errors for workspace since ProcErrors is set to false", obskit.WorkspaceID(workspaceID)) errorJobs = append(errorJobs, ErrorJob{ jobs: jobsForWorkspace, errorOutput: StoreErrorOutputT{ @@ -184,48 +187,64 @@ func (st *HandleT) storeErrorsToObjectStorage(jobs []*jobsdb.JobT) (errorJob []E }) continue } - path := fmt.Sprintf("%v%v.json.gz", tmpDirPath+localTmpDirName, fmt.Sprintf("%v.%v.%v.%v.%v", time.Now().Unix(), config.GetString("INSTANCE_ID", "1"), fmt.Sprintf("%v-%v", jobs[0].JobID, jobs[len(jobs)-1].JobID), uuid, workspaceID)) - dumps[workspaceID] = path + path := filepath.Join( + tmpDirPath, + localTmpDirName, + fmt.Sprintf( + "%v.%v.%v.%v.%v.json.gz", + time.Now().Unix(), + config.GetString("INSTANCE_ID", "1"), + fmt.Sprintf("%v-%v", jobs[0].JobID, jobs[len(jobs)-1].JobID), + uuid, + workspaceID, + ), + ) + if err := os.MkdirAll(filepath.Dir(path), os.ModePerm); err != nil { + panic(fmt.Errorf("creating gz file %q: mkdir error: %w", path, err)) + } + writer, err := misc.CreateGZ(path) + if err != nil { + panic(err) + } + writerMap[workspaceID] = path newline := []byte("\n") lo.ForEach(jobsForWorkspace, func(job *jobsdb.JobT, _ int) { rawJob, err := json.Marshal(job) if err != nil { panic(err) } - if _, err := gzWriter.Write(path, append(rawJob, newline...)); err != nil { + if _, err := writer.Write(append(rawJob, newline...)); err != nil { + _ = writer.Close() panic(err) } }) + if err := writer.Close(); err != nil { + panic(err) + } } - err = gzWriter.Close() - if err != nil { - panic(err) - } defer func() { - for _, path := range dumps { - os.Remove(path) + for _, path := range writerMap { + _ = os.Remove(path) } }() - g, ctx := errgroup.WithContext(ctx) - g.SetLimit(config.GetInt("Processor.errorBackupWorkers", 100)) + g, ctx := kitsync.NewEagerGroup(ctx, config.GetInt("Processor.errorBackupWorkers", 100)) var mu sync.Mutex - for workspaceID, filePath := range dumps { - wrkId := workspaceID - path := filePath - errFileUploader, err := st.fileuploader.GetFileManager(ctx, wrkId) + for workspaceID, path := range writerMap { + errFileUploader, err := st.fileuploader.GetFileManager(ctx, workspaceID) if err != nil { - st.logger.Errorf("Skipping Storing errors for workspace: %s since no file manager is found", workspaceID) - mu.Lock() - errorJobs = append(errorJobs, ErrorJob{ - jobs: jobsPerWorkspace[workspaceID], - errorOutput: StoreErrorOutputT{ - Location: "", - Error: err, - }, - }) - mu.Unlock() + st.logger.Errorn("Skipping Storing errors for workspace since no file manager is found", obskit.WorkspaceID(workspaceID), obskit.Error(err)) + if !errors.Is(err, fileuploader.ErrNotSubscribed) { + mu.Lock() + errorJobs = append(errorJobs, ErrorJob{ + jobs: jobsPerWorkspace[workspaceID], + errorOutput: StoreErrorOutputT{ + Error: err, + }, + }) + mu.Unlock() + } continue } g.Go(crash.Wrapper(func() error { @@ -235,10 +254,13 @@ func (st *HandleT) storeErrorsToObjectStorage(jobs []*jobsdb.JobT) (errorJob []E } prefixes := []string{"rudder-proc-err-logs", time.Now().Format("01-02-2006")} uploadOutput, err := errFileUploader.Upload(ctx, outputFile, prefixes...) - st.logger.Infof("Uploaded error logs to %s for workspaceId %s", uploadOutput.Location, wrkId) + st.logger.Infon("Uploaded error logs for workspaceId", + logger.NewStringField("location", uploadOutput.Location), + obskit.WorkspaceID(workspaceID), + ) mu.Lock() errorJobs = append(errorJobs, ErrorJob{ - jobs: jobsPerWorkspace[wrkId], + jobs: jobsPerWorkspace[workspaceID], errorOutput: StoreErrorOutputT{ Location: uploadOutput.Location, Error: err, diff --git a/router/batchrouter/handle.go b/router/batchrouter/handle.go index 5cc9996e22..3b55638667 100644 --- a/router/batchrouter/handle.go +++ b/router/batchrouter/handle.go @@ -274,9 +274,17 @@ func (brt *Handle) upload(provider string, batchJobs *BatchedJobs, isWarehouse b if err != nil { panic(err) } - path := fmt.Sprintf("%v%v.json", tmpDirPath+localTmpDirName, fmt.Sprintf("%v.%v.%v", time.Now().Unix(), batchJobs.Connection.Source.ID, uuid)) + gzipFilePath := filepath.Join( + tmpDirPath, + localTmpDirName, + fmt.Sprintf( + "%v.%v.%v.json.gz", + time.Now().Unix(), + batchJobs.Connection.Source.ID, + uuid, + ), + ) - gzipFilePath := fmt.Sprintf(`%v.gz`, path) err = os.MkdirAll(filepath.Dir(gzipFilePath), os.ModePerm) if err != nil { panic(err) diff --git a/services/fileuploader/gzwriter.go b/services/fileuploader/gzwriter.go deleted file mode 100644 index bf6e5d743b..0000000000 --- a/services/fileuploader/gzwriter.go +++ /dev/null @@ -1,62 +0,0 @@ -package fileuploader - -import ( - "fmt" - "os" - "path/filepath" - - "github.com/rudderlabs/rudder-server/utils/misc" -) - -// MultiFileWriter can write to multiple paths at the same time. -type MultiFileWriter interface { - // Write writes the given data to the file at the given path. - Write(path string, data []byte) (int, error) - // Close closes all open files. - Close() error - - // Count returns the number of open files. - Count() int -} - -type gzFileHandler struct { - // gzWriters is a map of path to GZipWriter. - gzWriters map[string]misc.GZipWriter -} - -// NewGzMultiFileWriter creates a new MultiFileWriter that writes to multiple gz-compressed files. -func NewGzMultiFileWriter() MultiFileWriter { - return &gzFileHandler{ - gzWriters: make(map[string]misc.GZipWriter), - } -} - -func (g *gzFileHandler) Write(path string, data []byte) (count int, err error) { - if _, ok := g.gzWriters[path]; !ok { - - err := os.MkdirAll(filepath.Dir(path), os.ModePerm) - if err != nil { - return 0, fmt.Errorf("creating gz file %q: mkdir error: %w", path, err) - } - - g.gzWriters[path], err = misc.CreateGZ(path) - if err != nil { - return 0, err - } - } - return g.gzWriters[path].Write(data) -} - -func (g *gzFileHandler) Close() error { - for path, writer := range g.gzWriters { - if err := writer.CloseGZ(); err != nil { - return fmt.Errorf("closing gz file %q: %w", path, err) - } - } - g.gzWriters = make(map[string]misc.GZipWriter) - return nil -} - -func (g *gzFileHandler) Count() int { - return len(g.gzWriters) -} diff --git a/services/fileuploader/gzwriter_test.go b/services/fileuploader/gzwriter_test.go deleted file mode 100644 index 4b8caf139a..0000000000 --- a/services/fileuploader/gzwriter_test.go +++ /dev/null @@ -1,33 +0,0 @@ -package fileuploader - -import ( - "os" - "testing" - - . "github.com/onsi/gomega" -) - -func TestGzWriter(t *testing.T) { - RegisterTestingT(t) - gzWriter := NewGzMultiFileWriter() - err := os.MkdirAll("./temp", os.ModePerm) - Expect(err).To(BeNil()) - - _, err = gzWriter.Write("./temp/test", []byte("test1")) - Expect(err).To(BeNil()) - Expect(gzWriter.Count()).To(Equal(1)) - - _, err = gzWriter.Write("./temp/test", []byte("test2")) - Expect(err).To(BeNil()) - Expect(gzWriter.Count()).To(Equal(1)) - - _, err = gzWriter.Write("./temp/make", []byte("test3")) - Expect(err).To(BeNil()) - Expect(gzWriter.Count()).To(Equal(2)) - - err = gzWriter.Close() - Expect(err).To(BeNil()) - - Expect(gzWriter.Count()).To(Equal(0)) - os.RemoveAll("./temp/") -} From f89b32c043aee1044f85f5008cd18f4b872b4c0c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 4 Sep 2024 22:51:34 +0530 Subject: [PATCH 11/29] chore(deps): bump the go-deps group across 1 directory with 6 updates (#5058) --- go.mod | 52 +++++++++++++------------- go.sum | 116 ++++++++++++++++++++++++++++----------------------------- 2 files changed, 84 insertions(+), 84 deletions(-) diff --git a/go.mod b/go.mod index 974c3101af..93cfa008d1 100644 --- a/go.mod +++ b/go.mod @@ -37,7 +37,7 @@ require ( github.com/aws/aws-sdk-go v1.55.5 github.com/cenkalti/backoff v2.2.1+incompatible github.com/cenkalti/backoff/v4 v4.3.0 - github.com/confluentinc/confluent-kafka-go/v2 v2.5.0 + github.com/confluentinc/confluent-kafka-go/v2 v2.5.3 github.com/databricks/databricks-sql-go v1.6.1 github.com/denisenkom/go-mssqldb v0.12.3 github.com/dgraph-io/badger/v4 v4.3.0 @@ -74,11 +74,11 @@ require ( github.com/phayes/freeport v0.0.0-20220201140144-74d24b5ae9f5 github.com/prometheus/client_model v0.6.1 github.com/redis/go-redis/v9 v9.6.1 - github.com/rs/cors v1.11.0 + github.com/rs/cors v1.11.1 github.com/rudderlabs/analytics-go v3.3.3+incompatible github.com/rudderlabs/bing-ads-go-sdk v0.2.3 github.com/rudderlabs/compose-test v0.1.3 - github.com/rudderlabs/rudder-go-kit v0.38.0 + github.com/rudderlabs/rudder-go-kit v0.39.1 github.com/rudderlabs/rudder-observability-kit v0.0.3 github.com/rudderlabs/rudder-schemas v0.5.1 github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32 @@ -88,7 +88,7 @@ require ( github.com/segmentio/go-hll v1.0.1 github.com/segmentio/kafka-go v0.4.47 github.com/segmentio/ksuid v1.0.4 - github.com/snowflakedb/gosnowflake v1.11.0 + github.com/snowflakedb/gosnowflake v1.11.1 github.com/sony/gobreaker v1.0.0 github.com/spaolacci/murmur3 v1.1.0 github.com/spf13/cast v1.7.0 @@ -110,8 +110,8 @@ require ( golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 golang.org/x/oauth2 v0.22.0 golang.org/x/sync v0.8.0 - google.golang.org/api v0.194.0 - google.golang.org/genproto/googleapis/rpc v0.0.0-20240814211410-ddb44dafa142 + google.golang.org/api v0.196.0 + google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.34.2 ) @@ -124,10 +124,10 @@ require ( require ( cloud.google.com/go v0.115.1 // indirect - cloud.google.com/go/auth v0.9.1 // indirect + cloud.google.com/go/auth v0.9.3 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.4 // indirect cloud.google.com/go/compute/metadata v0.5.0 // indirect - cloud.google.com/go/iam v1.1.13 // indirect + cloud.google.com/go/iam v1.2.0 // indirect dario.cat/mergo v1.0.0 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect @@ -138,7 +138,7 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2 // indirect github.com/Azure/go-ansiterm v0.0.0-20230124172434-306776ec8161 // indirect - github.com/DataDog/zstd v1.5.5 // indirect + github.com/DataDog/zstd v1.5.6 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/Microsoft/go-winio v0.6.2 // indirect github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5 // indirect @@ -225,11 +225,11 @@ require ( github.com/google/pprof v0.0.0-20240827171923-fa2c70bbbfe5 // indirect github.com/google/s2a-go v0.1.8 // indirect github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.3 // indirect github.com/googleapis/gax-go/v2 v2.13.0 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed // indirect - github.com/hamba/avro/v2 v2.22.2-0.20240625062549-66aad10411d9 // indirect + github.com/hamba/avro/v2 v2.24.0 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect @@ -270,7 +270,7 @@ require ( github.com/pkg/sftp v1.13.6 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 // indirect - github.com/prometheus/client_golang v1.20.1 // indirect + github.com/prometheus/client_golang v1.20.2 // indirect github.com/prometheus/common v0.55.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/rivo/uniseg v0.4.7 // indirect @@ -304,18 +304,18 @@ require ( github.com/zeebo/xxh3 v1.0.2 // indirect go.etcd.io/etcd/client/pkg/v3 v3.5.15 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.52.0 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.52.0 // indirect - go.opentelemetry.io/otel v1.28.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.28.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.28.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.28.0 // indirect - go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.28.0 // indirect - go.opentelemetry.io/otel/exporters/zipkin v1.28.0 // indirect - go.opentelemetry.io/otel/metric v1.28.0 // indirect - go.opentelemetry.io/otel/sdk v1.28.0 // indirect - go.opentelemetry.io/otel/sdk/metric v1.28.0 // indirect - go.opentelemetry.io/otel/trace v1.28.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.54.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0 // indirect + go.opentelemetry.io/otel v1.29.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.29.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.29.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.29.0 // indirect + go.opentelemetry.io/otel/exporters/zipkin v1.29.0 // indirect + go.opentelemetry.io/otel/metric v1.29.0 // indirect + go.opentelemetry.io/otel/sdk v1.29.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.29.0 // indirect + go.opentelemetry.io/otel/trace v1.29.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect @@ -328,8 +328,8 @@ require ( golang.org/x/time v0.6.0 // indirect golang.org/x/tools v0.24.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect - google.golang.org/genproto v0.0.0-20240814211410-ddb44dafa142 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142 // indirect + google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect gopkg.in/alexcesaro/statsd.v2 v2.0.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect diff --git a/go.sum b/go.sum index 25e3faebb0..f05ce001e6 100644 --- a/go.sum +++ b/go.sum @@ -32,8 +32,8 @@ cloud.google.com/go v0.100.1/go.mod h1:fs4QogzfH5n2pBXBP9vRiU+eCny7lD2vmFZy79Iuw cloud.google.com/go v0.100.2/go.mod h1:4Xra9TjzAeYHrl5+oeLlzbM2k3mjVhZh4UqTZ//w99A= cloud.google.com/go v0.115.1 h1:Jo0SM9cQnSkYfp44+v+NQXHpcHqlnRJk2qxh6yvxxxQ= cloud.google.com/go v0.115.1/go.mod h1:DuujITeaufu3gL68/lOFIirVNJwQeyf5UXyi+Wbgknc= -cloud.google.com/go/auth v0.9.1 h1:+pMtLEV2k0AXKvs/tGZojuj6QaioxfUjOpMsG5Gtx+w= -cloud.google.com/go/auth v0.9.1/go.mod h1:Sw8ocT5mhhXxFklyhT12Eiy0ed6tTrPMCJjSI8KhYLk= +cloud.google.com/go/auth v0.9.3 h1:VOEUIAADkkLtyfr3BLa3R8Ed/j6w1jTBmARx+wb5w5U= +cloud.google.com/go/auth v0.9.3/go.mod h1:7z6VY+7h3KUdRov5F1i8NDP5ZzWKYmEPO842BgCsmTk= cloud.google.com/go/auth/oauth2adapt v0.2.4 h1:0GWE/FUsXhf6C+jAkWgYm7X9tK8cuEIfy19DBn6B6bY= cloud.google.com/go/auth/oauth2adapt v0.2.4/go.mod h1:jC/jOpwFP6JBxhB3P5Rr0a9HLMC/Pe3eaL4NmdvqPtc= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= @@ -50,22 +50,22 @@ cloud.google.com/go/compute v1.3.0/go.mod h1:cCZiE1NHEtai4wiufUhW8I8S1JKkAnhnQJW cloud.google.com/go/compute v1.5.0/go.mod h1:9SMHyhJlzhlkJqrPAc839t2BZFTSk6Jdj6mkzQJeu0M= cloud.google.com/go/compute/metadata v0.5.0 h1:Zr0eK8JbFv6+Wi4ilXAR8FJ3wyNdpxHKJNPos6LTZOY= cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= -cloud.google.com/go/datacatalog v1.21.0 h1:vl0pQT9TZ5rKi9e69FgtXNCR7I8MVRj4+CnbeXhz6UQ= -cloud.google.com/go/datacatalog v1.21.0/go.mod h1:DB0QWF9nelpsbB0eR/tA0xbHZZMvpoFD1XFy3Qv/McI= +cloud.google.com/go/datacatalog v1.22.0 h1:7e5/0B2LYbNx0BcUJbiCT8K2wCtcB5993z/v1JeLIdc= +cloud.google.com/go/datacatalog v1.22.0/go.mod h1:4Wff6GphTY6guF5WphrD76jOdfBiflDiRGFAxq7t//I= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/firestore v1.6.1/go.mod h1:asNXNOzBdyVQmEU+ggO8UPodTkEVFW5Qx+rwHnAz+EY= cloud.google.com/go/iam v0.1.0/go.mod h1:vcUNEa0pEm0qRVpmWepWaFMIAI8/hjB9mO8rNCJtF6c= cloud.google.com/go/iam v0.1.1/go.mod h1:CKqrcnI/suGpybEHxZ7BMehL0oA4LpdyJdUlTl9jVMw= cloud.google.com/go/iam v0.3.0/go.mod h1:XzJPvDayI+9zsASAFO68Hk07u3z+f+JrT2xXNdp4bnY= -cloud.google.com/go/iam v1.1.13 h1:7zWBXG9ERbMLrzQBRhFliAV+kjcRToDTgQT3CTwYyv4= -cloud.google.com/go/iam v1.1.13/go.mod h1:K8mY0uSXwEXS30KrnVb+j54LB/ntfZu1dr+4zFMNbus= +cloud.google.com/go/iam v1.2.0 h1:kZKMKVNk/IsSSc/udOb83K0hL/Yh/Gcqpz+oAkoIFN8= +cloud.google.com/go/iam v1.2.0/go.mod h1:zITGuWgsLZxd8OwAlX+eMFgZDXzBm7icj1PVTYG766Q= cloud.google.com/go/kms v1.1.0/go.mod h1:WdbppnCDMDpOvoYBMn1+gNmOeEoZYqAv+HeuKARGCXI= cloud.google.com/go/kms v1.4.0/go.mod h1:fajBHndQ+6ubNw6Ss2sSd+SWvjL26RNo/dr7uxsnnOA= -cloud.google.com/go/kms v1.18.4 h1:dYN3OCsQ6wJLLtOnI8DGUwQ5shMusXsWCCC+s09ATsk= -cloud.google.com/go/kms v1.18.4/go.mod h1:SG1bgQ3UWW6/KdPo9uuJnzELXY5YTTMJtDYvajiQ22g= -cloud.google.com/go/longrunning v0.5.11 h1:Havn1kGjz3whCfoD8dxMLP73Ph5w+ODyZB9RUsDxtGk= -cloud.google.com/go/longrunning v0.5.11/go.mod h1:rDn7//lmlfWV1Dx6IB4RatCPenTwwmqXuiP0/RgoEO4= +cloud.google.com/go/kms v1.19.0 h1:x0OVJDl6UH1BSX4THKlMfdcFWoE4ruh90ZHuilZekrU= +cloud.google.com/go/kms v1.19.0/go.mod h1:e4imokuPJUc17Trz2s6lEXFDt8bgDmvpVynH39bdrHM= +cloud.google.com/go/longrunning v0.6.0 h1:mM1ZmaNsQsnb+5n1DNPeL0KwQd9jQRqSqSDEkBZr+aI= +cloud.google.com/go/longrunning v0.6.0/go.mod h1:uHzSZqW89h7/pasCWNYdUpwGz3PcVWhrWupreVPYLts= cloud.google.com/go/monitoring v1.1.0/go.mod h1:L81pzz7HKn14QCMaCs6NTQkdBnE87TElyanS95vIcl4= cloud.google.com/go/monitoring v1.4.0/go.mod h1:y6xnxfwI3hTFWOdkOaD7nfJVlwuC3/mS/5kvtT131p4= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= @@ -178,8 +178,8 @@ github.com/ClickHouse/clickhouse-go/v2 v2.25.0 h1:rKscwqgQHzWBTZySZDcHKxgs0Ad+xF github.com/ClickHouse/clickhouse-go/v2 v2.25.0/go.mod h1:iDTViXk2Fgvf1jn2dbJd1ys+fBkdD1UMRnXlwmhijhQ= github.com/DATA-DOG/go-sqlmock v1.5.2 h1:OcvFkGmslmlZibjAjaHm3L//6LiuBgolP7OputlJIzU= github.com/DATA-DOG/go-sqlmock v1.5.2/go.mod h1:88MAG/4G7SMwSE3CeA0ZKzrT5CiOU3OJ+JlNzwDqpNU= -github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= -github.com/DataDog/zstd v1.5.5/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= +github.com/DataDog/zstd v1.5.6 h1:LbEglqepa/ipmmQJUDnSsfvA8e8IStVcGaFWDuxvGOY= +github.com/DataDog/zstd v1.5.6/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/GoogleCloudPlatform/cloudsql-proxy v1.29.0/go.mod h1:spvB9eLJH9dutlbPSRmHvSXXHOwGRyeXh1jVdquA2G8= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= @@ -378,8 +378,8 @@ github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMe github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/compose-spec/compose-go/v2 v2.1.0 h1:qdW2qISQlCQG8v1O2TChcdxgAWTUGgUX/CPSO+ES9+E= github.com/compose-spec/compose-go/v2 v2.1.0/go.mod h1:bEPizBkIojlQ20pi2vNluBa58tevvj0Y18oUSHPyfdc= -github.com/confluentinc/confluent-kafka-go/v2 v2.5.0 h1:PM18lA9g6u6Qcz06DpXmGRlxXTvWlHqnlAkQi1chPUo= -github.com/confluentinc/confluent-kafka-go/v2 v2.5.0/go.mod h1:Hyo+IIQ/tmsfkOcRP8T6VlSeOW3T33v0Me8Xvq4u90Y= +github.com/confluentinc/confluent-kafka-go/v2 v2.5.3 h1:3AAMHvhiv3d4ajW4fSnZw+liipffhtunkEWz23zTXXM= +github.com/confluentinc/confluent-kafka-go/v2 v2.5.3/go.mod h1:QxYLPRKR1MVlkXCCjzjjrpXb0VyFNfVaZXi0obZykJ0= github.com/containerd/console v1.0.4 h1:F2g4+oChYvBTsASRTz8NP6iIAi97J3TtSAsLbIFn4ro= github.com/containerd/console v1.0.4/go.mod h1:YynlIjWYF8myEu6sdkwKIvGQq+cOckRm6So2avqoYAk= github.com/containerd/containerd v1.6.32 h1:zc3RN2fLrPZPH4mZziyaRVIqCN6zoFpeN0SOVDEgCTA= @@ -730,8 +730,8 @@ github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/wire v0.5.0/go.mod h1:ngWDr9Qvq3yZA10YrxfyGELY/AFWGVpy9c1LTRi1EoU= -github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= -github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= +github.com/googleapis/enterprise-certificate-proxy v0.3.3 h1:QRje2j5GZimBzlbhGA2V2QlGNgL8G6e+wGo/+/2bWI0= +github.com/googleapis/enterprise-certificate-proxy v0.3.3/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= @@ -757,8 +757,8 @@ github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8 github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed h1:5upAirOpQc1Q53c0bnx2ufif5kANL7bfZWcc6VJWJd8= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed/go.mod h1:tMWxXQ9wFIaZeTI9F+hmhFiGpFmhOHzyShyFUhRm0H4= -github.com/hamba/avro/v2 v2.22.2-0.20240625062549-66aad10411d9 h1:NEoabXt33PDWK4fXryK4e+XX+fSKDmmu9vg3yb9YI2M= -github.com/hamba/avro/v2 v2.22.2-0.20240625062549-66aad10411d9/go.mod h1:fQVdB2mFZBhPW1D5Abej41LMvrErARGrrdjOnKbm5yw= +github.com/hamba/avro/v2 v2.24.0 h1:axTlaYDkcSY0dVekRSy8cdrsj5MG86WqosUQacKCids= +github.com/hamba/avro/v2 v2.24.0/go.mod h1:7vDfy/2+kYCE8WUHoj2et59GTv0ap7ptktMXu0QHePI= github.com/hanwen/go-fuse v1.0.0/go.mod h1:unqXarDXqzAk0rt98O2tVndEPIpUgLD9+rwFisZH3Ok= github.com/hanwen/go-fuse/v2 v2.1.0/go.mod h1:oRyA5eK+pvJyv5otpO/DgccS8y/RvYMaO00GgRLGryc= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= @@ -1092,8 +1092,8 @@ github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 h1:o4JXh1EVt github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= -github.com/prometheus/client_golang v1.20.1 h1:IMJXHOD6eARkQpxo8KkhgEVFlBNm+nkrFUyGlIu7Na8= -github.com/prometheus/client_golang v1.20.1/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= +github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= @@ -1115,8 +1115,8 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= -github.com/rs/cors v1.11.0 h1:0B9GE/r9Bc2UxRMMtymBkHTenPkHDv0CW4Y98GBY+po= -github.com/rs/cors v1.11.0/go.mod h1:XyqrcTp5zjWr1wsJ8PIRZssZ8b/WMcMf71DJnit4EMU= +github.com/rs/cors v1.11.1 h1:eU3gRzXLRK57F5rKMGMZURNdIG4EoAmX8k94r9wXWHA= +github.com/rs/cors v1.11.1/go.mod h1:XyqrcTp5zjWr1wsJ8PIRZssZ8b/WMcMf71DJnit4EMU= github.com/rs/xid v1.2.1/go.mod h1:+uKXf+4Djp6Md1KODXJxgGQPKngRmWyn10oCKFzNHOQ= github.com/rs/xid v1.4.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= @@ -1134,8 +1134,8 @@ github.com/rudderlabs/compose-test v0.1.3 h1:uyep6jDCIF737sfv4zIaMsKRQKX95IDz5Xb github.com/rudderlabs/compose-test v0.1.3/go.mod h1:tuvS1eQdSfwOYv1qwyVAcpdJxPLQXJgy5xGDd/9XmMg= github.com/rudderlabs/parquet-go v0.0.2 h1:ZXRdZdimB0PdJtmxeSSxfI0fDQ3kZjwzBxRi6Ut1J8k= github.com/rudderlabs/parquet-go v0.0.2/go.mod h1:g6guum7o8uhj/uNhunnt7bw5Vabu/goI5i21/3fnxWQ= -github.com/rudderlabs/rudder-go-kit v0.38.0 h1:WVm1nhnt4j2SGEf+PRl/UDFjOtlngAOUoZbiJ7XapyM= -github.com/rudderlabs/rudder-go-kit v0.38.0/go.mod h1:dRi+dtkpLoRCq319RFU467aKf9VkDpskCA1nskXEllA= +github.com/rudderlabs/rudder-go-kit v0.39.1 h1:aNuGHuYJD8opRNfWVrLh+rWt48cE4ZBb2Km5k64eAcU= +github.com/rudderlabs/rudder-go-kit v0.39.1/go.mod h1:3iiVwUvC/+QUyPbhM5m4NrwL6rTcsAOeDCupLAueS+U= github.com/rudderlabs/rudder-observability-kit v0.0.3 h1:vZtuZRkGX+6rjaeKtxxFE2YYP6QlmAcVcgecTOjvz+Q= github.com/rudderlabs/rudder-observability-kit v0.0.3/go.mod h1:6UjAh3H6rkE0fFLh7z8ZGQEQbKtUkRfhWOf/OUhfqW8= github.com/rudderlabs/rudder-schemas v0.5.1 h1:g4I5wp2yA6ZWQZ1MjSNn4zby3XctG/TOgbYUW3dS4z4= @@ -1194,8 +1194,8 @@ github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykE github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/snowflakedb/gosnowflake v1.11.0 h1:qyqunGCVyq/Qyx40KQT+6sJ1CAGuuG2qv3WiCTLTctI= -github.com/snowflakedb/gosnowflake v1.11.0/go.mod h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A= +github.com/snowflakedb/gosnowflake v1.11.1 h1:E91s8vBOSroaSTLsyjO4QPkEuzGmZcCxEFQLg214mvk= +github.com/snowflakedb/gosnowflake v1.11.1/go.mod h1:WFe+8mpsapDaQjHX6BqJBKtfQCGlGD3lHKeDsKfpx2A= github.com/sony/gobreaker v1.0.0 h1:feX5fGGXSl3dYd4aHZItw+FpHLvvoaqkawKjVNiFMNQ= github.com/sony/gobreaker v1.0.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= github.com/sourcegraph/conc v0.3.0 h1:OQTbbt6P72L20UqAkXXuLOj79LfEanQ+YQFNpLA9ySo= @@ -1330,41 +1330,41 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.52.0 h1:vS1Ao/R55RNV4O7TA2Qopok8yN+X0LIP6RVWLFkprck= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.52.0/go.mod h1:BMsdeOxN04K0L5FNUBfjFdvwWGNe/rkmSwH4Aelu/X0= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.54.0 h1:r6I7RJCN86bpD/FQwedZ0vSixDpwuWREjW9oRMsmqDc= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.54.0/go.mod h1:B9yO6b04uB80CzjedvewuqDhxJxi11s7/GtiGa8bAjI= go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.46.1 h1:gbhw/u49SS3gkPWiYweQNJGm/uJN5GkI/FrosxSHT7A= go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.46.1/go.mod h1:GnOaBaFQ2we3b9AGWJpsBa7v1S5RlQzlC3O7dRMxZhM= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.52.0 h1:9l89oX4ba9kHbBol3Xin3leYJ+252h0zszDtBwyKe2A= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.52.0/go.mod h1:XLZfZboOJWHNKUv7eH0inh0E9VV6eWDFB/9yJyTLPp0= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0 h1:TT4fX+nBOA/+LUkobKGW1ydGcn+G3vRw9+g5HwCphpk= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0/go.mod h1:L7UH0GbB0p47T4Rri3uHjbpCFYrVrwc1I25QhNPiGK8= go.opentelemetry.io/otel v0.14.0/go.mod h1:vH5xEuwy7Rts0GNtsCW3HYQoZDY+OmBJ6t1bFGGlxgw= -go.opentelemetry.io/otel v1.28.0 h1:/SqNcYk+idO0CxKEUOtKQClMK/MimZihKYMruSMViUo= -go.opentelemetry.io/otel v1.28.0/go.mod h1:q68ijF8Fc8CnMHKyzqL6akLO46ePnjkgfIMIjUIX9z4= +go.opentelemetry.io/otel v1.29.0 h1:PdomN/Al4q/lN6iBJEN3AwPvUiHPMlt93c8bqTG5Llw= +go.opentelemetry.io/otel v1.29.0/go.mod h1:N/WtXPs1CNCUEx+Agz5uouwCba+i+bJGFicT8SR4NP8= go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.42.0 h1:ZtfnDL+tUrs1F0Pzfwbg2d59Gru9NCH3bgSHBM6LDwU= go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.42.0/go.mod h1:hG4Fj/y8TR/tlEDREo8tWstl9fO9gcFkn4xrx0Io8xU= -go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.28.0 h1:U2guen0GhqH8o/G2un8f/aG/y++OuW6MyCo6hT9prXk= -go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.28.0/go.mod h1:yeGZANgEcpdx/WK0IvvRFC+2oLiMS2u4L/0Rj2M2Qr0= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.29.0 h1:k6fQVDQexDE+3jG2SfCQjnHS7OamcP73YMoxEVq5B6k= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.29.0/go.mod h1:t4BrYLHU450Zo9fnydWlIuswB1bm7rM8havDpWOJeDo= go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.42.0 h1:wNMDy/LVGLj2h3p6zg4d0gypKfWKSWI14E1C4smOgl8= go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.42.0/go.mod h1:YfbDdXAAkemWJK3H/DshvlrxqFB2rtW4rY6ky/3x/H0= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.28.0 h1:3Q/xZUyC1BBkualc9ROb4G8qkH90LXEIICcs5zv1OYY= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.28.0/go.mod h1:s75jGIWA9OfCMzF0xr+ZgfrB5FEbbV7UuYo32ahUiFI= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.28.0 h1:R3X6ZXmNPRR8ul6i3WgFURCHzaXjHdm0karRG/+dj3s= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.28.0/go.mod h1:QWFXnDavXWwMx2EEcZsf3yxgEKAqsxQ+Syjp+seyInw= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.29.0 h1:dIIDULZJpgdiHz5tXrTgKIMLkus6jEFa7x5SOKcyR7E= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.29.0/go.mod h1:jlRVBe7+Z1wyxFSUs48L6OBQZ5JwH2Hg/Vbl+t9rAgI= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0 h1:nSiV3s7wiCam610XcLbYOmMfJxB9gO4uK3Xgv5gmTgg= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0/go.mod h1:hKn/e/Nmd19/x1gvIHwtOwVWM+VhuITSWip3JUDghj0= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0 h1:digkEZCJWobwBqMwC0cwCq8/wkkRy/OowZg5OArWZrM= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0/go.mod h1:/OpE/y70qVkndM0TrxT4KBoN3RsFZP0QaofcfYrj76I= go.opentelemetry.io/otel/exporters/prometheus v0.42.0 h1:jwV9iQdvp38fxXi8ZC+lNpxjK16MRcZlpDYvbuO1FiA= go.opentelemetry.io/otel/exporters/prometheus v0.42.0/go.mod h1:f3bYiqNqhoPxkvI2LrXqQVC546K7BuRDL/kKuxkujhA= -go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.28.0 h1:EVSnY9JbEEW92bEkIYOVMw4q1WJxIAGoFTrtYOzWuRQ= -go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.28.0/go.mod h1:Ea1N1QQryNXpCD0I1fdLibBAIpQuBkznMmkdKrapk1Y= -go.opentelemetry.io/otel/exporters/zipkin v1.28.0 h1:q86SrM4sgdc1eDABeA+307DUWy1qaT3fDCVbeKYGfY4= -go.opentelemetry.io/otel/exporters/zipkin v1.28.0/go.mod h1:mkxt8tmE/1YujUHsMIgTPvBN2HVE3kXlRZWeKsTsFgI= -go.opentelemetry.io/otel/metric v1.28.0 h1:f0HGvSl1KRAU1DLgLGFjrwVyismPlnuU6JD6bOeuA5Q= -go.opentelemetry.io/otel/metric v1.28.0/go.mod h1:Fb1eVBFZmLVTMb6PPohq3TO9IIhUisDsbJoL/+uQW4s= -go.opentelemetry.io/otel/sdk v1.28.0 h1:b9d7hIry8yZsgtbmM0DKyPWMMUMlK9NEKuIG4aBqWyE= -go.opentelemetry.io/otel/sdk v1.28.0/go.mod h1:oYj7ClPUA7Iw3m+r7GeEjz0qckQRJK2B8zjcZEfu7Pg= -go.opentelemetry.io/otel/sdk/metric v1.28.0 h1:OkuaKgKrgAbYrrY0t92c+cC+2F6hsFNnCQArXCKlg08= -go.opentelemetry.io/otel/sdk/metric v1.28.0/go.mod h1:cWPjykihLAPvXKi4iZc1dpER3Jdq2Z0YLse3moQUCpg= -go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= -go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= +go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.29.0 h1:X3ZjNp36/WlkSYx0ul2jw4PtbNEDDeLskw3VPsrpYM0= +go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.29.0/go.mod h1:2uL/xnOXh0CHOBFCWXz5u1A4GXLiW+0IQIzVbeOEQ0U= +go.opentelemetry.io/otel/exporters/zipkin v1.29.0 h1:rqaUJdM9ItWf6DGrelaShXnJpb8rd3HTbcZWptvcsWA= +go.opentelemetry.io/otel/exporters/zipkin v1.29.0/go.mod h1:wDIyU6DjrUYqUgnmzjWnh1HOQGZCJ6YXMIJCdMc+T9Y= +go.opentelemetry.io/otel/metric v1.29.0 h1:vPf/HFWTNkPu1aYeIsc98l4ktOQaL6LeSoeV2g+8YLc= +go.opentelemetry.io/otel/metric v1.29.0/go.mod h1:auu/QWieFVWx+DmQOUMgj0F8LHWdgalxXqvp7BII/W8= +go.opentelemetry.io/otel/sdk v1.29.0 h1:vkqKjk7gwhS8VaWb0POZKmIEDimRCMsopNYnriHyryo= +go.opentelemetry.io/otel/sdk v1.29.0/go.mod h1:pM8Dx5WKnvxLCb+8lG1PRNIDxu9g9b9g59Qr7hfAAok= +go.opentelemetry.io/otel/sdk/metric v1.29.0 h1:K2CfmJohnRgvZ9UAj2/FhIf/okdWcNdBwe1m8xFXiSY= +go.opentelemetry.io/otel/sdk/metric v1.29.0/go.mod h1:6zZLdCl2fkauYoZIOn/soQIDSWFmNSRcICarHfuhNJQ= +go.opentelemetry.io/otel/trace v1.29.0 h1:J/8ZNK4XgR7a21DZUAsbF8pZ5Jcw1VhACmnYt39JTi4= +go.opentelemetry.io/otel/trace v1.29.0/go.mod h1:eHl3w0sp3paPkYstJOmAimxhiFXPg+MMTlEh3nsQgWQ= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v1.3.1 h1:TrMUixzpM0yuc/znrFTP9MMRh8trP93mkCiDVeXrui0= go.opentelemetry.io/proto/otlp v1.3.1/go.mod h1:0X1WI4de4ZsLrrJNLAQbFeLCm3T7yBkR0XqQ7niQU+8= @@ -1730,8 +1730,8 @@ google.golang.org/api v0.69.0/go.mod h1:boanBiw+h5c3s+tBPgEzLDRHfFLWV0qXxRHz3ws7 google.golang.org/api v0.70.0/go.mod h1:Bs4ZM2HGifEvXwd50TtW70ovgJffJYw2oRCOFU/SkfA= google.golang.org/api v0.71.0/go.mod h1:4PyU6e6JogV1f9eA4voyrTY2batOLdgZ5qZ5HOCc4j8= google.golang.org/api v0.74.0/go.mod h1:ZpfMZOVRMywNyvJFeqL9HRWBgAuRfSjJFpe9QtRRyDs= -google.golang.org/api v0.194.0 h1:dztZKG9HgtIpbI35FhfuSNR/zmaMVdxNlntHj1sIS4s= -google.golang.org/api v0.194.0/go.mod h1:AgvUFdojGANh3vI+P7EVnxj3AISHllxGCJSFmggmnd0= +google.golang.org/api v0.196.0 h1:k/RafYqebaIJBO3+SMnfEGtFVlvp5vSgqTUF54UN/zg= +google.golang.org/api v0.196.0/go.mod h1:g9IL21uGkYgvQ5BZg6BAtoGJQIm8r6EgaAbpNey5wBE= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1825,12 +1825,12 @@ google.golang.org/genproto v0.0.0-20220304144024-325a89244dc8/go.mod h1:kGP+zUP2 google.golang.org/genproto v0.0.0-20220310185008-1973136f34c6/go.mod h1:kGP+zUP2Ddo0ayMi4YuN7C3WZyJvGLZRh8Z5wnAqvEI= google.golang.org/genproto v0.0.0-20220324131243-acbaeb5b85eb/go.mod h1:hAL49I2IFola2sVEjAn7MEwsja0xp51I0tlGAf9hz4E= google.golang.org/genproto v0.0.0-20220401170504-314d38edb7de/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= -google.golang.org/genproto v0.0.0-20240814211410-ddb44dafa142 h1:oLiyxGgE+rt22duwci1+TG7bg2/L1LQsXwfjPlmuJA0= -google.golang.org/genproto v0.0.0-20240814211410-ddb44dafa142/go.mod h1:G11eXq53iI5Q+kyNOmCvnzBaxEA2Q/Ik5Tj7nqBE8j4= -google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142 h1:wKguEg1hsxI2/L3hUYrpo1RVi48K+uTyzKqprwLXsb8= -google.golang.org/genproto/googleapis/api v0.0.0-20240814211410-ddb44dafa142/go.mod h1:d6be+8HhtEtucleCbxpPW9PA9XwISACu8nvpPqF0BVo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240814211410-ddb44dafa142 h1:e7S5W7MGGLaSu8j3YjdezkZ+m1/Nm0uRVRMEMGk26Xs= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240814211410-ddb44dafa142/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1 h1:BulPr26Jqjnd4eYDVe+YvyR7Yc2vJGkO5/0UxD0/jZU= +google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:hL97c3SYopEHblzpxRL4lSs523++l8DYxGM1FQiYmb4= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 h1:pPJltXNxVzT4pK9yD8vR9X75DaWYYmLGMsEvBfFQZzQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 1cf2bf575cca868c3e4331b313a44d173c353321 Mon Sep 17 00:00:00 2001 From: devops-github-rudderstack <88187154+devops-github-rudderstack@users.noreply.github.com> Date: Wed, 4 Sep 2024 22:13:19 -0500 Subject: [PATCH 12/29] chore: sync release v1.33.0 to main branch (#5054) --- CHANGELOG.md | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 261947248e..7f9e0a909a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,33 @@ # Changelog +## [1.33.0](https://github.com/rudderlabs/rudder-server/compare/v1.32.0...v1.33.0) (2024-09-02) + + +### Features + +* add support for headers to webhooks ([#5018](https://github.com/rudderlabs/rudder-server/issues/5018)) ([15fa31f](https://github.com/rudderlabs/rudder-server/commit/15fa31f4fdbda218ad3d511e1e309cbef42c3f38)) +* onboarding lytics bulk upload ([#5000](https://github.com/rudderlabs/rudder-server/issues/5000)) ([5c38dab](https://github.com/rudderlabs/rudder-server/commit/5c38dabb4d59a060f1243a2f0f1179e00be18d66)) +* remove bugsnag for reporting panics ([#5014](https://github.com/rudderlabs/rudder-server/issues/5014)) ([d68ceb3](https://github.com/rudderlabs/rudder-server/commit/d68ceb3503bbcfd260276ebee1fa6bebe721cea1)) +* stats for schema size ([#5031](https://github.com/rudderlabs/rudder-server/issues/5031)) ([01b84f4](https://github.com/rudderlabs/rudder-server/commit/01b84f422ffffc95e77800b3655d9a5415ee01ce)) + + +### Bug Fixes + +* take tacking plan id from dgsourceTPconfig ([#5041](https://github.com/rudderlabs/rudder-server/issues/5041)) ([f6782c3](https://github.com/rudderlabs/rudder-server/commit/f6782c3a74f9ec1b3fb99fd910d088b9223c504c)) + + +### Miscellaneous + +* add sourceCategory label in event_delivery_time metric ([#5004](https://github.com/rudderlabs/rudder-server/issues/5004)) ([a5adab3](https://github.com/rudderlabs/rudder-server/commit/a5adab32e22a39f2cff50eb1d947ab0ab7a5b6b8)) +* avoid using global config ([#5001](https://github.com/rudderlabs/rudder-server/issues/5001)) ([26228d8](https://github.com/rudderlabs/rudder-server/commit/26228d80522f3ff713bd4e18c857fbb5aa788ecf)) +* avoid using warehouse global config ([26228d8](https://github.com/rudderlabs/rudder-server/commit/26228d80522f3ff713bd4e18c857fbb5aa788ecf)) +* **deps:** bump the go-deps group across 1 directory with 18 updates ([#5038](https://github.com/rudderlabs/rudder-server/issues/5038)) ([c18669b](https://github.com/rudderlabs/rudder-server/commit/c18669b6735d936323d873579c6ab47fcec83ec4)) +* ignore all x-forwarded headers ([#5032](https://github.com/rudderlabs/rudder-server/issues/5032)) ([e832eae](https://github.com/rudderlabs/rudder-server/commit/e832eaede8356634a74e6b370c5eff5a2b6e7a6f)) +* refactor warehouse integration tests ([#5021](https://github.com/rudderlabs/rudder-server/issues/5021)) ([0ca6dbf](https://github.com/rudderlabs/rudder-server/commit/0ca6dbfcd1f316ac91b826e99d9f6ef0c94bb178)) +* sync release v1.32.0 to main branch ([#5010](https://github.com/rudderlabs/rudder-server/issues/5010)) ([3865506](https://github.com/rudderlabs/rudder-server/commit/3865506850836c2bdeaf20c90bd1ab26b5b65bcd)) +* update event_delivery_time histogram buckets for warehouse ([#5015](https://github.com/rudderlabs/rudder-server/issues/5015)) ([983403c](https://github.com/rudderlabs/rudder-server/commit/983403cf74a65bdcb7a8ba7953f15567141fadad)) +* update rudder-go-kit port binding ([#4998](https://github.com/rudderlabs/rudder-server/issues/4998)) ([5327d44](https://github.com/rudderlabs/rudder-server/commit/5327d4413e8aa92ab737243bb45c5971a39e5db8)) + ## [1.32.0](https://github.com/rudderlabs/rudder-server/compare/v1.31.0...v1.32.0) (2024-08-19) From 861506a5a4745bfe3f209b2da8b7e1222be90909 Mon Sep 17 00:00:00 2001 From: Akash Chetty Date: Thu, 5 Sep 2024 16:31:28 +0530 Subject: [PATCH 13/29] chore: trino hive cursor error (#5024) --- .../integrations/datalake/datalake_test.go | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) diff --git a/warehouse/integrations/datalake/datalake_test.go b/warehouse/integrations/datalake/datalake_test.go index 80dbdc2cec..369979cd52 100644 --- a/warehouse/integrations/datalake/datalake_test.go +++ b/warehouse/integrations/datalake/datalake_test.go @@ -11,6 +11,7 @@ import ( "time" "github.com/minio/minio-go/v7" + "github.com/trinodb/trino-go-client/trino" "cloud.google.com/go/storage" @@ -539,6 +540,94 @@ func TestIntegration(t *testing.T) { 1*time.Second, ) require.Equal(t, int64(8), count) + + t.Log("By default parquet_use_column_index=true and parquet_ignore_statistics=false") + t.Log("parquet_use_column_index=true") + require.Eventually(t, func() bool { + err := db.QueryRowContext(ctx, ` + select + count(*) + from + minio.rudderstack.tracks + where + context_library_name = 'http' + `).Scan(&count) + if err != nil { + t.Log("select count with where clause: ", err) + } + + var e *trino.ErrQueryFailed + if err != nil && errors.As(err, &e) && e.StatusCode == 200 { + var ei *trino.ErrTrino + if errors.As(e.Reason, &ei) && ei.ErrorName == "HIVE_CURSOR_ERROR" { + return true + } + } + return false + }, + 60*time.Second, + 1*time.Second, + ) + + t.Log("parquet_use_column_index=false") + dsnWithoutIndex := fmt.Sprintf("http://user@localhost:%d?catalog=minio&schema=default&session_properties=minio.parquet_use_column_index=false", + c.Port("trino", 8080), + ) + dbWithoutIndex, err := sql.Open("trino", dsnWithoutIndex) + require.NoError(t, err) + t.Cleanup(func() { + _ = dbWithoutIndex.Close() + }) + + require.Eventually(t, func() bool { + err := dbWithoutIndex.QueryRowContext(ctx, ` + select + count(*) + from + minio.rudderstack.tracks + where + context_library_name = 'http' + `).Scan(&count) + if err != nil { + t.Log("select count with where clause: ", err) + return false + } + return true + }, + 60*time.Second, + 1*time.Second, + ) + require.Equal(t, int64(3), count) + + t.Logf("parquet_ignore_statistics=true") + dsnIgnoreStatistics := fmt.Sprintf("http://user@localhost:%d?catalog=minio&schema=default&session_properties=minio.parquet_ignore_statistics=true", + c.Port("trino", 8080), + ) + dbIgnoreStatistics, err := sql.Open("trino", dsnIgnoreStatistics) + require.NoError(t, err) + t.Cleanup(func() { + _ = dbIgnoreStatistics.Close() + }) + + require.Eventually(t, func() bool { + err := dbIgnoreStatistics.QueryRowContext(ctx, ` + select + count(*) + from + minio.rudderstack.tracks + where + context_library_name = 'http' + `).Scan(&count) + if err != nil { + t.Log("select count with where clause: ", err) + return false + } + return true + }, + 60*time.Second, + 1*time.Second, + ) + require.Equal(t, int64(3), count) }) t.Run("Spark", func(t *testing.T) { From a3ed090d66d19cb0ffa80aa5ede1e4653db06298 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Sep 2024 16:32:27 +0530 Subject: [PATCH 14/29] chore(deps): bump the go-deps group with 2 updates (#5059) --- go.mod | 6 +++--- go.sum | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/go.mod b/go.mod index 93cfa008d1..172d07668c 100644 --- a/go.mod +++ b/go.mod @@ -78,7 +78,7 @@ require ( github.com/rudderlabs/analytics-go v3.3.3+incompatible github.com/rudderlabs/bing-ads-go-sdk v0.2.3 github.com/rudderlabs/compose-test v0.1.3 - github.com/rudderlabs/rudder-go-kit v0.39.1 + github.com/rudderlabs/rudder-go-kit v0.39.2 github.com/rudderlabs/rudder-observability-kit v0.0.3 github.com/rudderlabs/rudder-schemas v0.5.1 github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32 @@ -108,7 +108,7 @@ require ( go.uber.org/goleak v1.3.0 go.uber.org/mock v0.4.0 golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 - golang.org/x/oauth2 v0.22.0 + golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 google.golang.org/api v0.196.0 google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 @@ -260,7 +260,7 @@ require ( github.com/mtibben/percent v0.2.1 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.1.0 // indirect - github.com/opencontainers/runc v1.1.13 // indirect + github.com/opencontainers/runc v1.1.14 // indirect github.com/openzipkin/zipkin-go v0.4.3 // indirect github.com/pelletier/go-toml/v2 v2.2.2 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect diff --git a/go.sum b/go.sum index f05ce001e6..96f7c6817b 100644 --- a/go.sum +++ b/go.sum @@ -1134,8 +1134,8 @@ github.com/rudderlabs/compose-test v0.1.3 h1:uyep6jDCIF737sfv4zIaMsKRQKX95IDz5Xb github.com/rudderlabs/compose-test v0.1.3/go.mod h1:tuvS1eQdSfwOYv1qwyVAcpdJxPLQXJgy5xGDd/9XmMg= github.com/rudderlabs/parquet-go v0.0.2 h1:ZXRdZdimB0PdJtmxeSSxfI0fDQ3kZjwzBxRi6Ut1J8k= github.com/rudderlabs/parquet-go v0.0.2/go.mod h1:g6guum7o8uhj/uNhunnt7bw5Vabu/goI5i21/3fnxWQ= -github.com/rudderlabs/rudder-go-kit v0.39.1 h1:aNuGHuYJD8opRNfWVrLh+rWt48cE4ZBb2Km5k64eAcU= -github.com/rudderlabs/rudder-go-kit v0.39.1/go.mod h1:3iiVwUvC/+QUyPbhM5m4NrwL6rTcsAOeDCupLAueS+U= +github.com/rudderlabs/rudder-go-kit v0.39.2 h1:Sisiyeawi43XQOo++5BkcE1+X5XcmCpq1VBwX5EiiiM= +github.com/rudderlabs/rudder-go-kit v0.39.2/go.mod h1:GtOYIFfVvNcXabgGytoGdsjdpKTH6PipFIom0bY94WQ= github.com/rudderlabs/rudder-observability-kit v0.0.3 h1:vZtuZRkGX+6rjaeKtxxFE2YYP6QlmAcVcgecTOjvz+Q= github.com/rudderlabs/rudder-observability-kit v0.0.3/go.mod h1:6UjAh3H6rkE0fFLh7z8ZGQEQbKtUkRfhWOf/OUhfqW8= github.com/rudderlabs/rudder-schemas v0.5.1 h1:g4I5wp2yA6ZWQZ1MjSNn4zby3XctG/TOgbYUW3dS4z4= @@ -1474,8 +1474,8 @@ golang.org/x/oauth2 v0.0.0-20211005180243-6b3c2da341f1/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= golang.org/x/oauth2 v0.0.0-20220309155454-6242fa91716a/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= -golang.org/x/oauth2 v0.22.0 h1:BzDx2FehcG7jJwgWLELCdmLuxk2i+x9UDpSiss2u0ZA= -golang.org/x/oauth2 v0.22.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= +golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From 09aeccfea7425a3ce804c0e30291e1e2fada137e Mon Sep 17 00:00:00 2001 From: Dilip Kola <33080863+koladilip@users.noreply.github.com> Date: Fri, 6 Sep 2024 17:42:55 +0530 Subject: [PATCH 15/29] fix: webhook integration tests (#5061) --- gateway/webhook/integration_test.go | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/gateway/webhook/integration_test.go b/gateway/webhook/integration_test.go index 6a3ae97c4c..14cec08a29 100644 --- a/gateway/webhook/integration_test.go +++ b/gateway/webhook/integration_test.go @@ -89,6 +89,7 @@ func TestIntegrationWebhook(t *testing.T) { "gateway", jobsdb.WithDBHandle(p.DB), ) + require.NoError(t, gatewayDB.Start()) defer gatewayDB.TearDown() @@ -123,6 +124,7 @@ func TestIntegrationWebhook(t *testing.T) { bcs := make(map[string]backendconfig.ConfigT) testSetup := testcases.Load(t) + sourceConfigs := make([]backendconfig.SourceT, len(testSetup.Cases)) for i, tc := range testSetup.Cases { @@ -284,15 +286,19 @@ func TestIntegrationWebhook(t *testing.T) { assert.JSONEq(t, string(p), string(batch.Batch[0])) } - r, err = errDB.GetUnprocessed(ctx, jobsdb.GetQueryParams{ - WorkspaceID: workspaceID, - // ParameterFilters: []jobsdb.ParameterFilterT{{ - // Name: "source_id", - // Value: sourceID, - // }}, - JobsLimit: 1, - }) - require.NoError(t, err) + require.Eventually(t, func() bool { + r, err = errDB.GetUnprocessed(ctx, jobsdb.GetQueryParams{ + WorkspaceID: workspaceID, + ParameterFilters: []jobsdb.ParameterFilterT{{ + Name: "source_id", + Value: sourceID, + }}, + JobsLimit: 10, + }) + require.NoError(t, err) + return len(r.Jobs) == len(tc.Output.ErrQueue) + }, time.Second, time.Millisecond*10) + assert.Len(t, r.Jobs, len(tc.Output.ErrQueue)) for i, p := range tc.Output.ErrQueue { errPayload, err := json.Marshal(struct { From d9c5c7997de9964551a2d1101a6e638920d80db2 Mon Sep 17 00:00:00 2001 From: Rohith BCS Date: Mon, 9 Sep 2024 16:40:16 +0530 Subject: [PATCH 16/29] chore: delete load files post successful upload (#5050) --- .../warehouse/integration_test.go | 127 ------------------ warehouse/integrations/testhelper/setup.go | 1 - warehouse/integrations/testhelper/verify.go | 50 ------- warehouse/router/upload.go | 21 +-- 4 files changed, 11 insertions(+), 188 deletions(-) diff --git a/integration_test/warehouse/integration_test.go b/integration_test/warehouse/integration_test.go index e015a7ac0c..6c4f90a594 100644 --- a/integration_test/warehouse/integration_test.go +++ b/integration_test/warehouse/integration_test.go @@ -109,10 +109,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -174,10 +170,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -231,10 +223,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -283,10 +271,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -412,10 +396,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -526,10 +506,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -602,10 +578,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -803,10 +775,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events+(events/2), []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events+(events/2), []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -929,10 +897,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -954,10 +918,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1018,10 +978,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1043,10 +999,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1107,10 +1059,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1132,10 +1080,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1196,10 +1140,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1221,10 +1161,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1288,10 +1224,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1313,10 +1245,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1380,10 +1308,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1405,10 +1329,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1472,10 +1392,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1497,10 +1413,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*2, []lo.Tuple2[string, any]{ {A: "status", B: exportedData}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1574,10 +1486,6 @@ func TestUploads(t *testing.T) { {A: "destination_id", B: destinationID}, {A: "status", B: succeeded}, }...) - requireLoadFileEventsCount(t, ctx, db, events*3, []lo.Tuple2[string, any]{ - {A: "source_id", B: sourceID}, - {A: "destination_id", B: destinationID}, - }...) requireTableUploadEventsCount(t, ctx, db, events*3, []lo.Tuple2[string, any]{ {A: "status", B: waiting}, {A: "wh_uploads.source_id", B: sourceID}, @@ -1787,41 +1695,6 @@ func requireStagingFileEventsCount( ) } -// nolint:unparam -func requireLoadFileEventsCount( - t testing.TB, - ctx context.Context, - db *sqlmw.DB, - expectedCount int, - filters ...lo.Tuple2[string, any], -) { - t.Helper() - - query := "SELECT COALESCE(sum(total_events), 0) FROM wh_load_files WHERE 1 = 1" - query += strings.Join(lo.Map(filters, func(t lo.Tuple2[string, any], index int) string { - return fmt.Sprintf(" AND %s = $%d", t.A, index+1) - }), "") - queryArgs := lo.Map(filters, func(t lo.Tuple2[string, any], _ int) any { - return t.B - }) - - require.Eventuallyf(t, - func() bool { - var eventsCount int - err := db.QueryRowContext(ctx, query, queryArgs...).Scan(&eventsCount) - if err != nil { - t.Logf("error getting load file events count: %v", err) - return false - } - t.Logf("Load file events count: %d", eventsCount) - return eventsCount == expectedCount - }, - 10*time.Second, - 250*time.Millisecond, - "expected load file events count to be %d", expectedCount, - ) -} - // nolint:unparam func requireTableUploadEventsCount( t testing.TB, diff --git a/warehouse/integrations/testhelper/setup.go b/warehouse/integrations/testhelper/setup.go index 588c42b21c..8dfa3c8275 100644 --- a/warehouse/integrations/testhelper/setup.go +++ b/warehouse/integrations/testhelper/setup.go @@ -77,7 +77,6 @@ func (w *TestConfig) VerifyEvents(t testing.TB) { createStagingFile(t, w) verifyEventsInStagingFiles(t, w) - verifyEventsInLoadFiles(t, w) verifyEventsInTableUploads(t, w) if w.SourceJob { diff --git a/warehouse/integrations/testhelper/verify.go b/warehouse/integrations/testhelper/verify.go index 1d703a19a7..1374afd96a 100644 --- a/warehouse/integrations/testhelper/verify.go +++ b/warehouse/integrations/testhelper/verify.go @@ -69,56 +69,6 @@ func verifyEventsInStagingFiles(t testing.TB, testConfig *TestConfig) { t.Logf("Completed verifying events in staging files") } -func verifyEventsInLoadFiles(t testing.TB, testConfig *TestConfig) { - t.Helper() - - t.Logf("Started verifying events in load file") - - for _, table := range testConfig.Tables { - sqlStatement := ` - SELECT - SUM(total_events) AS sum - FROM - wh_load_files - WHERE - source_id = $1 - AND destination_id = $2 - AND created_at > $3 - AND table_name = $4;` - t.Logf("Checking events in load files for SrcID: %s, DestID: %s, TimestampBeforeSendingEvents: %s, "+ - "Table: %s, sqlStatement: %s", - testConfig.SourceID, testConfig.DestinationID, testConfig.TimestampBeforeSendingEvents, - whutils.ToProviderCase(testConfig.DestinationType, table), sqlStatement, - ) - - var err error - var count sql.NullInt64 - expectedCount := int64(testConfig.LoadFilesEventsMap[table]) - - operation := func() bool { - err = testConfig.JobsDB.QueryRow(sqlStatement, - testConfig.SourceID, testConfig.DestinationID, testConfig.TimestampBeforeSendingEvents, - whutils.ToProviderCase(testConfig.DestinationType, table), - ).Scan(&count) - - if err == nil && count.Int64 != expectedCount { - t.Logf("Expected load files events count for table %q is %d, got %d", - table, expectedCount, count.Int64, - ) - } - - return err == nil && count.Int64 == expectedCount - } - // Expected load files events count for table _groups is 0, got 0: - require.Eventuallyf(t, operation, WaitFor10Minute, DefaultQueryFrequency, - "Expected load files events count for table %q is %d, got %d: %v", - table, expectedCount, count, err, - ) - } - - t.Logf("Completed verifying events in load files") -} - func verifyEventsInTableUploads(t testing.TB, testConfig *TestConfig) { t.Helper() diff --git a/warehouse/router/upload.go b/warehouse/router/upload.go index b8558985ac..683c096605 100644 --- a/warehouse/router/upload.go +++ b/warehouse/router/upload.go @@ -10,6 +10,8 @@ import ( "sync" "time" + obskit "github.com/rudderlabs/rudder-observability-kit/go/labels" + "github.com/cenkalti/backoff/v4" "github.com/rudderlabs/rudder-go-kit/config" @@ -287,16 +289,14 @@ func (job *UploadJob) run() (err error) { defer whManager.Cleanup(job.ctx) if err = job.recovery.Recover(job.ctx, whManager, job.warehouse); err != nil { - job.logger.Warnw("Error during recovery (dangling staging table cleanup)", - logfield.DestinationID, job.warehouse.Destination.ID, - logfield.DestinationType, job.warehouse.Destination.DestinationDefinition.Name, - logfield.SourceID, job.warehouse.Source.ID, - logfield.SourceType, job.warehouse.Source.SourceDefinition.Name, - logfield.DestinationID, job.warehouse.Destination.ID, - logfield.DestinationType, job.warehouse.Destination.DestinationDefinition.Name, - logfield.WorkspaceID, job.warehouse.WorkspaceID, - logfield.Namespace, job.warehouse.Namespace, - logfield.Error, err.Error(), + job.logger.Warnn("Error during recovery (dangling staging table cleanup)", + obskit.DestinationID(job.warehouse.Destination.ID), + obskit.DestinationType(job.warehouse.Destination.DestinationDefinition.Name), + obskit.WorkspaceID(job.warehouse.WorkspaceID), + obskit.Namespace(job.warehouse.Namespace), + obskit.Error(err), + obskit.SourceID(job.warehouse.Source.ID), + obskit.SourceType(job.warehouse.Source.SourceDefinition.Name), ) _, _ = job.setUploadError(err, InternalProcessingFailed) return err @@ -424,6 +424,7 @@ func (job *UploadJob) run() (err error) { job.timerStat(nextUploadState.inProgress).SendTiming(time.Since(stateStartTime)) if newStatus == model.ExportedData { + _ = job.loadFilesRepo.DeleteByStagingFiles(job.ctx, job.stagingFileIDs) break } From df7e02f77db50c397e42ea6524099187e4519152 Mon Sep 17 00:00:00 2001 From: Mihir Gandhi Date: Mon, 9 Sep 2024 20:28:13 +0530 Subject: [PATCH 17/29] chore: sync 1.33.1 to master (#5071) * chore: release 1.33.0 (#5047) * fix: revert support for webhook headers (#5064) * fix: revert support for webhook headers * chore: ran go mod tidy * chore: release 1.33.1 (#5065) --------- Co-authored-by: devops-github-rudderstack <88187154+devops-github-rudderstack@users.noreply.github.com> Co-authored-by: Dilip Kola <33080863+koladilip@users.noreply.github.com> --- CHANGELOG.md | 7 +++++++ gateway/webhook/integration_test.go | 6 +++--- gateway/webhook/webhook.go | 20 -------------------- gateway/webhook/webhook_test.go | 6 ------ go.mod | 2 +- go.sum | 4 ++-- 6 files changed, 13 insertions(+), 32 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7f9e0a909a..1aafa99c98 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,12 @@ # Changelog +## [1.33.1](https://github.com/rudderlabs/rudder-server/compare/v1.33.0...v1.33.1) (2024-09-06) + + +### Bug Fixes + +* revert support for webhook headers ([#5064](https://github.com/rudderlabs/rudder-server/issues/5064)) ([56eca34](https://github.com/rudderlabs/rudder-server/commit/56eca3475e24ab9ba12624d2b4015c9d15be7249)) + ## [1.33.0](https://github.com/rudderlabs/rudder-server/compare/v1.32.0...v1.33.0) (2024-09-02) diff --git a/gateway/webhook/integration_test.go b/gateway/webhook/integration_test.go index 14cec08a29..51d0230476 100644 --- a/gateway/webhook/integration_test.go +++ b/gateway/webhook/integration_test.go @@ -293,12 +293,12 @@ func TestIntegrationWebhook(t *testing.T) { Name: "source_id", Value: sourceID, }}, - JobsLimit: 10, + JobsLimit: 1, }) - require.NoError(t, err) - return len(r.Jobs) == len(tc.Output.ErrQueue) + return err == nil && len(r.Jobs) == len(tc.Output.ErrQueue) }, time.Second, time.Millisecond*10) + require.NoError(t, err) assert.Len(t, r.Jobs, len(tc.Output.ErrQueue)) for i, p := range tc.Output.ErrQueue { errPayload, err := json.Marshal(struct { diff --git a/gateway/webhook/webhook.go b/gateway/webhook/webhook.go index ccd6fc9bbf..fd05420eaf 100644 --- a/gateway/webhook/webhook.go +++ b/gateway/webhook/webhook.go @@ -269,17 +269,6 @@ func (webhook *HandleT) batchRequests(sourceDef string, requestQ chan *webhookT) } } -func getXHeaders(req *http.Request) map[string]string { - xHeaders := make(map[string]string) - for key, values := range req.Header { - lowerCaseKey := strings.ToLower(key) - if !strings.HasPrefix(lowerCaseKey, "x-forwarded-") && strings.HasPrefix(lowerCaseKey, "x-") { - xHeaders[key] = strings.Join(values, ",") - } - } - return xHeaders -} - func prepareRequestBody(req *http.Request, sourceType string, sourceListForParsingParams []string) ([]byte, error) { defer func() { _ = req.Body.Close() @@ -303,15 +292,6 @@ func prepareRequestBody(req *http.Request, sourceType string, sourceListForParsi } } - xHeaders := getXHeaders(req) - if len(xHeaders) > 0 { - body, err = sjson.SetBytes(body, "headers", xHeaders) - if err != nil { - return nil, errors.New(response.InvalidJSON) - } - - } - return body, nil } diff --git a/gateway/webhook/webhook_test.go b/gateway/webhook/webhook_test.go index 7774f651f2..5fa7a64028 100644 --- a/gateway/webhook/webhook_test.go +++ b/gateway/webhook/webhook_test.go @@ -549,12 +549,6 @@ func TestPrepareRequestBody(t *testing.T) { sourceType: "shopify", expectedResponse: `{"key":"value","query_parameters":{}}`, }, - { - name: "Some payload with headers for shopify", - req: createRequest(requestOpts{method: http.MethodPost, target: "http://example.com", body: strings.NewReader(`{"key":"value"}`), headers: map[string]string{"X-Key": "header-value"}}), - sourceType: "shopify", - expectedResponse: `{"key":"value","query_parameters":{},"headers":{"X-Key":"header-value"}}`, - }, { name: "Some payload with query parameters for Adjust", req: createRequest(requestOpts{method: http.MethodPost, target: "http://example.com", body: strings.NewReader(`{"key1":"value1"}`), params: map[string]string{"key2": "value2"}}), diff --git a/go.mod b/go.mod index 172d07668c..5e79bebfb5 100644 --- a/go.mod +++ b/go.mod @@ -81,7 +81,7 @@ require ( github.com/rudderlabs/rudder-go-kit v0.39.2 github.com/rudderlabs/rudder-observability-kit v0.0.3 github.com/rudderlabs/rudder-schemas v0.5.1 - github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32 + github.com/rudderlabs/rudder-transformer/go v0.0.0-20240906042448-f7783d8fb300 github.com/rudderlabs/sql-tunnels v0.1.7 github.com/rudderlabs/sqlconnect-go v1.9.0 github.com/samber/lo v1.47.0 diff --git a/go.sum b/go.sum index 96f7c6817b..4d50929df3 100644 --- a/go.sum +++ b/go.sum @@ -1140,8 +1140,8 @@ github.com/rudderlabs/rudder-observability-kit v0.0.3 h1:vZtuZRkGX+6rjaeKtxxFE2Y github.com/rudderlabs/rudder-observability-kit v0.0.3/go.mod h1:6UjAh3H6rkE0fFLh7z8ZGQEQbKtUkRfhWOf/OUhfqW8= github.com/rudderlabs/rudder-schemas v0.5.1 h1:g4I5wp2yA6ZWQZ1MjSNn4zby3XctG/TOgbYUW3dS4z4= github.com/rudderlabs/rudder-schemas v0.5.1/go.mod h1:JoDTB9nCDXwRz+G+aYwP3Fj42HLssKARxsFFm+qqgb4= -github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32 h1:hsvbOvwfA8wr9GjNBWuh3lZnpqMgIzLU7HNESB87wSg= -github.com/rudderlabs/rudder-transformer/go v0.0.0-20240903092449-3f2a5a664d32/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= +github.com/rudderlabs/rudder-transformer/go v0.0.0-20240906042448-f7783d8fb300 h1:SmOLUqSCCcYs8QXYdZlHXCSCw77xhQ6qjNBsSA3bDKI= +github.com/rudderlabs/rudder-transformer/go v0.0.0-20240906042448-f7783d8fb300/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= github.com/rudderlabs/sql-tunnels v0.1.7 h1:wDCRl6zY4M5gfWazf7XkSTGQS3yjBzUiUgEMBIfHNDA= github.com/rudderlabs/sql-tunnels v0.1.7/go.mod h1:5f7+YL49JHYgteP4rAgqKnr4K2OadB0oIpUS+Tt3sPM= github.com/rudderlabs/sqlconnect-go v1.9.0 h1:icLgqvVQ15Vh+oP7epA0b0yK6sIzxRVwPlRzOoDNVRA= From ff271e58c91bcf7036ae030887a6b6086f3b214e Mon Sep 17 00:00:00 2001 From: Akash Chetty Date: Tue, 10 Sep 2024 15:00:46 +0530 Subject: [PATCH 18/29] chore: for bigquery FORCE_RUN_INTEGRATION_TESTS (#5060) --- warehouse/integrations/bigquery/bigquery_test.go | 3 +++ warehouse/integrations/bigquery/middleware/middleware_test.go | 3 +++ 2 files changed, 6 insertions(+) diff --git a/warehouse/integrations/bigquery/bigquery_test.go b/warehouse/integrations/bigquery/bigquery_test.go index 355d8bc1a8..828d4301ad 100644 --- a/warehouse/integrations/bigquery/bigquery_test.go +++ b/warehouse/integrations/bigquery/bigquery_test.go @@ -45,6 +45,9 @@ func TestIntegration(t *testing.T) { t.Skip("Skipping tests. Add 'SLOW=1' env var to run test.") } if _, exists := os.LookupEnv(bqHelper.TestKey); !exists { + if os.Getenv("FORCE_RUN_INTEGRATION_TESTS") == "true" { + t.Fatalf("%s environment variable not set", bqHelper.TestKey) + } t.Skipf("Skipping %s as %s is not set", t.Name(), bqHelper.TestKey) } diff --git a/warehouse/integrations/bigquery/middleware/middleware_test.go b/warehouse/integrations/bigquery/middleware/middleware_test.go index 8fec7adff9..0837d023a4 100644 --- a/warehouse/integrations/bigquery/middleware/middleware_test.go +++ b/warehouse/integrations/bigquery/middleware/middleware_test.go @@ -20,6 +20,9 @@ import ( func TestQueryWrapper(t *testing.T) { if _, exists := os.LookupEnv(bqHelper.TestKey); !exists { + if os.Getenv("FORCE_RUN_INTEGRATION_TESTS") == "true" { + t.Fatalf("%s environment variable not set", bqHelper.TestKey) + } t.Skipf("Skipping %s as %s is not set", t.Name(), bqHelper.TestKey) } From 57dd8c18ec0fb11775020841611908d9c1d4d9a3 Mon Sep 17 00:00:00 2001 From: Akash Chetty Date: Tue, 10 Sep 2024 15:01:16 +0530 Subject: [PATCH 19/29] fix: datalake spark flaky test (#5075) --- .../integrations/datalake/testdata/docker-compose.spark.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/warehouse/integrations/datalake/testdata/docker-compose.spark.yml b/warehouse/integrations/datalake/testdata/docker-compose.spark.yml index 6b4a9752d0..de21014667 100644 --- a/warehouse/integrations/datalake/testdata/docker-compose.spark.yml +++ b/warehouse/integrations/datalake/testdata/docker-compose.spark.yml @@ -24,6 +24,8 @@ services: ports: - "8080" - "7077" + depends_on: + - hive-metastore deploy: resources: limits: From 9a4b8439dfea3ab46aa6da4d179192502337b230 Mon Sep 17 00:00:00 2001 From: Leonidas Vrachnis Date: Tue, 10 Sep 2024 13:52:57 +0200 Subject: [PATCH 20/29] fix: prevent badgerdb hang when closing without prior operations (#5074) --- services/dedup/badger/badger.go | 28 +++++++++++++++++++--------- services/dedup/badger/badger_test.go | 8 ++++++++ 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/services/dedup/badger/badger.go b/services/dedup/badger/badger.go index c776506d91..fc3154e122 100644 --- a/services/dedup/badger/badger.go +++ b/services/dedup/badger/badger.go @@ -1,6 +1,7 @@ package badger import ( + "context" "fmt" "strconv" "sync" @@ -23,11 +24,12 @@ type BadgerDB struct { logger loggerForBadger badgerDB *badger.DB window config.ValueLoader[time.Duration] - close chan struct{} - gcDone chan struct{} path string opts badger.Options once sync.Once + wg sync.WaitGroup + bgCtx context.Context + cancel context.CancelFunc } // DefaultPath returns the default path for the deduplication service's badger DB @@ -57,14 +59,16 @@ func NewBadgerDB(conf *config.Config, stats stats.Stats, path string) *Dedup { WithSyncWrites(conf.GetBool("BadgerDB.syncWrites", false)). WithDetectConflicts(conf.GetBool("BadgerDB.detectConflicts", false)) + bgCtx, cancel := context.WithCancel(context.Background()) db := &BadgerDB{ stats: stats, logger: loggerForBadger{log}, path: path, - gcDone: make(chan struct{}), - close: make(chan struct{}), window: dedupWindow, opts: badgerOpts, + wg: sync.WaitGroup{}, + bgCtx: bgCtx, + cancel: cancel, } return &Dedup{ badgerDB: db, @@ -114,9 +118,11 @@ func (d *BadgerDB) Set(kvs []types.KeyValue) error { } func (d *BadgerDB) Close() { - close(d.close) - <-d.gcDone - _ = d.badgerDB.Close() + d.cancel() + d.wg.Wait() + if d.badgerDB != nil { + _ = d.badgerDB.Close() + } } func (d *BadgerDB) init() error { @@ -127,9 +133,10 @@ func (d *BadgerDB) init() error { if err != nil { return } + d.wg.Add(1) rruntime.Go(func() { + defer d.wg.Done() d.gcLoop() - close(d.gcDone) }) }) return err @@ -138,12 +145,15 @@ func (d *BadgerDB) init() error { func (d *BadgerDB) gcLoop() { for { select { - case <-d.close: + case <-d.bgCtx.Done(): _ = d.badgerDB.RunValueLogGC(0.5) return case <-time.After(5 * time.Minute): } again: + if d.bgCtx.Err() != nil { + return + } // One call would only result in removal of at max one log file. // As an optimization, you could also immediately re-run it whenever it returns nil error // (this is why `goto again` is used). diff --git a/services/dedup/badger/badger_test.go b/services/dedup/badger/badger_test.go index f97bb4ced7..8618a2a383 100644 --- a/services/dedup/badger/badger_test.go +++ b/services/dedup/badger/badger_test.go @@ -46,3 +46,11 @@ func Test_Badger(t *testing.T) { require.False(t, found) }) } + +func TestBadgerClose(t *testing.T) { + badger := NewBadgerDB(config.New(), stats.NOP, t.TempDir()) + require.NotNil(t, badger) + + t.Log("close badger without any other operation") + badger.Close() +} From 6493beb18c92373b0796b3413849ecf6e1e37683 Mon Sep 17 00:00:00 2001 From: Akash Chetty Date: Wed, 11 Sep 2024 20:11:39 +0530 Subject: [PATCH 21/29] chore: deps upgrade (#5081) --- Dockerfile | 2 +- go.mod | 149 ++++----- go.sum | 290 +++++++++--------- .../backend_config_unavailability_test.go | 3 +- suppression-backup-service/Dockerfile | 2 +- 5 files changed, 229 insertions(+), 217 deletions(-) diff --git a/Dockerfile b/Dockerfile index 749277686c..6c8e85ac51 100644 --- a/Dockerfile +++ b/Dockerfile @@ -2,7 +2,7 @@ # syntax=docker/dockerfile:1 # GO_VERSION is updated automatically to match go.mod, see Makefile -ARG GO_VERSION=1.23.0 +ARG GO_VERSION=1.23.1 ARG ALPINE_VERSION=3.20 FROM golang:${GO_VERSION}-alpine${ALPINE_VERSION} AS builder ARG VERSION diff --git a/go.mod b/go.mod index 5e79bebfb5..4746ddb719 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/rudderlabs/rudder-server -go 1.23.0 +go 1.23.1 // Addressing snyk vulnerabilities in indirect dependencies // When upgrading a dependency, please make sure that @@ -11,7 +11,7 @@ replace ( github.com/gin-gonic/gin => github.com/gin-gonic/gin v1.10.0 github.com/go-jose/go-jose/v3 => github.com/go-jose/go-jose/v3 v3.0.3 github.com/gocql/gocql => github.com/scylladb/gocql v1.14.2 - github.com/opencontainers/runc => github.com/opencontainers/runc v1.1.12 + github.com/opencontainers/runc => github.com/opencontainers/runc v1.1.14 github.com/satori/go.uuid => github.com/satori/go.uuid v1.2.0 github.com/xitongsys/parquet-go => github.com/rudderlabs/parquet-go v0.0.2 golang.org/x/crypto => golang.org/x/crypto v0.23.0 @@ -25,7 +25,7 @@ replace ( require ( cloud.google.com/go/bigquery v1.62.0 - cloud.google.com/go/pubsub v1.42.0 + cloud.google.com/go/pubsub v1.43.0 cloud.google.com/go/storage v1.43.0 github.com/Azure/azure-storage-blob-go v0.15.0 github.com/ClickHouse/clickhouse-go v1.5.4 @@ -41,12 +41,12 @@ require ( github.com/databricks/databricks-sql-go v1.6.1 github.com/denisenkom/go-mssqldb v0.12.3 github.com/dgraph-io/badger/v4 v4.3.0 - github.com/docker/docker v27.2.0+incompatible + github.com/docker/docker v27.2.1+incompatible github.com/go-chi/chi/v5 v5.1.0 github.com/go-redis/redis v6.15.9+incompatible github.com/go-redis/redis/v8 v8.11.5 github.com/gocql/gocql v1.14.2 - github.com/golang-migrate/migrate/v4 v4.17.1 + github.com/golang-migrate/migrate/v4 v4.18.1 github.com/golang/mock v1.6.0 github.com/gomodule/redigo v1.9.2 github.com/google/uuid v1.6.0 @@ -63,7 +63,7 @@ require ( github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.0 github.com/manifoldco/promptui v0.9.0 - github.com/marcboeker/go-duckdb v1.7.1 + github.com/marcboeker/go-duckdb v1.8.0 github.com/minio/minio-go/v7 v7.0.76 github.com/mitchellh/mapstructure v1.5.0 github.com/olekukonko/tablewriter v0.0.5 @@ -78,10 +78,10 @@ require ( github.com/rudderlabs/analytics-go v3.3.3+incompatible github.com/rudderlabs/bing-ads-go-sdk v0.2.3 github.com/rudderlabs/compose-test v0.1.3 - github.com/rudderlabs/rudder-go-kit v0.39.2 + github.com/rudderlabs/rudder-go-kit v0.41.1 github.com/rudderlabs/rudder-observability-kit v0.0.3 github.com/rudderlabs/rudder-schemas v0.5.1 - github.com/rudderlabs/rudder-transformer/go v0.0.0-20240906042448-f7783d8fb300 + github.com/rudderlabs/rudder-transformer/go v0.0.0-20240910055720-f77d2ab4125a github.com/rudderlabs/sql-tunnels v0.1.7 github.com/rudderlabs/sqlconnect-go v1.9.0 github.com/samber/lo v1.47.0 @@ -99,44 +99,38 @@ require ( github.com/trinodb/trino-go-client v0.316.0 github.com/urfave/cli/v2 v2.27.4 github.com/viney-shih/go-lock v1.1.2 - github.com/xitongsys/parquet-go v1.5.1 + github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20240122235623-d6294584ab18 - go.etcd.io/etcd/api/v3 v3.5.15 - go.etcd.io/etcd/client/v3 v3.5.15 + go.etcd.io/etcd/api/v3 v3.5.16 + go.etcd.io/etcd/client/v3 v3.5.16 go.uber.org/atomic v1.11.0 go.uber.org/automaxprocs v1.5.3 go.uber.org/goleak v1.3.0 go.uber.org/mock v0.4.0 - golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 + golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 google.golang.org/api v0.196.0 google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 - google.golang.org/grpc v1.66.0 + google.golang.org/grpc v1.66.1 google.golang.org/protobuf v1.34.2 ) -require ( - github.com/apache/arrow/go/v17 v17.0.0 // indirect - github.com/moby/sys/userns v0.1.0 // indirect - github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect -) - require ( cloud.google.com/go v0.115.1 // indirect cloud.google.com/go/auth v0.9.3 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.4 // indirect cloud.google.com/go/compute/metadata v0.5.0 // indirect cloud.google.com/go/iam v1.2.0 // indirect - dario.cat/mergo v1.0.0 // indirect + dario.cat/mergo v1.0.1 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect github.com/99designs/keyring v1.2.2 // indirect - github.com/AthenZ/athenz v1.11.59 // indirect + github.com/AthenZ/athenz v1.11.65 // indirect github.com/Azure/azure-pipeline-go v0.2.3 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 // indirect - github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.0 // indirect github.com/Azure/go-ansiterm v0.0.0-20230124172434-306776ec8161 // indirect github.com/DataDog/zstd v1.5.6 // indirect github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect @@ -147,49 +141,51 @@ require ( github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40 // indirect github.com/apache/arrow/go/v12 v12.0.1 // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect + github.com/apache/arrow/go/v17 v17.0.0 // indirect github.com/apache/thrift v0.20.0 // indirect github.com/ardielle/ardielle-go v1.5.2 // indirect github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5 // indirect - github.com/aws/aws-sdk-go-v2 v1.30.4 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.2 // indirect - github.com/aws/aws-sdk-go-v2/config v1.27.27 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.17.27 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.16.21 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.16 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.16 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.7 // indirect + github.com/aws/aws-sdk-go-v2 v1.30.5 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.4 // indirect + github.com/aws/aws-sdk-go-v2/config v1.27.33 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.17.32 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.13 // indirect + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.18 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.17 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.17 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.17 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.4 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.9 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.18 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.7 // indirect - github.com/aws/aws-sdk-go-v2/service/redshiftdata v1.27.3 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.54.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.30.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.19 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.19 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.17 // indirect + github.com/aws/aws-sdk-go-v2/service/redshiftdata v1.28.2 // indirect + github.com/aws/aws-sdk-go-v2/service/s3 v1.61.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.22.7 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.7 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.30.7 // indirect github.com/aws/smithy-go v1.20.4 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bitfield/gotestdox v0.2.2 // indirect - github.com/bits-and-blooms/bitset v1.13.0 // indirect + github.com/bits-and-blooms/bitset v1.14.3 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/chzyer/readline v1.5.1 // indirect github.com/cloudflare/golz4 v0.0.0-20150217214814-ef862a3cdc58 // indirect - github.com/containerd/containerd v1.7.18 // indirect + github.com/containerd/containerd v1.7.22 // indirect github.com/containerd/continuity v0.4.3 // indirect github.com/containerd/log v0.1.0 // indirect - github.com/coreos/go-oidc/v3 v3.10.0 // indirect + github.com/coreos/go-oidc/v3 v3.11.0 // indirect github.com/coreos/go-semver v0.3.1 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.4 // indirect - github.com/danieljoos/wincred v1.2.1 // indirect + github.com/danieljoos/wincred v1.2.2 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dgraph-io/ristretto v0.1.2-0.20240116140435-c67e07994f91 // indirect github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect github.com/dlclark/regexp2 v1.11.4 // indirect github.com/dnephin/pflag v1.0.7 // indirect - github.com/docker/cli v26.1.4+incompatible // indirect + github.com/docker/cli v27.2.1+incompatible // indirect + github.com/docker/cli-docs-tool v0.8.0 // indirect github.com/docker/go-connections v0.5.0 // indirect github.com/docker/go-units v0.5.0 // indirect github.com/dustin/go-humanize v1.0.1 // indirect @@ -198,17 +194,19 @@ require ( github.com/fatih/color v1.17.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/fsnotify/fsnotify v1.7.0 // indirect - github.com/gabriel-vasile/mimetype v1.4.4 // indirect + github.com/fxamacker/cbor/v2 v2.7.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.5 // indirect github.com/go-ini/ini v1.67.0 // indirect - github.com/go-jose/go-jose/v4 v4.0.2 // indirect + github.com/go-jose/go-jose/v4 v4.0.4 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect github.com/go-playground/locales v0.14.1 // indirect github.com/go-playground/universal-translator v0.18.1 // indirect - github.com/go-playground/validator/v10 v10.22.0 // indirect + github.com/go-playground/validator/v10 v10.22.1 // indirect github.com/go-sql-driver/mysql v1.8.1 // indirect github.com/go-task/slim-sprig/v3 v3.0.0 // indirect + github.com/go-viper/mapstructure/v2 v2.1.0 // indirect github.com/goccy/go-json v0.10.3 // indirect github.com/goccy/go-reflect v1.2.0 // indirect github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect @@ -222,14 +220,14 @@ require ( github.com/google/flatbuffers v24.3.25+incompatible // indirect github.com/google/go-cmp v0.6.0 // indirect github.com/google/gofuzz v1.2.0 // indirect - github.com/google/pprof v0.0.0-20240827171923-fa2c70bbbfe5 // indirect + github.com/google/pprof v0.0.0-20240910150728-a0b0bb1d4134 // indirect github.com/google/s2a-go v0.1.8 // indirect github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.3.3 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.4 // indirect github.com/googleapis/gax-go/v2 v2.13.0 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed // indirect - github.com/hamba/avro/v2 v2.24.0 // indirect + github.com/hamba/avro/v2 v2.25.2 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect @@ -244,25 +242,27 @@ require ( github.com/klauspost/cpuid/v2 v2.2.8 // indirect github.com/kr/fs v0.1.0 // indirect github.com/leodido/go-urn v1.4.0 // indirect - github.com/lufia/plan9stats v0.0.0-20240513124658-fba389f38bae // indirect + github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/mattn/go-colorable v0.1.13 // indirect github.com/mattn/go-ieproxy v0.0.12 // indirect github.com/mattn/go-isatty v0.0.20 // indirect - github.com/mattn/go-runewidth v0.0.15 // indirect + github.com/mattn/go-runewidth v0.0.16 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect github.com/minio/md5-simd v1.1.2 // indirect github.com/moby/docker-image-spec v1.3.1 // indirect + github.com/moby/sys/userns v0.1.0 // indirect github.com/moby/term v0.5.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mtibben/percent v0.2.1 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.1.0 // indirect github.com/opencontainers/runc v1.1.14 // indirect github.com/openzipkin/zipkin-go v0.4.3 // indirect - github.com/pelletier/go-toml/v2 v2.2.2 // indirect + github.com/pelletier/go-toml/v2 v2.2.3 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect @@ -270,14 +270,14 @@ require ( github.com/pkg/sftp v1.13.6 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 // indirect - github.com/prometheus/client_golang v1.20.2 // indirect - github.com/prometheus/common v0.55.0 // indirect + github.com/prometheus/client_golang v1.20.3 // indirect + github.com/prometheus/common v0.59.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/rs/xid v1.6.0 // indirect github.com/rs/zerolog v1.33.0 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect - github.com/sagikazarmark/locafero v0.4.0 // indirect + github.com/sagikazarmark/locafero v0.6.0 // indirect github.com/sagikazarmark/slog-shim v0.1.0 // indirect github.com/segmentio/backo-go v1.1.0 // indirect github.com/shirou/gopsutil/v3 v3.24.5 // indirect @@ -291,6 +291,7 @@ require ( github.com/tidwall/pretty v1.2.1 // indirect github.com/tklauser/go-sysconf v0.3.14 // indirect github.com/tklauser/numcpus v0.8.0 // indirect + github.com/x448/float16 v0.8.4 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/scram v1.1.2 // indirect github.com/xdg-go/stringprep v1.0.4 // indirect @@ -299,10 +300,10 @@ require ( github.com/xeipuuv/gojsonschema v1.2.0 // indirect github.com/xrash/smetrics v0.0.0-20240521201337-686a1a2994c1 // indirect github.com/xtgo/uuid v0.0.0-20140804021211-a0b114877d4c // indirect - github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76 // indirect + github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.15 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.16 // indirect go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.54.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0 // indirect @@ -319,17 +320,17 @@ require ( go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect - golang.org/x/crypto v0.26.0 // indirect - golang.org/x/mod v0.20.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/term v0.23.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/crypto v0.27.0 // indirect + golang.org/x/mod v0.21.0 // indirect + golang.org/x/net v0.29.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/term v0.24.0 // indirect + golang.org/x/text v0.18.0 // indirect golang.org/x/time v0.6.0 // indirect - golang.org/x/tools v0.24.0 // indirect - golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect + golang.org/x/tools v0.25.0 // indirect + golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240903143218-8af14fe29dc1 // indirect gopkg.in/alexcesaro/statsd.v2 v2.0.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect @@ -341,10 +342,10 @@ require ( gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect gotest.tools/gotestsum v1.12.0 // indirect - k8s.io/apimachinery v0.30.1 // indirect - k8s.io/client-go v0.30.1 // indirect - k8s.io/klog/v2 v2.120.1 // indirect - k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0 // indirect + k8s.io/apimachinery v0.31.0 // indirect + k8s.io/client-go v0.31.0 // indirect + k8s.io/klog/v2 v2.130.1 // indirect + k8s.io/utils v0.0.0-20240902221715-702e33fdd3c3 // indirect sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd // indirect sigs.k8s.io/structured-merge-diff/v4 v4.4.1 // indirect ) diff --git a/go.sum b/go.sum index 4d50929df3..aa7ce7a3f2 100644 --- a/go.sum +++ b/go.sum @@ -73,8 +73,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= cloud.google.com/go/pubsub v1.19.0/go.mod h1:/O9kmSe9bb9KRnIAWkzmqhPjHo6LtzGOBYd/kr06XSs= -cloud.google.com/go/pubsub v1.42.0 h1:PVTbzorLryFL5ue8esTS2BfehUs0ahyNOY9qcd+HMOs= -cloud.google.com/go/pubsub v1.42.0/go.mod h1:KADJ6s4MbTwhXmse/50SebEhE4SmUwHi48z3/dHar1Y= +cloud.google.com/go/pubsub v1.43.0 h1:s3Qx+F96J7Kwey/uVHdK3QxFLIlOvvw4SfMYw2jFjb4= +cloud.google.com/go/pubsub v1.43.0/go.mod h1:LNLfqItblovg7mHWgU5g84Vhza4J8kTxx0YqIeTzcXY= cloud.google.com/go/secretmanager v1.3.0/go.mod h1:+oLTkouyiYiabAQNugCeTS3PAArGiMJuBqvJnJsyH+U= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= @@ -90,8 +90,8 @@ cloud.google.com/go/trace v1.2.0/go.mod h1:Wc8y/uYyOhPy12KEnXG9XGrvfMz5F5SrYecQl contrib.go.opencensus.io/exporter/aws v0.0.0-20200617204711-c478e41e60e9/go.mod h1:uu1P0UCM/6RbsMrgPa98ll8ZcHM858i/AD06a9aLRCA= contrib.go.opencensus.io/exporter/stackdriver v0.13.10/go.mod h1:I5htMbyta491eUxufwwZPQdcKvvgzMB4O9ni41YnIM8= contrib.go.opencensus.io/integrations/ocsql v0.1.7/go.mod h1:8DsSdjz3F+APR+0z0WkU1aRorQCFfRxvqjUUPMbF3fE= -dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk= -dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= +dario.cat/mergo v1.0.1 h1:Ra4+bf83h2ztPIQYNP99R6m+Y7KfnARDfID+a+vLl4s= +dario.cat/mergo v1.0.1/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= filippo.io/edwards25519 v1.1.0 h1:FNf4tywRC1HmFuKW5xopWpigGjJKiJSV0Cqo0cJWDaA= filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4VDT4= @@ -102,8 +102,8 @@ github.com/99designs/keyring v1.2.2 h1:pZd3neh/EmUzWONb35LxQfvuY7kiSXAq3HQd97+XB github.com/99designs/keyring v1.2.2/go.mod h1:wes/FrByc8j7lFOAGLGSNEg8f/PaI3cgTBqhFkHUrPk= github.com/AlecAivazis/survey/v2 v2.3.7 h1:6I/u8FvytdGsgonrYsVn2t8t4QiRnh6QSTqkkhIiSjQ= github.com/AlecAivazis/survey/v2 v2.3.7/go.mod h1:xUTIdE4KCOIjsBAE1JYsUPoCqYdZ1reCfTwbto0Fduo= -github.com/AthenZ/athenz v1.11.59 h1:0YyXYZ0RXI5hLXWVMbwj21AdyAEQoTH1Obefu+sfBPE= -github.com/AthenZ/athenz v1.11.59/go.mod h1:IiXKag9zpVJTs/bPcuVilt9S2Uzpz02NiRp/e+fth+4= +github.com/AthenZ/athenz v1.11.65 h1:LV8zGlszam5Jccza/JbLgWsK+HwI2W836IyCTlfEKuw= +github.com/AthenZ/athenz v1.11.65/go.mod h1:G3Wk2obA0hau0mbt8MTYT64mZg58i3jccrQMLLBCgXE= github.com/Azure/azure-amqp-common-go/v3 v3.2.1/go.mod h1:O6X1iYHP7s2x7NjUKsXVhkwWrQhxrd+d8/3rRadj4CI= github.com/Azure/azure-amqp-common-go/v3 v3.2.2/go.mod h1:O6X1iYHP7s2x7NjUKsXVhkwWrQhxrd+d8/3rRadj4CI= github.com/Azure/azure-pipeline-go v0.2.3 h1:7U9HBg1JFK3jHl5qmo4CTZKFTVgMwdFHMVtCdfBE21U= @@ -114,26 +114,26 @@ github.com/Azure/azure-sdk-for-go v59.3.0+incompatible/go.mod h1:9XXNKU+eRnpl9mo github.com/Azure/azure-sdk-for-go/sdk/azcore v0.19.0/go.mod h1:h6H6c8enJmmocHUbLiiGY6sx7f9i+X3m1CHdd5c6Rdw= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.0.0/go.mod h1:uGG2W01BaETf0Ozp+QxxKJdMBNRWPdstHG0Fmdwn1/U= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.6.0/go.mod h1:bjGvMhVMb+EEm3VRNQawDMUyMMjo+S5ewNjflkep/0Q= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 h1:E+OJmp2tPvt1W+amx48v1eqbjDYsgN+RzP4q16yV5eM= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1/go.mod h1:a6xsAQUZg+VsS3TJ05SRp524Hs4pZ/AeFSr5ENf0Yjo= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0 h1:nyQWyZvwGTvunIMxi1Y9uXkcyr+I7TeNrr/foo4Kpk8= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.14.0/go.mod h1:l38EPgmsp71HHLq9j7De57JcKOWPyhrsW1Awm1JS6K0= github.com/Azure/azure-sdk-for-go/sdk/azidentity v0.11.0/go.mod h1:HcM1YX14R7CJcghJGOYCgdezslRSVzqwLf/q+4Y2r/0= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.0.0/go.mod h1:+6sju8gk8FRmSajX3Oz4G5Gm7P+mbqE9FVaXXFYTkCM= github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.3.0/go.mod h1:OQeznEEkTZ9OrhHJoDD8ZDq51FHgXjqtP9z6bEwBq9U= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 h1:sO0/P7g68FrryJzljemN+6GTssUXdANk6aJ7T1ZxnsQ= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1/go.mod h1:h8hyGFDsU5HMivxiS2iYFZsgDbU9OnnJ163x5UGVKYo= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0 h1:tfLQ34V6F7tVSwoTf/4lH5sE0o6eCJuNDTmH09nDpbc= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg= github.com/Azure/azure-sdk-for-go/sdk/internal v0.7.0/go.mod h1:yqy467j36fJxcRV2TzfVZ1pCb5vxm4BtZPUdYWe/Xo8= github.com/Azure/azure-sdk-for-go/sdk/internal v1.0.0/go.mod h1:eWRD7oawr1Mu1sLCawqVc0CUiF43ia3qQMxLscsKQ9w= github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0/go.mod h1:okt5dMMTOFjX/aovMlrjvvXoPMBVSPzk9185BT0+eZM= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 h1:jBQA3cKT4L2rWMpgE7Yt3Hwh2aUj8KXjIGLxjHeYNNo= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0/go.mod h1:4OG6tQ9EOP/MT0NMjDlRzWoVFxfu9rN9B2X+tlSVktg= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0 h1:ywEEhmNahHBihViHepv3xPBn1663uRv2t2q/ESv9seY= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.10.0/go.mod h1:iZDifYGJTIgIIkYRNWPENUnqx6bJ2xnSDFI2tjwZNuY= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/internal v1.0.0/go.mod h1:ceIuwmxDWptoW3eCqSXlnPsZFKh4X+R38dWPv7GS9Vs= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/resources/armresources v1.0.0/go.mod h1:s1tW/At+xHqjNFvWU4G0c0Qv33KOhvbGNj0RCTQDV8s= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.2.0/go.mod h1:c+Lifp3EDEamAkPVzMooRNOK6CZjNSdEnf1A7jsI9u4= -github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.5.0 h1:AifHbc4mg0x9zW52WOpKbsHaDKuRhlI7TVl47thgQ70= -github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.5.0/go.mod h1:T5RfihdXtBDxt1Ch2wobif3TvzTdumDy29kahv6AV9A= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.6.0 h1:PiSrjRPpkQNjrM8H0WwKMnZUdu1RGMtd/LdGKUrOo+c= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/storage/armstorage v1.6.0/go.mod h1:oDrbWx4ewMylP7xHivfgixbfGBT6APAwsSoHRKotnIc= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0/go.mod h1:7QJP7dr2wznCMeqIrhMgWGf7XpAQnVrJqDm9nvV3Cu4= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2 h1:YUUxeiOWgdAQE3pXt2H7QXzZs0q8UBjgRbl56qo8GYM= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.2/go.mod h1:dmXQgZuiSubAecswZE+Sm8jkvEa7kQgTPVRvwL/nd0E= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.0 h1:Be6KInmFEKV81c0pOAEbRYehLMwmmGI1exuFj248AMk= +github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.4.0/go.mod h1:WCPBHsOXfBVnivScjs2ypRfimjEW0qPVLGgJkZlrIOA= github.com/Azure/azure-service-bus-go v0.11.5/go.mod h1:MI6ge2CuQWBVq+ly456MY7XqNLJip5LO1iSFodbNLbU= github.com/Azure/azure-storage-blob-go v0.14.0/go.mod h1:SMqIBi+SuiQH32bvyjngEewEeXoPfKMgWlBDaYf6fck= github.com/Azure/azure-storage-blob-go v0.15.0 h1:rXtgp8tN1p29GvpGgfJetavIG0V7OgcSXPpwp3tx6qk= @@ -166,8 +166,8 @@ github.com/Azure/go-autorest/tracing v0.6.0 h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUM github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= github.com/AzureAD/microsoft-authentication-library-for-go v0.4.0/go.mod h1:Vt9sXTKwMyGcOxSmLDMnGPgqsUg7m8pe215qMLrDXw4= github.com/AzureAD/microsoft-authentication-library-for-go v1.0.0/go.mod h1:kgDmCTgBzIEPFElEF+FK0SdjAor06dRq2Go927dnQ6o= -github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 h1:DzHpqpoJVaCgOUdVHxE8QB52S6NiVdDQvGlny1qvPqA= -github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/ClickHouse/ch-go v0.61.5 h1:zwR8QbYI0tsMiEcze/uIMK+Tz1D3XZXLdNrlaOpeEI4= @@ -237,81 +237,81 @@ github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v1.16.2/go.mod h1:ytwTPBG6fXTZLxxeeCCWj2/EMYp/xDUgX+OET6TLNNU= github.com/aws/aws-sdk-go-v2 v1.23.0/go.mod h1:i1XDttT4rnf6vxc9AuskLc6s7XBee8rlLilKlc03uAA= -github.com/aws/aws-sdk-go-v2 v1.30.4 h1:frhcagrVNrzmT95RJImMHgabt99vkXGslubDaDagTk8= -github.com/aws/aws-sdk-go-v2 v1.30.4/go.mod h1:CT+ZPWXbYrci8chcARI3OmI/qgd+f6WtuLOoaIA8PR0= +github.com/aws/aws-sdk-go-v2 v1.30.5 h1:mWSRTwQAb0aLE17dSzztCVJWI9+cRMgqebndjwDyK0g= +github.com/aws/aws-sdk-go-v2 v1.30.5/go.mod h1:CT+ZPWXbYrci8chcARI3OmI/qgd+f6WtuLOoaIA8PR0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.1/go.mod h1:n8Bs1ElDD2wJ9kCRTczA83gYbBmjSwZp3umc6zF4EeM= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.2 h1:x6xsQXGSmW6frevwDA+vi/wqhp1ct18mVXYN08/93to= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.2/go.mod h1:lPprDr1e6cJdyYeGXnRaJoP4Md+cDBvi2eOj00BlGmg= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.4 h1:70PVAiL15/aBMh5LThwgXdSQorVr91L127ttckI9QQU= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.4/go.mod h1:/MQxMqci8tlqDH+pjmoLu1i0tbWCUP1hhyMRuFxpQCw= github.com/aws/aws-sdk-go-v2/config v1.15.3/go.mod h1:9YL3v07Xc/ohTsxFXzan9ZpFpdTOFl4X65BAKYaz8jg= github.com/aws/aws-sdk-go-v2/config v1.25.3/go.mod h1:tAByZy03nH5jcq0vZmkcVoo6tRzRHEwSFx3QW4NmDw8= -github.com/aws/aws-sdk-go-v2/config v1.27.27 h1:HdqgGt1OAP0HkEDDShEl0oSYa9ZZBSOmKpdpsDMdO90= -github.com/aws/aws-sdk-go-v2/config v1.27.27/go.mod h1:MVYamCg76dFNINkZFu4n4RjDixhVr51HLj4ErWzrVwg= +github.com/aws/aws-sdk-go-v2/config v1.27.33 h1:Nof9o/MsmH4oa0s2q9a0k7tMz5x/Yj5k06lDODWz3BU= +github.com/aws/aws-sdk-go-v2/config v1.27.33/go.mod h1:kEqdYzRb8dd8Sy2pOdEbExTTF5v7ozEXX0McgPE7xks= github.com/aws/aws-sdk-go-v2/credentials v1.11.2/go.mod h1:j8YsY9TXTm31k4eFhspiQicfXPLZ0gYXA50i4gxPE8g= github.com/aws/aws-sdk-go-v2/credentials v1.16.2/go.mod h1:sDdvGhXrSVT5yzBDR7qXz+rhbpiMpUYfF3vJ01QSdrc= -github.com/aws/aws-sdk-go-v2/credentials v1.17.27 h1:2raNba6gr2IfA0eqqiP2XiQ0UVOpGPgDSi0I9iAP+UI= -github.com/aws/aws-sdk-go-v2/credentials v1.17.27/go.mod h1:gniiwbGahQByxan6YjQUMcW4Aov6bLC3m+evgcoN4r4= +github.com/aws/aws-sdk-go-v2/credentials v1.17.32 h1:7Cxhp/BnT2RcGy4VisJ9miUPecY+lyE9I8JvcZofn9I= +github.com/aws/aws-sdk-go-v2/credentials v1.17.32/go.mod h1:P5/QMF3/DCHbXGEGkdbilXHsyTBX5D3HSwcrSc9p20I= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.12.3/go.mod h1:uk1vhHHERfSVCUnqSqz8O48LBYDSC+k6brng09jcMOk= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.4/go.mod h1:t4i+yGHMCcUNIX1x7YVYa6bH/Do7civ5I6cG/6PMfyA= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 h1:KreluoV8FZDEtI6Co2xuNk/UqI9iwMrOx/87PBNIKqw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11/go.mod h1:SeSUYBLsMYFoRvHE0Tjvn7kbxaUhl75CJi1sbfhMxkU= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.13 h1:pfQ2sqNpMVK6xz2RbqLEL0GH87JOwSxPV2rzm8Zsb74= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.13/go.mod h1:NG7RXPUlqfsCLLFfi0+IpKN4sCB9D9fw/qTaSB+xRoU= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.3/go.mod h1:0dHuD2HZZSiwfJSy1FO5bX1hQ1TxVV1QXXjpn3XUE44= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.14.0/go.mod h1:UcgIwJ9KHquYxs6Q5skC9qXjhYMK+JASDYcXQ4X7JZE= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.16.21 h1:1v8Ii0MRVGYB/sdhkbxrtolCA7Tp+lGh+5OJTs5vmZ8= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.16.21/go.mod h1:cxdd1rc8yxCjKz28hi30XN1jDXr2DxZvD44vLxTz/bg= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.18 h1:9DIp7vhmOPmueCDwpXa45bEbLHHTt1kcxChdTJWWxvI= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.18/go.mod h1:aJv/Fwz8r56ozwYFRC4bzoeL1L17GYQYemfblOBux1M= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.9/go.mod h1:AnVH5pvai0pAF4lXRq0bmhbes1u9R8wTE+g+183bZNM= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.3/go.mod h1:7sGSz1JCKHWWBHq98m6sMtWQikmYPpxjqOydDemiVoM= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.16 h1:TNyt/+X43KJ9IJJMjKfa3bNTiZbUP7DeCxfbTROESwY= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.16/go.mod h1:2DwJF39FlNAUiX5pAc0UNeiz16lK2t7IaFcm0LFHEgc= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.17 h1:pI7Bzt0BJtYA0N/JEC6B8fJ4RBrEMi1LBrkMdFYNSnQ= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.17/go.mod h1:Dh5zzJYMtxfIjYW+/evjQ8uj2OyR/ve2KROHGHlSFqE= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.3/go.mod h1:ssOhaLpRlh88H3UmEcsBoVKq309quMvm3Ds8e9d4eJM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.3/go.mod h1:ify42Rb7nKeDDPkFjKn7q1bPscVPu/+gmHH8d2c+anU= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.16 h1:jYfy8UPmd+6kJW5YhY0L1/KftReOGxI/4NtVSTh9O/I= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.16/go.mod h1:7ZfEPZxkW42Afq4uQB8H2E2e6ebh6mXTueEpYzjCzcs= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.17 h1:Mqr/V5gvrhA2gvgnF42Zh5iMiQNcOYthFYwCyrnuWlc= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.17/go.mod h1:aLJpZlCmjE+V+KtN1q1uyZkfnUWpQGpbsn89XPKyzfU= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.10/go.mod h1:8DcYQcz0+ZJaSxANlHIsbbi6S+zMwjwdDqwW3r9AzaE= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= -github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= -github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= +github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1 h1:VaRN3TlFdd6KxX1x3ILT5ynH6HvKgqdiXoTxAF4HQcQ= +github.com/aws/aws-sdk-go-v2/internal/ini v1.8.1/go.mod h1:FbtygfRFze9usAadmnGJNc8KsP346kEe+y2/oyhGAGc= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.7 h1:/FUtT3xsoHO3cfh+I/kCbcMCN98QZRsiFet/V8QkWSs= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.7/go.mod h1:MaCAgWpGooQoCWZnMur97rGn5dp350w2+CeiV5406wE= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.17 h1:Roo69qTpfu8OlJ2Tb7pAYVuF0CpuUMB0IYWwYP/4DZM= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.17/go.mod h1:NcWPxQzGM1USQggaTVwz6VpqMZPX1CvDJLDh6jnOCa4= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.1/go.mod h1:GeUru+8VzrTXV/83XyMJ80KpH8xO89VPoUileyNQ+tc= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.1/go.mod h1:l9ymW25HOqymeU2m1gbUQ3rUIsTwKs8gYHXkqDQUhiI= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.4 h1:KypMCbLPPHEmf9DgMGw51jMj77VfGPAN2Kv4cfhlfgI= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.4/go.mod h1:Vz1JQXliGcQktFTN/LN6uGppAIRoLBR2bMvIMP0gOjc= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.3/go.mod h1:Seb8KNmD6kVTjwRjVEgOT5hPin6sq+v4C2ycJQDwuH8= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.9 h1:UXqEWQI0n+q0QixzU0yUUQBZXRd5037qdInTIHFTl98= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.9/go.mod h1:xP6Gq6fzGZT8w/ZN+XvGMZ2RU1LeEs7b2yUP5DN8NY4= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.19 h1:FLMkfEiRjhgeDTCjjLoc3URo/TBkgeQbocA78lfkzSI= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.3.19/go.mod h1:Vx+GucNSsdhaxs3aZIKfSUjKVGsxN25nX2SRcdhuw08= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.3/go.mod h1:wlY6SVjuwvh3TVRpTqdy4I1JpBFLX4UGeKZdWntaocw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.3/go.mod h1:Owv1I59vaghv1Ax8zz8ELY8DN7/Y0rGS+WWAmjgi950= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.18 h1:tJ5RnkHCiSH0jyd6gROjlJtNwov0eGYNz8s8nFcR0jQ= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.18/go.mod h1:++NHzT+nAF7ZPrHPsA+ENvsXkOO8wEu+C6RXltAG4/c= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.19 h1:rfprUlsdzgl7ZL2KlXiUAoJnI/VxfHCvDFr2QDFj6u4= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.19/go.mod h1:SCWkEdRq8/7EK60NcvvQ6NXKuTcchAD4ROAsC37VEZE= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.13.3/go.mod h1:Bm/v2IaN6rZ+Op7zX+bOUMdL4fsrYZiD0dsjLhNKwZc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.7 h1:uO5XR6QGBcmPyo2gxofYJLFkcVQ4izOoGDNenlZhTEk= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.7/go.mod h1:feeeAYfAcwTReM6vbwjEyDmiGho+YgBhaFULuXDW8kc= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.17 h1:u+EfGmksnJc/x5tq3A+OD7LrMbSSR/5TrKLvkdy/fhY= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.17.17/go.mod h1:VaMx6302JHax2vHJWgRo+5n9zvbacs3bLU/23DNQrTY= github.com/aws/aws-sdk-go-v2/service/kms v1.16.3/go.mod h1:QuiHPBqlOFCi4LqdSskYYAWpQlx3PKmohy+rE2F+o5g= -github.com/aws/aws-sdk-go-v2/service/redshiftdata v1.27.3 h1:rtX1ZHGPpqbQGZlPuN1u7nA+0zjq0DB7QTVNlYY/gfw= -github.com/aws/aws-sdk-go-v2/service/redshiftdata v1.27.3/go.mod h1:8Ah7aUFE9G0dppkn6ZXn1iExeHUV4369IJ2GRi7++Y0= +github.com/aws/aws-sdk-go-v2/service/redshiftdata v1.28.2 h1:oNarSIarQfMAZHeUhD2JOkdEpPfUFfoPKmb1GBK17Kc= +github.com/aws/aws-sdk-go-v2/service/redshiftdata v1.28.2/go.mod h1:9HLbgBikxAqW0V3Q8eQMQvoW1XRq0J7TjqYe8Lpiwx4= github.com/aws/aws-sdk-go-v2/service/s3 v1.26.3/go.mod h1:g1qvDuRsJY+XghsV6zg00Z4KJ7DtFFCx8fJD2a491Ak= github.com/aws/aws-sdk-go-v2/service/s3 v1.43.0/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= -github.com/aws/aws-sdk-go-v2/service/s3 v1.54.3 h1:57NtjG+WLims0TxIQbjTqebZUKDM03DfM11ANAekW0s= -github.com/aws/aws-sdk-go-v2/service/s3 v1.54.3/go.mod h1:739CllldowZiPPsDFcJHNF4FXrVxaSGVnZ9Ez9Iz9hc= +github.com/aws/aws-sdk-go-v2/service/s3 v1.61.2 h1:Kp6PWAlXwP1UvIflkIP6MFZYBNDCa4mFCGtxrpICVOg= +github.com/aws/aws-sdk-go-v2/service/s3 v1.61.2/go.mod h1:5FmD/Dqq57gP+XwaUnd5WFPipAuzrf0HmupX27Gvjvc= github.com/aws/aws-sdk-go-v2/service/secretsmanager v1.15.4/go.mod h1:PJc8s+lxyU8rrre0/4a0pn2wgwiDvOEzoOjcJUBr67o= github.com/aws/aws-sdk-go-v2/service/sns v1.17.4/go.mod h1:kElt+uCcXxcqFyc+bQqZPFD9DME/eC6oHBXvFzQ9Bcw= github.com/aws/aws-sdk-go-v2/service/sqs v1.18.3/go.mod h1:skmQo0UPvsjsuYYSYMVmrPc1HWCbHUJyrCEp+ZaLzqM= github.com/aws/aws-sdk-go-v2/service/ssm v1.24.1/go.mod h1:NR/xoKjdbRJ+qx0pMR4mI+N/H1I1ynHwXnO6FowXJc0= github.com/aws/aws-sdk-go-v2/service/sso v1.11.3/go.mod h1:7UQ/e69kU7LDPtY40OyoHYgRmgfGM4mgsLYtcObdveU= github.com/aws/aws-sdk-go-v2/service/sso v1.17.2/go.mod h1:/pE21vno3q1h4bbhUOEi+6Zu/aT26UK2WKkDXd+TssQ= -github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 h1:BXx0ZIxvrJdSgSvKTZ+yRBeSqqgPM89VPlulEcl37tM= -github.com/aws/aws-sdk-go-v2/service/sso v1.22.4/go.mod h1:ooyCOXjvJEsUw7x+ZDHeISPMhtwI3ZCB7ggFMcFfWLU= +github.com/aws/aws-sdk-go-v2/service/sso v1.22.7 h1:pIaGg+08llrP7Q5aiz9ICWbY8cqhTkyy+0SHvfzQpTc= +github.com/aws/aws-sdk-go-v2/service/sso v1.22.7/go.mod h1:eEygMHnTKH/3kNp9Jr1n3PdejuSNcgwLe1dWgQtO0VQ= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.20.0/go.mod h1:dWqm5G767qwKPuayKfzm4rjzFmVjiBFbOJrpSPnAMDs= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 h1:yiwVzJW2ZxZTurVbYWA7QOrAaCYQR72t0wrSBfoesUE= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4/go.mod h1:0oxfLkpz3rQ/CHlx5hB7H69YUpFiI1tql6Q6Ne+1bCw= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.7 h1:/Cfdu0XV3mONYKaOt1Gr0k1KvQzkzPyiKUdlWJqy+J4= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.7/go.mod h1:bCbAxKDqNvkHxRaIMnyVPXPo+OaPRwvmgzMxbz1VKSA= github.com/aws/aws-sdk-go-v2/service/sts v1.16.3/go.mod h1:bfBj0iVmsUyUg4weDB4NxktD9rDGeKSVWnjTnwbx9b8= github.com/aws/aws-sdk-go-v2/service/sts v1.25.3/go.mod h1:4EqRHDCKP78hq3zOnmFXu5k0j4bXbRFfCh/zQ6KnEfQ= -github.com/aws/aws-sdk-go-v2/service/sts v1.30.5 h1:OMsEmCyz2i89XwRwPouAJvhj81wINh+4UK+k/0Yo/q8= -github.com/aws/aws-sdk-go-v2/service/sts v1.30.5/go.mod h1:vmSqFK+BVIwVpDAGZB3CoCXHzurt4qBE8lf+I/kRTh0= +github.com/aws/aws-sdk-go-v2/service/sts v1.30.7 h1:NKTa1eqZYw8tiHSRGpP0VtTdub/8KNk8sDkNPFaOKDE= +github.com/aws/aws-sdk-go-v2/service/sts v1.30.7/go.mod h1:NXi1dIAGteSaRLqYgarlhP/Ij0cFT+qmCwiJqWh/U5o= github.com/aws/smithy-go v1.11.2/go.mod h1:3xHYmszWVx2c0kIwQeEVf9uSm4fYZt67FBJnwub1bgM= github.com/aws/smithy-go v1.17.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/aws/smithy-go v1.20.4 h1:2HK1zBdPgRbjFOHlfeQZfpC4r72MOb9bZkiFwggKO+4= @@ -323,8 +323,8 @@ github.com/bitfield/gotestdox v0.2.2 h1:x6RcPAbBbErKLnapz1QeAlf3ospg8efBsedU93CD github.com/bitfield/gotestdox v0.2.2/go.mod h1:D+gwtS0urjBrzguAkTM2wodsTQYFHdpx8eqRJ3N+9pY= github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932 h1:mXoPYz/Ul5HYEDvkta6I8/rnYM5gSdSV2tJ6XbZuEtY= github.com/bitly/go-hostpool v0.0.0-20171023180738-a3a6125de932/go.mod h1:NOuUCSz6Q9T7+igc/hlvDOUdtWKryOrtFyIVABv/p7k= -github.com/bits-and-blooms/bitset v1.13.0 h1:bAQ9OPNFYbGHV6Nez0tmNI0RiEu7/hxlYJRUA0wFAVE= -github.com/bits-and-blooms/bitset v1.13.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/bits-and-blooms/bitset v1.14.3 h1:Gd2c8lSNf9pKXom5JtD7AaKO8o7fGQ2LtFj1436qilA= +github.com/bits-and-blooms/bitset v1.14.3/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/bkaradzic/go-lz4 v1.0.0 h1:RXc4wYsyz985CkXXeX04y4VnZFGG8Rd43pRaHsOXAKk= github.com/bkaradzic/go-lz4 v1.0.0/go.mod h1:0YdlkowM3VswSROI7qDxhRvJ3sLhlFrRRwjwegp5jy4= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY= @@ -395,8 +395,8 @@ github.com/containerd/ttrpc v1.2.3/go.mod h1:ieWsXucbb8Mj9PH0rXCw1i8IunRbbAiDkpX github.com/containerd/typeurl v1.0.2 h1:Chlt8zIieDbzQFzXzAeBEF92KhExuE4p9p92/QmY7aY= github.com/containerd/typeurl/v2 v2.1.1 h1:3Q4Pt7i8nYwy2KmQWIw2+1hTvwTE/6w9FqcttATPO/4= github.com/containerd/typeurl/v2 v2.1.1/go.mod h1:IDp2JFvbwZ31H8dQbEIY7sDl2L3o3HZj1hsSQlywkQ0= -github.com/coreos/go-oidc/v3 v3.10.0 h1:tDnXHnLyiTVyT/2zLDGj09pFPkhND8Gl8lnTRhoEaJU= -github.com/coreos/go-oidc/v3 v3.10.0/go.mod h1:5j11xcw0D3+SGxn6Z/WFADsgcWVMyNAlSQupk0KK3ac= +github.com/coreos/go-oidc/v3 v3.11.0 h1:Ia3MxdwpSw702YW0xgfmP1GVCMA9aEFWu12XUZ3/OtI= +github.com/coreos/go-oidc/v3 v3.11.0/go.mod h1:gE3LgjOgFoHi9a4ce4/tJczr0Ai2/BoDhf0r5lltWI0= github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= @@ -411,8 +411,8 @@ github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46t github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= github.com/creack/pty v1.1.18/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= -github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= -github.com/danieljoos/wincred v1.2.1/go.mod h1:uGaFL9fDn3OLTvzCGulzE+SzjEe5NGlh5FdCcyfPwps= +github.com/danieljoos/wincred v1.2.2 h1:774zMFJrqaeYCK2W57BgAem/MLi6mtSE47MB6BOJ0i0= +github.com/danieljoos/wincred v1.2.2/go.mod h1:w7w4Utbrz8lqeMbDAK0lkNJUv5sAOkFi7nd/ogr0Uh8= github.com/databricks/databricks-sql-go v1.6.1 h1:SOAwVdw/N3AZ5ECJYI49SBUncNy61WzOpzlJFZ17O5g= github.com/databricks/databricks-sql-go v1.6.1/go.mod h1:/FB8hVRN/KGnWStEyz19r2r7TmfBsK8nUv6yMid//tU= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -431,8 +431,8 @@ github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WA github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= -github.com/dhui/dktest v0.4.1 h1:/w+IWuDXVymg3IrRJCHHOkMK10m9aNVMOyD0X12YVTg= -github.com/dhui/dktest v0.4.1/go.mod h1:DdOqcUpL7vgyP4GlF3X3w7HbSlz8cEQzwewPveYEQbA= +github.com/dhui/dktest v0.4.3 h1:wquqUxAFdcUgabAVLvSCOKOlag5cIZuaOjYIBOWdsR0= +github.com/dhui/dktest v0.4.3/go.mod h1:zNK8IwktWzQRm6I/l2Wjp7MakiyaFWv4G1hjmodmMTs= github.com/dimchansky/utfbom v1.1.0/go.mod h1:rO41eb7gLfo8SF1jd9F8HplJm1Fewwi4mQvIirEdv+8= github.com/dimchansky/utfbom v1.1.1/go.mod h1:SxdoEBH5qIqFocHMyGOXVAybYJdr71b1Q/j0mACtrfE= github.com/dimfeld/httptreemux v5.0.1+incompatible h1:Qj3gVcDNoOthBAqftuD596rm4wg/adLLz5xh5CmpiCA= @@ -447,14 +447,16 @@ github.com/dnephin/pflag v1.0.7 h1:oxONGlWxhmUct0YzKTgrpQv9AUA1wtPBn7zuSjJqptk= github.com/dnephin/pflag v1.0.7/go.mod h1:uxE91IoWURlOiTUIA8Mq5ZZkAv3dPUfZNaT80Zm7OQE= github.com/docker/buildx v0.14.0 h1:FxqcfE7xgeEC4oQlKLpuvfobRDVDXrHE3jByM+mdyqk= github.com/docker/buildx v0.14.0/go.mod h1:Vy/2lC9QsJvo33+7KKkN/GDE5WxnVqW0/dpcN7ZqPJY= -github.com/docker/cli v26.1.4+incompatible h1:I8PHdc0MtxEADqYJZvhBrW9bo8gawKwwenxRM7/rLu8= -github.com/docker/cli v26.1.4+incompatible/go.mod h1:JLrzqnKDaYBop7H2jaqPtU4hHvMKP+vjCwu2uszcLI8= +github.com/docker/cli v27.2.1+incompatible h1:U5BPtiD0viUzjGAjV1p0MGB8eVA3L3cbIrnyWmSJI70= +github.com/docker/cli v27.2.1+incompatible/go.mod h1:JLrzqnKDaYBop7H2jaqPtU4hHvMKP+vjCwu2uszcLI8= +github.com/docker/cli-docs-tool v0.8.0 h1:YcDWl7rQJC3lJ7WVZRwSs3bc9nka97QLWfyJQli8yJU= +github.com/docker/cli-docs-tool v0.8.0/go.mod h1:8TQQ3E7mOXoYUs811LiPdUnAhXrcVsBIrW21a5pUbdk= github.com/docker/compose/v2 v2.27.0 h1:FKyClQdErCxUZULC2zo6Jn5ve+epFPe/Y0HaxjmUzNg= github.com/docker/compose/v2 v2.27.0/go.mod h1:uaqwmY6haO8wXWHk+LAsqqDapX6boH4izRKqj/E7+Bo= github.com/docker/distribution v2.8.3+incompatible h1:AtKxIZ36LoNK51+Z6RpzLpddBirtxJnzDrHLEKxTAYk= github.com/docker/distribution v2.8.3+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= -github.com/docker/docker v27.2.0+incompatible h1:Rk9nIVdfH3+Vz4cyI/uhbINhEZ/oLmc+CBXmH6fbNk4= -github.com/docker/docker v27.2.0+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v27.2.1+incompatible h1:fQdiLfW7VLscyoeYEBz7/J8soYFDZV1u6VW6gJEjNMI= +github.com/docker/docker v27.2.1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/docker-credential-helpers v0.8.0 h1:YQFtbBQb4VrpoPxhFuzEBPQ9E16qz5SpHLS+uswaCp8= github.com/docker/docker-credential-helpers v0.8.0/go.mod h1:UGFXcuoQ5TxPiB54nHOZ32AWRqQdECoh/Mg0AlEYb40= github.com/docker/go v1.5.1-1.0.20160303222718-d30aec9fd63c h1:lzqkGL9b3znc+ZUgi7FlLnqjQhcXxkNM/quxIjBVMD0= @@ -509,9 +511,11 @@ github.com/fsouza/fake-gcs-server v1.49.3 h1:RPt94uYjWb+t19dlZg4PVRJFCvqf7px0YZD github.com/fsouza/fake-gcs-server v1.49.3/go.mod h1:WsE7OZKNd5WXgiry01oJO6mDvljOr+YLPR3VQtM2sDY= github.com/fvbommel/sortorder v1.0.2 h1:mV4o8B2hKboCdkJm+a7uX/SIpZob4JzUpc5GGnM45eo= github.com/fvbommel/sortorder v1.0.2/go.mod h1:uk88iVf1ovNn1iLfgUVU2F9o5eO30ui720w+kxuqRs0= +github.com/fxamacker/cbor/v2 v2.7.0 h1:iM5WgngdRBanHcxugY4JySA0nk1wZorNOpTgCMedv5E= +github.com/fxamacker/cbor/v2 v2.7.0/go.mod h1:pxXPTn3joSm21Gbwsv0w9OSA2y1HFR9qXEeXQVeNoDQ= github.com/gabriel-vasile/mimetype v1.4.3/go.mod h1:d8uq/6HKRL6CGdk+aubisF/M5GcPfT7nKyLpA0lbSSk= -github.com/gabriel-vasile/mimetype v1.4.4 h1:QjV6pZ7/XZ7ryI2KuyeEDE8wnh7fHP9YnQy+R0LnH8I= -github.com/gabriel-vasile/mimetype v1.4.4/go.mod h1:JwLei5XPtWdGiMFB5Pjle1oEeoSeEuJfJE+TtfvdB/s= +github.com/gabriel-vasile/mimetype v1.4.5 h1:J7wGKdGu33ocBOhGy0z653k/lFKLFDPJMG8Gql0kxn4= +github.com/gabriel-vasile/mimetype v1.4.5/go.mod h1:ibHel+/kbxn9x2407k1izTA1S81ku1z/DlgOW2QE0M4= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.10.0/go.mod h1:4PMNQiOhvDRa013RKVbsiNwoyezlm2rm0uX/T7kzp5Y= @@ -533,8 +537,8 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 github.com/go-ini/ini v1.25.4/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= github.com/go-ini/ini v1.67.0 h1:z6ZrTEZqSWOTyH2FlglNbNgARyHG8oLW9gMELqKr06A= github.com/go-ini/ini v1.67.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= -github.com/go-jose/go-jose/v4 v4.0.2 h1:R3l3kkBds16bO7ZFAEEcofK0MkrAJt3jlJznWZG0nvk= -github.com/go-jose/go-jose/v4 v4.0.2/go.mod h1:WVf9LFMHh/QVrmqrOfqun0C45tMe3RoiKJMPvgWwLfY= +github.com/go-jose/go-jose/v4 v4.0.4 h1:VsjPI33J0SB9vQM6PLmNjoHqMQNGPiZ0rHL7Ni7Q6/E= +github.com/go-jose/go-jose/v4 v4.0.4/go.mod h1:NKb5HO1EZccyMpiZNbdUw/14tiXNyUJh188dfnMCAfc= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= @@ -550,8 +554,8 @@ github.com/go-openapi/jsonpointer v0.19.6 h1:eCs3fxoIi3Wh6vtgmLTOjdhSpiqphQ+DaPn github.com/go-openapi/jsonpointer v0.19.6/go.mod h1:osyAmYz/mB/C3I+WsTTSgw1ONzaLJoLCyoi6/zppojs= github.com/go-openapi/jsonreference v0.20.2 h1:3sVjiK66+uXK/6oQ8xgcRKcFgQ5KXa2KvnJRumpMGbE= github.com/go-openapi/jsonreference v0.20.2/go.mod h1:Bl1zwGIM8/wsvqjsOQLJ/SH+En5Ap4rVB5KVcIDZG2k= -github.com/go-openapi/swag v0.22.3 h1:yMBqmnQ0gyZvEb/+KzuWZOXgllrXT4SADYbvDaXHv/g= -github.com/go-openapi/swag v0.22.3/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14= +github.com/go-openapi/swag v0.22.4 h1:QLMzNJnMGPRNDCbySlcj1x01tzU8/9LTTL9hZZZogBU= +github.com/go-openapi/swag v0.22.4/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14= github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= @@ -559,8 +563,8 @@ github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/Nu github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.20.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= -github.com/go-playground/validator/v10 v10.22.0 h1:k6HsTZ0sTnROkhS//R0O+55JgM8C4Bx7ia+JlgcnOao= -github.com/go-playground/validator/v10 v10.22.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= +github.com/go-playground/validator/v10 v10.22.1 h1:40JcKH+bBNGFczGuoBYgX4I6m/i27HYW8P9FDk5PbgA= +github.com/go-playground/validator/v10 v10.22.1/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= github.com/go-redis/redis v6.15.8+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA= github.com/go-redis/redis v6.15.9+incompatible h1:K0pv1D7EQUjfyoMql+r/jZqCLizCGKFlFgcHWWmHQjg= github.com/go-redis/redis v6.15.9+incompatible/go.mod h1:NAIEuMOZ/fxfXJIrKDQDz8wamY7mA7PouImQ2Jvg6kA= @@ -575,6 +579,8 @@ github.com/go-sql-driver/mysql v1.8.1/go.mod h1:wEBSXgmK//2ZFJyE+qWnIsVGmvmEKlqw github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-task/slim-sprig/v3 v3.0.0 h1:sUs3vkvUymDpBKi3qH1YSqBQk9+9D/8M2mN1vB6EwHI= github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZiAzKg9hl15HA8= +github.com/go-viper/mapstructure/v2 v2.1.0 h1:gHnMa2Y/pIxElCH2GlZZ1lZSsn6XMtufpGyP1XxdC/w= +github.com/go-viper/mapstructure/v2 v2.1.0/go.mod h1:oJDH3BJKyqBA2TXFhDsKDGDTlndYOZ6rGS0BRZIxGhM= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= @@ -605,8 +611,8 @@ github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOW github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= -github.com/golang-migrate/migrate/v4 v4.17.1 h1:4zQ6iqL6t6AiItphxJctQb3cFqWiSpMnX7wLTPnnYO4= -github.com/golang-migrate/migrate/v4 v4.17.1/go.mod h1:m8hinFyWBn0SA4QKHuKh175Pm9wjmxj3S2Mia7dbXzM= +github.com/golang-migrate/migrate/v4 v4.18.1 h1:JML/k+t4tpHCpQTCAD62Nu43NUFzHY4CV3uAuvHGC+Y= +github.com/golang-migrate/migrate/v4 v4.18.1/go.mod h1:HAX6m3sQgcdO81tdjn5exv20+3Kb13cmGli1hrD6hks= github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= github.com/golang-sql/civil v0.0.0-20220223132316-b832511892a9 h1:au07oEsX2xN0ktxqI+Sida1w446QrXBRJ0nee3SNZlA= github.com/golang-sql/civil v0.0.0-20220223132316-b832511892a9/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= @@ -712,8 +718,8 @@ github.com/google/pprof v0.0.0-20210506205249-923b5ab0fc1a/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20240827171923-fa2c70bbbfe5 h1:5iH8iuqE5apketRbSFBy+X1V0o+l+8NF1avt4HWl7cA= -github.com/google/pprof v0.0.0-20240827171923-fa2c70bbbfe5/go.mod h1:vavhavw2zAxS5dIdcRluK6cSGGPlZynqzFM8NdvU144= +github.com/google/pprof v0.0.0-20240910150728-a0b0bb1d4134 h1:c5FlPPgxOn7kJz3VoPLkQYQXGBS3EklQ4Zfi57uOuqQ= +github.com/google/pprof v0.0.0-20240910150728-a0b0bb1d4134/go.mod h1:vavhavw2zAxS5dIdcRluK6cSGGPlZynqzFM8NdvU144= github.com/google/renameio v0.1.0 h1:GOZbcHa3HfsPKPlmyPyN2KEohoMXOhdMbHrvbpl2QaA= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= @@ -730,8 +736,8 @@ github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/wire v0.5.0/go.mod h1:ngWDr9Qvq3yZA10YrxfyGELY/AFWGVpy9c1LTRi1EoU= -github.com/googleapis/enterprise-certificate-proxy v0.3.3 h1:QRje2j5GZimBzlbhGA2V2QlGNgL8G6e+wGo/+/2bWI0= -github.com/googleapis/enterprise-certificate-proxy v0.3.3/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= +github.com/googleapis/enterprise-certificate-proxy v0.3.4 h1:XYIDZApgAnrN1c855gTgghdIA6Stxb52D5RnLI1SLyw= +github.com/googleapis/enterprise-certificate-proxy v0.3.4/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= @@ -757,8 +763,8 @@ github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8 github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed h1:5upAirOpQc1Q53c0bnx2ufif5kANL7bfZWcc6VJWJd8= github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed/go.mod h1:tMWxXQ9wFIaZeTI9F+hmhFiGpFmhOHzyShyFUhRm0H4= -github.com/hamba/avro/v2 v2.24.0 h1:axTlaYDkcSY0dVekRSy8cdrsj5MG86WqosUQacKCids= -github.com/hamba/avro/v2 v2.24.0/go.mod h1:7vDfy/2+kYCE8WUHoj2et59GTv0ap7ptktMXu0QHePI= +github.com/hamba/avro/v2 v2.25.2 h1:28dqbOCB7wA/3+J1ZN4GQ40tzsFtbtItkTPWgl97el0= +github.com/hamba/avro/v2 v2.25.2/go.mod h1:I8glyswHnpED3Nlx2ZdUe+4LJnCOOyiCzLMno9i/Uu0= github.com/hanwen/go-fuse v1.0.0/go.mod h1:unqXarDXqzAk0rt98O2tVndEPIpUgLD9+rwFisZH3Ok= github.com/hanwen/go-fuse/v2 v2.1.0/go.mod h1:oRyA5eK+pvJyv5otpO/DgccS8y/RvYMaO00GgRLGryc= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= @@ -925,16 +931,16 @@ github.com/lib/pq v1.10.9 h1:YXG7RB+JIjhP29X+OtkiDnYaXQwpS4JEWq7dtCCRUEw= github.com/lib/pq v1.10.9/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= github.com/linkedin/goavro/v2 v2.13.0 h1:L8eI8GcuciwUkt41Ej62joSZS4kKaYIUdze+6for9NU= github.com/linkedin/goavro/v2 v2.13.0/go.mod h1:KXx+erlq+RPlGSPmLF7xGo6SAbh8sCQ53x064+ioxhk= -github.com/lufia/plan9stats v0.0.0-20240513124658-fba389f38bae h1:dIZY4ULFcto4tAFlj1FYZl8ztUZ13bdq+PLY+NOfbyI= -github.com/lufia/plan9stats v0.0.0-20240513124658-fba389f38bae/go.mod h1:ilwx/Dta8jXAgpFYFvSWEMwxmbWXyiUHkd5FwyKhb5k= +github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 h1:7UMa6KCCMjZEMDtTVdcGu0B1GmmC7QJKiCCjyTAWQy0= +github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683/go.mod h1:ilwx/Dta8jXAgpFYFvSWEMwxmbWXyiUHkd5FwyKhb5k= github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0= github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= github.com/manifoldco/promptui v0.9.0 h1:3V4HzJk1TtXW1MTZMP7mdlwbBpIinw3HztaIlYthEiA= github.com/manifoldco/promptui v0.9.0/go.mod h1:ka04sppxSGFAtxX0qhlYQjISsg9mR4GWtQEhdbn6Pgg= -github.com/marcboeker/go-duckdb v1.7.1 h1:m9/nKfP7cG9AptcQ95R1vfacRuhtrZE5pZF8BPUb/Iw= -github.com/marcboeker/go-duckdb v1.7.1/go.mod h1:2oV8BZv88S16TKGKM+Lwd0g7DX84x0jMxjTInThC8Is= +github.com/marcboeker/go-duckdb v1.8.0 h1:iOWv1wTL0JIMqpyns6hCf5XJJI4fY6lmJNk+itx5RRo= +github.com/marcboeker/go-duckdb v1.8.0/go.mod h1:2oV8BZv88S16TKGKM+Lwd0g7DX84x0jMxjTInThC8Is= github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ= github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= @@ -953,8 +959,8 @@ github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.10/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= -github.com/mattn/go-runewidth v0.0.15 h1:UNAjwbU9l54TA3KzvqLGxwWjHmMgBUVhBiTjelZgg3U= -github.com/mattn/go-runewidth v0.0.15/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= +github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mattn/go-shellwords v1.0.12 h1:M2zGm7EW6UQJvDeQxo4T51eKPurbeFbe8WtebGE2xrk= github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= github.com/mattn/go-sqlite3 v1.9.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= @@ -991,8 +997,8 @@ github.com/moby/locker v1.0.1 h1:fOXqR41zeveg4fFODix+1Ch4mj/gT0NE1XJbp/epuBg= github.com/moby/locker v1.0.1/go.mod h1:S7SDdo5zpBK84bzzVlKr2V0hz+7x9hWbYC/kq7oQppc= github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= github.com/moby/patternmatcher v0.6.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= -github.com/moby/spdystream v0.2.0 h1:cjW1zVyyoiM0T7b6UoySUFqzXMoqRckQtXwGPiBhOM8= -github.com/moby/spdystream v0.2.0/go.mod h1:f7i0iNDQJ059oMTcWxx8MA/zKFIuD/lY+0GqbN2Wy8c= +github.com/moby/spdystream v0.4.0 h1:Vy79D6mHeJJjiPdFEL2yku1kl0chZpJfZcPpb16BRl8= +github.com/moby/spdystream v0.4.0/go.mod h1:xBAYlnt/ay+11ShkdFKNAG7LsyK/tmNBVvVOwrfMgdI= github.com/moby/sys/mountinfo v0.7.1 h1:/tTvQaSJRr2FshkhXiIpux6fQ2Zvc4j7tAhMTStAG2g= github.com/moby/sys/mountinfo v0.7.1/go.mod h1:IJb6JQeOklcdMU9F5xQ8ZALD+CUr5VlGpwtX+VE0rpI= github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5lXtc= @@ -1046,8 +1052,8 @@ github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8 github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= github.com/opencontainers/image-spec v1.1.0 h1:8SG7/vwALn54lVB/0yZ/MMwhFrPYtpEHQb2IpWsCzug= github.com/opencontainers/image-spec v1.1.0/go.mod h1:W4s4sFTMaBeK1BQLXbG4AdM2szdn85PY75RI83NrTrM= -github.com/opencontainers/runc v1.1.12 h1:BOIssBaW1La0/qbNZHXOOa71dZfZEQOzW7dqQf3phss= -github.com/opencontainers/runc v1.1.12/go.mod h1:S+lQwSfncpBha7XTy/5lBwWgm5+y5Ma/O44Ekby9FK8= +github.com/opencontainers/runc v1.1.14 h1:rgSuzbmgz5DUJjeSnw337TxDbRuqjs6iqQck/2weR6w= +github.com/opencontainers/runc v1.1.14/go.mod h1:E4C2z+7BxR7GHXp0hAY53mek+x49X1LjPNeMTfRGvOA= github.com/openzipkin/zipkin-go v0.4.3 h1:9EGwpqkgnwdEIJ+Od7QVSEIH+ocmm5nPat0G7sjsSdg= github.com/openzipkin/zipkin-go v0.4.3/go.mod h1:M9wCJZFWCo2RiY+o1eBCEMe0Dp2S5LDHcMZmk3RmK7c= github.com/ory/dockertest/v3 v3.11.0 h1:OiHcxKAvSDUwsEVh2BjxQQc/5EHz9n0va9awCtNGuyA= @@ -1059,8 +1065,9 @@ github.com/paulmach/orb v0.11.1/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/En github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pelletier/go-toml v1.9.5 h1:4yBQzkHv+7BHq2PQUZF3Mx0IYxG7LsP222s7Agd3ve8= github.com/pelletier/go-toml v1.9.5/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= -github.com/pelletier/go-toml/v2 v2.2.2 h1:aYUidT7k73Pcl9nb2gScu7NSrKCSHIDE89b3+6Wq+LM= github.com/pelletier/go-toml/v2 v2.2.2/go.mod h1:1t835xjRzz80PqgE6HHgN2JOsmgYu/h4qDAS4n929Rs= +github.com/pelletier/go-toml/v2 v2.2.3 h1:YmeHyLY8mFWbdkNWwpr+qIL2bEqT0o95WSdkNHvL12M= +github.com/pelletier/go-toml/v2 v2.2.3/go.mod h1:MfCQTFTvCcUyyvvwm1+G6H/jORL20Xlb6rzQu9GuUkc= github.com/phayes/freeport v0.0.0-20220201140144-74d24b5ae9f5 h1:Ii+DKncOVM8Cu1Hc+ETb5K+23HdAMvESYE3ZJ5b5cMI= github.com/phayes/freeport v0.0.0-20220201140144-74d24b5ae9f5/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY= @@ -1092,13 +1099,13 @@ github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 h1:o4JXh1EVt github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= -github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= -github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.3 h1:oPksm4K8B+Vt35tUhw6GbSNSgVlVSBH0qELP/7u83l4= +github.com/prometheus/client_golang v1.20.3/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= -github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= -github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/common v0.59.1 h1:LXb1quJHWm1P6wq/U824uxYi4Sg0oGvNeUm1z5dJoX0= +github.com/prometheus/common v0.59.1/go.mod h1:GpWM7dewqmVYcd7SmRaiWVe9SSqjf0UrwnYnpEZNuT0= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/r3labs/sse v0.0.0-20210224172625-26fe804710bc h1:zAsgcP8MhzAbhMnB1QQ2O7ZhWYVGYSR2iVcjzQuPV+o= @@ -1134,14 +1141,14 @@ github.com/rudderlabs/compose-test v0.1.3 h1:uyep6jDCIF737sfv4zIaMsKRQKX95IDz5Xb github.com/rudderlabs/compose-test v0.1.3/go.mod h1:tuvS1eQdSfwOYv1qwyVAcpdJxPLQXJgy5xGDd/9XmMg= github.com/rudderlabs/parquet-go v0.0.2 h1:ZXRdZdimB0PdJtmxeSSxfI0fDQ3kZjwzBxRi6Ut1J8k= github.com/rudderlabs/parquet-go v0.0.2/go.mod h1:g6guum7o8uhj/uNhunnt7bw5Vabu/goI5i21/3fnxWQ= -github.com/rudderlabs/rudder-go-kit v0.39.2 h1:Sisiyeawi43XQOo++5BkcE1+X5XcmCpq1VBwX5EiiiM= -github.com/rudderlabs/rudder-go-kit v0.39.2/go.mod h1:GtOYIFfVvNcXabgGytoGdsjdpKTH6PipFIom0bY94WQ= +github.com/rudderlabs/rudder-go-kit v0.41.1 h1:rImVvHd1hnO4ht1eoVQ+Dgro2dJwuEYIqbH4gS6bBF8= +github.com/rudderlabs/rudder-go-kit v0.41.1/go.mod h1:+58f3uAIc3/6X3lGhv6G1yF8GmoK31l1sPw4eh30tpA= github.com/rudderlabs/rudder-observability-kit v0.0.3 h1:vZtuZRkGX+6rjaeKtxxFE2YYP6QlmAcVcgecTOjvz+Q= github.com/rudderlabs/rudder-observability-kit v0.0.3/go.mod h1:6UjAh3H6rkE0fFLh7z8ZGQEQbKtUkRfhWOf/OUhfqW8= github.com/rudderlabs/rudder-schemas v0.5.1 h1:g4I5wp2yA6ZWQZ1MjSNn4zby3XctG/TOgbYUW3dS4z4= github.com/rudderlabs/rudder-schemas v0.5.1/go.mod h1:JoDTB9nCDXwRz+G+aYwP3Fj42HLssKARxsFFm+qqgb4= -github.com/rudderlabs/rudder-transformer/go v0.0.0-20240906042448-f7783d8fb300 h1:SmOLUqSCCcYs8QXYdZlHXCSCw77xhQ6qjNBsSA3bDKI= -github.com/rudderlabs/rudder-transformer/go v0.0.0-20240906042448-f7783d8fb300/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= +github.com/rudderlabs/rudder-transformer/go v0.0.0-20240910055720-f77d2ab4125a h1:OZcvpApxEYNkB9UNXrKDUBufQ24Lsr2Cs0pw70tzXBw= +github.com/rudderlabs/rudder-transformer/go v0.0.0-20240910055720-f77d2ab4125a/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= github.com/rudderlabs/sql-tunnels v0.1.7 h1:wDCRl6zY4M5gfWazf7XkSTGQS3yjBzUiUgEMBIfHNDA= github.com/rudderlabs/sql-tunnels v0.1.7/go.mod h1:5f7+YL49JHYgteP4rAgqKnr4K2OadB0oIpUS+Tt3sPM= github.com/rudderlabs/sqlconnect-go v1.9.0 h1:icLgqvVQ15Vh+oP7epA0b0yK6sIzxRVwPlRzOoDNVRA= @@ -1149,8 +1156,8 @@ github.com/rudderlabs/sqlconnect-go v1.9.0/go.mod h1:pKhrcp7ewf2FeEObiyLWTi8Ra/I github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= -github.com/sagikazarmark/locafero v0.4.0 h1:HApY1R9zGo4DBgr7dqsTH/JJxLTTsOt7u6keLGt6kNQ= -github.com/sagikazarmark/locafero v0.4.0/go.mod h1:Pe1W6UlPYUk/+wc/6KFhbORCfqzgYEpgQ3O5fPuL3H4= +github.com/sagikazarmark/locafero v0.6.0 h1:ON7AQg37yzcRPU69mt7gwhFEBwxI6P9T4Qu3N51bwOk= +github.com/sagikazarmark/locafero v0.6.0/go.mod h1:77OmuIc6VTraTXKXIs/uvUxKGUXjE1GbemJYHqdNjX0= github.com/sagikazarmark/slog-shim v0.1.0 h1:diDBnUNK9N/354PgrxMywXnAwEr1QZcOr6gto+ugjYE= github.com/sagikazarmark/slog-shim v0.1.0/go.mod h1:SrcSrq8aKtyuqEI1uvTDTK1arOWRIczQRv+GVI1AkeQ= github.com/samber/lo v1.47.0 h1:z7RynLwP5nbyRscyvcD043DWYoOcYRv3mV8lBeqOCLc= @@ -1275,6 +1282,8 @@ github.com/urfave/cli/v2 v2.27.4 h1:o1owoI+02Eb+K107p27wEX9Bb8eqIoZCfLXloLUSWJ8= github.com/urfave/cli/v2 v2.27.4/go.mod h1:m4QzxcD2qpra4z7WhzEGn74WZLViBnMpb1ToCAKdGRQ= github.com/viney-shih/go-lock v1.1.2 h1:3TdGTiHZCPqBdTvFbQZQN/TRZzKF3KWw2rFEyKz3YqA= github.com/viney-shih/go-lock v1.1.2/go.mod h1:Yijm78Ljteb3kRiJrbLAxVntkUukGu5uzSxq/xV7OO8= +github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= +github.com/x448/float16 v0.8.4/go.mod h1:14CWIYCyZA/cWjXOioeEpHeN/83MdbZDRQHoFcYsOfg= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= @@ -1295,8 +1304,8 @@ github.com/xrash/smetrics v0.0.0-20240521201337-686a1a2994c1 h1:gEOO8jv9F4OT7lGC github.com/xrash/smetrics v0.0.0-20240521201337-686a1a2994c1/go.mod h1:Ohn+xnUBiLI6FVj/9LpzZWtj1/D6lUovWYBkxHVV3aM= github.com/xtgo/uuid v0.0.0-20140804021211-a0b114877d4c h1:3lbZUMbMiGUW/LMkfsEABsc5zNT9+b1CvsJx47JzJ8g= github.com/xtgo/uuid v0.0.0-20140804021211-a0b114877d4c/go.mod h1:UrdRz5enIKZ63MEE3IF9l2/ebyx59GyGgPi+tICQdmM= -github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76 h1:tBiBTKHnIjovYoLX/TPkcf+OjqqKGQrPtGT3Foz+Pgo= -github.com/youmark/pkcs8 v0.0.0-20240424034433-3c2c7870ae76/go.mod h1:SQliXeA7Dhkt//vS29v3zpbEwoa+zb2Cn5xj5uO4K5U= +github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 h1:ilQV1hzziu+LLM3zUTJ0trRztfwgjqKnBWNtSRkbmwM= +github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78/go.mod h1:aL8wCCfTfSfmXjznFBSZNN13rSJjlIOI1fUNAtF7rmI= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -1312,14 +1321,14 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= github.com/zenizh/go-capturer v0.0.0-20211219060012-52ea6c8fed04 h1:qXafrlZL1WsJW5OokjraLLRURHiw0OzKHD/RNdspp4w= github.com/zenizh/go-capturer v0.0.0-20211219060012-52ea6c8fed04/go.mod h1:FiwNQxz6hGoNFBC4nIx+CxZhI3nne5RmIOlT/MXcSD4= -go.einride.tech/aip v0.67.1 h1:d/4TW92OxXBngkSOwWS2CH5rez869KpKMaN44mdxkFI= -go.einride.tech/aip v0.67.1/go.mod h1:ZGX4/zKw8dcgzdLsrvpOOGxfxI2QSk12SlP7d6c0/XI= -go.etcd.io/etcd/api/v3 v3.5.15 h1:3KpLJir1ZEBrYuV2v+Twaa/e2MdDCEZ/70H+lzEiwsk= -go.etcd.io/etcd/api/v3 v3.5.15/go.mod h1:N9EhGzXq58WuMllgH9ZvnEr7SI9pS0k0+DHZezGp7jM= -go.etcd.io/etcd/client/pkg/v3 v3.5.15 h1:fo0HpWz/KlHGMCC+YejpiCmyWDEuIpnTDzpJLB5fWlA= -go.etcd.io/etcd/client/pkg/v3 v3.5.15/go.mod h1:mXDI4NAOwEiszrHCb0aqfAYNCrZP4e9hRca3d1YK8EU= -go.etcd.io/etcd/client/v3 v3.5.15 h1:23M0eY4Fd/inNv1ZfU3AxrbbOdW79r9V9Rl62Nm6ip4= -go.etcd.io/etcd/client/v3 v3.5.15/go.mod h1:CLSJxrYjvLtHsrPKsy7LmZEE+DK2ktfd2bN4RhBMwlU= +go.einride.tech/aip v0.68.0 h1:4seM66oLzTpz50u4K1zlJyOXQ3tCzcJN7I22tKkjipw= +go.einride.tech/aip v0.68.0/go.mod h1:7y9FF8VtPWqpxuAxl0KQWqaULxW4zFIesD6zF5RIHHg= +go.etcd.io/etcd/api/v3 v3.5.16 h1:WvmyJVbjWqK4R1E+B12RRHz3bRGy9XVfh++MgbN+6n0= +go.etcd.io/etcd/api/v3 v3.5.16/go.mod h1:1P4SlIP/VwkDmGo3OlOD7faPeP8KDIFhqvciH5EfN28= +go.etcd.io/etcd/client/pkg/v3 v3.5.16 h1:ZgY48uH6UvB+/7R9Yf4x574uCO3jIx0TRDyetSfId3Q= +go.etcd.io/etcd/client/pkg/v3 v3.5.16/go.mod h1:V8acl8pcEK0Y2g19YlOV9m9ssUe6MgiDSobSoaBAM0E= +go.etcd.io/etcd/client/v3 v3.5.16 h1:sSmVYOAHeC9doqi0gv7v86oY/BTld0SEFGaxsU9eRhE= +go.etcd.io/etcd/client/v3 v3.5.16/go.mod h1:X+rExSGkyqxvu276cr2OwPLBaeqFu1cIl4vmRjAD/50= go.opencensus.io v0.15.0/go.mod h1:UffZAU+4sDEINUGP/B7UfBBkq4fqLu9zXAX7ke6CHW0= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= @@ -1416,8 +1425,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56 h1:2dVuKD2vS7b0QIHQbpyTISPd0LeHDbnYEryqj5Q1ug8= -golang.org/x/exp v0.0.0-20240719175910-8a7402abbf56/go.mod h1:M4RDyNAINzryxdtnbRXRL/OHtkFuWGRjvuhBJpk2IlY= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 h1:e66Fs6Z+fZTbFBAxKfP3PALWBtpfqks2bwGcexMxgtk= +golang.org/x/exp v0.0.0-20240909161429-701f63a606c0/go.mod h1:2TbTHSBQa924w8M6Xs1QcRcFwyucIwBGpK1p2f1YFFY= golang.org/x/image v0.16.0/go.mod h1:ugSZItdV4nOxyqp56HmXwH0Ry0nBCpjnZdpDaIHdoPs= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -1450,8 +1459,8 @@ golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.16.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= -golang.org/x/mod v0.20.0 h1:utOm6MM3R3dnawAiJgn0y+xvuYRsm1RKM/4giyfDgV0= -golang.org/x/mod v0.20.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= +golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1581,13 +1590,13 @@ golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= -golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= -golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= +golang.org/x/term v0.24.0 h1:Mh5cbb+Zk2hqqXNO7S1iTjEphVL+jb8ZWaqh/g+JWkM= +golang.org/x/term v0.24.0/go.mod h1:lOBK/LVxemqiMij05LGJ0tzNr8xlmwBRJ81PX6wVLH8= golang.org/x/text v0.15.0 h1:h1V/4gjBv8v9cjcR6+AR5+/cIYK5N/WAgiv4xlsEtAk= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1668,16 +1677,16 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.11.0/go.mod h1:anzJrxPjNtfgiYQYirP2CPGzGLxrH2u2QBhn6Bf3qY8= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.19.0/go.mod h1:qoJWxmGSIBmAeriMx19ogtrEPrGtDbPK634QFIcLAhc= -golang.org/x/tools v0.24.0 h1:J1shsA93PJUEVaUSaay7UXAyE8aimq3GW0pjlolpa24= -golang.org/x/tools v0.24.0/go.mod h1:YhNqVBIfWHdzvTLs0d8LCuMhkKUgSUKldakyV7W/WDQ= +golang.org/x/tools v0.25.0 h1:oFU9pkj/iJgs+0DT+VMHrx+oBKs/LJMV+Uvg78sl+fE= +golang.org/x/tools v0.25.0/go.mod h1:/vtpO8WL1N9cQC3FN5zPqb//fRXskFHbLKk4OW1Q7rg= golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190513163551-3ee3066db522/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= -golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= +golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da h1:noIWHXmPHxILtqtCOPIhSt0ABwskkZKjD3bXGnZGpNY= +golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/gonum v0.9.3/go.mod h1:TZumC3NeyVQskjXqmyWt4S3bINhy7B4eYwW69EbyX+0= @@ -1827,8 +1836,8 @@ google.golang.org/genproto v0.0.0-20220324131243-acbaeb5b85eb/go.mod h1:hAL49I2I google.golang.org/genproto v0.0.0-20220401170504-314d38edb7de/go.mod h1:8w6bsBMX6yCPbAVTeqQHvzxW0EIFigd5lZyahWgyfDo= google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1 h1:BulPr26Jqjnd4eYDVe+YvyR7Yc2vJGkO5/0UxD0/jZU= google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:hL97c3SYopEHblzpxRL4lSs523++l8DYxGM1FQiYmb4= -google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= -google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/api v0.0.0-20240903143218-8af14fe29dc1 h1:hjSy6tcFQZ171igDaN5QHOw2n6vx40juYbC/x67CEhc= +google.golang.org/genproto/googleapis/api v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:qpvKtACPCQhAdu3PyQgV4l3LMXZEtft7y8QcarRsp9I= google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 h1:pPJltXNxVzT4pK9yD8vR9X75DaWYYmLGMsEvBfFQZzQ= google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= @@ -1860,8 +1869,8 @@ google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9K google.golang.org/grpc v1.40.1/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= google.golang.org/grpc v1.44.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= +google.golang.org/grpc v1.66.1 h1:hO5qAXR19+/Z44hmvIM4dQFMSYX9XcWsByfoxutBpAM= +google.golang.org/grpc v1.66.1/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1928,20 +1937,20 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -k8s.io/api v0.30.1 h1:kCm/6mADMdbAxmIh0LBjS54nQBE+U4KmbCfIkF5CpJY= -k8s.io/api v0.30.1/go.mod h1:ddbN2C0+0DIiPntan/bye3SW3PdwLa11/0yqwvuRrJM= -k8s.io/apimachinery v0.30.1 h1:ZQStsEfo4n65yAdlGTfP/uSHMQSoYzU/oeEbkmF7P2U= -k8s.io/apimachinery v0.30.1/go.mod h1:iexa2somDaxdnj7bha06bhb43Zpa6eWH8N8dbqVjTUc= +k8s.io/api v0.31.0 h1:b9LiSjR2ym/SzTOlfMHm1tr7/21aD7fSkqgD/CVJBCo= +k8s.io/api v0.31.0/go.mod h1:0YiFF+JfFxMM6+1hQei8FY8M7s1Mth+z/q7eF1aJkTE= +k8s.io/apimachinery v0.31.0 h1:m9jOiSr3FoSSL5WO9bjm1n6B9KROYYgNZOb4tyZ1lBc= +k8s.io/apimachinery v0.31.0/go.mod h1:rsPdaZJfTfLsNJSQzNHQvYoTmxhoOEofxtOsF3rtsMo= k8s.io/apiserver v0.29.2 h1:+Z9S0dSNr+CjnVXQePG8TcBWHr3Q7BmAr7NraHvsMiQ= k8s.io/apiserver v0.29.2/go.mod h1:B0LieKVoyU7ykQvPFm7XSdIHaCHSzCzQWPFa5bqbeMQ= -k8s.io/client-go v0.30.1 h1:uC/Ir6A3R46wdkgCV3vbLyNOYyCJ8oZnjtJGKfytl/Q= -k8s.io/client-go v0.30.1/go.mod h1:wrAqLNs2trwiCH/wxxmT/x3hKVH9PuV0GGW0oDoHVqc= -k8s.io/klog/v2 v2.120.1 h1:QXU6cPEOIslTGvZaXvFWiP9VKyeet3sawzTOvdXb4Vw= -k8s.io/klog/v2 v2.120.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= +k8s.io/client-go v0.31.0 h1:QqEJzNjbN2Yv1H79SsS+SWnXkBgVu4Pj3CJQgbx0gI8= +k8s.io/client-go v0.31.0/go.mod h1:Y9wvC76g4fLjmU0BA+rV+h2cncoadjvjjkkIGoTLcGU= +k8s.io/klog/v2 v2.130.1 h1:n9Xl7H1Xvksem4KFG4PYbdQCQxqc/tTUyrgXaOhHSzk= +k8s.io/klog/v2 v2.130.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340 h1:BZqlfIlq5YbRMFko6/PM7FjZpUb45WallggurYhKGag= k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340/go.mod h1:yD4MZYeKMBwQKVht279WycxKyM84kkAx2DPrTXaeb98= -k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0 h1:jgGTlFYnhF1PM1Ax/lAlxUPE+KfCIXHaathvJg1C3ak= -k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= +k8s.io/utils v0.0.0-20240902221715-702e33fdd3c3 h1:b2FmK8YH+QEwq/Sy2uAEhmqL5nPfGYbJOcaqjeYYZoA= +k8s.io/utils v0.0.0-20240902221715-702e33fdd3c3/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= nhooyr.io/websocket v1.8.7/go.mod h1:B70DZP8IakI65RVQ51MsWP/8jndNma26DVA/nFSCgW0= nullprogram.com/x/optparse v1.0.0/go.mod h1:KdyPE+Igbe0jQUrVfMqDMeJQIJZEuyV7pjYmp6pbG50= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= @@ -1952,7 +1961,8 @@ sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMm sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= sigs.k8s.io/structured-merge-diff/v4 v4.4.1 h1:150L+0vs/8DA78h1u02ooW1/fFq/Lwr+sGiqlzvrtq4= sigs.k8s.io/structured-merge-diff/v4 v4.4.1/go.mod h1:N8hJocpFajUSSeSJ9bOZ77VzejKZaXsTtZo4/u7Io08= -sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= +sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E= +sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY= tags.cncf.io/container-device-interface v0.6.2 h1:dThE6dtp/93ZDGhqaED2Pu374SOeUkBfuvkLuiTdwzg= tags.cncf.io/container-device-interface v0.6.2/go.mod h1:Shusyhjs1A5Na/kqPVLL0KqnHQHuunol9LFeUNkuGVE= diff --git a/integration_test/backendconfigunavailability/backend_config_unavailability_test.go b/integration_test/backendconfigunavailability/backend_config_unavailability_test.go index 68f22e5809..57fff40ce6 100644 --- a/integration_test/backendconfigunavailability/backend_config_unavailability_test.go +++ b/integration_test/backendconfigunavailability/backend_config_unavailability_test.go @@ -35,6 +35,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/testhelper/docker/resource/postgres" "github.com/rudderlabs/rudder-go-kit/testhelper/docker/resource/transformer" "github.com/rudderlabs/rudder-go-kit/testhelper/rand" + "github.com/rudderlabs/rudder-server/runner" ) @@ -141,7 +142,7 @@ func setup(t testing.TB) testConfig { }`, }) // initialise un-started config BE server, to be started later - unStartedTransformerConfigBEServer := httptest.NewUnStartedServer(transformerBEConfigHandler) + unStartedTransformerConfigBEServer := httptest.NewUnstartedServer(transformerBEConfigHandler) _, transformerConfigBEPort, err := net.SplitHostPort(unStartedTransformerConfigBEServer.Listener.Addr().String()) require.NoError(t, err) transformerConfigBEServerUrl := fmt.Sprintf("http://%s:%s", "localhost", transformerConfigBEPort) diff --git a/suppression-backup-service/Dockerfile b/suppression-backup-service/Dockerfile index 47a942816c..f2d7104ac7 100644 --- a/suppression-backup-service/Dockerfile +++ b/suppression-backup-service/Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1 # GO_VERSION is updated automatically to match go.mod, see Makefile -ARG GO_VERSION=1.23.0 +ARG GO_VERSION=1.23.1 ARG ALPINE_VERSION=3.20 FROM golang:${GO_VERSION}-alpine${ALPINE_VERSION} AS builder RUN mkdir /app From f6271a0cad08e55c5ac28a19cd2e034a4024ed7d Mon Sep 17 00:00:00 2001 From: Akash Chetty Date: Wed, 11 Sep 2024 20:11:49 +0530 Subject: [PATCH 22/29] chore: passing dependency for asyncdestinationmanager (#5080) --- .../bing-ads/audience/bingads_test.go | 31 +++++++------ .../bing-ads/audience/bulk_uploader.go | 14 +++--- .../bing-ads/audience/manager.go | 21 ++++++--- .../bing-ads/audience/types.go | 4 +- .../bing-ads/audience/util.go | 3 +- .../offline-conversions/bingads_test.go | 27 +++++++----- .../offline-conversions/bulk_uploader.go | 14 +++--- .../bing-ads/offline-conversions/manager.go | 16 ++++--- .../bing-ads/offline-conversions/types.go | 4 +- .../eloqua/bulk_uploader.go | 11 ++--- .../eloqua/eloqua_test.go | 27 +++++++----- .../asyncdestinationmanager/eloqua/manager.go | 12 ++--- .../asyncdestinationmanager/eloqua/types.go | 2 + .../klaviyobulkupload/klaviyobulkupload.go | 14 +++--- .../klaviyobulkupload_test.go | 6 ++- .../klaviyobulkupload/types.go | 3 ++ .../lytics_bulk_upload/lyticsBulkUpload.go | 7 +-- .../lytics_bulk_upload/lytics_test.go | 19 +++----- .../lytics_bulk_upload/manager.go | 11 +++-- .../lytics_bulk_upload/types.go | 4 ++ .../lytics_bulk_upload/utils.go | 3 +- .../asyncdestinationmanager/manager.go | 44 +++++++++++++------ .../marketo-bulk-upload/marketobulkupload.go | 21 +++++---- .../asyncdestinationmanager/sftp/manager.go | 22 +++++----- .../asyncdestinationmanager/sftp/sftp_test.go | 9 ++-- .../asyncdestinationmanager/sftp/types.go | 4 +- .../yandexmetrica/yandexmetrica.go | 15 ++++--- .../yandexmetrica/yandexmetrica_test.go | 15 ++++--- router/batchrouter/handle_lifecycle.go | 3 +- 29 files changed, 231 insertions(+), 155 deletions(-) diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bingads_test.go b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bingads_test.go index a952dd9b63..c1a33b7b4b 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bingads_test.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bingads_test.go @@ -13,6 +13,8 @@ import ( "go.uber.org/mock/gomock" + "github.com/rudderlabs/rudder-go-kit/stats" + . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" @@ -21,6 +23,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" mocks_oauth "github.com/rudderlabs/rudder-server/mocks/services/oauth" @@ -68,7 +71,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(&bingads_sdk.GetBulkUploadUrlResponse{ UploadUrl: "http://localhost/upload1", RequestId: misc.FastUUID().URN(), @@ -135,7 +138,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf("Error in getting bulk upload url")) bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf("Error in getting bulk upload url")) @@ -188,7 +191,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) ClientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &ClientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &ClientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf("unable to get bulk upload url, check your credentials")) bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, fmt.Errorf("unable to get bulk upload url, check your credentials")) @@ -242,7 +245,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(&bingads_sdk.GetBulkUploadUrlResponse{ UploadUrl: "http://localhost/upload1", RequestId: misc.FastUUID().URN(), @@ -307,7 +310,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(100), @@ -330,7 +333,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(nil, fmt.Errorf("failed to get bulk upload status:")) pollInput := common.AsyncPoll{ @@ -349,7 +352,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(100), @@ -378,7 +381,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(0), @@ -405,7 +408,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(0), @@ -432,7 +435,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId456").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(100), @@ -478,7 +481,7 @@ var _ = Describe("Bing ads Audience", func() { client := ts.Client() modifiedURL := ts.URL // Use the test server URL clientI := Client{client: client, URL: modifiedURL} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) UploadStatsInput := common.GetUploadStatsInput{ FailedJobURLs: modifiedURL, @@ -528,7 +531,7 @@ var _ = Describe("Bing ads Audience", func() { client := ts.Client() modifiedURL := ts.URL // Use the test server URL clientI := Client{client: client, URL: modifiedURL} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) UploadStatsInput := common.GetUploadStatsInput{ FailedJobURLs: modifiedURL, @@ -582,7 +585,7 @@ var _ = Describe("Bing ads Audience", func() { }, }) - bingAdsUploader, err := newManagerInternal(&destination, oauthService, nil) + bingAdsUploader, err := newManagerInternal(logger.NOP, stats.NOP, &destination, oauthService, nil) Expect(err).To(BeNil()) Expect(bingAdsUploader).ToNot(BeNil()) }) @@ -592,7 +595,7 @@ var _ = Describe("Bing ads Audience", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(&bingads_sdk.GetBulkUploadUrlResponse{ UploadUrl: "http://localhost/upload", RequestId: misc.FastUUID().URN(), diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bulk_uploader.go b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bulk_uploader.go index 0ce56fbfac..1ed233543e 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bulk_uploader.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/bulk_uploader.go @@ -14,17 +14,19 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" router_utils "github.com/rudderlabs/rudder-server/router/utils" "github.com/rudderlabs/rudder-server/utils/misc" ) -func NewBingAdsBulkUploader(destName string, service bingads.BulkServiceI, client *Client) *BingAdsBulkUploader { +func NewBingAdsBulkUploader(logger logger.Logger, statsFactory stats.Stats, destName string, service bingads.BulkServiceI, client *Client) *BingAdsBulkUploader { return &BingAdsBulkUploader{ destName: destName, service: service, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("BingAds").Child("BingAdsBulkUploader"), + logger: logger.Child("BingAds").Child("BingAdsBulkUploader"), + statsFactory: statsFactory, client: *client, fileSizeLimit: common.GetBatchRouterConfigInt64("MaxUploadLimit", destName, 100*bytesize.MB), eventsLimit: common.GetBatchRouterConfigInt64("MaxEventsLimit", destName, 4000000), @@ -56,7 +58,7 @@ func (b *BingAdsBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr DestinationID: destination.ID, } } - uploadRetryableStat := stats.Default.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ + uploadRetryableStat := b.statsFactory.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) @@ -74,7 +76,7 @@ func (b *BingAdsBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr continue } - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ + uploadTimeStat := b.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ "module": "batch_router", "destType": b.destName, }) @@ -220,13 +222,13 @@ func (b *BingAdsBulkUploader) getUploadStatsOfSingleImport(filePath string) (com }, } - eventsAbortedStat := stats.Default.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ + eventsAbortedStat := b.statsFactory.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) eventsAbortedStat.Count(len(eventStatsResponse.Metadata.FailedKeys)) - eventsSuccessStat := stats.Default.NewTaggedStat("success_job_count", stats.CountType, map[string]string{ + eventsSuccessStat := b.statsFactory.NewTaggedStat("success_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/manager.go b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/manager.go index 61b2ed2585..8932c216c4 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/manager.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/manager.go @@ -10,13 +10,14 @@ import ( "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/bing-ads/common" "github.com/rudderlabs/rudder-server/services/oauth" oauthv2 "github.com/rudderlabs/rudder-server/services/oauth/v2" ) -func newManagerInternal(destination *backendconfig.DestinationT, oauthClient oauth.Authorizer, oauthClientV2 oauthv2.Authorizer) (*BingAdsBulkUploader, error) { +func newManagerInternal(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT, oauthClient oauth.Authorizer, oauthClientV2 oauthv2.Authorizer) (*BingAdsBulkUploader, error) { destConfig := DestinationConfig{} jsonConfig, err := stdjson.Marshal(destination.Config) if err != nil { @@ -49,16 +50,22 @@ func newManagerInternal(destination *backendconfig.DestinationT, oauthClient oau session := bingads.NewSession(sessionConfig) clientNew := Client{} - bingUploader := NewBingAdsBulkUploader(destination.DestinationDefinition.Name, bingads.NewBulkService(session), &clientNew) + bingUploader := NewBingAdsBulkUploader(logger, statsFactory, destination.DestinationDefinition.Name, bingads.NewBulkService(session), &clientNew) return bingUploader, nil } -func NewManager(destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (*BingAdsBulkUploader, error) { +func NewManager( + conf *config.Config, + logger logger.Logger, + statsFactory stats.Stats, + destination *backendconfig.DestinationT, + backendConfig backendconfig.BackendConfig, +) (*BingAdsBulkUploader, error) { oauthClient := oauth.NewOAuthErrorHandler(backendConfig) oauthClientV2 := oauthv2.NewOAuthHandler(backendConfig, - oauthv2.WithLogger(logger.NewLogger().Child("BatchRouter")), - oauthv2.WithCPConnectorTimeout(config.GetDuration("HttpClient.oauth.timeout", 30, time.Second)), - oauthv2.WithStats(stats.Default), + oauthv2.WithLogger(logger), + oauthv2.WithCPConnectorTimeout(conf.GetDuration("HttpClient.oauth.timeout", 30, time.Second)), + oauthv2.WithStats(statsFactory), ) - return newManagerInternal(destination, oauthClient, oauthClientV2) + return newManagerInternal(logger, statsFactory, destination, oauthClient, oauthClientV2) } diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/types.go b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/types.go index ae3857ee6d..ddeb9cf77f 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/types.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/types.go @@ -6,8 +6,9 @@ import ( "net/http" "os" - bingads "github.com/rudderlabs/bing-ads-go-sdk/bingads" + "github.com/rudderlabs/bing-ads-go-sdk/bingads" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" ) type Client struct { @@ -18,6 +19,7 @@ type BingAdsBulkUploader struct { destName string service bingads.BulkServiceI logger logger.Logger + statsFactory stats.Stats client Client fileSizeLimit int64 eventsLimit int64 diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/util.go b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/util.go index 554a280a48..03f4de47c8 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/audience/util.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/audience/util.go @@ -18,6 +18,7 @@ import ( "github.com/samber/lo" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/utils/misc" ) @@ -161,7 +162,7 @@ func (b *BingAdsBulkUploader) createZipFile(filePath, audienceId string) ([]*Act return nil, err } - payloadSizeStat := stats.Default.NewTaggedStat("payload_size", stats.HistogramType, + payloadSizeStat := b.statsFactory.NewTaggedStat("payload_size", stats.HistogramType, map[string]string{ "module": "batch_router", "destType": b.destName, diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go index 425809eb07..98985d454d 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bingads_test.go @@ -14,6 +14,8 @@ import ( "go.uber.org/mock/gomock" + "github.com/rudderlabs/rudder-go-kit/stats" + . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" @@ -22,6 +24,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" mocks_oauth "github.com/rudderlabs/rudder-server/mocks/services/oauth" @@ -71,7 +74,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { URL: "http://localhost/upload1", client: &http.Client{}, } - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(&bingads_sdk.GetBulkUploadUrlResponse{ UploadUrl: "http://localhost/upload1", RequestId: misc.FastUUID().URN(), @@ -136,7 +139,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) errorMsg := "Error in getting bulk upload url" bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errorMsg)) bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errorMsg)) @@ -179,7 +182,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) ClientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &ClientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &ClientI) errMsg := "unable to get bulk upload url, check your credentials" bingAdsService.EXPECT().GetBulkUploadUrl().Return(nil, errors.New(errMsg)) @@ -225,7 +228,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadUrl().Return(&bingads_sdk.GetBulkUploadUrlResponse{ UploadUrl: "http://localhost/upload1", RequestId: misc.FastUUID().URN(), @@ -283,7 +286,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(100), @@ -306,7 +309,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(nil, fmt.Errorf("failed to get bulk upload status:")) pollInput := common.AsyncPoll{ @@ -325,7 +328,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(100), @@ -354,7 +357,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(0), @@ -381,7 +384,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId123").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(0), @@ -408,7 +411,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { ctrl := gomock.NewController(GinkgoT()) bingAdsService := mock_bulkservice.NewMockBulkServiceI(ctrl) clientI := Client{} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) bingAdsService.EXPECT().GetBulkUploadStatus("dummyRequestId456").Return(&bingads_sdk.GetBulkUploadStatusResponse{ PercentComplete: int64(100), @@ -454,7 +457,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { client := ts.Client() modifiedURL := ts.URL // Use the test server URL clientI := Client{client: client, URL: modifiedURL} - bulkUploader := NewBingAdsBulkUploader("BING_ADS", bingAdsService, &clientI) + bulkUploader := NewBingAdsBulkUploader(logger.NOP, stats.NOP, "BING_ADS", bingAdsService, &clientI) UploadStatsInput := common.GetUploadStatsInput{ FailedJobURLs: modifiedURL, @@ -522,7 +525,7 @@ var _ = Describe("Bing ads Offline Conversions", func() { }, }) - bingAdsUploader, err := newManagerInternal(&destination, oauthService, nil) + bingAdsUploader, err := newManagerInternal(logger.NOP, stats.NOP, &destination, oauthService, nil) Expect(err).To(BeNil()) Expect(bingAdsUploader).ToNot(BeNil()) }) diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bulk_uploader.go b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bulk_uploader.go index 0dd566c8a9..7ea421d839 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bulk_uploader.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/bulk_uploader.go @@ -14,15 +14,17 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) -func NewBingAdsBulkUploader(destName string, service bingads.BulkServiceI, client *Client) *BingAdsBulkUploader { +func NewBingAdsBulkUploader(logger logger.Logger, statsFactory stats.Stats, destName string, service bingads.BulkServiceI, client *Client) *BingAdsBulkUploader { return &BingAdsBulkUploader{ destName: destName, service: service, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("BingAds").Child("BingAdsBulkUploader"), + logger: logger.Child("BingAds").Child("BingAdsBulkUploader"), + statsFactory: statsFactory, client: *client, fileSizeLimit: common.GetBatchRouterConfigInt64("MaxUploadLimit", destName, 100*bytesize.MB), eventsLimit: common.GetBatchRouterConfigInt64("MaxEventsLimit", destName, 1000), @@ -115,7 +117,7 @@ func (b *BingAdsBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr DestinationID: destination.ID, } } - uploadRetryableStat := stats.Default.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ + uploadRetryableStat := b.statsFactory.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) @@ -133,7 +135,7 @@ func (b *BingAdsBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr continue } - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ + uploadTimeStat := b.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ "module": "batch_router", "destType": b.destName, }) @@ -284,13 +286,13 @@ func (b *BingAdsBulkUploader) getUploadStatsOfSingleImport(filePath string) (com }, } - eventsAbortedStat := stats.Default.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ + eventsAbortedStat := b.statsFactory.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) eventsAbortedStat.Count(len(eventStatsResponse.Metadata.FailedKeys)) - eventsSuccessStat := stats.Default.NewTaggedStat("success_job_count", stats.CountType, map[string]string{ + eventsSuccessStat := b.statsFactory.NewTaggedStat("success_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/manager.go b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/manager.go index 52117ce1b4..c304130206 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/manager.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/manager.go @@ -10,13 +10,15 @@ import ( "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/bing-ads/common" + asynccommon "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" "github.com/rudderlabs/rudder-server/services/oauth" oauthv2 "github.com/rudderlabs/rudder-server/services/oauth/v2" ) -func newManagerInternal(destination *backendconfig.DestinationT, oauthClient oauth.Authorizer, oauthClientV2 oauthv2.Authorizer) (*BingAdsBulkUploader, error) { +func newManagerInternal(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT, oauthClient oauth.Authorizer, oauthClientV2 oauthv2.Authorizer) (*BingAdsBulkUploader, error) { destConfig := DestinationConfig{} jsonConfig, err := stdjson.Marshal(destination.Config) if err != nil { @@ -49,16 +51,16 @@ func newManagerInternal(destination *backendconfig.DestinationT, oauthClient oau session := bingads.NewSession(sessionConfig) clientNew := Client{} - bingUploader := NewBingAdsBulkUploader(destination.DestinationDefinition.Name, bingads.NewBulkService(session), &clientNew) + bingUploader := NewBingAdsBulkUploader(logger, statsFactory, destination.DestinationDefinition.Name, bingads.NewBulkService(session), &clientNew) return bingUploader, nil } -func NewManager(destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (*BingAdsBulkUploader, error) { +func NewManager(conf *config.Config, logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (asynccommon.AsyncDestinationManager, error) { oauthClient := oauth.NewOAuthErrorHandler(backendConfig) oauthClientV2 := oauthv2.NewOAuthHandler(backendConfig, - oauthv2.WithLogger(logger.NewLogger().Child("BatchRouter")), - oauthv2.WithCPConnectorTimeout(config.GetDuration("HttpClient.oauth.timeout", 30, time.Second)), - oauthv2.WithStats(stats.Default), + oauthv2.WithLogger(logger), + oauthv2.WithCPConnectorTimeout(conf.GetDuration("HttpClient.oauth.timeout", 30, time.Second)), + oauthv2.WithStats(statsFactory), ) - return newManagerInternal(destination, oauthClient, oauthClientV2) + return newManagerInternal(logger, statsFactory, destination, oauthClient, oauthClientV2) } diff --git a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/types.go b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/types.go index b51e75f4ef..220069d2ff 100644 --- a/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/types.go +++ b/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions/types.go @@ -5,8 +5,9 @@ import ( "encoding/json" "net/http" - bingads "github.com/rudderlabs/bing-ads-go-sdk/bingads" + "github.com/rudderlabs/bing-ads-go-sdk/bingads" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" ) type Client struct { @@ -17,6 +18,7 @@ type BingAdsBulkUploader struct { destName string service bingads.BulkServiceI logger logger.Logger + statsFactory stats.Stats client Client fileSizeLimit int64 eventsLimit int64 diff --git a/router/batchrouter/asyncdestinationmanager/eloqua/bulk_uploader.go b/router/batchrouter/asyncdestinationmanager/eloqua/bulk_uploader.go index 2295cecafb..48d9437c6c 100644 --- a/router/batchrouter/asyncdestinationmanager/eloqua/bulk_uploader.go +++ b/router/batchrouter/asyncdestinationmanager/eloqua/bulk_uploader.go @@ -11,6 +11,7 @@ import ( "github.com/tidwall/gjson" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) @@ -31,7 +32,7 @@ func (*EloquaBulkUploader) Transform(job *jobsdb.JobT) (string, error) { func (b *EloquaBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStruct) common.AsyncUploadOutput { destination := asyncDestStruct.Destination - uploadRetryableStat := stats.Default.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ + uploadRetryableStat := b.statsFactory.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) @@ -94,14 +95,14 @@ func (b *EloquaBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStru if err != nil { return b.createAsyncUploadErrorOutput("unable to create csv file. ", err, destination.ID, asyncDestStruct) } - CSVFileSizeStat := stats.Default.NewTaggedStat("csv_file_size", stats.HistogramType, + CSVFileSizeStat := b.statsFactory.NewTaggedStat("csv_file_size", stats.HistogramType, map[string]string{ "module": "batch_router", "destType": b.destName, }) CSVFileSizeStat.Observe(float64(fileSize)) defer os.Remove(filePAth) - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ + uploadTimeStat := b.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ "module": "batch_router", "destType": b.destName, }) @@ -216,13 +217,13 @@ func (b *EloquaBulkUploader) GetUploadStats(UploadStatsInput common.GetUploadSta var uploadStatusResponse common.GetUploadStatsResponse defer func() { b.clearJobToCsvMap() - eventsAbortedStat := stats.Default.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ + eventsAbortedStat := b.statsFactory.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) eventsAbortedStat.Count(len(uploadStatusResponse.Metadata.FailedKeys)) - eventsSuccessStat := stats.Default.NewTaggedStat("success_job_count", stats.CountType, map[string]string{ + eventsSuccessStat := b.statsFactory.NewTaggedStat("success_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": b.destName, }) diff --git a/router/batchrouter/asyncdestinationmanager/eloqua/eloqua_test.go b/router/batchrouter/asyncdestinationmanager/eloqua/eloqua_test.go index c7d98ea06b..9679d874ae 100644 --- a/router/batchrouter/asyncdestinationmanager/eloqua/eloqua_test.go +++ b/router/batchrouter/asyncdestinationmanager/eloqua/eloqua_test.go @@ -9,6 +9,8 @@ import ( "go.uber.org/mock/gomock" + "github.com/rudderlabs/rudder-go-kit/stats" + . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" @@ -16,6 +18,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" mock_bulkservice "github.com/rudderlabs/rudder-server/mocks/router/eloqua" @@ -72,7 +75,7 @@ var _ = Describe("Eloqua test", func() { initEloqua() ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023}, FailedJobIDs: []int64{}, @@ -95,7 +98,7 @@ var _ = Describe("Eloqua test", func() { initEloqua() ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023}, FailedJobIDs: []int64{}, @@ -121,7 +124,7 @@ var _ = Describe("Eloqua test", func() { initEloqua() ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023}, FailedJobIDs: []int64{}, @@ -164,7 +167,7 @@ var _ = Describe("Eloqua test", func() { initEloqua() ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023}, FailedJobIDs: []int64{}, @@ -211,7 +214,7 @@ var _ = Describe("Eloqua test", func() { initEloqua() ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023}, FailedJobIDs: []int64{}, @@ -258,7 +261,7 @@ var _ = Describe("Eloqua test", func() { initEloqua() ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023}, FailedJobIDs: []int64{}, @@ -321,7 +324,7 @@ var _ = Describe("Eloqua test", func() { It("TestEloquaFailedToGetSyncStatus", func() { ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) pollInput := common.AsyncPoll{ ImportId: "/syncs/384:/contacts/imports/384", } @@ -339,7 +342,7 @@ var _ = Describe("Eloqua test", func() { It("TestEloquaReceivedSuccess", func() { ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) pollInput := common.AsyncPoll{ ImportId: "/syncs/384:/contacts/imports/384", } @@ -358,7 +361,7 @@ var _ = Describe("Eloqua test", func() { It("TestEloquaReceivedPending", func() { ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) pollInput := common.AsyncPoll{ ImportId: "/syncs/384:/contacts/imports/384", } @@ -383,7 +386,7 @@ var _ = Describe("Eloqua test", func() { It("TestEloquaErrorOccurredWhileUploading", func() { ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) pollInput := common.GetUploadStatsInput{ FailedJobURLs: "/syncs/384", @@ -415,7 +418,7 @@ var _ = Describe("Eloqua test", func() { It("TestEloquaFailedToFetchRejectedData", func() { ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) pollInput := common.GetUploadStatsInput{ WarningJobURLs: "/syncs/384", @@ -432,7 +435,7 @@ var _ = Describe("Eloqua test", func() { It("TestEloquaSucceedToFetchRejectedData", func() { ctrl := gomock.NewController(GinkgoT()) eloquaService := mock_bulkservice.NewMockEloquaService(ctrl) - bulkUploader := eloqua.NewEloquaBulkUploader("Eloqua", "", "", eloquaService) + bulkUploader := eloqua.NewEloquaBulkUploader(logger.NOP, stats.NOP, "Eloqua", "", "", eloquaService) pollInput := common.GetUploadStatsInput{ WarningJobURLs: "/syncs/384", ImportingList: jobs, diff --git a/router/batchrouter/asyncdestinationmanager/eloqua/manager.go b/router/batchrouter/asyncdestinationmanager/eloqua/manager.go index 1d385e61ff..fca044371a 100644 --- a/router/batchrouter/asyncdestinationmanager/eloqua/manager.go +++ b/router/batchrouter/asyncdestinationmanager/eloqua/manager.go @@ -8,11 +8,12 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) -func NewManager(destination *backendconfig.DestinationT) (*EloquaBulkUploader, error) { +func NewManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (*EloquaBulkUploader, error) { destConfig := DestinationConfig{} jsonConfig, err := stdjson.Marshal(destination.Config) if err != nil { @@ -30,7 +31,7 @@ func NewManager(destination *backendconfig.DestinationT) (*EloquaBulkUploader, e } eloqua := NewEloquaServiceImpl("2.0") baseEndpoint, err := eloqua.GetBaseEndpoint(&eloquaData) - unableToGetBaseEndpointStat := stats.Default.NewTaggedStat("unable_to_get_base_endpoint", stats.CountType, map[string]string{ + unableToGetBaseEndpointStat := statsFactory.NewTaggedStat("unable_to_get_base_endpoint", stats.CountType, map[string]string{ "module": "batch_router", "destType": destName, }) @@ -39,13 +40,14 @@ func NewManager(destination *backendconfig.DestinationT) (*EloquaBulkUploader, e return nil, fmt.Errorf("error in getting base endpoint: %v", err) } - return NewEloquaBulkUploader(destName, encodedAuthorizationString, baseEndpoint, eloqua), nil + return NewEloquaBulkUploader(logger, statsFactory, destName, encodedAuthorizationString, baseEndpoint, eloqua), nil } -func NewEloquaBulkUploader(destinationName, authorization, baseEndpoint string, eloqua EloquaService) *EloquaBulkUploader { +func NewEloquaBulkUploader(logger logger.Logger, statsFactory stats.Stats, destinationName, authorization, baseEndpoint string, eloqua EloquaService) *EloquaBulkUploader { return &EloquaBulkUploader{ destName: destinationName, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("Eloqua").Child("EloquaBulkUploader"), + logger: logger.Child("Eloqua").Child("EloquaBulkUploader"), + statsFactory: statsFactory, authorization: authorization, baseEndpoint: baseEndpoint, fileSizeLimit: common.GetBatchRouterConfigInt64("MaxUploadLimit", destinationName, 32*bytesize.MB), diff --git a/router/batchrouter/asyncdestinationmanager/eloqua/types.go b/router/batchrouter/asyncdestinationmanager/eloqua/types.go index 26aebe064c..b431bfb1f7 100644 --- a/router/batchrouter/asyncdestinationmanager/eloqua/types.go +++ b/router/batchrouter/asyncdestinationmanager/eloqua/types.go @@ -7,6 +7,7 @@ import ( "time" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" ) type EloquaService interface { @@ -23,6 +24,7 @@ type EloquaService interface { type EloquaBulkUploader struct { destName string logger logger.Logger + statsFactory stats.Stats authorization string baseEndpoint string fileSizeLimit int64 diff --git a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go index 5f977facf9..6c98da9337 100644 --- a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go +++ b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload.go @@ -17,6 +17,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" @@ -59,11 +60,12 @@ func createFinalPayload(combinedProfiles []Profile, listId string) Payload { return payload } -func NewManager(destination *backendconfig.DestinationT) (*KlaviyoBulkUploader, error) { +func NewManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (*KlaviyoBulkUploader, error) { return &KlaviyoBulkUploader{ destName: destination.DestinationDefinition.Name, destinationConfig: destination.Config, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("KlaviyoBulkUpload").Child("KlaviyoBulkUploader"), + logger: logger.Child("KlaviyoBulkUpload").Child("KlaviyoBulkUploader"), + statsFactory: statsFactory, }, nil } @@ -264,7 +266,7 @@ func (kbu *KlaviyoBulkUploader) GetUploadStats(UploadStatsInput common.GetUpload } func (kbu *KlaviyoBulkUploader) generateKlaviyoErrorOutput(errorString string, err error, importingJobIds []int64, destinationID string) common.AsyncUploadOutput { - eventsAbortedStat := stats.Default.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ + eventsAbortedStat := kbu.statsFactory.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": "KLAVIYO_BULK_UPLOAD", }) @@ -336,7 +338,7 @@ func (kbu *KlaviyoBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationS chunks, _ := chunkBySizeAndElements(combinedProfiles, MAXPAYLOADSIZE, BATCHSIZE) - eventsSuccessStat := stats.Default.NewTaggedStat("success_job_count", stats.CountType, statLabels) + eventsSuccessStat := kbu.statsFactory.NewTaggedStat("success_job_count", stats.CountType, statLabels) var importIds []string // DelimitedImportIds is : separated importIds @@ -359,8 +361,8 @@ func (kbu *KlaviyoBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationS req.Header.Set("Authorization", "Klaviyo-API-Key "+destination.Config["privateApiKey"].(string)) req.Header.Set("revision", "2024-05-15") - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, statLabels) - payloadSizeStat := stats.Default.NewTaggedStat("payload_size", stats.HistogramType, statLabels) + uploadTimeStat := kbu.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, statLabels) + payloadSizeStat := kbu.statsFactory.NewTaggedStat("payload_size", stats.HistogramType, statLabels) payloadSizeStat.Observe(float64(len(outputJSON))) resp, err := client.Do(req) diff --git a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload_test.go b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload_test.go index fb836e7a46..330dbcc937 100644 --- a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload_test.go +++ b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/klaviyobulkupload_test.go @@ -7,6 +7,10 @@ import ( "github.com/stretchr/testify/assert" + "github.com/rudderlabs/rudder-go-kit/stats" + + "github.com/rudderlabs/rudder-go-kit/logger" + "go.uber.org/mock/gomock" backendconfig "github.com/rudderlabs/rudder-server/backend-config" @@ -30,7 +34,7 @@ var destination = &backendconfig.DestinationT{ } func TestNewManagerSuccess(t *testing.T) { - manager, err := klaviyobulkupload.NewManager(destination) + manager, err := klaviyobulkupload.NewManager(logger.NOP, stats.NOP, destination) assert.NoError(t, err) assert.NotNil(t, manager) assert.Equal(t, "KLAVIYO_BULK_UPLOAD", destination.Name) diff --git a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/types.go b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/types.go index 6da62133ee..9cf55edd88 100644 --- a/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/types.go +++ b/router/batchrouter/asyncdestinationmanager/klaviyobulkupload/types.go @@ -6,6 +6,8 @@ import ( "net/http" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) @@ -33,6 +35,7 @@ type KlaviyoBulkUploader struct { destName string destinationConfig map[string]interface{} logger logger.Logger + statsFactory stats.Stats Client *http.Client jobIdToIdentifierMap map[string]int64 } diff --git a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lyticsBulkUpload.go b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lyticsBulkUpload.go index 871342be12..a12a82262d 100644 --- a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lyticsBulkUpload.go +++ b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lyticsBulkUpload.go @@ -11,6 +11,7 @@ import ( "github.com/tidwall/gjson" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) @@ -74,7 +75,7 @@ func (u *LyticsBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStru filePath := asyncDestStruct.FileName destConfig, err := jsoniter.Marshal(destination.Config) if err != nil { - eventsAbortedStat := stats.Default.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ + eventsAbortedStat := u.statsFactory.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": u.destName, }) @@ -108,14 +109,14 @@ func (u *LyticsBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStru DestinationID: destination.ID, } } - uploadRetryableStat := stats.Default.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ + uploadRetryableStat := u.statsFactory.NewTaggedStat("events_over_prescribed_limit", stats.CountType, map[string]string{ "module": "batch_router", "destType": u.destName, }) uploadRetryableStat.Count(len(actionFiles.FailedJobIDs)) - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ + uploadTimeStat := u.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ "module": "batch_router", "destType": u.destName, }) diff --git a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lytics_test.go b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lytics_test.go index 83a3c1ed11..c21c33c1c5 100644 --- a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lytics_test.go +++ b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/lytics_test.go @@ -8,12 +8,15 @@ import ( "go.uber.org/mock/gomock" + "github.com/rudderlabs/rudder-go-kit/stats" + . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" mocks "github.com/rudderlabs/rudder-server/mocks/router/lytics_bulk_upload" @@ -67,7 +70,7 @@ var _ = Describe("LYTICS_BULK_UPLOAD test", func() { LyticsServiceImpl := lyticsBulkUpload.LyticsServiceImpl{ BulkApi: "https://bulk.lytics.io/collect/bulk/test?timestamp_field=timestamp", } - bulkUploader := common.SimpleAsyncDestinationManager{UploaderAndTransformer: lyticsBulkUpload.NewLyticsBulkUploader("LYTICS_BULK_UPLOAD", destination.Config["lyticsApiKey"].(string), LyticsServiceImpl.BulkApi, lyticsService)} + bulkUploader := common.SimpleAsyncDestinationManager{UploaderAndTransformer: lyticsBulkUpload.NewLyticsBulkUploader(logger.NOP, stats.NOP, "LYTICS_BULK_UPLOAD", destination.Config["lyticsApiKey"].(string), LyticsServiceImpl.BulkApi, lyticsService)} asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1, 2, 3, 4}, FailedJobIDs: []int64{}, @@ -102,12 +105,7 @@ var _ = Describe("LYTICS_BULK_UPLOAD test", func() { // Set up the SimpleAsyncDestinationManager with the LyticsBulkUploader bulkUploader := common.SimpleAsyncDestinationManager{ - UploaderAndTransformer: lyticsBulkUpload.NewLyticsBulkUploader( - "LYTICS_BULK_UPLOAD", - destination.Config["lyticsApiKey"].(string), - LyticsServiceImpl.BulkApi, - lyticsService, - ), + UploaderAndTransformer: lyticsBulkUpload.NewLyticsBulkUploader(logger.NOP, stats.NOP, "LYTICS_BULK_UPLOAD", destination.Config["lyticsApiKey"].(string), LyticsServiceImpl.BulkApi, lyticsService), } // Set up the AsyncDestinationStruct with test data @@ -163,12 +161,7 @@ var _ = Describe("LYTICS_BULK_UPLOAD test", func() { // Set up the SimpleAsyncDestinationManager with the LyticsBulkUploader bulkUploader := common.SimpleAsyncDestinationManager{ - UploaderAndTransformer: lyticsBulkUpload.NewLyticsBulkUploader( - "LYTICS_BULK_UPLOAD", - destination.Config["lyticsApiKey"].(string), - LyticsServiceImpl.BulkApi, - lyticsService, - ), + UploaderAndTransformer: lyticsBulkUpload.NewLyticsBulkUploader(logger.NOP, stats.NOP, "LYTICS_BULK_UPLOAD", destination.Config["lyticsApiKey"].(string), LyticsServiceImpl.BulkApi, lyticsService), } // Set up the AsyncDestinationStruct with test data diff --git a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/manager.go b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/manager.go index 5cb6a1a72f..02189e1e8c 100644 --- a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/manager.go +++ b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/manager.go @@ -6,14 +6,17 @@ import ( "github.com/rudderlabs/rudder-go-kit/bytesize" "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) -func NewLyticsBulkUploader(destinationName, authorization, endpoint string, lytics LyticsService) common.AsyncUploadAndTransformManager { +func NewLyticsBulkUploader(logger logger.Logger, statsFactory stats.Stats, destinationName, authorization, endpoint string, lytics LyticsService) common.AsyncUploadAndTransformManager { return &LyticsBulkUploader{ destName: destinationName, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("Lytics").Child("LyticsBulkUploader"), + logger: logger.Child("Lytics").Child("LyticsBulkUploader"), + statsFactory: statsFactory, authorization: authorization, baseEndpoint: endpoint, fileSizeLimit: common.GetBatchRouterConfigInt64("MaxUploadLimit", destinationName, 10*bytesize.MB), @@ -22,7 +25,7 @@ func NewLyticsBulkUploader(destinationName, authorization, endpoint string, lyti } } -func NewManager(destination *backendconfig.DestinationT) (common.AsyncDestinationManager, error) { +func NewManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (common.AsyncDestinationManager, error) { destConfig := DestinationConfig{} jsonConfig, err := json.Marshal(destination.Config) if err != nil { @@ -38,6 +41,6 @@ func NewManager(destination *backendconfig.DestinationT) (common.AsyncDestinatio lyticsImpl := lyticsService.getBulkApi(destConfig) return common.SimpleAsyncDestinationManager{ - UploaderAndTransformer: NewLyticsBulkUploader(destName, destConfig.LyticsApiKey, lyticsImpl.BulkApi, lyticsService), + UploaderAndTransformer: NewLyticsBulkUploader(logger, statsFactory, destName, destConfig.LyticsApiKey, lyticsImpl.BulkApi, lyticsService), }, nil } diff --git a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/types.go b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/types.go index c55dbcbe9a..20aee2ecac 100644 --- a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/types.go +++ b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/types.go @@ -10,13 +10,17 @@ import ( jsoniter "github.com/json-iterator/go" + "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" ) type LyticsBulkUploader struct { destName string logger logger.Logger + statsFactory stats.Stats authorization string baseEndpoint string fileSizeLimit int64 diff --git a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/utils.go b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/utils.go index 2ed93a9b1f..653b1ca174 100644 --- a/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/utils.go +++ b/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload/utils.go @@ -12,6 +12,7 @@ import ( "github.com/tidwall/gjson" "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/utils/misc" ) @@ -156,7 +157,7 @@ func (u *LyticsBulkUploader) createCSVFile(existingFilePath string, streamTraits } // Calculate the payload size and observe it - payloadSizeStat := stats.Default.NewTaggedStat("payload_size", stats.HistogramType, + payloadSizeStat := u.statsFactory.NewTaggedStat("payload_size", stats.HistogramType, map[string]string{ "module": "batch_router", "destType": u.destName, diff --git a/router/batchrouter/asyncdestinationmanager/manager.go b/router/batchrouter/asyncdestinationmanager/manager.go index 76292c14d6..59dd7fe748 100644 --- a/router/batchrouter/asyncdestinationmanager/manager.go +++ b/router/batchrouter/asyncdestinationmanager/manager.go @@ -3,52 +3,68 @@ package asyncdestinationmanager import ( "errors" + "github.com/rudderlabs/rudder-go-kit/config" + "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" bingads_audience "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/bing-ads/audience" bingads_offline_conversions "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/bing-ads/offline-conversions" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/eloqua" - klaviyobulkupload "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/klaviyobulkupload" + "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/klaviyobulkupload" lyticsBulkUpload "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/lytics_bulk_upload" marketobulkupload "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/marketo-bulk-upload" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/sftp" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/yandexmetrica" ) -func newRegularManager(destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (common.AsyncDestinationManager, error) { +func newRegularManager( + conf *config.Config, + logger logger.Logger, + statsFactory stats.Stats, + destination *backendconfig.DestinationT, + backendConfig backendconfig.BackendConfig, +) (common.AsyncDestinationManager, error) { switch destination.DestinationDefinition.Name { case "BINGADS_AUDIENCE": - return bingads_audience.NewManager(destination, backendConfig) + return bingads_audience.NewManager(conf, logger, statsFactory, destination, backendConfig) case "BINGADS_OFFLINE_CONVERSIONS": - return bingads_offline_conversions.NewManager(destination, backendConfig) + return bingads_offline_conversions.NewManager(conf, logger, statsFactory, destination, backendConfig) case "MARKETO_BULK_UPLOAD": - return marketobulkupload.NewManager(destination) + return marketobulkupload.NewManager(conf, logger, statsFactory, destination) case "ELOQUA": - return eloqua.NewManager(destination) + return eloqua.NewManager(logger, statsFactory, destination) case "YANDEX_METRICA_OFFLINE_EVENTS": - return yandexmetrica.NewManager(destination, backendConfig) + return yandexmetrica.NewManager(logger, statsFactory, destination, backendConfig) case "KLAVIYO_BULK_UPLOAD": - return klaviyobulkupload.NewManager(destination) + return klaviyobulkupload.NewManager(logger, statsFactory, destination) case "LYTICS_BULK_UPLOAD": - return lyticsBulkUpload.NewManager(destination) + return lyticsBulkUpload.NewManager(logger, statsFactory, destination) } return nil, errors.New("invalid destination type") } -func newSFTPManager(destination *backendconfig.DestinationT) (common.AsyncDestinationManager, error) { +func newSFTPManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (common.AsyncDestinationManager, error) { switch destination.DestinationDefinition.Name { case "SFTP": - return sftp.NewManager(destination) + return sftp.NewManager(logger, statsFactory, destination) } return nil, errors.New("invalid destination type") } -func NewManager(destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (common.AsyncDestinationManager, error) { +func NewManager( + conf *config.Config, + logger logger.Logger, + statsFactory stats.Stats, + destination *backendconfig.DestinationT, + backendConfig backendconfig.BackendConfig, +) (common.AsyncDestinationManager, error) { switch { case common.IsAsyncRegularDestination(destination.DestinationDefinition.Name): - return newRegularManager(destination, backendConfig) + return newRegularManager(conf, logger, statsFactory, destination, backendConfig) case common.IsSFTPDestination(destination.DestinationDefinition.Name): - return newSFTPManager(destination) + return newSFTPManager(logger, statsFactory, destination) } return nil, errors.New("invalid destination type") } diff --git a/router/batchrouter/asyncdestinationmanager/marketo-bulk-upload/marketobulkupload.go b/router/batchrouter/asyncdestinationmanager/marketo-bulk-upload/marketobulkupload.go index ce1ba4c2e7..d7528cad3e 100644 --- a/router/batchrouter/asyncdestinationmanager/marketo-bulk-upload/marketobulkupload.go +++ b/router/batchrouter/asyncdestinationmanager/marketo-bulk-upload/marketobulkupload.go @@ -16,6 +16,7 @@ import ( "github.com/tidwall/gjson" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" @@ -28,18 +29,22 @@ type MarketoBulkUploader struct { destinationConfig map[string]interface{} transformUrl string pollUrl string + conf *config.Config logger logger.Logger + statsFactory stats.Stats timeout time.Duration } -func NewManager(destination *backendconfig.DestinationT) (*MarketoBulkUploader, error) { +func NewManager(conf *config.Config, logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (*MarketoBulkUploader, error) { marketoBulkUpload := &MarketoBulkUploader{ destName: destination.DestinationDefinition.Name, destinationConfig: destination.Config, pollUrl: "/pollStatus", - transformUrl: config.GetString("DEST_TRANSFORM_URL", "http://localhost:9090"), - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("Marketo").Child("MarketoBulkUploader"), - timeout: config.GetDuration("HttpClient.marketoBulkUpload.timeout", 30, time.Second), + transformUrl: conf.GetString("DEST_TRANSFORM_URL", "http://localhost:9090"), + conf: conf, + logger: logger.Child("Marketo").Child("MarketoBulkUploader"), + statsFactory: statsFactory, + timeout: conf.GetDuration("HttpClient.marketoBulkUpload.timeout", 30, time.Second), } return marketoBulkUpload, nil } @@ -287,12 +292,12 @@ func (b *MarketoBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr } } - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ + uploadTimeStat := b.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, map[string]string{ "module": "batch_router", "destType": destType, }) - payloadSizeStat := stats.Default.NewTaggedStat("payload_size", stats.HistogramType, map[string]string{ + payloadSizeStat := b.statsFactory.NewTaggedStat("payload_size", stats.HistogramType, map[string]string{ "module": "batch_router", "destType": destType, }) @@ -300,7 +305,7 @@ func (b *MarketoBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr startTime := time.Now() payloadSizeStat.Observe(float64(len(payload))) b.logger.Debugf("[Async Destination Manager] File Upload Started for Dest Type %v", destType) - responseBody, statusCodeHTTP := misc.HTTPCallWithRetryWithTimeout(uploadURL, payload, config.GetDuration("HttpClient.marketoBulkUpload.timeout", 10, time.Minute)) + responseBody, statusCodeHTTP := misc.HTTPCallWithRetryWithTimeout(uploadURL, payload, b.conf.GetDuration("HttpClient.marketoBulkUpload.timeout", 10, time.Minute)) b.logger.Debugf("[Async Destination Manager] File Upload Finished for Dest Type %v", destType) uploadTimeStat.Since(startTime) var bodyBytes []byte @@ -365,7 +370,7 @@ func (b *MarketoBulkUploader) Upload(asyncDestStruct *common.AsyncDestinationStr DestinationID: destinationID, } } - eventsAbortedStat := stats.Default.NewTaggedStat("events_delivery_aborted", stats.CountType, map[string]string{ + eventsAbortedStat := b.statsFactory.NewTaggedStat("events_delivery_aborted", stats.CountType, map[string]string{ "module": "batch_router", "destType": destType, }) diff --git a/router/batchrouter/asyncdestinationmanager/sftp/manager.go b/router/batchrouter/asyncdestinationmanager/sftp/manager.go index b0301b8b00..9bfffa63f9 100644 --- a/router/batchrouter/asyncdestinationmanager/sftp/manager.go +++ b/router/batchrouter/asyncdestinationmanager/sftp/manager.go @@ -11,6 +11,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/sftp" "github.com/rudderlabs/rudder-go-kit/stats" obskit "github.com/rudderlabs/rudder-observability-kit/go/labels" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" @@ -73,9 +74,9 @@ func (d *defaultManager) Upload(asyncDestStruct *common.AsyncDestinationStruct) "destType": destType, } - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, statLabels) - payloadSizeStat := stats.Default.NewTaggedStat("payload_size", stats.HistogramType, statLabels) - eventsSuccessStat := stats.Default.NewTaggedStat("success_job_count", stats.CountType, statLabels) + uploadTimeStat := d.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, statLabels) + payloadSizeStat := d.statsFactory.NewTaggedStat("payload_size", stats.HistogramType, statLabels) + eventsSuccessStat := d.statsFactory.NewTaggedStat("success_job_count", stats.CountType, statLabels) payloadSizeStat.Observe(float64(fileInfo.Size())) d.logger.Debugn("File Upload Started", obskit.DestinationID(destinationID)) @@ -96,14 +97,15 @@ func (d *defaultManager) Upload(asyncDestStruct *common.AsyncDestinationStruct) } } -func newDefaultManager(fileManager sftp.FileManager) *defaultManager { +func newDefaultManager(logger logger.Logger, statsFactory stats.Stats, fileManager sftp.FileManager) *defaultManager { return &defaultManager{ - FileManager: fileManager, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("SFTP").Child("Manager"), + FileManager: fileManager, + logger: logger.Child("SFTP").Child("Manager"), + statsFactory: statsFactory, } } -func newInternalManager(destination *backendconfig.DestinationT) (common.AsyncUploadAndTransformManager, error) { +func newInternalManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (common.AsyncUploadAndTransformManager, error) { sshConfig, err := createSSHConfig(destination) if err != nil { return nil, fmt.Errorf("creating SSH config: %w", err) @@ -114,11 +116,11 @@ func newInternalManager(destination *backendconfig.DestinationT) (common.AsyncUp return nil, fmt.Errorf("creating file manager: %w", err) } - return newDefaultManager(fileManager), nil + return newDefaultManager(logger, statsFactory, fileManager), nil } -func NewManager(destination *backendconfig.DestinationT) (common.AsyncDestinationManager, error) { - sftpManager, err := newInternalManager(destination) +func NewManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT) (common.AsyncDestinationManager, error) { + sftpManager, err := newInternalManager(logger, statsFactory, destination) if err != nil { return nil, err } diff --git a/router/batchrouter/asyncdestinationmanager/sftp/sftp_test.go b/router/batchrouter/asyncdestinationmanager/sftp/sftp_test.go index 1207143360..10b5ccdb35 100644 --- a/router/batchrouter/asyncdestinationmanager/sftp/sftp_test.go +++ b/router/batchrouter/asyncdestinationmanager/sftp/sftp_test.go @@ -12,9 +12,12 @@ import ( . "github.com/onsi/gomega" "go.uber.org/mock/gomock" + "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-go-kit/config" "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/sftp/mock_sftp" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" "github.com/rudderlabs/rudder-server/utils/misc" @@ -133,7 +136,7 @@ var _ = Describe("SFTP test", func() { initSFTP() ctrl := gomock.NewController(GinkgoT()) fileManager := mock_sftp.NewMockFileManager(ctrl) - defaultManager := newDefaultManager(fileManager) + defaultManager := newDefaultManager(logger.NOP, stats.NOP, fileManager) manager := common.SimpleAsyncDestinationManager{UploaderAndTransformer: defaultManager} asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1014, 1015, 1016, 1017}, @@ -155,7 +158,7 @@ var _ = Describe("SFTP test", func() { initSFTP() ctrl := gomock.NewController(GinkgoT()) fileManager := mock_sftp.NewMockFileManager(ctrl) - defaultManager := newDefaultManager(fileManager) + defaultManager := newDefaultManager(logger.NOP, stats.NOP, fileManager) fileManager.EXPECT().Upload(gomock.Any(), gomock.Any()).Return(fmt.Errorf("root directory does not exists")) manager := common.SimpleAsyncDestinationManager{UploaderAndTransformer: defaultManager} asyncDestination := common.AsyncDestinationStruct{ @@ -178,7 +181,7 @@ var _ = Describe("SFTP test", func() { initSFTP() ctrl := gomock.NewController(GinkgoT()) fileManager := mock_sftp.NewMockFileManager(ctrl) - defaultManager := newDefaultManager(fileManager) + defaultManager := newDefaultManager(logger.NOP, stats.NOP, fileManager) now := time.Now() filePath := fmt.Sprintf("/tmp/testDir1/destination_id_1_someJobRunId_1/file_%d_%02d_%02d_%d_1.csv", now.Year(), now.Month(), now.Day(), now.Unix()) fileManager.EXPECT().Upload(gomock.Any(), filePath).Return(nil) diff --git a/router/batchrouter/asyncdestinationmanager/sftp/types.go b/router/batchrouter/asyncdestinationmanager/sftp/types.go index a515d6d9b1..f3c67587d1 100644 --- a/router/batchrouter/asyncdestinationmanager/sftp/types.go +++ b/router/batchrouter/asyncdestinationmanager/sftp/types.go @@ -3,12 +3,14 @@ package sftp import ( "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/sftp" + "github.com/rudderlabs/rudder-go-kit/stats" ) // defaultManager is the default manager for SFTP type defaultManager struct { - FileManager sftp.FileManager logger logger.Logger + statsFactory stats.Stats + FileManager sftp.FileManager filePathPrefix string } diff --git a/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go b/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go index 9a9153bd02..dc2b6a27bb 100644 --- a/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go +++ b/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica.go @@ -23,6 +23,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" @@ -86,11 +87,12 @@ func getID(id interface{}, headerName string) (idStruct, error) { type YandexMetricaBulkUploader struct { logger logger.Logger + statsFactory stats.Stats Client *http.Client destinationInfo *oauthv2.DestinationInfo } -func NewManager(destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (*YandexMetricaBulkUploader, error) { +func NewManager(logger logger.Logger, statsFactory stats.Stats, destination *backendconfig.DestinationT, backendConfig backendconfig.BackendConfig) (*YandexMetricaBulkUploader, error) { destinationInfo := &oauthv2.DestinationInfo{ Config: destination.Config, DefinitionConfig: destination.DestinationDefinition.Config, @@ -100,7 +102,8 @@ func NewManager(destination *backendconfig.DestinationT, backendConfig backendco } yandexUploadManager := &YandexMetricaBulkUploader{ destinationInfo: destinationInfo, - logger: logger.NewLogger().Child("batchRouter").Child("AsyncDestinationManager").Child("YandexMetrica").Child("YandexMetricaBulkUploader"), + logger: logger.Child("YandexMetrica").Child("YandexMetricaBulkUploader"), + statsFactory: statsFactory, } cache := oauthv2.NewCache() optionalArgs := &oauthv2httpclient.HttpClientOptionalArgs{ @@ -259,7 +262,7 @@ func (ym *YandexMetricaBulkUploader) uploadFileToDestination(uploadURL, csvFileP } func (ym *YandexMetricaBulkUploader) generateErrorOutput(errorString string, err error, importingJobIds []int64) common.AsyncUploadOutput { - eventsAbortedStat := stats.Default.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ + eventsAbortedStat := ym.statsFactory.NewTaggedStat("failed_job_count", stats.CountType, map[string]string{ "module": "batch_router", "destType": ym.destinationInfo.DefinitionName, }) @@ -330,11 +333,11 @@ func (ym *YandexMetricaBulkUploader) Upload(asyncDestStruct *common.AsyncDestina return ym.generateErrorOutput("joining uploadUrl with counterId", err, importingJobIDs) } - uploadTimeStat := stats.Default.NewTaggedStat("async_upload_time", stats.TimerType, statLabels) + uploadTimeStat := ym.statsFactory.NewTaggedStat("async_upload_time", stats.TimerType, statLabels) - payloadSizeStat := stats.Default.NewTaggedStat("payload_size", stats.HistogramType, statLabels) + payloadSizeStat := ym.statsFactory.NewTaggedStat("payload_size", stats.HistogramType, statLabels) - eventsSuccessStat := stats.Default.NewTaggedStat("success_job_count", stats.CountType, statLabels) + eventsSuccessStat := ym.statsFactory.NewTaggedStat("success_job_count", stats.CountType, statLabels) payloadSizeStat.Observe(float64(len(ympayload))) ym.logger.Debugf("[Async Destination Manager] File Upload Started for Dest Type %v\n", destType) diff --git a/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica_test.go b/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica_test.go index 07ae04f714..98ac2aa6de 100644 --- a/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica_test.go +++ b/router/batchrouter/asyncdestinationmanager/yandexmetrica/yandexmetrica_test.go @@ -14,6 +14,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" kitsync "github.com/rudderlabs/rudder-go-kit/sync" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" mockoauthv2 "github.com/rudderlabs/rudder-server/mocks/services/oauthV2" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager/common" @@ -49,7 +50,7 @@ var ( var _ = Describe("Antisymmetric", func() { Describe("NewManager function test", func() { It("should return yandexmetrica manager", func() { - yandexmetrica, err := yandexmetrica.NewManager(destination, backendconfig.DefaultBackendConfig) + yandexmetrica, err := yandexmetrica.NewManager(logger.NOP, stats.NOP, destination, backendconfig.DefaultBackendConfig) Expect(err).To(BeNil()) Expect(yandexmetrica).NotTo(BeNil()) }) @@ -73,8 +74,8 @@ var _ = Describe("Antisymmetric", func() { oauthHandler := oauthv2.NewOAuthHandler(mockTokenProvider, oauthv2.WithCache(oauthv2.NewCache()), oauthv2.WithLocker(kitsync.NewPartitionRWLocker()), - oauthv2.WithStats(stats.Default), - oauthv2.WithLogger(logger.NewLogger().Child("MockOAuthHandler")), + oauthv2.WithStats(stats.NOP), + oauthv2.WithLogger(logger.NOP), oauthv2.WithCpConnector(mockCpConnector), ) optionalArgs := httpClient.HttpClientOptionalArgs{ @@ -83,7 +84,7 @@ var _ = Describe("Antisymmetric", func() { OAuthHandler: oauthHandler, } httpClient := httpClient.NewOAuthHttpClient(&http.Client{}, oauthv2common.RudderFlowDelivery, &cache, backendconfig.DefaultBackendConfig, augmenter.GetAuthErrorCategoryForYandex, &optionalArgs) - yandexmetrica, _ := yandexmetrica.NewManager(destination, backendconfig.DefaultBackendConfig) + yandexmetrica, _ := yandexmetrica.NewManager(logger.NOP, stats.NOP, destination, backendconfig.DefaultBackendConfig) yandexmetrica.Client = httpClient asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1, 2, 3, 4}, @@ -115,8 +116,8 @@ var _ = Describe("Antisymmetric", func() { oauthHandler := oauthv2.NewOAuthHandler(mockTokenProvider, oauthv2.WithCache(oauthv2.NewCache()), oauthv2.WithLocker(kitsync.NewPartitionRWLocker()), - oauthv2.WithStats(stats.Default), - oauthv2.WithLogger(logger.NewLogger().Child("MockOAuthHandler")), + oauthv2.WithStats(stats.NOP), + oauthv2.WithLogger(logger.NOP), oauthv2.WithCpConnector(mockCpConnector), ) optionalArgs := httpClient.HttpClientOptionalArgs{ @@ -125,7 +126,7 @@ var _ = Describe("Antisymmetric", func() { OAuthHandler: oauthHandler, } httpClient := httpClient.NewOAuthHttpClient(&http.Client{}, oauthv2common.RudderFlowDelivery, &cache, backendconfig.DefaultBackendConfig, augmenter.GetAuthErrorCategoryForYandex, &optionalArgs) - yandexmetrica, _ := yandexmetrica.NewManager(destination, backendconfig.DefaultBackendConfig) + yandexmetrica, _ := yandexmetrica.NewManager(logger.NOP, stats.NOP, destination, backendconfig.DefaultBackendConfig) yandexmetrica.Client = httpClient asyncDestination := common.AsyncDestinationStruct{ ImportingJobIDs: []int64{1, 2, 3, 4}, diff --git a/router/batchrouter/handle_lifecycle.go b/router/batchrouter/handle_lifecycle.go index e0b507026f..06851978d7 100644 --- a/router/batchrouter/handle_lifecycle.go +++ b/router/batchrouter/handle_lifecycle.go @@ -24,6 +24,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" kitsync "github.com/rudderlabs/rudder-go-kit/sync" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/jobsdb" "github.com/rudderlabs/rudder-server/router/batchrouter/asyncdestinationmanager" @@ -261,7 +262,7 @@ func (brt *Handle) Shutdown() { func (brt *Handle) initAsyncDestinationStruct(destination *backendconfig.DestinationT) { _, ok := brt.asyncDestinationStruct[destination.ID] - manager, err := asyncdestinationmanager.NewManager(destination, brt.backendConfig) + manager, err := asyncdestinationmanager.NewManager(brt.conf, brt.logger.Child("asyncdestinationmanager"), stats.Default, destination, brt.backendConfig) if err != nil { brt.logger.Errorf("BRT: Error initializing async destination struct for %s destination: %v", destination.Name, err) destInitFailStat := stats.Default.NewTaggedStat("destination_initialization_fail", stats.CountType, map[string]string{ From f41b9d125d83368ff9603c54966eaab54e722592 Mon Sep 17 00:00:00 2001 From: Rohith BCS Date: Thu, 12 Sep 2024 14:28:29 +0530 Subject: [PATCH 23/29] chore: propogate source category to router (#5085) --- gateway/gateway_test.go | 2 +- gateway/handle.go | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/gateway/gateway_test.go b/gateway/gateway_test.go index dc4cce0cd3..09a1baa012 100644 --- a/gateway/gateway_test.go +++ b/gateway/gateway_test.go @@ -559,7 +559,7 @@ var _ = Describe("Gateway", func() { var paramsMap, expectedParamsMap map[string]interface{} _ = json.Unmarshal(job.Parameters, ¶msMap) expectedStr := []byte(fmt.Sprintf( - `{"source_id": "%v", "source_job_run_id": "", "source_task_run_id": "", "traceparent": ""}`, + `{"source_id": "%v", "source_job_run_id": "", "source_task_run_id": "","source_category": "webhook", "traceparent": ""}`, SourceIDEnabled, )) _ = json.Unmarshal(expectedStr, &expectedParamsMap) diff --git a/gateway/handle.go b/gateway/handle.go index 1495684b7d..42c75e3dd0 100644 --- a/gateway/handle.go +++ b/gateway/handle.go @@ -286,6 +286,7 @@ func (gw *Handle) getJobDataFromRequest(req *webRequestT) (jobData *jobFromReq, // values retrieved from first event in batch sourcesJobRunID, sourcesTaskRunID = req.authContext.SourceJobRunID, req.authContext.SourceTaskRunID + sourceCategory = req.authContext.SourceCategory // tracing traceParent = req.traceParent @@ -459,6 +460,7 @@ func (gw *Handle) getJobDataFromRequest(req *webRequestT) (jobData *jobFromReq, "source_job_run_id": sourcesJobRunID, "source_task_run_id": sourcesTaskRunID, "traceparent": traceParent, + "source_category": sourceCategory, } if len(destinationID) != 0 { params["destination_id"] = destinationID @@ -747,6 +749,7 @@ func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byt UserID string `json:"user_id"` TraceParent string `json:"traceparent"` DestinationID string `json:"destination_id,omitempty"` + SourceCategory string `json:"source_category"` } type singularEventBatch struct { @@ -808,6 +811,7 @@ func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byt UserID: msg.Properties.UserID, TraceParent: msg.Properties.TraceID, DestinationID: msg.Properties.DestinationID, + SourceCategory: msg.Properties.SourceType, } writeKey, ok := gw.getWriteKeyFromSourceID(msg.Properties.SourceID) From ec92f3120d2b4595af0530f16310f725a66248bf Mon Sep 17 00:00:00 2001 From: Satish Kumar <97025614+satishrudderstack@users.noreply.github.com> Date: Thu, 12 Sep 2024 17:31:57 +0530 Subject: [PATCH 24/29] fix: reporting from warehouse module for failing uploads (#5044) --- integration_test/warehouse/integration_test.go | 2 +- warehouse/router/upload.go | 11 +++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/integration_test/warehouse/integration_test.go b/integration_test/warehouse/integration_test.go index 6c4f90a594..2b64911686 100644 --- a/integration_test/warehouse/integration_test.go +++ b/integration_test/warehouse/integration_test.go @@ -660,7 +660,7 @@ func TestUploads(t *testing.T) { {A: "in_pu", B: "batch_router"}, {A: "pu", B: "warehouse"}, {A: "initial_state", B: false}, - {A: "terminal_state", B: true}, + {A: "terminal_state", B: false}, }...) requireReportsCount(t, ctx, db, events, []lo.Tuple2[string, any]{ {A: "source_id", B: sourceID}, diff --git a/warehouse/router/upload.go b/warehouse/router/upload.go index 683c096605..2c12ca1b71 100644 --- a/warehouse/router/upload.go +++ b/warehouse/router/upload.go @@ -680,8 +680,11 @@ func (job *UploadJob) setUploadError(statusError error, state string) (string, e failCount := inputCount - outputCount reportingStatus := jobsdb.Failed.State + isTerminalPU := false + if state == model.Aborted { reportingStatus = jobsdb.Aborted.State + isTerminalPU = true } reportingMetrics := []*types.PUReportedMetric{{ ConnectionDetails: types.ConnectionDetails{ @@ -694,7 +697,7 @@ func (job *UploadJob) setUploadError(statusError error, state string) (string, e PUDetails: types.PUDetails{ InPU: types.BATCH_ROUTER, PU: types.WAREHOUSE, - TerminalPU: true, + TerminalPU: isTerminalPU, }, StatusDetail: &types.StatusDetail{ Status: reportingStatus, @@ -716,12 +719,12 @@ func (job *UploadJob) setUploadError(statusError error, state string) (string, e PUDetails: types.PUDetails{ InPU: types.BATCH_ROUTER, PU: types.WAREHOUSE, - TerminalPU: true, + TerminalPU: isTerminalPU, }, StatusDetail: &types.StatusDetail{ Status: jobsdb.Succeeded.State, - StatusCode: 400, // TODO: Change this to error specific code - Count: failCount, + StatusCode: 200, // TODO: Change this to error specific code + Count: outputCount, SampleEvent: []byte("{}"), SampleResponse: string(serializedErr), }, From 63c403395cff4fbff21feef93f743f9401f42907 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 13 Sep 2024 18:16:12 +0530 Subject: [PATCH 25/29] chore(deps): bump the go-deps group across 1 directory with 4 updates (#5090) --- go.mod | 32 ++++++++++++++++-------------- go.sum | 62 ++++++++++++++++++++++++++++++---------------------------- 2 files changed, 49 insertions(+), 45 deletions(-) diff --git a/go.mod b/go.mod index 4746ddb719..05743cf345 100644 --- a/go.mod +++ b/go.mod @@ -78,12 +78,12 @@ require ( github.com/rudderlabs/analytics-go v3.3.3+incompatible github.com/rudderlabs/bing-ads-go-sdk v0.2.3 github.com/rudderlabs/compose-test v0.1.3 - github.com/rudderlabs/rudder-go-kit v0.41.1 + github.com/rudderlabs/rudder-go-kit v0.42.0 github.com/rudderlabs/rudder-observability-kit v0.0.3 - github.com/rudderlabs/rudder-schemas v0.5.1 + github.com/rudderlabs/rudder-schemas v0.5.2 github.com/rudderlabs/rudder-transformer/go v0.0.0-20240910055720-f77d2ab4125a github.com/rudderlabs/sql-tunnels v0.1.7 - github.com/rudderlabs/sqlconnect-go v1.9.0 + github.com/rudderlabs/sqlconnect-go v1.10.0 github.com/samber/lo v1.47.0 github.com/segmentio/go-hll v1.0.1 github.com/segmentio/kafka-go v0.4.47 @@ -110,12 +110,14 @@ require ( golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 - google.golang.org/api v0.196.0 + google.golang.org/api v0.197.0 google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 - google.golang.org/grpc v1.66.1 + google.golang.org/grpc v1.66.2 google.golang.org/protobuf v1.34.2 ) +require github.com/rudderlabs/goqu/v10 v10.3.0 // indirect + require ( cloud.google.com/go v0.115.1 // indirect cloud.google.com/go/auth v0.9.3 // indirect @@ -307,16 +309,16 @@ require ( go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.54.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0 // indirect - go.opentelemetry.io/otel v1.29.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.29.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.29.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0 // indirect - go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.29.0 // indirect - go.opentelemetry.io/otel/exporters/zipkin v1.29.0 // indirect - go.opentelemetry.io/otel/metric v1.29.0 // indirect - go.opentelemetry.io/otel/sdk v1.29.0 // indirect - go.opentelemetry.io/otel/sdk/metric v1.29.0 // indirect - go.opentelemetry.io/otel/trace v1.29.0 // indirect + go.opentelemetry.io/otel v1.30.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.30.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.30.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.30.0 // indirect + go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.30.0 // indirect + go.opentelemetry.io/otel/exporters/zipkin v1.30.0 // indirect + go.opentelemetry.io/otel/metric v1.30.0 // indirect + go.opentelemetry.io/otel/sdk v1.30.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.30.0 // indirect + go.opentelemetry.io/otel/trace v1.30.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect diff --git a/go.sum b/go.sum index aa7ce7a3f2..0a956a1274 100644 --- a/go.sum +++ b/go.sum @@ -1139,20 +1139,22 @@ github.com/rudderlabs/bing-ads-go-sdk v0.2.3 h1:jR85Ep6X6SkiesaI7Q7WJHs/65SgByZb github.com/rudderlabs/bing-ads-go-sdk v0.2.3/go.mod h1:38Yig/202ni4GcloXhaTeH1LqUyFPEx6iljnFa+IDQI= github.com/rudderlabs/compose-test v0.1.3 h1:uyep6jDCIF737sfv4zIaMsKRQKX95IDz5Xbxor+x0kU= github.com/rudderlabs/compose-test v0.1.3/go.mod h1:tuvS1eQdSfwOYv1qwyVAcpdJxPLQXJgy5xGDd/9XmMg= +github.com/rudderlabs/goqu/v10 v10.3.0 h1:FaZioS8fRYJVYoLO5lieuyiVvEHhmQ6jP5sKwPIcKSs= +github.com/rudderlabs/goqu/v10 v10.3.0/go.mod h1:LH2vI5gGHBxEQuESqFyk5ZA2anGINc8o25hbidDWOYw= github.com/rudderlabs/parquet-go v0.0.2 h1:ZXRdZdimB0PdJtmxeSSxfI0fDQ3kZjwzBxRi6Ut1J8k= github.com/rudderlabs/parquet-go v0.0.2/go.mod h1:g6guum7o8uhj/uNhunnt7bw5Vabu/goI5i21/3fnxWQ= -github.com/rudderlabs/rudder-go-kit v0.41.1 h1:rImVvHd1hnO4ht1eoVQ+Dgro2dJwuEYIqbH4gS6bBF8= -github.com/rudderlabs/rudder-go-kit v0.41.1/go.mod h1:+58f3uAIc3/6X3lGhv6G1yF8GmoK31l1sPw4eh30tpA= +github.com/rudderlabs/rudder-go-kit v0.42.0 h1:j84n8y+iQ7amCuZ5s5Lq+T6DTgSaQsxbKYocZx8ouZQ= +github.com/rudderlabs/rudder-go-kit v0.42.0/go.mod h1:CbDfW+wgQJ/A+5iqOWFv/Allo1SDNOgGOd1Y+uOuz0k= github.com/rudderlabs/rudder-observability-kit v0.0.3 h1:vZtuZRkGX+6rjaeKtxxFE2YYP6QlmAcVcgecTOjvz+Q= github.com/rudderlabs/rudder-observability-kit v0.0.3/go.mod h1:6UjAh3H6rkE0fFLh7z8ZGQEQbKtUkRfhWOf/OUhfqW8= -github.com/rudderlabs/rudder-schemas v0.5.1 h1:g4I5wp2yA6ZWQZ1MjSNn4zby3XctG/TOgbYUW3dS4z4= -github.com/rudderlabs/rudder-schemas v0.5.1/go.mod h1:JoDTB9nCDXwRz+G+aYwP3Fj42HLssKARxsFFm+qqgb4= +github.com/rudderlabs/rudder-schemas v0.5.2 h1:qmbQrd+/2au5X+04vX8Qj4BGL7aK+TedeMdcyQyrLtY= +github.com/rudderlabs/rudder-schemas v0.5.2/go.mod h1:iUpjG/Zb+ioZcNLvXNYXSKQ2LpPlsIDBfxfCDH9ue/E= github.com/rudderlabs/rudder-transformer/go v0.0.0-20240910055720-f77d2ab4125a h1:OZcvpApxEYNkB9UNXrKDUBufQ24Lsr2Cs0pw70tzXBw= github.com/rudderlabs/rudder-transformer/go v0.0.0-20240910055720-f77d2ab4125a/go.mod h1:3NGitPz4pYRRZ6Xt09S+8hb0tHK/9pZcKJ3OgOTaSmE= github.com/rudderlabs/sql-tunnels v0.1.7 h1:wDCRl6zY4M5gfWazf7XkSTGQS3yjBzUiUgEMBIfHNDA= github.com/rudderlabs/sql-tunnels v0.1.7/go.mod h1:5f7+YL49JHYgteP4rAgqKnr4K2OadB0oIpUS+Tt3sPM= -github.com/rudderlabs/sqlconnect-go v1.9.0 h1:icLgqvVQ15Vh+oP7epA0b0yK6sIzxRVwPlRzOoDNVRA= -github.com/rudderlabs/sqlconnect-go v1.9.0/go.mod h1:pKhrcp7ewf2FeEObiyLWTi8Ra/I+GVE/1I2mYcY0O1s= +github.com/rudderlabs/sqlconnect-go v1.10.0 h1:PbYUdr8vDnBWYzaXCC1cpaJMYi7ggMp44/utuMVfmIs= +github.com/rudderlabs/sqlconnect-go v1.10.0/go.mod h1:6vfDgv/fTUtJxX4zvNYtXt254HlOgDw/ybzBxkKWbdQ= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= @@ -1346,34 +1348,34 @@ go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0. go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0 h1:TT4fX+nBOA/+LUkobKGW1ydGcn+G3vRw9+g5HwCphpk= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.54.0/go.mod h1:L7UH0GbB0p47T4Rri3uHjbpCFYrVrwc1I25QhNPiGK8= go.opentelemetry.io/otel v0.14.0/go.mod h1:vH5xEuwy7Rts0GNtsCW3HYQoZDY+OmBJ6t1bFGGlxgw= -go.opentelemetry.io/otel v1.29.0 h1:PdomN/Al4q/lN6iBJEN3AwPvUiHPMlt93c8bqTG5Llw= -go.opentelemetry.io/otel v1.29.0/go.mod h1:N/WtXPs1CNCUEx+Agz5uouwCba+i+bJGFicT8SR4NP8= +go.opentelemetry.io/otel v1.30.0 h1:F2t8sK4qf1fAmY9ua4ohFS/K+FUuOPemHUIXHtktrts= +go.opentelemetry.io/otel v1.30.0/go.mod h1:tFw4Br9b7fOS+uEao81PJjVMjW/5fvNCbpsDIXqP0pc= go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.42.0 h1:ZtfnDL+tUrs1F0Pzfwbg2d59Gru9NCH3bgSHBM6LDwU= go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.42.0/go.mod h1:hG4Fj/y8TR/tlEDREo8tWstl9fO9gcFkn4xrx0Io8xU= -go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.29.0 h1:k6fQVDQexDE+3jG2SfCQjnHS7OamcP73YMoxEVq5B6k= -go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.29.0/go.mod h1:t4BrYLHU450Zo9fnydWlIuswB1bm7rM8havDpWOJeDo= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.30.0 h1:WypxHH02KX2poqqbaadmkMYalGyy/vil4HE4PM4nRJc= +go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v1.30.0/go.mod h1:U79SV99vtvGSEBeeHnpgGJfTsnsdkWLpPN/CcHAzBSI= go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.42.0 h1:wNMDy/LVGLj2h3p6zg4d0gypKfWKSWI14E1C4smOgl8= go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.42.0/go.mod h1:YfbDdXAAkemWJK3H/DshvlrxqFB2rtW4rY6ky/3x/H0= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.29.0 h1:dIIDULZJpgdiHz5tXrTgKIMLkus6jEFa7x5SOKcyR7E= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.29.0/go.mod h1:jlRVBe7+Z1wyxFSUs48L6OBQZ5JwH2Hg/Vbl+t9rAgI= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0 h1:nSiV3s7wiCam610XcLbYOmMfJxB9gO4uK3Xgv5gmTgg= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.29.0/go.mod h1:hKn/e/Nmd19/x1gvIHwtOwVWM+VhuITSWip3JUDghj0= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.30.0 h1:lsInsfvhVIfOI6qHVyysXMNDnjO9Npvl7tlDPJFBVd4= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.30.0/go.mod h1:KQsVNh4OjgjTG0G6EiNi1jVpnaeeKsKMRwbLN+f1+8M= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.30.0 h1:m0yTiGDLUvVYaTFbAvCkVYIYcvwKt3G7OLoN77NUs/8= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.30.0/go.mod h1:wBQbT4UekBfegL2nx0Xk1vBcnzyBPsIVm9hRG4fYcr4= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0 h1:digkEZCJWobwBqMwC0cwCq8/wkkRy/OowZg5OArWZrM= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0/go.mod h1:/OpE/y70qVkndM0TrxT4KBoN3RsFZP0QaofcfYrj76I= go.opentelemetry.io/otel/exporters/prometheus v0.42.0 h1:jwV9iQdvp38fxXi8ZC+lNpxjK16MRcZlpDYvbuO1FiA= go.opentelemetry.io/otel/exporters/prometheus v0.42.0/go.mod h1:f3bYiqNqhoPxkvI2LrXqQVC546K7BuRDL/kKuxkujhA= -go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.29.0 h1:X3ZjNp36/WlkSYx0ul2jw4PtbNEDDeLskw3VPsrpYM0= -go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.29.0/go.mod h1:2uL/xnOXh0CHOBFCWXz5u1A4GXLiW+0IQIzVbeOEQ0U= -go.opentelemetry.io/otel/exporters/zipkin v1.29.0 h1:rqaUJdM9ItWf6DGrelaShXnJpb8rd3HTbcZWptvcsWA= -go.opentelemetry.io/otel/exporters/zipkin v1.29.0/go.mod h1:wDIyU6DjrUYqUgnmzjWnh1HOQGZCJ6YXMIJCdMc+T9Y= -go.opentelemetry.io/otel/metric v1.29.0 h1:vPf/HFWTNkPu1aYeIsc98l4ktOQaL6LeSoeV2g+8YLc= -go.opentelemetry.io/otel/metric v1.29.0/go.mod h1:auu/QWieFVWx+DmQOUMgj0F8LHWdgalxXqvp7BII/W8= -go.opentelemetry.io/otel/sdk v1.29.0 h1:vkqKjk7gwhS8VaWb0POZKmIEDimRCMsopNYnriHyryo= -go.opentelemetry.io/otel/sdk v1.29.0/go.mod h1:pM8Dx5WKnvxLCb+8lG1PRNIDxu9g9b9g59Qr7hfAAok= -go.opentelemetry.io/otel/sdk/metric v1.29.0 h1:K2CfmJohnRgvZ9UAj2/FhIf/okdWcNdBwe1m8xFXiSY= -go.opentelemetry.io/otel/sdk/metric v1.29.0/go.mod h1:6zZLdCl2fkauYoZIOn/soQIDSWFmNSRcICarHfuhNJQ= -go.opentelemetry.io/otel/trace v1.29.0 h1:J/8ZNK4XgR7a21DZUAsbF8pZ5Jcw1VhACmnYt39JTi4= -go.opentelemetry.io/otel/trace v1.29.0/go.mod h1:eHl3w0sp3paPkYstJOmAimxhiFXPg+MMTlEh3nsQgWQ= +go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.30.0 h1:kn1BudCgwtE7PxLqcZkErpD8GKqLZ6BSzeW9QihQJeM= +go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.30.0/go.mod h1:ljkUDtAMdleoi9tIG1R6dJUpVwDcYjw3J2Q6Q/SuiC0= +go.opentelemetry.io/otel/exporters/zipkin v1.30.0 h1:1uYaSfxiCLdJATlGEtYjQe4jZYfqCjVwxeSTMXe8VF4= +go.opentelemetry.io/otel/exporters/zipkin v1.30.0/go.mod h1:r/4BhMc3kiKxD61wGh9J3NVQ3/cZ45F2NHkQgVnql48= +go.opentelemetry.io/otel/metric v1.30.0 h1:4xNulvn9gjzo4hjg+wzIKG7iNFEaBMX00Qd4QIZs7+w= +go.opentelemetry.io/otel/metric v1.30.0/go.mod h1:aXTfST94tswhWEb+5QjlSqG+cZlmyXy/u8jFpor3WqQ= +go.opentelemetry.io/otel/sdk v1.30.0 h1:cHdik6irO49R5IysVhdn8oaiR9m8XluDaJAs4DfOrYE= +go.opentelemetry.io/otel/sdk v1.30.0/go.mod h1:p14X4Ok8S+sygzblytT1nqG98QG2KYKv++HE0LY/mhg= +go.opentelemetry.io/otel/sdk/metric v1.30.0 h1:QJLT8Pe11jyHBHfSAgYH7kEmT24eX792jZO1bo4BXkM= +go.opentelemetry.io/otel/sdk/metric v1.30.0/go.mod h1:waS6P3YqFNzeP01kuo/MBBYqaoBJl7efRQHOaydhy1Y= +go.opentelemetry.io/otel/trace v1.30.0 h1:7UBkkYzeg3C7kQX8VAidWh2biiQbtAKjyIML8dQ9wmc= +go.opentelemetry.io/otel/trace v1.30.0/go.mod h1:5EyKqTzzmyqB9bwtCCq6pDLktPK6fmGf/Dph+8VI02o= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v1.3.1 h1:TrMUixzpM0yuc/znrFTP9MMRh8trP93mkCiDVeXrui0= go.opentelemetry.io/proto/otlp v1.3.1/go.mod h1:0X1WI4de4ZsLrrJNLAQbFeLCm3T7yBkR0XqQ7niQU+8= @@ -1739,8 +1741,8 @@ google.golang.org/api v0.69.0/go.mod h1:boanBiw+h5c3s+tBPgEzLDRHfFLWV0qXxRHz3ws7 google.golang.org/api v0.70.0/go.mod h1:Bs4ZM2HGifEvXwd50TtW70ovgJffJYw2oRCOFU/SkfA= google.golang.org/api v0.71.0/go.mod h1:4PyU6e6JogV1f9eA4voyrTY2batOLdgZ5qZ5HOCc4j8= google.golang.org/api v0.74.0/go.mod h1:ZpfMZOVRMywNyvJFeqL9HRWBgAuRfSjJFpe9QtRRyDs= -google.golang.org/api v0.196.0 h1:k/RafYqebaIJBO3+SMnfEGtFVlvp5vSgqTUF54UN/zg= -google.golang.org/api v0.196.0/go.mod h1:g9IL21uGkYgvQ5BZg6BAtoGJQIm8r6EgaAbpNey5wBE= +google.golang.org/api v0.197.0 h1:x6CwqQLsFiA5JKAiGyGBjc2bNtHtLddhJCE2IKuhhcQ= +google.golang.org/api v0.197.0/go.mod h1:AuOuo20GoQ331nq7DquGHlU6d+2wN2fZ8O0ta60nRNw= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1869,8 +1871,8 @@ google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9K google.golang.org/grpc v1.40.1/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= google.golang.org/grpc v1.44.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= -google.golang.org/grpc v1.66.1 h1:hO5qAXR19+/Z44hmvIM4dQFMSYX9XcWsByfoxutBpAM= -google.golang.org/grpc v1.66.1/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= +google.golang.org/grpc v1.66.2 h1:3QdXkuq3Bkh7w+ywLdLvM56cmGvQHUMZpiCzt6Rqaoo= +google.golang.org/grpc v1.66.2/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= From 40a5446fa269a7861773dbaa14c8acc123c8112e Mon Sep 17 00:00:00 2001 From: Sankeerth Date: Mon, 16 Sep 2024 14:10:50 +0530 Subject: [PATCH 26/29] fix: error reporting lag and pii handle (#5028) * fix: error detail reporting lag & pii handling - introduce a lag metric in error detail reporting - use ui configuration while storing sample event in error detail reports * chore: lint fixes * fix: address comments * chore: add test for testing piienable configuration in error_detail_reports table * chore: lint errors fix * chore: store last reported time when context has some error while fetching report data - store last reported time as the main loop start time when no rows are returned * chore: address comments --------- Co-authored-by: Sai Sankeerth --- enterprise/reporting/error_reporting.go | 189 ++++++++++++------- enterprise/reporting/error_reporting_test.go | 178 +++++++++++++++++ 2 files changed, 297 insertions(+), 70 deletions(-) diff --git a/enterprise/reporting/error_reporting.go b/enterprise/reporting/error_reporting.go index d713b24665..110641041d 100644 --- a/enterprise/reporting/error_reporting.go +++ b/enterprise/reporting/error_reporting.go @@ -5,6 +5,7 @@ import ( "context" "database/sql" "encoding/json" + "errors" "fmt" "io" "net/http" @@ -17,6 +18,7 @@ import ( "github.com/cenkalti/backoff/v4" "github.com/lib/pq" "github.com/samber/lo" + "go.uber.org/atomic" "golang.org/x/sync/errgroup" "github.com/rudderlabs/rudder-go-kit/bytesize" @@ -24,6 +26,7 @@ import ( "github.com/rudderlabs/rudder-go-kit/logger" "github.com/rudderlabs/rudder-go-kit/stats" + obskit "github.com/rudderlabs/rudder-observability-kit/go/labels" migrator "github.com/rudderlabs/rudder-server/services/sql-migrator" "github.com/rudderlabs/rudder-server/utils/httputil" . "github.com/rudderlabs/rudder-server/utils/tx" //nolint:staticcheck @@ -163,6 +166,22 @@ func (edr *ErrorDetailReporter) DatabaseSyncer(c types.SyncerConfig) types.Repor } } +func (edr *ErrorDetailReporter) emitLagMetric(ctx context.Context, c types.SyncerConfig, lastReportedAtTime *atomic.Time) error { + // for monitoring reports pileups + reportingLag := edr.stats.NewTaggedStat( + "error_detail_reports_metrics_lag_seconds", stats.GaugeType, stats.Tags{"client": c.Label}, + ) + for { + lag := time.Since(lastReportedAtTime.Load()) + reportingLag.Gauge(lag.Seconds()) + select { + case <-ctx.Done(): + return ctx.Err() + case <-time.After(2 * time.Minute): + } + } +} + func (edr *ErrorDetailReporter) GetSyncer(syncerKey string) *types.SyncSource { edr.syncersMu.RLock() defer edr.syncersMu.RUnlock() @@ -199,8 +218,12 @@ func (edr *ErrorDetailReporter) Report(ctx context.Context, metrics []*types.PUR workspaceID := edr.configSubscriber.WorkspaceIDFromSource(metric.ConnectionDetails.SourceID) metric := *metric + if edr.IsPIIReportingDisabled(workspaceID) { + edr.log.Debugn("PII setting is disabled for workspaceId:", obskit.WorkspaceID(workspaceID)) + return nil + } destinationDetail := edr.configSubscriber.GetDestDetail(metric.ConnectionDetails.DestinationID) - edr.log.Debugf("For DestId: %v -> DestDetail: %v", metric.ConnectionDetails.DestinationID, destinationDetail) + edr.log.Debugn("DestinationId & DestDetail details", obskit.DestinationID(metric.ConnectionDetails.DestinationID), logger.NewField("destinationDetail", destinationDetail)) // extract error-message & error-code errDets := edr.extractErrorDetails(metric.StatusDetail.SampleResponse) @@ -241,16 +264,15 @@ func (edr *ErrorDetailReporter) Report(ctx context.Context, metrics []*types.PUR _, err = stmt.ExecContext(ctx) if err != nil { - edr.log.Errorf("Failed during statement preparation: %v", err) + edr.log.Errorf("Failed during statement execution: %v", err) return fmt.Errorf("executing final statement: %v", err) } return nil } -func (*ErrorDetailReporter) IsPIIReportingDisabled(_ string) bool { - // Since we don't see the necessity for error detail reporting, we are implementing a kind of NOOP method - return false +func (ed *ErrorDetailReporter) IsPIIReportingDisabled(workspaceID string) bool { + return ed.configSubscriber.IsPIIReportingDisabled(workspaceID) } func (edr *ErrorDetailReporter) migrate(c types.SyncerConfig) (*sql.DB, error) { @@ -317,82 +339,109 @@ func (edr *ErrorDetailReporter) mainLoop(ctx context.Context, c types.SyncerConf edr.errorDetailReportsQueryTime = edr.stats.NewTaggedStat("error_detail_reports_query_time", stats.TimerType, tags) edr.edReportingRequestLatency = edr.stats.NewTaggedStat("error_detail_reporting_request_latency", stats.TimerType, tags) - // In infinite loop - // Get Reports - // Aggregate - // Send in a separate go-routine - // Delete in a separate go-routine - for { - if ctx.Err() != nil { - edr.log.Infof("stopping mainLoop for syncer %s : %s", c.Label, ctx.Err()) - return - } - requestChan := make(chan struct{}, edr.maxConcurrentRequests.Load()) - loopStart := time.Now() - currentMs := time.Now().UTC().Unix() / 60 + var lastReportedAtTime atomic.Time + lastReportedAtTime.Store(time.Now()) - getReportsStart := time.Now() - reports, reportedAt := edr.getReports(ctx, currentMs, c.ConnInfo) - getReportsTimer.Since(getReportsStart) - getReportsSize.Observe(float64(len(reports))) + g, ctx := errgroup.WithContext(ctx) - if len(reports) == 0 { - select { - case <-ctx.Done(): + g.Go(func() error { + return edr.emitLagMetric(ctx, c, &lastReportedAtTime) + }) + + g.Go(func() error { + // In infinite loop + // Get Reports + // Aggregate + // Send in a separate go-routine + // Delete in a separate go-routine + for { + if ctx.Err() != nil { edr.log.Infof("stopping mainLoop for syncer %s : %s", c.Label, ctx.Err()) - return - case <-time.After(edr.sleepInterval.Load()): + return ctx.Err() } - continue - } - - aggregationStart := time.Now() - metrics := edr.aggregate(reports) - aggregateTimer.Since(aggregationStart) - getAggregatedReportsSize.Observe(float64(len(metrics))) - - errGroup, errCtx := errgroup.WithContext(ctx) - for _, metric := range metrics { - metricToSend := metric - requestChan <- struct{}{} - if errCtx.Err() != nil { - // if any of errGroup's goroutines fail - don't send anymore requests for this batch - break + requestChan := make(chan struct{}, edr.maxConcurrentRequests.Load()) + loopStart := time.Now() + currentMs := time.Now().UTC().Unix() / 60 + + getReportsStart := time.Now() + reports, reportedAt := edr.getReports(ctx, currentMs, c.ConnInfo) + if ctx.Err() != nil { + edr.log.Errorw("getting reports", "error", ctx.Err()) + select { + case <-ctx.Done(): + edr.log.Infof("stopping mainLoop for syncer %s : %s", c.Label, ctx.Err()) + return ctx.Err() + case <-time.After(edr.mainLoopSleepInterval.Load()): + } + continue } - errGroup.Go(func() error { - err := edr.sendMetric(errCtx, c.Label, metricToSend) - if err != nil { - edr.log.Error("Error while sending to Reporting service:", err) + getReportsTimer.Since(getReportsStart) + getReportsSize.Observe(float64(len(reports))) + + if len(reports) == 0 { + lastReportedAtTime.Store(loopStart) + select { + case <-ctx.Done(): + edr.log.Infof("stopping mainLoop for syncer %s : %s", c.Label, ctx.Err()) + return ctx.Err() + case <-time.After(edr.sleepInterval.Load()): } - <-requestChan - return err - }) - } - - err := errGroup.Wait() - if err == nil { - // sqlStatement := fmt.Sprintf(`DELETE FROM %s WHERE reported_at = %d`, ErrorDetailReportsTable, reportedAt) - dbHandle, err := edr.getDBHandle(c.ConnInfo) - if err != nil { - edr.log.Errorf("error reports deletion getDbhandle failed: %v", err) continue } - deleteReportsStart := time.Now() - _, err = dbHandle.ExecContext(ctx, `DELETE FROM `+ErrorDetailReportsTable+` WHERE reported_at = $1`, reportedAt) - errorDetailReportsDeleteQueryTimer.Since(deleteReportsStart) - if err != nil { - edr.log.Errorf("[ Error Detail Reporting ]: Error deleting local reports from %s: %v", ErrorDetailReportsTable, err) + lastReportedAtTime.Store(time.Unix(reportedAt*60, 0)) + + aggregationStart := time.Now() + metrics := edr.aggregate(reports) + aggregateTimer.Since(aggregationStart) + getAggregatedReportsSize.Observe(float64(len(metrics))) + + errGroup, errCtx := errgroup.WithContext(ctx) + for _, metric := range metrics { + metricToSend := metric + requestChan <- struct{}{} + if errCtx.Err() != nil { + // if any of errGroup's goroutines fail - don't send anymore requests for this batch + break + } + errGroup.Go(func() error { + err := edr.sendMetric(errCtx, c.Label, metricToSend) + if err != nil { + edr.log.Error("Error while sending to Reporting service:", err) + } + <-requestChan + return err + }) } - // vacuum error_reports_details table - edr.vacuum(ctx, dbHandle, c) - } - mainLoopTimer.Since(loopStart) - select { - case <-ctx.Done(): - return - case <-time.After(edr.mainLoopSleepInterval.Load()): + err := errGroup.Wait() + if err == nil { + // sqlStatement := fmt.Sprintf(`DELETE FROM %s WHERE reported_at = %d`, ErrorDetailReportsTable, reportedAt) + dbHandle, err := edr.getDBHandle(c.ConnInfo) + if err != nil { + edr.log.Errorf("error reports deletion getDbhandle failed: %v", err) + continue + } + deleteReportsStart := time.Now() + _, err = dbHandle.ExecContext(ctx, `DELETE FROM `+ErrorDetailReportsTable+` WHERE reported_at = $1`, reportedAt) + errorDetailReportsDeleteQueryTimer.Since(deleteReportsStart) + if err != nil { + edr.log.Errorf("[ Error Detail Reporting ]: Error deleting local reports from %s: %v", ErrorDetailReportsTable, err) + } + // vacuum error_reports_details table + edr.vacuum(ctx, dbHandle, c) + } + + mainLoopTimer.Since(loopStart) + select { + case <-ctx.Done(): + return ctx.Err() + case <-time.After(edr.mainLoopSleepInterval.Load()): + } } + }) + err := g.Wait() + if err != nil && !errors.Is(err, context.Canceled) { + panic(err) } } diff --git a/enterprise/reporting/error_reporting_test.go b/enterprise/reporting/error_reporting_test.go index 0bb29004e5..58c0c68f1e 100644 --- a/enterprise/reporting/error_reporting_test.go +++ b/enterprise/reporting/error_reporting_test.go @@ -1,12 +1,26 @@ package reporting import ( + "context" "net/http" "testing" + "github.com/DATA-DOG/go-sqlmock" . "github.com/onsi/gomega" + "go.uber.org/mock/gomock" + backendconfig "github.com/rudderlabs/rudder-server/backend-config" + mocksBackendConfig "github.com/rudderlabs/rudder-server/mocks/backend-config" + + "github.com/rudderlabs/rudder-go-kit/config" + "github.com/rudderlabs/rudder-go-kit/logger" + "github.com/rudderlabs/rudder-go-kit/stats" + "github.com/rudderlabs/rudder-server/utils/pubsub" "github.com/rudderlabs/rudder-server/utils/types" + + "github.com/stretchr/testify/assert" + + utilsTx "github.com/rudderlabs/rudder-server/utils/tx" ) func TestShouldReport(t *testing.T) { @@ -44,3 +58,167 @@ func TestShouldReport(t *testing.T) { } Expect(shouldReport(metric4)).To(BeFalse()) } + +func TestErrorDetailReporter_Report(t *testing.T) { + db, dbMock, err := sqlmock.New() + if err != nil { + t.Fatalf("an error '%s' was not expected when opening a stub database connection", err) + } + defer db.Close() + + dbMock.ExpectBegin() + defer dbMock.ExpectClose() + + tx, _ := db.Begin() + mockTx := &utilsTx.Tx{Tx: tx} + + tests := []struct { + name string + metrics []*types.PUReportedMetric + expectExecution bool + }{ + { + name: "PII Reporting Enabled, should report it to error_detail_reports table", + metrics: []*types.PUReportedMetric{ + { + ConnectionDetails: types.ConnectionDetails{ + SourceID: "source1", + DestinationID: "dest1", + }, + StatusDetail: &types.StatusDetail{ + StatusCode: 400, + Count: 1, + }, + }, + }, + expectExecution: true, + }, + { + name: "PII Reporting Disabled, should not report it to error_detail_reports table", + metrics: []*types.PUReportedMetric{ + { + ConnectionDetails: types.ConnectionDetails{ + SourceID: "source2", + DestinationID: "dest2", + }, + StatusDetail: &types.StatusDetail{ + StatusCode: 400, + Count: 1, + }, + }, + }, + expectExecution: false, + }, + } + configSubscriber := newConfigSubscriber(logger.NOP) + mockCtrl := gomock.NewController(t) + mockBackendConfig := mocksBackendConfig.NewMockBackendConfig(mockCtrl) + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + mockBackendConfig.EXPECT().Subscribe(gomock.Any(), gomock.Any()).DoAndReturn(func(ctx context.Context, topic backendconfig.Topic) pubsub.DataChannel { + ch := make(chan pubsub.DataEvent, 1) + ch <- pubsub.DataEvent{ + Data: map[string]backendconfig.ConfigT{ + "workspace1": { + WorkspaceID: "workspace1", + Sources: []backendconfig.SourceT{ + { + ID: "source1", + Enabled: true, + Destinations: []backendconfig.DestinationT{ + { + ID: "dest1", + Enabled: true, + DestinationDefinition: backendconfig.DestinationDefinitionT{ + ID: "destDef1", + Name: "destType", + }, + }, + }, + }, + }, + Settings: backendconfig.Settings{ + DataRetention: backendconfig.DataRetention{ + DisableReportingPII: false, + }, + }, + }, + "workspace2": { + WorkspaceID: "workspace2", + Sources: []backendconfig.SourceT{ + { + ID: "source2", + Enabled: true, + Destinations: []backendconfig.DestinationT{ + { + ID: "dest2", + Enabled: true, + DestinationDefinition: backendconfig.DestinationDefinitionT{ + ID: "destDef1", + Name: "destType", + }, + }, + }, + }, + }, + Settings: backendconfig.Settings{ + DataRetention: backendconfig.DataRetention{ + DisableReportingPII: true, + }, + }, + }, + }, + Topic: string(backendconfig.TopicBackendConfig), + } + close(ch) + return ch + }).AnyTimes() + + configSubscriber.Subscribe(context.TODO(), mockBackendConfig) + + edr := NewErrorDetailReporter( + context.TODO(), + configSubscriber, + stats.NOP, + config.New(), + ) + + ctx := context.Background() + + copyStmt := dbMock.ExpectPrepare(`COPY "error_detail_reports" \("workspace_id", "namespace", "instance_id", "source_definition_id", "source_id", "destination_definition_id", "destination_id", "dest_type", "pu", "reported_at", "count", "status_code", "event_type", "error_code", "error_message", "sample_response", "sample_event", "event_name"\) FROM STDIN`) + var tableRow int64 = 0 + if tt.expectExecution { + for _, metric := range tt.metrics { + copyStmt.ExpectExec().WithArgs( + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + metric.ConnectionDetails.SourceID, + sqlmock.AnyArg(), + metric.ConnectionDetails.DestinationID, + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + metric.StatusDetail.Count, + metric.StatusDetail.StatusCode, + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + sqlmock.AnyArg(), + ).WillReturnResult(sqlmock.NewResult(tableRow, 1)) + tableRow++ + } + copyStmt.ExpectExec().WithoutArgs().WillReturnResult(sqlmock.NewResult(tableRow, 1)) // ExecContext + } + copyStmt.WillBeClosed() + err := edr.Report(ctx, tt.metrics, mockTx) + assert.NoError(t, err) + }) + err = dbMock.ExpectationsWereMet() + assert.NoError(t, err) + } +} From bbaaabd03a7aef6e26227adbdfdba457a71fdd39 Mon Sep 17 00:00:00 2001 From: "siddarth.msv" <82795818+Sidddddarth@users.noreply.github.com> Date: Mon, 16 Sep 2024 15:01:39 +0530 Subject: [PATCH 27/29] fix: gateway internal batch endpoint stats (#5089) --- gateway/gateway_test.go | 244 +++++++++++++++++++++++++++++++- gateway/handle.go | 137 ++++++++++-------- gateway/handle_observability.go | 10 -- 3 files changed, 314 insertions(+), 77 deletions(-) diff --git a/gateway/gateway_test.go b/gateway/gateway_test.go index 09a1baa012..d837ae8ac7 100644 --- a/gateway/gateway_test.go +++ b/gateway/gateway_test.go @@ -40,6 +40,7 @@ import ( "github.com/rudderlabs/rudder-server/app" backendconfig "github.com/rudderlabs/rudder-server/backend-config" "github.com/rudderlabs/rudder-server/enterprise/suppress-user/model" + gwstats "github.com/rudderlabs/rudder-server/gateway/internal/stats" gwtypes "github.com/rudderlabs/rudder-server/gateway/internal/types" "github.com/rudderlabs/rudder-server/gateway/response" webhookModel "github.com/rudderlabs/rudder-server/gateway/webhook/model" @@ -1837,6 +1838,96 @@ var _ = Describe("Gateway", func() { }, }, })) + Expect(statStore.GetByName("gateway.write_key_events")).To(Equal([]memstats.Metric{ + { + Name: "gateway.write_key_events", + Tags: map[string]string{ + "source": "", + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + }, + Value: 1, + }, + })) + Expect(statStore.GetByName("gateway.write_key_successful_events")).To(Equal([]memstats.Metric{ + { + Name: "gateway.write_key_successful_events", + Tags: map[string]string{ + "source": "", + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + }, + Value: 1, + }, + })) + Expect(statStore.GetByName("gateway.write_key_requests")).To(Equal([]memstats.Metric{ + { + Name: "gateway.write_key_requests", + Tags: map[string]string{ + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + }, + Value: 1, + }, + })) + Expect(statStore.GetByName("gateway.write_key_successful_requests")).To(Equal([]memstats.Metric{ + { + Name: "gateway.write_key_successful_requests", + Tags: map[string]string{ + "source": "", + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + }, + Value: 1, + }, + })) + Expect(statStore.GetByName("gateway.write_key_failed_requests")).To(Equal([]memstats.Metric{ + { + Name: "gateway.write_key_failed_requests", + Tags: map[string]string{ + "source": "", + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + }, + Value: 0, + }, + })) + Expect(statStore.GetByName("gateway.write_key_failed_events")).To(Equal([]memstats.Metric{ + { + Name: "gateway.write_key_failed_events", + Tags: map[string]string{ + "source": "", + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + }, + Value: 0, + }, + })) }) It("Successful request, without debugger", func() { @@ -1859,6 +1950,18 @@ var _ = Describe("Gateway", func() { defer httputil.CloseResponse(resp) Expect(err).To(BeNil()) Expect(string(respData)).Should(ContainSubstring(response.NotRudderEvent)) + failedRequestStat := statStore.Get("gateway.write_key_failed_requests", map[string]string{ + "writeKey": "", + "reqType": "internalBatch", + "reason": response.NotRudderEvent, + "workspaceId": "", + "sourceID": "", + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(failedRequestStat).To(Not(BeNil())) + Expect(failedRequestStat.Values()).To(Equal([]float64{1})) }) It("request failed unmarshall error", func() { @@ -1871,6 +1974,29 @@ var _ = Describe("Gateway", func() { defer httputil.CloseResponse(resp) Expect(err).To(BeNil()) Expect(string(respData)).Should(ContainSubstring(response.InvalidJSON)) + failedRequestStat := statStore.Get("gateway.write_key_failed_requests", map[string]string{ + "writeKey": "", + "reqType": "internalBatch", + "reason": response.InvalidJSON, + "workspaceId": "", + "sourceID": "", + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(failedRequestStat).To(Not(BeNil())) + Expect(failedRequestStat.Values()).To(Equal([]float64{1})) + failedEventStat := statStore.Get("gateway.write_key_failed_events", map[string]string{ + "writeKey": "", + "reqType": "internalBatch", + "reason": response.InvalidJSON, + "workspaceId": "", + "sourceID": "", + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(failedEventStat).To(BeNil()) }) It("request failed message validation error", func() { @@ -1883,6 +2009,18 @@ var _ = Describe("Gateway", func() { defer httputil.CloseResponse(resp) Expect(err).To(BeNil()) Expect(string(respData)).Should(ContainSubstring(response.InvalidStreamMessage)) + failedRequestStat := statStore.Get("gateway.write_key_failed_requests", map[string]string{ + "writeKey": "", + "reqType": "internalBatch", + "reason": response.InvalidStreamMessage, + "workspaceId": "", + "sourceID": "", + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(failedRequestStat).To(Not(BeNil())) + Expect(failedRequestStat.Values()).To(Equal([]float64{1})) }) It("request success - suppressed user", func() { @@ -1892,6 +2030,17 @@ var _ = Describe("Gateway", func() { resp, err := client.Do(req) Expect(err).To(BeNil()) Expect(http.StatusOK, resp.StatusCode) + successfulReqStat := statStore.Get("gateway.write_key_successful_requests", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(successfulReqStat).To(Not(BeNil())) + Expect(successfulReqStat.Values()).To(Equal([]float64{1})) }) It("request success - multiple messages", func() { @@ -1903,6 +2052,39 @@ var _ = Describe("Gateway", func() { resp, err := client.Do(req) Expect(err).To(BeNil()) Expect(http.StatusOK, resp.StatusCode) + successfulReqStat := statStore.Get("gateway.write_key_successful_requests", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(successfulReqStat).To(Not(BeNil())) + Expect(successfulReqStat.LastValue()).To(Equal(float64(3))) + successfulEventStat := statStore.Get("gateway.write_key_successful_events", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(successfulEventStat).To(Not(BeNil())) + Expect(successfulEventStat.LastValue()).To(Equal(float64(3))) + eventsStat := statStore.Get("gateway.write_key_events", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(eventsStat).To(Not(BeNil())) + Expect(eventsStat.Values()).To(Equal([]float64{1, 2, 3})) }) It("request failed db error", func() { @@ -1913,6 +2095,41 @@ var _ = Describe("Gateway", func() { resp, err := client.Do(req) Expect(err).To(BeNil()) Expect(http.StatusInternalServerError, resp.StatusCode) + failedReqStat := statStore.Get("gateway.write_key_failed_requests", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + "reason": "storeFailed", + }) + Expect(failedReqStat).To(Not(BeNil())) + Expect(failedReqStat.Values()).To(Equal([]float64{1})) + failedEventStat := statStore.Get("gateway.write_key_failed_events", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + "reason": "storeFailed", + }) + Expect(failedEventStat).To(Not(BeNil())) + Expect(failedEventStat.Values()).To(Equal([]float64{1})) + eventsStat := statStore.Get("gateway.write_key_events", map[string]string{ + "writeKey": WriteKeyEnabled, + "reqType": "internalBatch", + "workspaceId": WorkspaceID, + "sourceID": SourceIDEnabled, + "sourceType": "", + "sdkVersion": "", + "source": "", + }) + Expect(eventsStat).To(Not(BeNil())) + Expect(eventsStat.Values()).To(Equal([]float64{1, 2, 3, 4})) }) }) @@ -1955,8 +2172,16 @@ var _ = Describe("Gateway", func() { done: make(chan<- string), requestPayload: payload, } - jobForm, err := gateway.extractJobsFromInternalBatchPayload("batch", req.requestPayload) - Expect(err).To(BeNil()) + jobsWithStats, err := gateway.extractJobsFromInternalBatchPayload("batch", req.requestPayload) + Expect(err).To(BeNil()) + Expect(jobsWithStats).To(HaveLen(1)) + Expect(jobsWithStats[0].stat).To(Equal( + gwstats.SourceStat{ + SourceID: "sourceID", + WorkspaceID: "workspaceID", + ReqType: "batch", + }, + )) var job struct { Batch []struct { @@ -1964,8 +2189,8 @@ var _ = Describe("Gateway", func() { RequestIP string `json:"request_ip"` } `json:"batch"` } - Expect(jobForm).To(HaveLen(1)) - err = json.Unmarshal(jobForm[0].EventPayload, &job) + jobForm := jobsWithStats[0].job + err = json.Unmarshal(jobForm.EventPayload, &job) Expect(err).To(BeNil()) Expect(job.Batch).To(HaveLen(1)) Expect(job.Batch[0].ReceivedAt).To(ContainSubstring("dummyReceivedAtFromPayload")) @@ -1995,8 +2220,14 @@ var _ = Describe("Gateway", func() { done: make(chan<- string), requestPayload: payload, } - jobForm, err := gateway.extractJobsFromInternalBatchPayload("batch", req.requestPayload) + jobsWithStats, err := gateway.extractJobsFromInternalBatchPayload("batch", req.requestPayload) Expect(err).To(BeNil()) + Expect(jobsWithStats).To(HaveLen(1)) + Expect(jobsWithStats[0].stat).To(Equal(gwstats.SourceStat{ + SourceID: "sourceID", + WorkspaceID: "workspaceID", + ReqType: "batch", + })) var job struct { Batch []struct { @@ -2004,8 +2235,7 @@ var _ = Describe("Gateway", func() { RequestIP string `json:"request_ip"` } `json:"batch"` } - Expect(jobForm).To(HaveLen(1)) - err = json.Unmarshal(jobForm[0].EventPayload, &job) + err = json.Unmarshal(jobsWithStats[0].job.EventPayload, &job) Expect(err).To(BeNil()) Expect(job.Batch).To(HaveLen(1)) Expect(job.Batch[0].ReceivedAt).To(ContainSubstring("2024-01-01T01:01:01.000Z")) diff --git a/gateway/handle.go b/gateway/handle.go index 42c75e3dd0..be2e04d712 100644 --- a/gateway/handle.go +++ b/gateway/handle.go @@ -646,65 +646,58 @@ func (gw *Handle) addToWebRequestQ(_ *http.ResponseWriter, req *http.Request, do func (gw *Handle) internalBatchHandlerFunc() http.HandlerFunc { return func(w http.ResponseWriter, r *http.Request) { var ( - ctx = r.Context() - reqType = ctx.Value(gwtypes.CtxParamCallType).(string) - jobs []*jobsdb.JobT - body []byte - err error - status int - errorMessage string - responseBody string + ctx = r.Context() + reqType = ctx.Value(gwtypes.CtxParamCallType).(string) + jobsWithStats []jobWithStat + body []byte + err error + status int + errorMessage string + responseBody string + stat = gwstats.SourceStat{ReqType: reqType} ) // TODO: add tracing gw.logger.LogRequest(r) body, err = gw.getPayloadFromRequest(r) if err != nil { - stat := gwstats.SourceStat{ - ReqType: reqType, - } stat.RequestFailed("requestBodyReadFailed") stat.Report(gw.stats) goto requestError } - jobs, err = gw.extractJobsFromInternalBatchPayload(reqType, body) + jobsWithStats, err = gw.extractJobsFromInternalBatchPayload(reqType, body) if err != nil { goto requestError } - if len(jobs) > 0 { + if len(jobsWithStats) > 0 { + jobs := lo.Map(jobsWithStats, func(jws jobWithStat, _ int) *jobsdb.JobT { + return jws.job + }) if err = gw.storeJobs(ctx, jobs); err != nil { - gw.stats.NewTaggedStat( - "gateway.write_key_failed_events", - stats.CountType, - gw.newReqTypeStatsTagsWithReason(reqType, "storeFailed"), - ).Count(len(jobs)) + for _, jws := range jobsWithStats { + jws.stat.RequestEventsFailed(1, "storeFailed") + jws.stat.Report(gw.stats) + } goto requestError } - gw.stats.NewTaggedStat( - "gateway.write_key_successful_events", - stats.CountType, - gw.newReqTypeStatsTagsWithReason(reqType, ""), - ).Count(len(jobs)) - - // Sending events to config backend - for _, job := range jobs { - writeKey := gjson.GetBytes(job.EventPayload, "writeKey").String() - if writeKey == "" { + for _, jws := range jobsWithStats { + jws.stat.RequestEventsSucceeded(1) + jws.stat.Report(gw.stats) + // Sending events to config backend + if jws.stat.WriteKey == "" { gw.logger.Errorn("writeKey not found in event payload") continue } - gw.sourcehandle.RecordEvent(writeKey, job.EventPayload) + gw.sourcehandle.RecordEvent(jws.stat.WriteKey, jws.job.EventPayload) } + } else { + stat.RequestEventsSucceeded(0) + stat.Report(gw.stats) } status = http.StatusOK responseBody = response.GetStatus(response.Ok) - gw.stats.NewTaggedStat( - "gateway.write_key_successful_requests", - stats.CountType, - gw.newReqTypeStatsTagsWithReason(reqType, ""), - ).Increment() gw.logger.Debugn("response", logger.NewStringField("ip", kithttputil.GetRequestIP(r)), logger.NewStringField("path", r.URL.Path), @@ -718,11 +711,6 @@ func (gw *Handle) internalBatchHandlerFunc() http.HandlerFunc { errorMessage = err.Error() status = response.GetErrorStatusCode(errorMessage) responseBody = response.GetStatus(errorMessage) - gw.stats.NewTaggedStat( - "gateway.write_key_failed_requests", - stats.CountType, - gw.newReqTypeStatsTagsWithReason(reqType, errorMessage), - ).Increment() gw.logger.Infon("response", logger.NewStringField("ip", kithttputil.GetRequestIP(r)), logger.NewStringField("path", r.URL.Path), @@ -740,7 +728,14 @@ func (gw *Handle) internalBatchHandlerFunc() http.HandlerFunc { } } -func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byte) ([]*jobsdb.JobT, error) { +type jobWithStat struct { + job *jobsdb.JobT + stat gwstats.SourceStat +} + +func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byte) ( + []jobWithStat, error, +) { type params struct { MessageID string `json:"message_id"` SourceID string `json:"source_id"` @@ -762,27 +757,45 @@ func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byt var ( messages []stream.Message isUserSuppressed = gw.memoizedIsUserSuppressed() - jobs []*jobsdb.JobT + res []jobWithStat + stat = gwstats.SourceStat{ReqType: reqType} ) err := jsonfast.Unmarshal(body, &messages) if err != nil { + stat.RequestFailed(response.InvalidJSON) + stat.Report(gw.stats) return nil, errors.New(response.InvalidJSON) } gw.requestSizeStat.Observe(float64(len(body))) if len(messages) == 0 { + stat.RequestFailed(response.NotRudderEvent) + stat.Report(gw.stats) return nil, errors.New(response.NotRudderEvent) } - jobs = make([]*jobsdb.JobT, 0, len(messages)) + res = make([]jobWithStat, 0, len(messages)) for _, msg := range messages { + stat := gwstats.SourceStat{ReqType: reqType} err := gw.streamMsgValidator(&msg) if err != nil { gw.logger.Errorn("invalid message in request", logger.NewErrorField(err)) + stat.RequestEventsFailed(1, response.InvalidStreamMessage) + stat.Report(gw.stats) return nil, errors.New(response.InvalidStreamMessage) } + writeKey, ok := gw.getWriteKeyFromSourceID(msg.Properties.SourceID) + if !ok { + // only live-events will not work if writeKey is not found + gw.logger.Errorn("unable to get writeKey for job", + logger.NewStringField("messageId", msg.Properties.MessageID), + obskit.SourceID(msg.Properties.SourceID)) + } + stat.SourceID = msg.Properties.SourceID + stat.WorkspaceID = msg.Properties.WorkspaceID + stat.WriteKey = writeKey if isUserSuppressed(msg.Properties.WorkspaceID, msg.Properties.UserID, msg.Properties.SourceID) { sourceConfig := gw.getSourceConfigFromSourceID(msg.Properties.SourceID) gw.logger.Infon("suppressed event", @@ -814,14 +827,6 @@ func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byt SourceCategory: msg.Properties.SourceType, } - writeKey, ok := gw.getWriteKeyFromSourceID(msg.Properties.SourceID) - if !ok { - // only live-events will not work if writeKey is not found - gw.logger.Errorn("unable to get writeKey for job", - logger.NewStringField("messageId", msg.Properties.MessageID), - obskit.SourceID(msg.Properties.SourceID)) - } - marshalledParams, err := json.Marshal(jobsDBParams) if err != nil { gw.logger.Errorn("[Gateway] Failed to marshal parameters map", @@ -835,10 +840,16 @@ func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byt msg.Payload, err = fillReceivedAt(msg.Payload, msg.Properties.ReceivedAt) if err != nil { + err = fmt.Errorf("filling receivedAt: %w", err) + stat.RequestEventsFailed(1, err.Error()) + stat.Report(gw.stats) return nil, fmt.Errorf("filling receivedAt: %w", err) } msg.Payload, err = fillRequestIP(msg.Payload, msg.Properties.RequestIP) if err != nil { + err = fmt.Errorf("filling request_ip: %w", err) + stat.RequestEventsFailed(1, err.Error()) + stat.Report(gw.stats) return nil, fmt.Errorf("filling request_ip: %w", err) } @@ -851,24 +862,30 @@ func (gw *Handle) extractJobsFromInternalBatchPayload(reqType string, body []byt payload, err := json.Marshal(eventBatch) if err != nil { + err = fmt.Errorf("marshalling event batch: %w", err) + stat.RequestEventsFailed(1, err.Error()) + stat.Report(gw.stats) return nil, fmt.Errorf("marshalling event batch: %w", err) } jobUUID := uuid.New() - jobs = append(jobs, &jobsdb.JobT{ - UUID: jobUUID, - UserID: msg.Properties.RoutingKey, - Parameters: marshalledParams, - CustomVal: customVal, - EventPayload: payload, - EventCount: len(eventBatch.Batch), - WorkspaceId: msg.Properties.WorkspaceID, + res = append(res, jobWithStat{ + stat: stat, + job: &jobsdb.JobT{ + UUID: jobUUID, + UserID: msg.Properties.RoutingKey, + Parameters: marshalledParams, + CustomVal: customVal, + EventPayload: payload, + EventCount: len(eventBatch.Batch), + WorkspaceId: msg.Properties.WorkspaceID, + }, }) } - if len(jobs) == 0 { // events suppressed - but return success + if len(res) == 0 { // events suppressed - but return success return nil, nil } - return jobs, nil + return res, nil } func fillReceivedAt(event []byte, receivedAt time.Time) ([]byte, error) { diff --git a/gateway/handle_observability.go b/gateway/handle_observability.go index e32c517e38..e8f3b821ee 100644 --- a/gateway/handle_observability.go +++ b/gateway/handle_observability.go @@ -34,13 +34,3 @@ func (gw *Handle) newSourceStatTagsWithReason(s *backendconfig.SourceT, reqType, } return tags } - -func (gw *Handle) newReqTypeStatsTagsWithReason(reqType, reason string) stats.Tags { - tags := stats.Tags{ - "req_type": reqType, - } - if reason != "" { - tags["reason"] = reason - } - return tags -} From 7d7418320869b5ed038c29cd5353806408cf17ea Mon Sep 17 00:00:00 2001 From: "siddarth.msv" <82795818+Sidddddarth@users.noreply.github.com> Date: Mon, 16 Sep 2024 15:48:30 +0530 Subject: [PATCH 28/29] chore: update router reports payload behind a flag, emit stats to observe sizes (#5067) --- router/handle.go | 1 + router/handle_lifecycle.go | 1 + router/router_test.go | 4 ++-- router/worker.go | 27 ++++++++++++++++++++------- runner/buckets.go | 4 ++++ utils/types/reporting_types.go | 2 +- 6 files changed, 29 insertions(+), 10 deletions(-) diff --git a/router/handle.go b/router/handle.go index d0f26bd112..889bcc96f5 100644 --- a/router/handle.go +++ b/router/handle.go @@ -75,6 +75,7 @@ type Handle struct { drainConcurrencyLimit config.ValueLoader[int] workerInputBufferSize int saveDestinationResponse bool + reportJobsdbPayload config.ValueLoader[bool] diagnosisTickerTime time.Duration diff --git a/router/handle_lifecycle.go b/router/handle_lifecycle.go index 261f93d71d..af56a621c3 100644 --- a/router/handle_lifecycle.go +++ b/router/handle_lifecycle.go @@ -108,6 +108,7 @@ func (rt *Handle) Setup( rt.eventOrderKeyThreshold = config.GetReloadableIntVar(200, 1, "Router."+destType+".eventOrderKeyThreshold", "Router.eventOrderKeyThreshold") rt.eventOrderDisabledStateDuration = config.GetReloadableDurationVar(20, time.Minute, "Router."+destType+".eventOrderDisabledStateDuration", "Router.eventOrderDisabledStateDuration") rt.eventOrderHalfEnabledStateDuration = config.GetReloadableDurationVar(10, time.Minute, "Router."+destType+".eventOrderHalfEnabledStateDuration", "Router.eventOrderHalfEnabledStateDuration") + rt.reportJobsdbPayload = config.GetReloadableBoolVar(true, "Router."+destType+".reportJobsdbPayload", "Router.reportJobsdbPayload") statTags := stats.Tags{"destType": rt.destType} rt.tracer = stats.Default.NewTracer("router") diff --git a/router/router_test.go b/router/router_test.go index 74ac5dd9db..32376862e6 100644 --- a/router/router_test.go +++ b/router/router_test.go @@ -1303,7 +1303,7 @@ var _ = Describe("router", func() { Expect(metrics).To(HaveLen(1)) Expect(metrics[0].StatusDetail.StatusCode).To(Equal(200)) Expect(metrics[0].StatusDetail.Status).To(Equal(jobsdb.Succeeded.State)) - Expect(metrics[0].StatusDetail.SampleEvent).To(Equal(json.RawMessage(`{"message": "some transformed message"}`))) + Expect(metrics[0].StatusDetail.SampleEvent).To(Equal(toRetryJobsList[0].EventPayload)) return nil }, ) @@ -1458,7 +1458,7 @@ var _ = Describe("router", func() { Expect(metrics[0].StatusDetail.StatusCode).To(Equal(500)) Expect(metrics[0].StatusDetail.Status).To(Equal(jobsdb.Failed.State)) Expect(metrics[0].StatusDetail.SampleEvent).To(Equal(json.RawMessage(gaPayload))) - Expect(metrics[0].StatusDetail.SampleResponse).To(ContainSubstring(`failureStage":"RudderStack Transformation Error"`)) + Expect(metrics[0].StatusDetail.SampleResponse).To(ContainSubstring(`"routerSubStage":"router_dest_transformer"`)) return nil }, diff --git a/router/worker.go b/router/worker.go index e4bbf8cc71..61e8907141 100644 --- a/router/worker.go +++ b/router/worker.go @@ -588,6 +588,11 @@ func (w *worker) processDestinationJobs() { respBody := strings.Join(respBodyArr, " ") respStatusCodes, respBodys = w.prepareResponsesForJobs(&destinationJob, respStatusCode, respBody) } + stats.Default.NewTaggedStat("router_delivery_payload_size_bytes", stats.HistogramType, stats.Tags{ + "destType": w.rt.destType, + "workspaceID": destinationJob.JobMetadataArray[0].WorkspaceID, + "destinationID": destinationJob.JobMetadataArray[0].DestinationID, + }).Observe(float64(len(destinationJob.Message))) } } ch <- struct{}{} @@ -985,13 +990,9 @@ func (w *worker) postStatusOnResponseQ(respStatusCode int, payload json.RawMessa } } - inputPayload := payload - switch errorAt { - case routerutils.ERROR_AT_TF: - inputPayload = destinationJobMetadata.JobT.EventPayload - status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "failureStage", "RudderStack Transformation Error") - default: // includes ERROR_AT_DEL, ERROR_AT_CUST - status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "failureStage", "Destination Error") + inputPayload := destinationJobMetadata.JobT.EventPayload + if !w.rt.reportJobsdbPayload.Load() { // TODO: update default/remove this flag after monitoring the payload sizes + inputPayload = payload } status.ErrorResponse = routerutils.EnhanceJSON(status.ErrorResponse, "firstAttemptedAt", firstAttemptedAtTime.Format(misc.RFC3339Milli)) @@ -1012,6 +1013,18 @@ func (w *worker) postStatusOnResponseQ(respStatusCode int, payload json.RawMessa } return } + if !isSuccessStatus(respStatusCode) { + switch errorAt { + case routerutils.ERROR_AT_TF: + inputPayload = destinationJobMetadata.JobT.EventPayload + status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "routerSubStage", "router_dest_transformer") + default: // includes ERROR_AT_DEL, ERROR_AT_CUST + status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "routerSubStage", "router_dest_delivery") + } + // TODO: update after observing the sizes of the payloads + status.ErrorResponse = misc.UpdateJSONWithNewKeyVal(status.ErrorResponse, "payloadStage", "router_input") + } + // Saving payload to DB only // 1. if job failed and // 2. if router job undergoes batching or dest transform. diff --git a/runner/buckets.go b/runner/buckets.go index 5deb831076..69df30b85e 100644 --- a/runner/buckets.go +++ b/runner/buckets.go @@ -128,5 +128,9 @@ var ( // 1microsecond, 2.5microsecond, 5microsecond, 1ms, 5ms, 10ms, 25ms, 50ms, 100ms, 250ms, 500ms, 1s 0.00001, 0.00025, 0.0005, 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, }, + "router_delivery_payload_size_bytes": { + float64(1 * bytesize.KB), float64(10 * bytesize.KB), float64(100 * bytesize.KB), + float64(1 * bytesize.MB), float64(3 * bytesize.MB), float64(5 * bytesize.MB), float64(10 * bytesize.MB), + }, } ) diff --git a/utils/types/reporting_types.go b/utils/types/reporting_types.go index 1f7a529f17..c07cbe9d45 100644 --- a/utils/types/reporting_types.go +++ b/utils/types/reporting_types.go @@ -23,7 +23,7 @@ const ( const MaxLengthExceeded = ":max-length-exceeded:" -var ( +const ( DiffStatus = "diff" // Module names From dc7d6ef441ae28a800fcb7f0c9f73fda33b5192f Mon Sep 17 00:00:00 2001 From: Subbarao Vakati <52436067+snarkychef@users.noreply.github.com> Date: Mon, 16 Sep 2024 05:37:45 -0500 Subject: [PATCH 29/29] fix: replace counter with gauge cron tracker alert (#5084) --- warehouse/router/tracker.go | 8 +++++--- warehouse/router/tracker_test.go | 7 ++++--- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/warehouse/router/tracker.go b/warehouse/router/tracker.go index 629a58ae8e..d66bc0ffe3 100644 --- a/warehouse/router/tracker.go +++ b/warehouse/router/tracker.go @@ -23,13 +23,14 @@ import ( // CronTracker Track the status of the staging file whether it has reached the terminal state or not for every warehouse // we pick the staging file which is oldest within the range NOW() - 2 * syncFrequency and NOW() - 3 * syncFrequency func (r *Router) CronTracker(ctx context.Context) error { - tick := r.statsFactory.NewTaggedStat("warehouse_cron_tracker_tick", stats.CountType, stats.Tags{ + cronTrackerExecTimestamp := r.statsFactory.NewTaggedStat("warehouse_cron_tracker_timestamp_seconds", stats.GaugeType, stats.Tags{ "module": moduleName, "destType": r.destType, }) for { - tick.Count(1) + execTime := time.Now() + cronTrackerExecTimestamp.Gauge(execTime.Unix()) r.configSubscriberLock.RLock() warehouses := append([]model.Warehouse{}, r.warehouses...) @@ -51,11 +52,12 @@ func (r *Router) CronTracker(ctx context.Context) error { } } + nextExecTime := execTime.Add(r.config.uploadStatusTrackFrequency) select { case <-ctx.Done(): r.logger.Infon("context is cancelled, stopped running tracking") return nil - case <-time.After(r.config.uploadStatusTrackFrequency): + case <-time.After(time.Until(nextExecTime)): } } } diff --git a/warehouse/router/tracker_test.go b/warehouse/router/tracker_test.go index 716a67d062..5de2942076 100644 --- a/warehouse/router/tracker_test.go +++ b/warehouse/router/tracker_test.go @@ -215,16 +215,17 @@ func TestRouter_CronTracker(t *testing.T) { mockLogger.EXPECT().Infon("context is cancelled, stopped running tracking").Times(1) + executionTime := time.Now().Unix() err = r.CronTracker(ctx) require.NoError(t, err) - m := statsStore.GetByName("warehouse_cron_tracker_tick") + m := statsStore.GetByName("warehouse_cron_tracker_timestamp_seconds") require.Equal(t, len(m), 1) - require.Equal(t, m[0].Name, "warehouse_cron_tracker_tick") + require.Equal(t, m[0].Name, "warehouse_cron_tracker_timestamp_seconds") require.Equal(t, m[0].Tags, stats.Tags{ "module": moduleName, "destType": warehouseutils.POSTGRES, }) - require.GreaterOrEqual(t, m[0].Value, 1.0) + require.GreaterOrEqual(t, m[0].Value, float64(executionTime)) }) }