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))
})
}