From bc2dcdc9ee8c1df9716b1bd00865e26b79b32412 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Thu, 18 Jan 2024 23:50:24 +0530 Subject: [PATCH 01/28] dynamically add new tables to CDC mirrors (#1084) 1. Call an activity to add the new tables to the publication. 2. Kick off a child `CDCFlow` with only the new tables as input along with the `InitialCopyOnly` flag set. This runs enough of the `SetupFlow` and `SnapshotFlow` to be fine, but also skips over some checks regarding primary keys and replica identities. The current idea is to have them in the UI instead, just like in the create mirror interface. 3. Patch the config of the parent workflow with the new tables and then resume. Currently there is no way to interface with this feature, will add a way to signal this in a different PR. UI interface will be added subsequently. --------- Co-authored-by: Kaushik Iska --- flow/activities/flowable.go | 21 +++++ flow/cmd/handler.go | 15 ++-- flow/cmd/mirror_status.go | 14 +-- flow/connectors/core.go | 3 + flow/connectors/postgres/client.go | 4 + flow/connectors/postgres/postgres.go | 63 ++++++++++++-- flow/connectors/utils/array.go | 23 ++++- flow/e2e/test_utils.go | 2 +- flow/shared/additional_tables.go | 27 ++++++ flow/shared/constants.go | 2 +- flow/workflows/cdc_flow.go | 124 ++++++++++++++++++++++----- flow/workflows/qrep_flow.go | 22 +++-- flow/workflows/setup_flow.go | 18 ++-- flow/workflows/xmin_flow.go | 8 +- nexus/flow-rs/src/grpc.rs | 3 +- nexus/server/src/main.rs | 89 ++++++------------- protos/flow.proto | 20 +++-- protos/route.proto | 2 +- 18 files changed, 324 insertions(+), 136 deletions(-) create mode 100644 flow/shared/additional_tables.go diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index e8cdebbcc3..8dffb9d710 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -986,3 +986,24 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, return currentSnapshotXmin, nil } + +func (a *FlowableActivity) AddTablesToPublication(ctx context.Context, cfg *protos.FlowConnectionConfigs, + additionalTableMappings []*protos.TableMapping, +) error { + ctx = context.WithValue(ctx, shared.FlowNameKey, cfg.FlowJobName) + srcConn, err := connectors.GetCDCPullConnector(ctx, cfg.Source) + if err != nil { + return fmt.Errorf("failed to get source connector: %w", err) + } + defer connectors.CloseConnector(srcConn) + + err = srcConn.AddTablesToPublication(&protos.AddTablesToPublicationInput{ + FlowJobName: cfg.FlowJobName, + PublicationName: cfg.PublicationName, + AdditionalTables: additionalTableMappings, + }) + if err != nil { + a.Alerter.LogFlowError(ctx, cfg.FlowJobName, err) + } + return err +} diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 08be7c724e..c5ed5db1f6 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -327,7 +327,7 @@ func (h *FlowRequestHandler) ShutdownFlow( ctx, req.WorkflowId, "", - shared.CDCFlowSignalName, + shared.FlowSignalName, shared.ShutdownSignal, ) if err != nil { @@ -442,8 +442,9 @@ func (h *FlowRequestHandler) FlowStateChange( if err != nil { return nil, err } + if req.RequestedFlowState == protos.FlowStatus_STATUS_PAUSED && - *currState == protos.FlowStatus_STATUS_RUNNING { + currState == protos.FlowStatus_STATUS_RUNNING { err = h.updateWorkflowStatus(ctx, workflowID, protos.FlowStatus_STATUS_PAUSING) if err != nil { return nil, err @@ -452,20 +453,20 @@ func (h *FlowRequestHandler) FlowStateChange( ctx, workflowID, "", - shared.CDCFlowSignalName, + shared.FlowSignalName, shared.PauseSignal, ) } else if req.RequestedFlowState == protos.FlowStatus_STATUS_RUNNING && - *currState == protos.FlowStatus_STATUS_PAUSED { + currState == protos.FlowStatus_STATUS_PAUSED { err = h.temporalClient.SignalWorkflow( ctx, workflowID, "", - shared.CDCFlowSignalName, + shared.FlowSignalName, shared.NoopSignal, ) } else if req.RequestedFlowState == protos.FlowStatus_STATUS_TERMINATED && - (*currState == protos.FlowStatus_STATUS_RUNNING || *currState == protos.FlowStatus_STATUS_PAUSED) { + (currState == protos.FlowStatus_STATUS_RUNNING || currState == protos.FlowStatus_STATUS_PAUSED) { err = h.updateWorkflowStatus(ctx, workflowID, protos.FlowStatus_STATUS_TERMINATING) if err != nil { return nil, err @@ -482,7 +483,7 @@ func (h *FlowRequestHandler) FlowStateChange( req.RequestedFlowState, currState) } if err != nil { - return nil, fmt.Errorf("unable to signal CDCFlow workflow: %w", err) + return nil, fmt.Errorf("unable to signal workflow: %w", err) } return &protos.FlowStateChangeResponse{ diff --git a/flow/cmd/mirror_status.go b/flow/cmd/mirror_status.go index 9c904ab716..1e17495085 100644 --- a/flow/cmd/mirror_status.go +++ b/flow/cmd/mirror_status.go @@ -51,7 +51,7 @@ func (h *FlowRequestHandler) MirrorStatus( Status: &protos.MirrorStatusResponse_CdcStatus{ CdcStatus: cdcStatus, }, - CurrentFlowState: *currState, + CurrentFlowState: currState, }, nil } else { qrepStatus, err := h.QRepFlowStatus(ctx, req) @@ -66,7 +66,7 @@ func (h *FlowRequestHandler) MirrorStatus( Status: &protos.MirrorStatusResponse_QrepStatus{ QrepStatus: qrepStatus, }, - CurrentFlowState: *currState, + CurrentFlowState: currState, }, nil } } @@ -334,17 +334,19 @@ func (h *FlowRequestHandler) isCDCFlow(ctx context.Context, flowJobName string) return false, nil } -func (h *FlowRequestHandler) getWorkflowStatus(ctx context.Context, workflowID string) (*protos.FlowStatus, error) { +func (h *FlowRequestHandler) getWorkflowStatus(ctx context.Context, workflowID string) (protos.FlowStatus, error) { res, err := h.temporalClient.QueryWorkflow(ctx, workflowID, "", shared.FlowStatusQuery) if err != nil { slog.Error(fmt.Sprintf("failed to get state in workflow with ID %s: %s", workflowID, err.Error())) - return nil, fmt.Errorf("failed to get state in workflow with ID %s: %w", workflowID, err) + return protos.FlowStatus_STATUS_UNKNOWN, + fmt.Errorf("failed to get state in workflow with ID %s: %w", workflowID, err) } - var state *protos.FlowStatus + var state protos.FlowStatus err = res.Get(&state) if err != nil { slog.Error(fmt.Sprintf("failed to get state in workflow with ID %s: %s", workflowID, err.Error())) - return nil, fmt.Errorf("failed to get state in workflow with ID %s: %w", workflowID, err) + return protos.FlowStatus_STATUS_UNKNOWN, + fmt.Errorf("failed to get state in workflow with ID %s: %w", workflowID, err) } return state, nil } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index a9c80b8a46..0f3e588cc7 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -49,6 +49,9 @@ type CDCPullConnector interface { // GetOpenConnectionsForUser returns the number of open connections for the user configured in the peer. GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) + + // AddTablesToPublication adds additional tables added to a mirror to the publication also + AddTablesToPublication(req *protos.AddTablesToPublicationInput) error } type CDCSyncConnector interface { diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 8c8113911b..a5c7419fbe 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -604,3 +604,7 @@ func (c *PostgresConnector) getCurrentLSN() (pglogrepl.LSN, error) { } return pglogrepl.ParseLSN(result.String) } + +func (c *PostgresConnector) getDefaultPublicationName(jobName string) string { + return fmt.Sprintf("peerflow_pub_%s", jobName) +} diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index d0a7f4db52..48d67ecfaf 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -5,6 +5,7 @@ import ( "fmt" "log/slog" "regexp" + "strings" "time" "github.com/PeerDB-io/peer-flow/connectors/utils" @@ -205,8 +206,7 @@ func (c *PostgresConnector) PullRecords(catalogPool *pgxpool.Pool, req *model.Pu slotName = req.OverrideReplicationSlotName } - // Publication name would be the job name prefixed with "peerflow_pub_" - publicationName := fmt.Sprintf("peerflow_pub_%s", req.FlowJobName) + publicationName := c.getDefaultPublicationName(req.FlowJobName) if req.OverridePublicationName != "" { publicationName = req.OverridePublicationName } @@ -788,6 +788,16 @@ func (c *PostgresConnector) EnsurePullability( return nil, err } + tableIdentifierMapping[tableName] = &protos.PostgresTableIdentifier{ + RelId: relID, + } + + if !req.CheckConstraints { + msg := fmt.Sprintf("[no-constraints] ensured pullability table %s", tableName) + utils.RecordHeartbeatWithRecover(c.ctx, msg) + continue + } + replicaIdentity, replErr := c.getReplicaIdentityType(schemaTable) if replErr != nil { return nil, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, replErr) @@ -799,13 +809,11 @@ func (c *PostgresConnector) EnsurePullability( } // we only allow no primary key if the table has REPLICA IDENTITY FULL + // this is ok for replica identity index as we populate the primary key columns if len(pKeyCols) == 0 && !(replicaIdentity == ReplicaIdentityFull) { return nil, fmt.Errorf("table %s has no primary keys and does not have REPLICA IDENTITY FULL", schemaTable) } - tableIdentifierMapping[tableName] = &protos.PostgresTableIdentifier{ - RelId: relID, - } utils.RecordHeartbeatWithRecover(c.ctx, fmt.Sprintf("ensured pullability table %s", tableName)) } @@ -826,8 +834,7 @@ func (c *PostgresConnector) SetupReplication(signal SlotSignal, req *protos.Setu slotName = req.ExistingReplicationSlotName } - // Publication name would be the job name prefixed with "peerflow_pub_" - publicationName := fmt.Sprintf("peerflow_pub_%s", req.FlowJobName) + publicationName := c.getDefaultPublicationName(req.FlowJobName) if req.ExistingPublicationName != "" { publicationName = req.ExistingPublicationName } @@ -859,8 +866,7 @@ func (c *PostgresConnector) PullFlowCleanup(jobName string) error { // Slotname would be the job name prefixed with "peerflow_slot_" slotName := fmt.Sprintf("peerflow_slot_%s", jobName) - // Publication name would be the job name prefixed with "peerflow_pub_" - publicationName := fmt.Sprintf("peerflow_pub_%s", jobName) + publicationName := c.getDefaultPublicationName(jobName) pullFlowCleanupTx, err := c.pool.Begin(c.ctx) if err != nil { @@ -938,3 +944,42 @@ func (c *PostgresConnector) GetOpenConnectionsForUser() (*protos.GetOpenConnecti CurrentOpenConnections: result.Int64, }, nil } + +func (c *PostgresConnector) AddTablesToPublication(req *protos.AddTablesToPublicationInput) error { + // don't modify custom publications + if req == nil || len(req.AdditionalTables) == 0 { + return nil + } + + additionalSrcTables := make([]string, 0, len(req.AdditionalTables)) + for _, additionalTableMapping := range req.AdditionalTables { + additionalSrcTables = append(additionalSrcTables, additionalTableMapping.SourceTableIdentifier) + } + + // just check if we have all the tables already in the publication + if req.PublicationName != "" { + rows, err := c.pool.Query(c.ctx, + "SELECT tablename FROM pg_publication_tables WHERE pubname=$1", req.PublicationName) + if err != nil { + return fmt.Errorf("failed to check tables in publication: %w", err) + } + + tableNames, err := pgx.CollectRows[string](rows, pgx.RowTo) + if err != nil { + return fmt.Errorf("failed to check tables in publication: %w", err) + } + notPresentTables := utils.ArrayMinus(tableNames, additionalSrcTables) + if len(notPresentTables) > 0 { + return fmt.Errorf("some additional tables not present in custom publication: %s", + strings.Join(notPresentTables, ", ")) + } + } + + additionalSrcTablesString := strings.Join(additionalSrcTables, ",") + _, err := c.pool.Exec(c.ctx, fmt.Sprintf("ALTER PUBLICATION %s ADD TABLE %s", + c.getDefaultPublicationName(req.FlowJobName), additionalSrcTablesString)) + if err != nil { + return fmt.Errorf("failed to alter publication: %w", err) + } + return nil +} diff --git a/flow/connectors/utils/array.go b/flow/connectors/utils/array.go index 3db4d53bb3..d131e3c65e 100644 --- a/flow/connectors/utils/array.go +++ b/flow/connectors/utils/array.go @@ -1,13 +1,14 @@ package utils -func ArrayMinus(first []string, second []string) []string { - lookup := make(map[string]struct{}, len(second)) +// first - second +func ArrayMinus[T comparable](first, second []T) []T { + lookup := make(map[T]struct{}, len(second)) // Add elements from arrayB to the lookup map for _, element := range second { lookup[element] = struct{}{} } // Iterate over arrayA and check if the element is present in the lookup map - var result []string + var result []T for _, element := range first { _, exists := lookup[element] if !exists { @@ -29,3 +30,19 @@ func ArrayChunks[T any](slice []T, size int) [][]T { return partitions } + +func ArraysHaveOverlap[T comparable](first, second []T) bool { + lookup := make(map[T]struct{}) + + for _, element := range second { + lookup[element] = struct{}{} + } + + for _, element := range first { + if _, exists := lookup[element]; exists { + return true + } + } + + return false +} diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 69107cdc8f..0aa5aaebbb 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -187,7 +187,7 @@ func SetupCDCFlowStatusQuery(t *testing.T, env *testsuite.TestWorkflowEnvironmen err = response.Get(&state) if err != nil { slog.Error(err.Error()) - } else if state.CurrentFlowState == protos.FlowStatus_STATUS_RUNNING { + } else if state.CurrentFlowStatus == protos.FlowStatus_STATUS_RUNNING { return } } else if counter > 15 { diff --git a/flow/shared/additional_tables.go b/flow/shared/additional_tables.go new file mode 100644 index 0000000000..4fd0a874b2 --- /dev/null +++ b/flow/shared/additional_tables.go @@ -0,0 +1,27 @@ +package shared + +import ( + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" +) + +func AdditionalTablesHasOverlap(currentTableMappings []*protos.TableMapping, + additionalTableMappings []*protos.TableMapping, +) bool { + currentSrcTables := make([]string, 0, len(currentTableMappings)) + currentDstTables := make([]string, 0, len(currentTableMappings)) + additionalSrcTables := make([]string, 0, len(additionalTableMappings)) + additionalDstTables := make([]string, 0, len(additionalTableMappings)) + + for _, currentTableMapping := range currentTableMappings { + currentSrcTables = append(currentSrcTables, currentTableMapping.SourceTableIdentifier) + currentDstTables = append(currentDstTables, currentTableMapping.DestinationTableIdentifier) + } + for _, additionalTableMapping := range additionalTableMappings { + currentSrcTables = append(currentSrcTables, additionalTableMapping.SourceTableIdentifier) + currentDstTables = append(currentDstTables, additionalTableMapping.DestinationTableIdentifier) + } + + return utils.ArraysHaveOverlap(currentSrcTables, additionalSrcTables) || + utils.ArraysHaveOverlap(currentDstTables, additionalDstTables) +} diff --git a/flow/shared/constants.go b/flow/shared/constants.go index 07cfea307a..119514fb76 100644 --- a/flow/shared/constants.go +++ b/flow/shared/constants.go @@ -12,7 +12,7 @@ const ( snapshotFlowTaskQueue = "snapshot-flow-task-queue" // Signals - CDCFlowSignalName = "peer-flow-signal" + FlowSignalName = "peer-flow-signal" CDCDynamicPropertiesSignalName = "cdc-dynamic-properties" // Queries diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 70ac1723ba..ad0685e910 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -14,6 +14,7 @@ import ( "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" + "google.golang.org/protobuf/proto" ) const ( @@ -47,11 +48,12 @@ type CDCFlowWorkflowState struct { // Global mapping of relation IDs to RelationMessages sent as a part of logical replication. // Needed to support schema changes. RelationMessageMapping model.RelationMessageMapping - // current workflow state - CurrentFlowState protos.FlowStatus + CurrentFlowStatus protos.FlowStatus // moved from config here, set by SetupFlow SrcTableIdNameMapping map[uint32]string TableNameSchemaMapping map[string]*protos.TableSchema + // flow config update request, set to nil after processed + FlowConfigUpdates []*protos.CDCFlowConfigUpdate } type SignalProps struct { @@ -75,9 +77,10 @@ func NewCDCFlowWorkflowState(numTables int) *CDCFlowWorkflowState { RelationName: "protobuf_workaround", }, }, - CurrentFlowState: protos.FlowStatus_STATUS_SETUP, - SrcTableIdNameMapping: make(map[uint32]string, numTables), - TableNameSchemaMapping: make(map[string]*protos.TableSchema, numTables), + CurrentFlowStatus: protos.FlowStatus_STATUS_SETUP, + SrcTableIdNameMapping: nil, + TableNameSchemaMapping: nil, + FlowConfigUpdates: nil, } } @@ -141,7 +144,7 @@ func GetChildWorkflowID( type CDCFlowWorkflowResult = CDCFlowWorkflowState func (w *CDCFlowWorkflowExecution) receiveAndHandleSignalAsync(ctx workflow.Context, state *CDCFlowWorkflowState) { - signalChan := workflow.GetSignalChannel(ctx, shared.CDCFlowSignalName) + signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) var signalVal shared.CDCFlowSignal ok := signalChan.ReceiveAsync(&signalVal) @@ -150,6 +153,79 @@ func (w *CDCFlowWorkflowExecution) receiveAndHandleSignalAsync(ctx workflow.Cont } } +func (w *CDCFlowWorkflowExecution) processCDCFlowConfigUpdates(ctx workflow.Context, + cfg *protos.FlowConnectionConfigs, state *CDCFlowWorkflowState, + limits *CDCFlowLimits, mirrorNameSearch *map[string]interface{}, +) error { + for _, flowConfigUpdate := range state.FlowConfigUpdates { + if len(flowConfigUpdate.AdditionalTables) == 0 { + continue + } + if shared.AdditionalTablesHasOverlap(cfg.TableMappings, flowConfigUpdate.AdditionalTables) { + return fmt.Errorf("duplicate source/destination tables found in additionalTables") + } + + alterPublicationAddAdditionalTablesCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 5 * time.Minute, + }) + alterPublicationAddAdditionalTablesFuture := workflow.ExecuteActivity( + alterPublicationAddAdditionalTablesCtx, + flowable.AddTablesToPublication, + cfg, flowConfigUpdate.AdditionalTables) + if err := alterPublicationAddAdditionalTablesFuture.Get(ctx, nil); err != nil { + w.logger.Error("failed to alter publication for additional tables: ", err) + return err + } + + additionalTablesWorkflowCfg := proto.Clone(cfg).(*protos.FlowConnectionConfigs) + additionalTablesWorkflowCfg.DoInitialSnapshot = true + additionalTablesWorkflowCfg.InitialSnapshotOnly = true + additionalTablesWorkflowCfg.TableMappings = flowConfigUpdate.AdditionalTables + additionalTablesWorkflowCfg.FlowJobName = fmt.Sprintf("%s_additional_tables_%s", cfg.FlowJobName, + strings.ToLower(shared.RandomString(8))) + + childAdditionalTablesCDCFlowID, + err := GetChildWorkflowID(ctx, "cdc-flow", additionalTablesWorkflowCfg.FlowJobName) + if err != nil { + return err + } + + // execute the sync flow as a child workflow + childAdditionalTablesCDCFlowOpts := workflow.ChildWorkflowOptions{ + WorkflowID: childAdditionalTablesCDCFlowID, + ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, + RetryPolicy: &temporal.RetryPolicy{ + MaximumAttempts: 20, + }, + SearchAttributes: *mirrorNameSearch, + WaitForCancellation: true, + } + childAdditionalTablesCDCFlowCtx := workflow.WithChildOptions(ctx, childAdditionalTablesCDCFlowOpts) + childAdditionalTablesCDCFlowFuture := workflow.ExecuteChildWorkflow( + childAdditionalTablesCDCFlowCtx, + CDCFlowWorkflowWithConfig, + additionalTablesWorkflowCfg, + nil, + limits, + ) + var res *CDCFlowWorkflowResult + if err := childAdditionalTablesCDCFlowFuture.Get(childAdditionalTablesCDCFlowCtx, &res); err != nil { + return err + } + + for tableID, tableName := range res.SrcTableIdNameMapping { + state.SrcTableIdNameMapping[tableID] = tableName + } + for tableName, tableSchema := range res.TableNameSchemaMapping { + state.TableNameSchemaMapping[tableName] = tableSchema + } + cfg.TableMappings = append(cfg.TableMappings, flowConfigUpdate.AdditionalTables...) + // finished processing, wipe it + state.FlowConfigUpdates = nil + } + return nil +} + func CDCFlowWorkflowWithConfig( ctx workflow.Context, cfg *protos.FlowConnectionConfigs, @@ -165,10 +241,6 @@ func CDCFlowWorkflowWithConfig( w := NewCDCFlowWorkflowExecution(ctx) - if limits.TotalSyncFlows == 0 { - limits.TotalSyncFlows = maxSyncFlowsPerCDCFlow - } - err := workflow.SetQueryHandler(ctx, shared.CDCFlowStateQuery, func() (CDCFlowWorkflowState, error) { return *state, nil }) @@ -176,19 +248,18 @@ func CDCFlowWorkflowWithConfig( return state, fmt.Errorf("failed to set `%s` query handler: %w", shared.CDCFlowStateQuery, err) } err = workflow.SetQueryHandler(ctx, shared.FlowStatusQuery, func() (protos.FlowStatus, error) { - return state.CurrentFlowState, nil + return state.CurrentFlowStatus, nil }) if err != nil { return state, fmt.Errorf("failed to set `%s` query handler: %w", shared.FlowStatusQuery, err) } err = workflow.SetUpdateHandler(ctx, shared.FlowStatusUpdate, func(status protos.FlowStatus) error { - state.CurrentFlowState = status + state.CurrentFlowStatus = status return nil }) if err != nil { return state, fmt.Errorf("failed to set `%s` update handler: %w", shared.FlowStatusUpdate, err) } - mirrorNameSearch := map[string]interface{}{ shared.MirrorNameSearchAttribute: cfg.FlowJobName, } @@ -197,7 +268,7 @@ func CDCFlowWorkflowWithConfig( // because Resync modifies TableMappings before Setup and also before Snapshot // for safety, rely on the idempotency of SetupFlow instead // also, no signals are being handled until the loop starts, so no PAUSE/DROP will take here. - if state.CurrentFlowState != protos.FlowStatus_STATUS_RUNNING { + if state.CurrentFlowStatus != protos.FlowStatus_STATUS_RUNNING { // if resync is true, alter the table name schema mapping to temporarily add // a suffix to the table names. if cfg.Resync { @@ -231,7 +302,7 @@ func CDCFlowWorkflowWithConfig( } state.SrcTableIdNameMapping = setupFlowOutput.SrcTableIdNameMapping state.TableNameSchemaMapping = setupFlowOutput.TableNameSchemaMapping - state.CurrentFlowState = protos.FlowStatus_STATUS_SNAPSHOT + state.CurrentFlowStatus = protos.FlowStatus_STATUS_SNAPSHOT // next part of the setup is to snapshot-initial-copy and setup replication slots. snapshotFlowID, err := GetChildWorkflowID(ctx, "snapshot-flow", cfg.FlowJobName) @@ -294,7 +365,7 @@ func CDCFlowWorkflowWithConfig( } } - state.CurrentFlowState = protos.FlowStatus_STATUS_RUNNING + state.CurrentFlowStatus = protos.FlowStatus_STATUS_RUNNING state.Progress = append(state.Progress, "executed setup flow and snapshot flow") // if initial_copy_only is opted for, we end the flow here. @@ -303,6 +374,10 @@ func CDCFlowWorkflowWithConfig( } } + if limits.TotalSyncFlows == 0 { + limits.TotalSyncFlows = maxSyncFlowsPerCDCFlow + } + syncFlowOptions := &protos.SyncFlowOptions{ BatchSize: limits.MaxBatchSize, IdleTimeoutSeconds: 0, @@ -349,8 +424,8 @@ func CDCFlowWorkflowWithConfig( if state.ActiveSignal == shared.PauseSignal { startTime := time.Now() - state.CurrentFlowState = protos.FlowStatus_STATUS_PAUSED - signalChan := workflow.GetSignalChannel(ctx, shared.CDCFlowSignalName) + state.CurrentFlowStatus = protos.FlowStatus_STATUS_PAUSED + signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) var signalVal shared.CDCFlowSignal for state.ActiveSignal == shared.PauseSignal { @@ -359,6 +434,13 @@ func CDCFlowWorkflowWithConfig( ok, _ := signalChan.ReceiveWithTimeout(ctx, 1*time.Minute, &signalVal) if ok { state.ActiveSignal = shared.FlowSignalHandler(state.ActiveSignal, signalVal, w.logger) + // only process config updates when going from STATUS_PAUSED to STATUS_RUNNING + if state.ActiveSignal == shared.NoopSignal { + err = w.processCDCFlowConfigUpdates(ctx, cfg, state, limits, &mirrorNameSearch) + if err != nil { + return state, err + } + } } else if err := ctx.Err(); err != nil { return nil, err } @@ -370,11 +452,11 @@ func CDCFlowWorkflowWithConfig( // check if the peer flow has been shutdown if state.ActiveSignal == shared.ShutdownSignal { w.logger.Info("peer flow has been shutdown") - state.CurrentFlowState = protos.FlowStatus_STATUS_TERMINATED + state.CurrentFlowStatus = protos.FlowStatus_STATUS_TERMINATED return state, nil } - state.CurrentFlowState = protos.FlowStatus_STATUS_RUNNING + state.CurrentFlowStatus = protos.FlowStatus_STATUS_RUNNING // check if total sync flows have been completed // since this happens immediately after we check for signals, the case of a signal being missed @@ -499,5 +581,5 @@ func CDCFlowWorkflowWithConfig( } state.TruncateProgress(w.logger) - return nil, workflow.NewContinueAsNewError(ctx, CDCFlowWorkflowWithConfig, cfg, limits, state) + return state, workflow.NewContinueAsNewError(ctx, CDCFlowWorkflowWithConfig, cfg, limits, state) } diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index b44e0df207..2fadcbbe62 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -42,6 +42,7 @@ func NewQRepFlowState() *protos.QRepFlowState { }, NumPartitionsProcessed: 0, NeedsResync: true, + CurrentFlowStatus: protos.FlowStatus_STATUS_RUNNING, } } @@ -367,7 +368,7 @@ func (q *QRepFlowExecution) handleTableRenameForResync(ctx workflow.Context, sta } func (q *QRepFlowExecution) receiveAndHandleSignalAsync(ctx workflow.Context) { - signalChan := workflow.GetSignalChannel(ctx, shared.CDCFlowSignalName) + signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) var signalVal shared.CDCFlowSignal ok := signalChan.ReceiveAsync(&signalVal) @@ -386,8 +387,8 @@ func setWorkflowQueries(ctx workflow.Context, state *protos.QRepFlowState) error } // Support a Query for the current status of the qrep flow. - err = workflow.SetQueryHandler(ctx, shared.FlowStatusQuery, func() (*protos.FlowStatus, error) { - return &state.CurrentFlowState, nil + err = workflow.SetQueryHandler(ctx, shared.FlowStatusQuery, func() (protos.FlowStatus, error) { + return state.CurrentFlowStatus, nil }) if err != nil { return fmt.Errorf("failed to set `%s` query handler: %w", shared.FlowStatusQuery, err) @@ -395,7 +396,7 @@ func setWorkflowQueries(ctx workflow.Context, state *protos.QRepFlowState) error // Support an Update for the current status of the qrep flow. err = workflow.SetUpdateHandler(ctx, shared.FlowStatusUpdate, func(status *protos.FlowStatus) error { - state.CurrentFlowState = *status + state.CurrentFlowStatus = *status return nil }) if err != nil { @@ -429,6 +430,15 @@ func QRepFlowWorkflow( return err } + // Support an Update for the current status of the qrep flow. + err = workflow.SetUpdateHandler(ctx, shared.FlowStatusUpdate, func(status *protos.FlowStatus) error { + state.CurrentFlowStatus = *status + return nil + }) + if err != nil { + return fmt.Errorf("failed to register query handler: %w", err) + } + // get qrep run uuid via side-effect runUUIDSideEffect := workflow.SideEffect(ctx, func(ctx workflow.Context) interface{} { return uuid.New().String() @@ -507,7 +517,8 @@ func QRepFlowWorkflow( q.receiveAndHandleSignalAsync(ctx) if q.activeSignal == shared.PauseSignal { startTime := time.Now() - signalChan := workflow.GetSignalChannel(ctx, shared.CDCFlowSignalName) + state.CurrentFlowStatus = protos.FlowStatus_STATUS_PAUSED + signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) var signalVal shared.CDCFlowSignal for q.activeSignal == shared.PauseSignal { @@ -521,6 +532,7 @@ func QRepFlowWorkflow( } if q.activeSignal == shared.ShutdownSignal { q.logger.Info("terminating workflow - ", config.FlowJobName) + state.CurrentFlowStatus = protos.FlowStatus_STATUS_TERMINATED return nil } diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index 0b9168a931..d5820a5cb0 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -103,6 +103,7 @@ func (s *SetupFlowExecution) checkConnectionsAndSetupMetadataTables( func (s *SetupFlowExecution) ensurePullability( ctx workflow.Context, config *protos.FlowConnectionConfigs, + checkConstraints bool, ) (map[uint32]string, error) { s.logger.Info("ensuring pullability for peer flow - ", s.cdcFlowName) @@ -119,6 +120,7 @@ func (s *SetupFlowExecution) ensurePullability( PeerConnectionConfig: config.Source, FlowJobName: s.cdcFlowName, SourceTableIdentifiers: srcTblIdentifiers, + CheckConstraints: checkConstraints, } future := workflow.ExecuteActivity(ctx, flowable.EnsurePullability, ensurePullabilityInput) @@ -264,16 +266,14 @@ func (s *SetupFlowExecution) executeSetupFlow( } setupFlowOutput := protos.SetupFlowOutput{} - // for initial copy only flows, we don't need to ensure pullability or create the raw table - // as we don't need the primary key requirement. - if !config.InitialSnapshotOnly { - // then ensure pullability - srcTableIdNameMapping, err := s.ensurePullability(ctx, config) - if err != nil { - return nil, fmt.Errorf("failed to ensure pullability: %w", err) - } - setupFlowOutput.SrcTableIdNameMapping = srcTableIdNameMapping + srcTableIdNameMapping, err := s.ensurePullability(ctx, config, !config.InitialSnapshotOnly) + if err != nil { + return nil, fmt.Errorf("failed to ensure pullability: %w", err) + } + setupFlowOutput.SrcTableIdNameMapping = srcTableIdNameMapping + // for initial copy only flows, we don't need to create the raw table + if !config.InitialSnapshotOnly { // then create the raw table if err := s.createRawTable(ctx, config); err != nil { return nil, fmt.Errorf("failed to create raw table: %w", err) diff --git a/flow/workflows/xmin_flow.go b/flow/workflows/xmin_flow.go index 1394d17353..387ab2e0a7 100644 --- a/flow/workflows/xmin_flow.go +++ b/flow/workflows/xmin_flow.go @@ -119,7 +119,8 @@ func XminFlowWorkflow( q.receiveAndHandleSignalAsync(ctx) if x.activeSignal == shared.PauseSignal { startTime := time.Now() - signalChan := workflow.GetSignalChannel(ctx, shared.CDCFlowSignalName) + state.CurrentFlowStatus = protos.FlowStatus_STATUS_PAUSED + signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) var signalVal shared.CDCFlowSignal for x.activeSignal == shared.PauseSignal { @@ -131,8 +132,9 @@ func XminFlowWorkflow( } } } - if x.activeSignal == shared.ShutdownSignal { - x.logger.Info("terminating workflow - ", config.FlowJobName) + if q.activeSignal == shared.ShutdownSignal { + q.logger.Info("terminating workflow - ", config.FlowJobName) + state.CurrentFlowStatus = protos.FlowStatus_STATUS_TERMINATED return nil } diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index deed768143..9b5e2f4d97 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -107,13 +107,14 @@ impl FlowGrpcClient { flow_job_name: &str, workflow_details: WorkflowDetails, state: pt::peerdb_flow::FlowStatus, + flow_config_update: Option, ) -> anyhow::Result<()> { let state_change_req = pt::peerdb_route::FlowStateChangeRequest { flow_job_name: flow_job_name.to_owned(), requested_flow_state: state.into(), source_peer: Some(workflow_details.source_peer), destination_peer: Some(workflow_details.destination_peer), - flow_state_update: None, + flow_config_update, }; let response = self.client.flow_state_change(state_change_req).await?; let state_change_response = response.into_inner(); diff --git a/nexus/server/src/main.rs b/nexus/server/src/main.rs index 4a1eebe7e1..24b03a4020 100644 --- a/nexus/server/src/main.rs +++ b/nexus/server/src/main.rs @@ -66,11 +66,7 @@ impl AuthSource for FixedPasswordAuthSource { // randomly generate a 4 byte salt let salt = rand::thread_rng().gen::<[u8; 4]>(); let password = &self.password; - let hash_password = hash_md5_password( - login_info.user().unwrap_or(""), - password, - &salt, - ); + let hash_password = hash_md5_password(login_info.user().unwrap_or(""), password, &salt); Ok(Password::new( Some(salt.to_vec()), hash_password.as_bytes().to_vec(), @@ -116,9 +112,9 @@ impl NexusBackend { ) -> PgWireResult>> { let res = executor.execute(stmt).await?; match res { - QueryOutput::AffectedRows(rows) => Ok(vec![Response::Execution( - Tag::new("OK").with_rows(rows), - )]), + QueryOutput::AffectedRows(rows) => { + Ok(vec![Response::Execution(Tag::new("OK").with_rows(rows))]) + } QueryOutput::Stream(rows) => { let schema = rows.schema(); let res = sendable_stream_to_query_response(schema, rows)?; @@ -134,17 +130,13 @@ impl NexusBackend { match cm { peer_cursor::CursorModification::Created(cursor_name) => { peer_cursors.add_cursor(cursor_name, peer_holder.unwrap()); - Ok(vec![Response::Execution(Tag::new( - "DECLARE CURSOR", - ))]) + Ok(vec![Response::Execution(Tag::new("DECLARE CURSOR"))]) } peer_cursor::CursorModification::Closed(cursors) => { for cursor_name in cursors { peer_cursors.remove_cursor(&cursor_name); } - Ok(vec![Response::Execution(Tag::new( - "CLOSE CURSOR", - ))]) + Ok(vec![Response::Execution(Tag::new("CLOSE CURSOR"))]) } } } @@ -187,9 +179,7 @@ impl NexusBackend { ) -> PgWireResult>> { if if_not_exists { let existing_mirror_success = "MIRROR ALREADY EXISTS"; - Ok(vec![Response::Execution(Tag::new( - existing_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(existing_mirror_success))]) } else { Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -272,6 +262,7 @@ impl NexusBackend { flow_job_name, workflow_details, pt::peerdb_flow::FlowStatus::StatusTerminated, + None, ) .await .map_err(|err| { @@ -288,14 +279,10 @@ impl NexusBackend { ) })?; let drop_mirror_success = format!("DROP MIRROR {}", flow_job_name); - Ok(vec![Response::Execution(Tag::new( - &drop_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&drop_mirror_success))]) } else if *if_exists { let no_mirror_success = "NO SUCH MIRROR"; - Ok(vec![Response::Execution(Tag::new( - no_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(no_mirror_success))]) } else { Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -347,16 +334,12 @@ impl NexusBackend { if qrep_flow_job.disabled { let create_mirror_success = format!("CREATE MIRROR {}", qrep_flow_job.name); - return Ok(vec![Response::Execution(Tag::new( - &create_mirror_success, - ))]); + return Ok(vec![Response::Execution(Tag::new(&create_mirror_success))]); } let _workflow_id = self.run_qrep_mirror(qrep_flow_job).await?; let create_mirror_success = format!("CREATE MIRROR {}", qrep_flow_job.name); - Ok(vec![Response::Execution(Tag::new( - &create_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&create_mirror_success))]) } else { Self::handle_mirror_existence(*if_not_exists, &qrep_flow_job.name) } @@ -396,9 +379,7 @@ impl NexusBackend { e.to_string(), ))) })?; - Ok(vec![Response::Execution(Tag::new( - "OK", - ))]) + Ok(vec![Response::Execution(Tag::new("OK"))]) } PeerDDL::CreateMirrorForCDC { if_not_exists, @@ -477,9 +458,7 @@ impl NexusBackend { })?; let create_mirror_success = format!("CREATE MIRROR {}", flow_job.name); - Ok(vec![Response::Execution(Tag::new( - &create_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&create_mirror_success))]) } else { Self::handle_mirror_existence(*if_not_exists, &flow_job.name) } @@ -506,9 +485,7 @@ impl NexusBackend { } { let workflow_id = self.run_qrep_mirror(&job).await?; let create_mirror_success = format!("STARTED WORKFLOW {}", workflow_id); - Ok(vec![Response::Execution(Tag::new( - &create_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&create_mirror_success))]) } else { Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -550,14 +527,10 @@ impl NexusBackend { PgWireError::ApiError(format!("unable to drop peer: {:?}", err).into()) })?; let drop_peer_success = format!("DROP PEER {}", peer_name); - Ok(vec![Response::Execution(Tag::new( - &drop_peer_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&drop_peer_success))]) } else if *if_exists { let no_peer_success = "NO SUCH PEER"; - Ok(vec![Response::Execution(Tag::new( - no_peer_success, - ))]) + Ok(vec![Response::Execution(Tag::new(no_peer_success))]) } else { Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -638,15 +611,11 @@ impl NexusBackend { })?; let resync_mirror_success = format!("RESYNC MIRROR {}", mirror_name); - Ok(vec![Response::Execution(Tag::new( - &resync_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&resync_mirror_success))]) } None => { let no_peer_success = "NO SUCH QREP MIRROR"; - Ok(vec![Response::Execution(Tag::new( - no_peer_success, - ))]) + Ok(vec![Response::Execution(Tag::new(no_peer_success))]) } } } @@ -687,6 +656,7 @@ impl NexusBackend { flow_job_name, workflow_details, pt::peerdb_flow::FlowStatus::StatusPaused, + None, ) .await .map_err(|err| { @@ -695,14 +665,10 @@ impl NexusBackend { ) })?; let drop_mirror_success = format!("PAUSE MIRROR {}", flow_job_name); - Ok(vec![Response::Execution(Tag::new( - &drop_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(&drop_mirror_success))]) } else if *if_exists { let no_mirror_success = "NO SUCH MIRROR"; - Ok(vec![Response::Execution(Tag::new( - no_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(no_mirror_success))]) } else { Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), @@ -748,6 +714,7 @@ impl NexusBackend { flow_job_name, workflow_details, pt::peerdb_flow::FlowStatus::StatusRunning, + None, ) .await .map_err(|err| { @@ -755,15 +722,11 @@ impl NexusBackend { format!("unable to resume flow job: {:?}", err).into(), ) })?; - let drop_mirror_success = format!("RESUME MIRROR {}", flow_job_name); - Ok(vec![Response::Execution(Tag::new( - &drop_mirror_success, - ))]) + let resume_mirror_success = format!("RESUME MIRROR {}", flow_job_name); + Ok(vec![Response::Execution(Tag::new(&resume_mirror_success))]) } else if *if_exists { let no_mirror_success = "NO SUCH MIRROR"; - Ok(vec![Response::Execution(Tag::new( - no_mirror_success, - ))]) + Ok(vec![Response::Execution(Tag::new(no_mirror_success))]) } else { Err(PgWireError::UserError(Box::new(ErrorInfo::new( "ERROR".to_owned(), diff --git a/protos/flow.proto b/protos/flow.proto index 60d15d9d91..cef3f5c7e9 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -136,6 +136,7 @@ message EnsurePullabilityBatchInput { peerdb_peers.Peer peer_connection_config = 1; string flow_job_name = 2; repeated string source_table_identifiers = 3; + bool check_constraints = 4; } message PostgresTableIdentifier { @@ -349,7 +350,7 @@ message QRepFlowState { uint64 num_partitions_processed = 2; bool needs_resync = 3; bool disable_wait_for_new_rows = 4; - FlowStatus current_flow_state = 5; + FlowStatus current_flow_status = 5; } message PeerDBColumns { @@ -387,16 +388,17 @@ enum FlowStatus { STATUS_TERMINATED = 7; } -message CDCFlowStateUpdate { +message CDCFlowConfigUpdate { + repeated TableMapping additional_tables = 1; } -message QRepFlowStateUpdate { +message QRepFlowConfigUpdate { } -message FlowStateUpdate { +message FlowConfigUpdate { oneof update { - CDCFlowStateUpdate cdc_flow_state_update = 1; - QRepFlowStateUpdate qrep_flow_state_update = 2; + CDCFlowConfigUpdate cdc_flow_config_update = 1; + QRepFlowConfigUpdate qrep_flow_config_update = 2; } } @@ -404,3 +406,9 @@ message SetupFlowOutput { map src_table_id_name_mapping = 1; map table_name_schema_mapping = 2; } + +message AddTablesToPublicationInput{ + string flow_job_name = 1; + string publication_name = 2; + repeated TableMapping additional_tables = 3; +} diff --git a/protos/route.proto b/protos/route.proto index 577be49a4f..6dcfbe14ad 100644 --- a/protos/route.proto +++ b/protos/route.proto @@ -203,7 +203,7 @@ message FlowStateChangeRequest { peerdb_peers.Peer source_peer = 3; peerdb_peers.Peer destination_peer = 4; // only can be sent in certain situations - optional peerdb_flow.FlowStateUpdate flow_state_update = 5; + optional peerdb_flow.FlowConfigUpdate flow_config_update = 5; } message FlowStateChangeResponse { From 9c801286ff41535bdb2f095284a7486e1a1bb147 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 18 Jan 2024 21:58:17 +0000 Subject: [PATCH 02/28] cdc_flow: listen for shutdown request while sync flow in progress (#1103) This became particularly important after sync flow began waiting for initial records Also move checking for cdc parameters after signal processing, updating cdc parameters to latest for next flow if parameters changed while pause --- flow/workflows/cdc_flow.go | 122 +++++++++++++++++++++---------------- 1 file changed, 69 insertions(+), 53 deletions(-) diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index ad0685e910..bc29eb412d 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -143,16 +143,6 @@ func GetChildWorkflowID( // CDCFlowWorkflowResult is the result of the PeerFlowWorkflow. type CDCFlowWorkflowResult = CDCFlowWorkflowState -func (w *CDCFlowWorkflowExecution) receiveAndHandleSignalAsync(ctx workflow.Context, state *CDCFlowWorkflowState) { - signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) - - var signalVal shared.CDCFlowSignal - ok := signalChan.ReceiveAsync(&signalVal) - if ok { - state.ActiveSignal = shared.FlowSignalHandler(state.ActiveSignal, signalVal, w.logger) - } -} - func (w *CDCFlowWorkflowExecution) processCDCFlowConfigUpdates(ctx workflow.Context, cfg *protos.FlowConnectionConfigs, state *CDCFlowWorkflowState, limits *CDCFlowLimits, mirrorNameSearch *map[string]interface{}, @@ -414,12 +404,24 @@ func CDCFlowWorkflowWithConfig( currentSyncFlowNum := 0 totalRecordsSynced := 0 - for { - // check and act on signals before a fresh flow starts. - w.receiveAndHandleSignalAsync(ctx, state) + var canceled bool + signalChan := workflow.GetSignalChannel(ctx, shared.FlowSignalName) + mainLoopSelector := workflow.NewSelector(ctx) + mainLoopSelector.AddReceive(signalChan, func(c workflow.ReceiveChannel, _ bool) { + var signalVal shared.CDCFlowSignal + c.ReceiveAsync(&signalVal) + state.ActiveSignal = shared.FlowSignalHandler(state.ActiveSignal, signalVal, w.logger) + }) + mainLoopSelector.AddReceive(ctx.Done(), func(_ workflow.ReceiveChannel, _ bool) { + canceled = true + }) - if err := ctx.Err(); err != nil { - return nil, err + for { + for !canceled && mainLoopSelector.HasPending() { + mainLoopSelector.Select(ctx) + } + if canceled { + break } if state.ActiveSignal == shared.PauseSignal { @@ -456,6 +458,7 @@ func CDCFlowWorkflowWithConfig( return state, nil } + cdcPropertiesSelector.Select(ctx) state.CurrentFlowStatus = protos.FlowStatus_STATUS_RUNNING // check if total sync flows have been completed @@ -498,54 +501,68 @@ func CDCFlowWorkflowWithConfig( syncFlowOptions, ) + var syncDone bool var childSyncFlowRes *model.SyncResponse - if err := childSyncFlowFuture.Get(syncCtx, &childSyncFlowRes); err != nil { - w.logger.Error("failed to execute sync flow: ", err) - state.SyncFlowErrors = append(state.SyncFlowErrors, err.Error()) - } else { - state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) - if childSyncFlowRes != nil { + mainLoopSelector.AddFuture(childSyncFlowFuture, func(f workflow.Future) { + syncDone = true + if err := f.Get(syncCtx, &childSyncFlowRes); err != nil { + w.logger.Error("failed to execute sync flow: ", err) + state.SyncFlowErrors = append(state.SyncFlowErrors, err.Error()) + } else if childSyncFlowRes != nil { + state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) state.RelationMessageMapping = childSyncFlowRes.RelationMessageMapping totalRecordsSynced += int(childSyncFlowRes.NumRecordsSynced) + w.logger.Info("Total records synced: ", totalRecordsSynced) } - } - - w.logger.Info("Total records synced: ", totalRecordsSynced) - var tableSchemaDeltas []*protos.TableSchemaDelta = nil - if childSyncFlowRes != nil { - tableSchemaDeltas = childSyncFlowRes.TableSchemaDeltas - } + var tableSchemaDeltas []*protos.TableSchemaDelta = nil + if childSyncFlowRes != nil { + tableSchemaDeltas = childSyncFlowRes.TableSchemaDeltas + } - // slightly hacky: table schema mapping is cached, so we need to manually update it if schema changes. - if tableSchemaDeltas != nil { - modifiedSrcTables := make([]string, 0) - modifiedDstTables := make([]string, 0) + // slightly hacky: table schema mapping is cached, so we need to manually update it if schema changes. + if tableSchemaDeltas != nil { + modifiedSrcTables := make([]string, 0) + modifiedDstTables := make([]string, 0) - for _, tableSchemaDelta := range tableSchemaDeltas { - modifiedSrcTables = append(modifiedSrcTables, tableSchemaDelta.SrcTableName) - modifiedDstTables = append(modifiedDstTables, tableSchemaDelta.DstTableName) - } + for _, tableSchemaDelta := range tableSchemaDeltas { + modifiedSrcTables = append(modifiedSrcTables, tableSchemaDelta.SrcTableName) + modifiedDstTables = append(modifiedDstTables, tableSchemaDelta.DstTableName) + } - getModifiedSchemaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ - StartToCloseTimeout: 5 * time.Minute, - }) - getModifiedSchemaFuture := workflow.ExecuteActivity(getModifiedSchemaCtx, flowable.GetTableSchema, - &protos.GetTableSchemaBatchInput{ - PeerConnectionConfig: cfg.Source, - TableIdentifiers: modifiedSrcTables, - FlowName: cfg.FlowJobName, + getModifiedSchemaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 5 * time.Minute, }) - - var getModifiedSchemaRes *protos.GetTableSchemaBatchOutput - if err := getModifiedSchemaFuture.Get(ctx, &getModifiedSchemaRes); err != nil { - w.logger.Error("failed to execute schema update at source: ", err) - state.SyncFlowErrors = append(state.SyncFlowErrors, err.Error()) - } else { - for i := range modifiedSrcTables { - state.TableNameSchemaMapping[modifiedDstTables[i]] = getModifiedSchemaRes.TableNameSchemaMapping[modifiedSrcTables[i]] + getModifiedSchemaFuture := workflow.ExecuteActivity(getModifiedSchemaCtx, flowable.GetTableSchema, + &protos.GetTableSchemaBatchInput{ + PeerConnectionConfig: cfg.Source, + TableIdentifiers: modifiedSrcTables, + FlowName: cfg.FlowJobName, + }) + + var getModifiedSchemaRes *protos.GetTableSchemaBatchOutput + if err := getModifiedSchemaFuture.Get(ctx, &getModifiedSchemaRes); err != nil { + w.logger.Error("failed to execute schema update at source: ", err) + state.SyncFlowErrors = append(state.SyncFlowErrors, err.Error()) + } else { + for i := range modifiedSrcTables { + state.TableNameSchemaMapping[modifiedDstTables[i]] = getModifiedSchemaRes.TableNameSchemaMapping[modifiedSrcTables[i]] + } } } + }) + + for !syncDone && !canceled && state.ActiveSignal != shared.ShutdownSignal { + mainLoopSelector.Select(ctx) + } + if canceled { + break + } + // check if the peer flow has been shutdown + if state.ActiveSignal == shared.ShutdownSignal { + w.logger.Info("peer flow has been shutdown") + state.CurrentFlowStatus = protos.FlowStatus_STATUS_TERMINATED + return state, nil } normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) @@ -577,7 +594,6 @@ func CDCFlowWorkflowWithConfig( } else { state.NormalizeFlowStatuses = append(state.NormalizeFlowStatuses, childNormalizeFlowRes) } - cdcPropertiesSelector.Select(ctx) } state.TruncateProgress(w.logger) From 1fda8914ea11f93c7031be0dde9f340160e7b630 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 19 Jan 2024 16:12:19 +0530 Subject: [PATCH 03/28] RegisterHStore shouldn't fail when hstore isn't present (#1108) Currently fails in initial load with something like `failed to begin transaction: no rows in result set` --- flow/connectors/utils/postgres.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flow/connectors/utils/postgres.go b/flow/connectors/utils/postgres.go index 476cabfcf0..c3cc48659f 100644 --- a/flow/connectors/utils/postgres.go +++ b/flow/connectors/utils/postgres.go @@ -62,6 +62,10 @@ func RegisterHStore(ctx context.Context, conn *pgx.Conn) error { var hstoreOID uint32 err := conn.QueryRow(context.Background(), `select oid from pg_type where typname = 'hstore'`).Scan(&hstoreOID) if err != nil { + // hstore isn't present, just proceed + if err == pgx.ErrNoRows { + return nil + } return err } From 6e7a58f17d6d9a44dec726858702d6d627dcf346 Mon Sep 17 00:00:00 2001 From: pankaj-peerdb <149565017+pankaj-peerdb@users.noreply.github.com> Date: Fri, 19 Jan 2024 18:09:31 +0530 Subject: [PATCH 04/28] fix mirror ui (#1109) Quick fix to fix the overflowing text problem. On smaller screens now the cards would be of different size but will not overflow Before Screenshot 2024-01-19 at 5 39 49 PM After Screenshot 2024-01-19 at 5 43 05 PM --- ui/app/mirrors/create/mirrorcards.tsx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ui/app/mirrors/create/mirrorcards.tsx b/ui/app/mirrors/create/mirrorcards.tsx index b01f5fc0ba..9e781324f1 100644 --- a/ui/app/mirrors/create/mirrorcards.tsx +++ b/ui/app/mirrors/create/mirrorcards.tsx @@ -46,7 +46,7 @@ const MirrorCards = ({ style={{ padding: '0.5rem', width: '35%', - height: '22vh', + minHeight: '22vh', marginRight: card.title === 'Query Replication' ? '0.5rem' : 'auto', marginLeft: From 390b98f11b23af7647f8bb35319ab88a0b692288 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 19 Jan 2024 19:14:37 +0530 Subject: [PATCH 05/28] minor fixes and improvements for pg connector (#1102) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 1. Use `pgx.CollectRows` for simple cases, can be extended to more complex cases with some structs. 2. Using upserts to update sync metadata now 3. Explicitly opt into `publish_via_partition_root` for Postgres 13+ --------- Co-authored-by: Philip Dubé --- flow/cmd/peer_data.go | 14 +--- flow/connectors/core.go | 2 +- flow/connectors/postgres/client.go | 84 ++++++------------- .../{qrep_sync_method.go => qrep_sql_sync.go} | 0 4 files changed, 29 insertions(+), 71 deletions(-) rename flow/connectors/postgres/{qrep_sync_method.go => qrep_sql_sync.go} (100%) diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index af29bd6d80..f91f37bf8c 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -9,6 +9,7 @@ import ( connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "google.golang.org/protobuf/proto" @@ -60,16 +61,9 @@ func (h *FlowRequestHandler) GetSchemas( return &protos.PeerSchemasResponse{Schemas: nil}, err } - defer rows.Close() - var schemas []string - for rows.Next() { - var schema string - err := rows.Scan(&schema) - if err != nil { - return &protos.PeerSchemasResponse{Schemas: nil}, err - } - - schemas = append(schemas, schema) + schemas, err := pgx.CollectRows[string](rows, pgx.RowTo) + if err != nil { + return &protos.PeerSchemasResponse{Schemas: nil}, err } return &protos.PeerSchemasResponse{Schemas: schemas}, nil } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 0f3e588cc7..8abc6f3bf8 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -35,7 +35,7 @@ type CDCPullConnector interface { EnsurePullability(req *protos.EnsurePullabilityBatchInput) ( *protos.EnsurePullabilityBatchOutput, error) - // Methods related to retrieving and pusing records for this connector as a source and destination. + // Methods related to retrieving and pushing records for this connector as a source and destination. // PullRecords pulls records from the source, and returns a RecordBatch. // This method should be idempotent, and should be able to be called multiple times with the same request. diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index a5c7419fbe..f18d38be52 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -35,9 +35,9 @@ const ( getLastSyncAndNormalizeBatchID_SQL = "SELECT sync_batch_id,normalize_batch_id FROM %s.%s WHERE mirror_job_name=$1" createNormalizedTableSQL = "CREATE TABLE IF NOT EXISTS %s(%s)" - insertJobMetadataSQL = "INSERT INTO %s.%s VALUES ($1,$2,$3,$4)" + upsertJobMetadataForSyncSQL = `INSERT INTO %s.%s AS j VALUES ($1,$2,$3,$4) + ON CONFLICT(mirror_job_name) DO UPDATE SET lsn_offset=GREATEST(j.lsn_offset, EXCLUDED.lsn_offset), sync_batch_id=EXCLUDED.sync_batch_id` checkIfJobMetadataExistsSQL = "SELECT COUNT(1)::TEXT::BOOL FROM %s.%s WHERE mirror_job_name=$1" - updateMetadataForSyncRecordsSQL = "UPDATE %s.%s SET lsn_offset=GREATEST(lsn_offset, $1), sync_batch_id=$2 WHERE mirror_job_name=$3" updateMetadataForNormalizeRecordsSQL = "UPDATE %s.%s SET normalize_batch_id=$1 WHERE mirror_job_name=$2" getDistinctDestinationTableNamesSQL = `SELECT DISTINCT _peerdb_destination_table_name FROM %s.%s WHERE @@ -160,8 +160,6 @@ func (c *PostgresConnector) getReplicaIdentityIndexColumns(relID uint32, schemaT // getColumnNamesForIndex returns the column names for a given index. func (c *PostgresConnector) getColumnNamesForIndex(indexOID oid.Oid) ([]string, error) { - var col pgtype.Text - cols := make([]string, 0) rows, err := c.pool.Query(c.ctx, `SELECT a.attname FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) @@ -170,14 +168,10 @@ func (c *PostgresConnector) getColumnNamesForIndex(indexOID oid.Oid) ([]string, if err != nil { return nil, fmt.Errorf("error getting columns for index %v: %w", indexOID, err) } - defer rows.Close() - for rows.Next() { - err = rows.Scan(&col) - if err != nil { - return nil, fmt.Errorf("error scanning column for index %v: %w", indexOID, err) - } - cols = append(cols, col.String) + cols, err := pgx.CollectRows[string](rows, pgx.RowTo) + if err != nil { + return nil, fmt.Errorf("error scanning column for index %v: %w", indexOID, err) } return cols, nil } @@ -309,9 +303,18 @@ func (c *PostgresConnector) createSlotAndPublication( tableNameString := strings.Join(srcTableNames, ", ") if !s.PublicationExists { + // check and enable publish_via_partition_root + supportsPubViaRoot, err := c.majorVersionCheck(130000) + if err != nil { + return fmt.Errorf("error checking Postgres version: %w", err) + } + var pubViaRootString string + if supportsPubViaRoot { + pubViaRootString = "WITH(publish_via_partition_root=true)" + } // Create the publication to help filter changes only for the given tables - stmt := fmt.Sprintf("CREATE PUBLICATION %s FOR TABLE %s", publication, tableNameString) - _, err := c.pool.Exec(c.ctx, stmt) + stmt := fmt.Sprintf("CREATE PUBLICATION %s FOR TABLE %s %s", publication, tableNameString, pubViaRootString) + _, err = c.pool.Exec(c.ctx, stmt) if err != nil { c.logger.Warn(fmt.Sprintf("Error creating publication '%s': %v", publication, err)) return fmt.Errorf("error creating publication '%s' : %w", publication, err) @@ -471,17 +474,6 @@ func (c *PostgresConnector) jobMetadataExists(jobName string) (bool, error) { return result.Bool, nil } -func (c *PostgresConnector) jobMetadataExistsTx(tx pgx.Tx, jobName string) (bool, error) { - var result pgtype.Bool - err := tx.QueryRow(c.ctx, - fmt.Sprintf(checkIfJobMetadataExistsSQL, c.metadataSchema, mirrorJobsTableIdentifier), jobName).Scan(&result) - if err != nil { - return false, fmt.Errorf("error reading result row: %w", err) - } - - return result.Bool, nil -} - func (c *PostgresConnector) majorVersionCheck(majorVersion int) (bool, error) { var version pgtype.Int8 err := c.pool.QueryRow(c.ctx, "SELECT current_setting('server_version_num')::INTEGER").Scan(&version) @@ -495,25 +487,11 @@ func (c *PostgresConnector) majorVersionCheck(majorVersion int) (bool, error) { func (c *PostgresConnector) updateSyncMetadata(flowJobName string, lastCP int64, syncBatchID int64, syncRecordsTx pgx.Tx, ) error { - jobMetadataExists, err := c.jobMetadataExistsTx(syncRecordsTx, flowJobName) + _, err := syncRecordsTx.Exec(c.ctx, + fmt.Sprintf(upsertJobMetadataForSyncSQL, c.metadataSchema, mirrorJobsTableIdentifier), + flowJobName, lastCP, syncBatchID, 0) if err != nil { - return fmt.Errorf("failed to get sync status for flow job: %w", err) - } - - if !jobMetadataExists { - _, err := syncRecordsTx.Exec(c.ctx, - fmt.Sprintf(insertJobMetadataSQL, c.metadataSchema, mirrorJobsTableIdentifier), - flowJobName, lastCP, syncBatchID, 0) - if err != nil { - return fmt.Errorf("failed to insert flow job status: %w", err) - } - } else { - _, err := syncRecordsTx.Exec(c.ctx, - fmt.Sprintf(updateMetadataForSyncRecordsSQL, c.metadataSchema, mirrorJobsTableIdentifier), - lastCP, syncBatchID, flowJobName) - if err != nil { - return fmt.Errorf("failed to update flow job status: %w", err) - } + return fmt.Errorf("failed to upsert flow job status: %w", err) } return nil @@ -522,15 +500,7 @@ func (c *PostgresConnector) updateSyncMetadata(flowJobName string, lastCP int64, func (c *PostgresConnector) updateNormalizeMetadata(flowJobName string, normalizeBatchID int64, normalizeRecordsTx pgx.Tx, ) error { - jobMetadataExists, err := c.jobMetadataExistsTx(normalizeRecordsTx, flowJobName) - if err != nil { - return fmt.Errorf("failed to get sync status for flow job: %w", err) - } - if !jobMetadataExists { - return fmt.Errorf("job metadata does not exist, unable to update") - } - - _, err = normalizeRecordsTx.Exec(c.ctx, + _, err := normalizeRecordsTx.Exec(c.ctx, fmt.Sprintf(updateMetadataForNormalizeRecordsSQL, c.metadataSchema, mirrorJobsTableIdentifier), normalizeBatchID, flowJobName) if err != nil { @@ -550,16 +520,10 @@ func (c *PostgresConnector) getDistinctTableNamesInBatch(flowJobName string, syn if err != nil { return nil, fmt.Errorf("error while retrieving table names for normalization: %w", err) } - defer rows.Close() - var result pgtype.Text - destinationTableNames := make([]string, 0) - for rows.Next() { - err = rows.Scan(&result) - if err != nil { - return nil, fmt.Errorf("failed to read row: %w", err) - } - destinationTableNames = append(destinationTableNames, result.String) + destinationTableNames, err := pgx.CollectRows[string](rows, pgx.RowTo) + if err != nil { + return nil, fmt.Errorf("failed to scan row: %w", err) } return destinationTableNames, nil } diff --git a/flow/connectors/postgres/qrep_sync_method.go b/flow/connectors/postgres/qrep_sql_sync.go similarity index 100% rename from flow/connectors/postgres/qrep_sync_method.go rename to flow/connectors/postgres/qrep_sql_sync.go From c8bce782bb9eba2d8d7223166beb61b618e27c6e Mon Sep 17 00:00:00 2001 From: pankaj-peerdb <149565017+pankaj-peerdb@users.noreply.github.com> Date: Fri, 19 Jan 2024 19:15:47 +0530 Subject: [PATCH 06/28] Clickhouse UI (#1022) --- ui/app/api/peers/route.ts | 7 + ui/app/dto/PeersDTO.ts | 2 + ui/app/peers/create/[peerType]/handlers.ts | 4 + ui/app/peers/create/[peerType]/helpers/ch.ts | 62 +++++++ .../peers/create/[peerType]/helpers/common.ts | 3 + ui/app/peers/create/[peerType]/page.tsx | 5 + ui/app/peers/create/[peerType]/schema.ts | 40 +++++ ui/components/PeerComponent.tsx | 2 + ui/components/PeerForms/ClickhouseConfig.tsx | 156 ++++++++++++++++++ ui/components/SelectSource.tsx | 4 +- ui/public/svgs/ch.svg | 1 + 11 files changed, 285 insertions(+), 1 deletion(-) create mode 100644 ui/app/peers/create/[peerType]/helpers/ch.ts create mode 100644 ui/components/PeerForms/ClickhouseConfig.tsx create mode 100644 ui/public/svgs/ch.svg diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index 03aa98ae4a..964157c71b 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -8,6 +8,7 @@ import { import prisma from '@/app/utils/prisma'; import { BigqueryConfig, + ClickhouseConfig, DBType, Peer, PostgresConfig, @@ -50,6 +51,12 @@ const constructPeer = ( type: DBType.BIGQUERY, bigqueryConfig: config as BigqueryConfig, }; + case 'CLICKHOUSE': + return { + name, + type: DBType.CLICKHOUSE, + clickhouseConfig: config as ClickhouseConfig, + }; case 'S3': return { name, diff --git a/ui/app/dto/PeersDTO.ts b/ui/app/dto/PeersDTO.ts index 80de38124b..339ba7a9b1 100644 --- a/ui/app/dto/PeersDTO.ts +++ b/ui/app/dto/PeersDTO.ts @@ -1,5 +1,6 @@ import { BigqueryConfig, + ClickhouseConfig, PostgresConfig, S3Config, SnowflakeConfig, @@ -41,6 +42,7 @@ export type PeerConfig = | PostgresConfig | SnowflakeConfig | BigqueryConfig + | ClickhouseConfig | S3Config; export type CatalogPeer = { id: number; diff --git a/ui/app/peers/create/[peerType]/handlers.ts b/ui/app/peers/create/[peerType]/handlers.ts index 2eeb657bbf..ab8718dc14 100644 --- a/ui/app/peers/create/[peerType]/handlers.ts +++ b/ui/app/peers/create/[peerType]/handlers.ts @@ -48,6 +48,10 @@ const validateFields = ( const bqConfig = bqSchema.safeParse(config); if (!bqConfig.success) validationErr = bqConfig.error.issues[0].message; break; + case 'CLICKHOUSE': + const chConfig = chSchema.safeParse(config); + if (!chConfig.success) validationErr = chConfig.error.issues[0].message; + break; case 'S3': const s3Config = s3Schema.safeParse(config); if (!s3Config.success) validationErr = s3Config.error.issues[0].message; diff --git a/ui/app/peers/create/[peerType]/helpers/ch.ts b/ui/app/peers/create/[peerType]/helpers/ch.ts new file mode 100644 index 0000000000..a85552c2b8 --- /dev/null +++ b/ui/app/peers/create/[peerType]/helpers/ch.ts @@ -0,0 +1,62 @@ +import { ClickhouseConfig, SSHConfig } from '@/grpc_generated/peers'; +import { Dispatch, SetStateAction } from 'react'; +import { PeerSetting } from './common'; + +export const clickhouseSetting: PeerSetting[] = [ + { + label: 'Host', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, host: value })), + tips: 'Specifies the IP host name or address on which postgres is to listen for TCP/IP connections from client applications. Ensure that this host has us whitelisted so we can connect to it.', + }, + { + label: 'Port', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, port: parseInt(value, 10) })), + type: 'number', // type for textfield + default: 5432, + tips: 'Specifies the TCP/IP port or local Unix domain socket file extension on which postgres is listening for connections from client applications.', + }, + { + label: 'User', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, user: value })), + tips: 'Specify the user that we should use to connect to this host.', + helpfulLink: 'https://www.postgresql.org/docs/8.0/user-manag.html', + }, + { + label: 'Password', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, password: value })), + type: 'password', + tips: 'Password associated with the user you provided.', + helpfulLink: 'https://www.postgresql.org/docs/current/auth-password.html', + }, + { + label: 'Database', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, database: value })), + tips: 'Specify which database to associate with this peer.', + helpfulLink: + 'https://www.postgresql.org/docs/current/sql-createdatabase.html', + }, + { + label: 'S3 Integration', + stateHandler: (value, setter) => + setter((curr) => ({ ...curr, s3Integration: value })), + optional: true, + tips: `This is needed only if you plan to run a mirror and you wish to stage AVRO files on S3.`, + helpfulLink: + 'https://docs.snowflake.com/en/user-guide/data-load-s3-config-storage-integration', + }, + +]; + +export const blankClickhouseSetting: ClickhouseConfig = { + host: '', + port: 5432, + user: '', + password: '', + database: '', + s3Integration:'' +}; \ No newline at end of file diff --git a/ui/app/peers/create/[peerType]/helpers/common.ts b/ui/app/peers/create/[peerType]/helpers/common.ts index b1c27e0edc..3b7a017283 100644 --- a/ui/app/peers/create/[peerType]/helpers/common.ts +++ b/ui/app/peers/create/[peerType]/helpers/common.ts @@ -3,6 +3,7 @@ import { blankBigquerySetting } from './bq'; import { blankPostgresSetting } from './pg'; import { blankS3Setting } from './s3'; import { blankSnowflakeSetting } from './sf'; +import {blankClickhouseSetting} from './ch'; export interface PeerSetting { label: string; @@ -22,6 +23,8 @@ export const getBlankSetting = (dbType: string): PeerConfig => { return blankSnowflakeSetting; case 'BIGQUERY': return blankBigquerySetting; + case 'CLICKHOUSE': + return blankClickhouseSetting; case 'S3': return blankS3Setting; default: diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index a611df87ab..4ea5f30262 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -4,6 +4,8 @@ import BigqueryForm from '@/components/PeerForms/BigqueryConfig'; import PostgresForm from '@/components/PeerForms/PostgresForm'; import S3Form from '@/components/PeerForms/S3Form'; import SnowflakeForm from '@/components/PeerForms/SnowflakeForm'; +import ClickhouseForm from '@/components/PeerForms/ClickhouseConfig'; + import { Button } from '@/lib/Button'; import { ButtonGroup } from '@/lib/ButtonGroup'; import { Label } from '@/lib/Label'; @@ -18,6 +20,7 @@ import { handleCreate, handleValidate } from './handlers'; import { getBlankSetting } from './helpers/common'; import { postgresSetting } from './helpers/pg'; import { snowflakeSetting } from './helpers/sf'; +import {clickhouseSetting} from './helpers/ch'; type CreateConfigProps = { params: { peerType: string }; @@ -44,6 +47,8 @@ export default function CreateConfig({ return ; case 'BIGQUERY': return ; + case 'CLICKHOUSE': + return ; case 'S3': return ; default: diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 5bedeaa26f..5b4e7f81af 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -233,6 +233,46 @@ export const bqSchema = z.object({ ), }); +export const chSchema = z.object({ + host: z + .string({ + required_error: 'Host is required', + invalid_type_error: 'Host must be a string', + }) + .min(1, { message: 'Host cannot be empty' }) + .max(255, 'Host must be less than 255 characters'), + port: z + .number({ + required_error: 'Port is required', + invalid_type_error: 'Port must be a number', + }) + .int() + .min(1, 'Port must be a positive integer') + .max(65535, 'Port must be below 65535'), + database: z + .string({ + required_error: 'Database is required', + invalid_type_error: 'Database must be a string', + }) + .min(1, { message: 'Database name should be non-empty' }) + .max(100, 'Database must be less than 100 characters'), + user: z + .string({ + required_error: 'User is required', + invalid_type_error: 'User must be a string', + }) + .min(1, 'User must be non-empty') + .max(64, 'User must be less than 64 characters'), + password: z + .string({ + required_error: 'Password is required', + invalid_type_error: 'Password must be a string', + }) + .min(1, 'Password must be non-empty') + .max(100, 'Password must be less than 100 characters'), +}); + + export const s3Schema = z.object({ url: z .string({ diff --git a/ui/components/PeerComponent.tsx b/ui/components/PeerComponent.tsx index 7378e91635..6cf36bde11 100644 --- a/ui/components/PeerComponent.tsx +++ b/ui/components/PeerComponent.tsx @@ -18,6 +18,8 @@ export const DBTypeToImageMapping = (peerType: DBType | string) => { case DBType.S3: case 'S3': return '/svgs/aws.svg'; + case 'CLICKHOUSE': + return '/svgs/ch.svg'; case DBType.EVENTHUB_GROUP: case DBType.EVENTHUB: return '/svgs/ms.svg'; diff --git a/ui/components/PeerForms/ClickhouseConfig.tsx b/ui/components/PeerForms/ClickhouseConfig.tsx new file mode 100644 index 0000000000..51ab891cd8 --- /dev/null +++ b/ui/components/PeerForms/ClickhouseConfig.tsx @@ -0,0 +1,156 @@ +'use client'; +import { PeerSetter } from '@/app/dto/PeersDTO'; +import { PeerSetting } from '@/app/peers/create/[peerType]/helpers/common'; +import { + blankSSHConfig, + sshSetting, +} from '@/app/peers/create/[peerType]/helpers/pg'; +import { SSHConfig } from '@/grpc_generated/peers'; +import { Label } from '@/lib/Label'; +import { RowWithTextField } from '@/lib/Layout'; +import { Switch } from '@/lib/Switch'; +import { TextField } from '@/lib/TextField'; +import { Tooltip } from '@/lib/Tooltip'; +import { useEffect, useState } from 'react'; +import { InfoPopover } from '../InfoPopover'; +interface ConfigProps { + settings: PeerSetting[]; + setter: PeerSetter; +} + +export default function PostgresForm({ settings, setter }: ConfigProps) { + const [showSSH, setShowSSH] = useState(false); + const [sshConfig, setSSHConfig] = useState(blankSSHConfig); + + const handleChange = ( + e: React.ChangeEvent, + setting: PeerSetting + ) => { + setting.stateHandler(e.target.value, setter); + }; + + useEffect(() => { + setter((prev) => { + return { + ...prev, + sshConfig: showSSH ? sshConfig : undefined, + }; + }); + }, [sshConfig, setter, showSSH]); + + return ( + <> + {settings.map((setting, id) => { + return ( + + {setting.label}{' '} + {!setting.optional && ( + + + + )} + + } + action={ +
+ ) => + handleChange(e, setting) + } + /> + {setting.tips && ( + + )} +
+ } + /> + ); + })} + + + +
+ + setShowSSH(state)} /> +
+ {showSSH && + sshSetting.map((sshParam, index) => ( + + {sshParam.label}{' '} + {!sshParam.optional && ( + + + + )} + + } + action={ +
+ ) => + sshParam.stateHandler(e.target.value, setSSHConfig) + } + type={sshParam.type} + defaultValue={ + (sshConfig as SSHConfig)[ + sshParam.label === 'BASE64 Private Key' + ? 'privateKey' + : (sshParam.label.toLowerCase() as + | 'host' + | 'port' + | 'user' + | 'password' + | 'privateKey') + ] || '' + } + /> + {sshParam.tips && } +
+ } + /> + ))} + + ); +} \ No newline at end of file diff --git a/ui/components/SelectSource.tsx b/ui/components/SelectSource.tsx index a83b7d64ab..a3f848bb3a 100644 --- a/ui/components/SelectSource.tsx +++ b/ui/components/SelectSource.tsx @@ -31,7 +31,9 @@ export default function SelectSource({ (value === 'POSTGRES' || value === 'SNOWFLAKE' || value === 'BIGQUERY' || - value === 'S3') + value === 'S3' || + value === 'CLICKHOUSE' + ) ) .map((value) => ({ label: value, value })); diff --git a/ui/public/svgs/ch.svg b/ui/public/svgs/ch.svg new file mode 100644 index 0000000000..f2144b5d7e --- /dev/null +++ b/ui/public/svgs/ch.svg @@ -0,0 +1 @@ + \ No newline at end of file From 91f96ac21e6e8d7373562f8894a1f40302d9e2d0 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 19 Jan 2024 22:03:06 +0530 Subject: [PATCH 07/28] UI FIx: add an import, formatting (#1111) Seems like we were missing an import and also UI needed to be formatted (prettier) --- ui/app/api/peers/route.ts | 2 +- ui/app/peers/create/[peerType]/handlers.ts | 3 ++- ui/app/peers/create/[peerType]/helpers/ch.ts | 8 +++----- ui/app/peers/create/[peerType]/helpers/common.ts | 2 +- ui/app/peers/create/[peerType]/page.tsx | 8 +++++--- ui/app/peers/create/[peerType]/schema.ts | 1 - ui/components/PeerComponent.tsx | 2 +- ui/components/PeerForms/ClickhouseConfig.tsx | 2 +- ui/components/SelectSource.tsx | 3 +-- 9 files changed, 15 insertions(+), 16 deletions(-) diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index 964157c71b..200aab3d79 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -56,7 +56,7 @@ const constructPeer = ( name, type: DBType.CLICKHOUSE, clickhouseConfig: config as ClickhouseConfig, - }; + }; case 'S3': return { name, diff --git a/ui/app/peers/create/[peerType]/handlers.ts b/ui/app/peers/create/[peerType]/handlers.ts index ab8718dc14..e43164aa96 100644 --- a/ui/app/peers/create/[peerType]/handlers.ts +++ b/ui/app/peers/create/[peerType]/handlers.ts @@ -7,6 +7,7 @@ import { S3Config } from '@/grpc_generated/peers'; import { Dispatch, SetStateAction } from 'react'; import { bqSchema, + chSchema, peerNameSchema, pgSchema, s3Schema, @@ -51,7 +52,7 @@ const validateFields = ( case 'CLICKHOUSE': const chConfig = chSchema.safeParse(config); if (!chConfig.success) validationErr = chConfig.error.issues[0].message; - break; + break; case 'S3': const s3Config = s3Schema.safeParse(config); if (!s3Config.success) validationErr = s3Config.error.issues[0].message; diff --git a/ui/app/peers/create/[peerType]/helpers/ch.ts b/ui/app/peers/create/[peerType]/helpers/ch.ts index a85552c2b8..57eee3c39a 100644 --- a/ui/app/peers/create/[peerType]/helpers/ch.ts +++ b/ui/app/peers/create/[peerType]/helpers/ch.ts @@ -1,5 +1,4 @@ -import { ClickhouseConfig, SSHConfig } from '@/grpc_generated/peers'; -import { Dispatch, SetStateAction } from 'react'; +import { ClickhouseConfig } from '@/grpc_generated/peers'; import { PeerSetting } from './common'; export const clickhouseSetting: PeerSetting[] = [ @@ -49,7 +48,6 @@ export const clickhouseSetting: PeerSetting[] = [ helpfulLink: 'https://docs.snowflake.com/en/user-guide/data-load-s3-config-storage-integration', }, - ]; export const blankClickhouseSetting: ClickhouseConfig = { @@ -58,5 +56,5 @@ export const blankClickhouseSetting: ClickhouseConfig = { user: '', password: '', database: '', - s3Integration:'' -}; \ No newline at end of file + s3Integration: '', +}; diff --git a/ui/app/peers/create/[peerType]/helpers/common.ts b/ui/app/peers/create/[peerType]/helpers/common.ts index 3b7a017283..a80e7bbbd6 100644 --- a/ui/app/peers/create/[peerType]/helpers/common.ts +++ b/ui/app/peers/create/[peerType]/helpers/common.ts @@ -1,9 +1,9 @@ import { PeerConfig, PeerSetter } from '@/app/dto/PeersDTO'; import { blankBigquerySetting } from './bq'; +import { blankClickhouseSetting } from './ch'; import { blankPostgresSetting } from './pg'; import { blankS3Setting } from './s3'; import { blankSnowflakeSetting } from './sf'; -import {blankClickhouseSetting} from './ch'; export interface PeerSetting { label: string; diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index 4ea5f30262..83097574a6 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -1,10 +1,10 @@ 'use client'; import { PeerConfig } from '@/app/dto/PeersDTO'; import BigqueryForm from '@/components/PeerForms/BigqueryConfig'; +import ClickhouseForm from '@/components/PeerForms/ClickhouseConfig'; import PostgresForm from '@/components/PeerForms/PostgresForm'; import S3Form from '@/components/PeerForms/S3Form'; import SnowflakeForm from '@/components/PeerForms/SnowflakeForm'; -import ClickhouseForm from '@/components/PeerForms/ClickhouseConfig'; import { Button } from '@/lib/Button'; import { ButtonGroup } from '@/lib/ButtonGroup'; @@ -17,10 +17,10 @@ import Link from 'next/link'; import { useRouter } from 'next/navigation'; import { useState } from 'react'; import { handleCreate, handleValidate } from './handlers'; +import { clickhouseSetting } from './helpers/ch'; import { getBlankSetting } from './helpers/common'; import { postgresSetting } from './helpers/pg'; import { snowflakeSetting } from './helpers/sf'; -import {clickhouseSetting} from './helpers/ch'; type CreateConfigProps = { params: { peerType: string }; @@ -48,7 +48,9 @@ export default function CreateConfig({ case 'BIGQUERY': return ; case 'CLICKHOUSE': - return ; + return ( + + ); case 'S3': return ; default: diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 5b4e7f81af..6a21892f39 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -272,7 +272,6 @@ export const chSchema = z.object({ .max(100, 'Password must be less than 100 characters'), }); - export const s3Schema = z.object({ url: z .string({ diff --git a/ui/components/PeerComponent.tsx b/ui/components/PeerComponent.tsx index 6cf36bde11..318aff7529 100644 --- a/ui/components/PeerComponent.tsx +++ b/ui/components/PeerComponent.tsx @@ -19,7 +19,7 @@ export const DBTypeToImageMapping = (peerType: DBType | string) => { case 'S3': return '/svgs/aws.svg'; case 'CLICKHOUSE': - return '/svgs/ch.svg'; + return '/svgs/ch.svg'; case DBType.EVENTHUB_GROUP: case DBType.EVENTHUB: return '/svgs/ms.svg'; diff --git a/ui/components/PeerForms/ClickhouseConfig.tsx b/ui/components/PeerForms/ClickhouseConfig.tsx index 51ab891cd8..1819e626df 100644 --- a/ui/components/PeerForms/ClickhouseConfig.tsx +++ b/ui/components/PeerForms/ClickhouseConfig.tsx @@ -153,4 +153,4 @@ export default function PostgresForm({ settings, setter }: ConfigProps) { ))} ); -} \ No newline at end of file +} diff --git a/ui/components/SelectSource.tsx b/ui/components/SelectSource.tsx index a3f848bb3a..04cc5bf3a3 100644 --- a/ui/components/SelectSource.tsx +++ b/ui/components/SelectSource.tsx @@ -32,8 +32,7 @@ export default function SelectSource({ value === 'SNOWFLAKE' || value === 'BIGQUERY' || value === 'S3' || - value === 'CLICKHOUSE' - ) + value === 'CLICKHOUSE') ) .map((value) => ({ label: value, value })); From c978de3711dbe4b437ac2a18e10733f15852930e Mon Sep 17 00:00:00 2001 From: pankaj-peerdb <149565017+pankaj-peerdb@users.noreply.github.com> Date: Fri, 19 Jan 2024 22:40:09 +0530 Subject: [PATCH 08/28] typo in s3 creation form (#1113) --- ui/components/PeerForms/S3Form.tsx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ui/components/PeerForms/S3Form.tsx b/ui/components/PeerForms/S3Form.tsx index cafef330f9..8bd2bdc426 100644 --- a/ui/components/PeerForms/S3Form.tsx +++ b/ui/components/PeerForms/S3Form.tsx @@ -152,7 +152,7 @@ const S3Form = ({ setter }: S3Props) => { You can also choose to use your own PostgreSQL database:
- + setShowMetadata(state)} />
{showMetadata && From 414508d5ec56921d1c837f13a4337c920f198971 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Sat, 20 Jan 2024 01:13:51 +0530 Subject: [PATCH 09/28] split and tidy up code for SF Avro consolidate (#1104) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Quite a bit of reorganization, moved functions from connector level to `ConsolidateHandler` level, reading from struct instead of unnecessary var passing --------- Co-authored-by: Philip Dubé --- .../snowflake/avro_transform_test.go | 2 +- flow/connectors/snowflake/qrep.go | 11 +- .../snowflake/qrep_avro_consolidate.go | 237 +++++++++++++++ flow/connectors/snowflake/qrep_avro_sync.go | 286 +----------------- flow/connectors/snowflake/snowflake.go | 2 +- 5 files changed, 255 insertions(+), 283 deletions(-) create mode 100644 flow/connectors/snowflake/qrep_avro_consolidate.go diff --git a/flow/connectors/snowflake/avro_transform_test.go b/flow/connectors/snowflake/avro_transform_test.go index ffbe896658..b7a487dcaa 100644 --- a/flow/connectors/snowflake/avro_transform_test.go +++ b/flow/connectors/snowflake/avro_transform_test.go @@ -11,7 +11,7 @@ func TestAvroTransform(t *testing.T) { `$1:"col3" AS "COL3",` + `($1:"camelCol4")::STRING AS "camelCol4",` + `CURRENT_TIMESTAMP AS "SYNC_COL"` - transform, cols := GetTransformSQL(colNames, colTypes, "sync_col") + transform, cols := getTransformSQL(colNames, colTypes, "sync_col") if transform != expectedTransform { t.Errorf("Transform SQL is not correct. Got: %v", transform) } diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index ada36cd59f..264cd3ea0e 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -47,7 +47,7 @@ func (c *SnowflakeConnector) SyncQRepRecords( return 0, nil } - avroSync := NewSnowflakeAvroSyncMethod(config, c) + avroSync := NewSnowflakeAvroSyncHandler(config, c) return avroSync.SyncQRepRecords(config, partition, tblSchema, stream) } @@ -258,13 +258,8 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig destTable := config.DestinationTableIdentifier stageName := c.getStageNameForJob(config.FlowJobName) - colNames, _, err := c.getColsFromTable(destTable) - if err != nil { - c.logger.Error(fmt.Sprintf("failed to get columns from table %s", destTable), slog.Any("error", err)) - return fmt.Errorf("failed to get columns from table %s: %w", destTable, err) - } - - err = CopyStageToDestination(c, config, destTable, stageName, colNames) + writeHandler := NewSnowflakeAvroConsolidateHandler(c, config, destTable, stageName) + err := writeHandler.CopyStageToDestination() if err != nil { c.logger.Error("failed to copy stage to destination", slog.Any("error", err)) return fmt.Errorf("failed to copy stage to destination: %w", err) diff --git a/flow/connectors/snowflake/qrep_avro_consolidate.go b/flow/connectors/snowflake/qrep_avro_consolidate.go new file mode 100644 index 0000000000..a77767eb53 --- /dev/null +++ b/flow/connectors/snowflake/qrep_avro_consolidate.go @@ -0,0 +1,237 @@ +package connsnowflake + +import ( + "fmt" + "log/slog" + "strings" + "time" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" +) + +type SnowflakeAvroConsolidateHandler struct { + connector *SnowflakeConnector + config *protos.QRepConfig + dstTableName string + stage string + allColNames []string + allColTypes []string +} + +// NewSnowflakeAvroConsolidateHandler creates a new SnowflakeAvroWriteHandler +func NewSnowflakeAvroConsolidateHandler( + connector *SnowflakeConnector, + config *protos.QRepConfig, + dstTableName string, + stage string, +) *SnowflakeAvroConsolidateHandler { + return &SnowflakeAvroConsolidateHandler{ + connector: connector, + config: config, + dstTableName: dstTableName, + stage: stage, + } +} + +func (s *SnowflakeAvroConsolidateHandler) CopyStageToDestination() error { + s.connector.logger.Info("Copying stage to destination " + s.dstTableName) + + colNames, colTypes, colsErr := s.connector.getColsFromTable(s.dstTableName) + if colsErr != nil { + return fmt.Errorf("failed to get columns from destination table: %w", colsErr) + } + s.allColNames = colNames + s.allColTypes = colTypes + + appendMode := true + if s.config.WriteMode != nil { + writeType := s.config.WriteMode.WriteType + if writeType == protos.QRepWriteType_QREP_WRITE_MODE_UPSERT { + appendMode = false + } + } + + if appendMode { + err := s.handleAppendMode() + if err != nil { + return fmt.Errorf("failed to handle append mode: %w", err) + } + } else { + err := s.handleUpsertMode() + if err != nil { + return fmt.Errorf("failed to handle upsert mode: %w", err) + } + } + + return nil +} + +func getTransformSQL(colNames []string, colTypes []string, syncedAtCol string) (string, string) { + transformations := make([]string, 0, len(colNames)) + columnOrder := make([]string, 0, len(colNames)) + for idx, avroColName := range colNames { + colType := colTypes[idx] + normalizedColName := SnowflakeIdentifierNormalize(avroColName) + columnOrder = append(columnOrder, normalizedColName) + if avroColName == syncedAtCol { + transformations = append(transformations, fmt.Sprintf("CURRENT_TIMESTAMP AS %s", normalizedColName)) + continue + } + + if utils.IsUpper(avroColName) { + avroColName = strings.ToLower(avroColName) + } + // Avro files are written with lowercase in mind, so don't normalize it like everything else + switch colType { + case "GEOGRAPHY": + transformations = append(transformations, + fmt.Sprintf("TO_GEOGRAPHY($1:\"%s\"::string, true) AS %s", avroColName, normalizedColName)) + case "GEOMETRY": + transformations = append(transformations, + fmt.Sprintf("TO_GEOMETRY($1:\"%s\"::string, true) AS %s", avroColName, normalizedColName)) + case "NUMBER": + transformations = append(transformations, + fmt.Sprintf("$1:\"%s\" AS %s", avroColName, normalizedColName)) + case "VARIANT": + transformations = append(transformations, + fmt.Sprintf("PARSE_JSON($1:\"%s\") AS %s", avroColName, normalizedColName)) + + default: + transformations = append(transformations, + fmt.Sprintf("($1:\"%s\")::%s AS %s", avroColName, colType, normalizedColName)) + } + } + transformationSQL := strings.Join(transformations, ",") + columnsSQL := strings.Join(columnOrder, ",") + + return transformationSQL, columnsSQL +} + +// copy to either the actual destination table or a tempTable +func (s *SnowflakeAvroConsolidateHandler) getCopyTransformation(copyDstTable string) string { + copyOpts := []string{ + "FILE_FORMAT = (TYPE = AVRO)", + "PURGE = TRUE", + "ON_ERROR = 'CONTINUE'", + } + transformationSQL, columnsSQL := getTransformSQL(s.allColNames, s.allColTypes, s.config.SyncedAtColName) + return fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", + copyDstTable, columnsSQL, transformationSQL, s.stage, strings.Join(copyOpts, ",")) +} + +func (s *SnowflakeAvroConsolidateHandler) handleAppendMode() error { + parsedDstTable, _ := utils.ParseSchemaTable(s.dstTableName) + copyCmd := s.getCopyTransformation(snowflakeSchemaTableNormalize(parsedDstTable)) + s.connector.logger.Info("running copy command: " + copyCmd) + _, err := s.connector.database.ExecContext(s.connector.ctx, copyCmd) + if err != nil { + return fmt.Errorf("failed to run COPY INTO command: %w", err) + } + + s.connector.logger.Info("copied file from stage " + s.stage + " to table " + s.dstTableName) + return nil +} + +func (s *SnowflakeAvroConsolidateHandler) generateUpsertMergeCommand( + tempTableName string, +) string { + upsertKeyCols := s.config.WriteMode.UpsertKeyColumns + // all cols are acquired from snowflake schema, so let us try to make upsert key cols match the case + // and also the watermark col, then the quoting should be fine + caseMatchedCols := map[string]string{} + for _, col := range s.allColNames { + caseMatchedCols[strings.ToLower(col)] = col + } + + for i, col := range upsertKeyCols { + upsertKeyCols[i] = caseMatchedCols[strings.ToLower(col)] + } + + upsertKeys := make([]string, 0, len(upsertKeyCols)) + partitionKeyCols := make([]string, 0, len(upsertKeyCols)) + for _, key := range upsertKeyCols { + quotedKey := utils.QuoteIdentifier(key) + upsertKeys = append(upsertKeys, fmt.Sprintf("dst.%s = src.%s", quotedKey, quotedKey)) + partitionKeyCols = append(partitionKeyCols, quotedKey) + } + upsertKeyClause := strings.Join(upsertKeys, " AND ") + + updateSetClauses := make([]string, 0, len(s.allColNames)) + insertColumnsClauses := make([]string, 0, len(s.allColNames)) + insertValuesClauses := make([]string, 0, len(s.allColNames)) + for _, column := range s.allColNames { + quotedColumn := utils.QuoteIdentifier(column) + updateSetClauses = append(updateSetClauses, fmt.Sprintf("%s = src.%s", quotedColumn, quotedColumn)) + insertColumnsClauses = append(insertColumnsClauses, quotedColumn) + insertValuesClauses = append(insertValuesClauses, fmt.Sprintf("src.%s", quotedColumn)) + } + updateSetClause := strings.Join(updateSetClauses, ", ") + insertColumnsClause := strings.Join(insertColumnsClauses, ", ") + insertValuesClause := strings.Join(insertValuesClauses, ", ") + selectCmd := fmt.Sprintf(` + SELECT * + FROM %s + QUALIFY ROW_NUMBER() OVER (PARTITION BY %s ORDER BY %s DESC) = 1 + `, tempTableName, strings.Join(partitionKeyCols, ","), partitionKeyCols[0]) + + mergeCmd := fmt.Sprintf(` + MERGE INTO %s dst + USING (%s) src + ON %s + WHEN MATCHED THEN UPDATE SET %s + WHEN NOT MATCHED THEN INSERT (%s) VALUES (%s) + `, s.dstTableName, selectCmd, upsertKeyClause, + updateSetClause, insertColumnsClause, insertValuesClause) + + return mergeCmd +} + +// handleUpsertMode handles the upsert mode +func (s *SnowflakeAvroConsolidateHandler) handleUpsertMode() error { + runID, err := shared.RandomUInt64() + if err != nil { + return fmt.Errorf("failed to generate run ID: %w", err) + } + + tempTableName := fmt.Sprintf("%s_temp_%d", s.dstTableName, runID) + + //nolint:gosec + createTempTableCmd := fmt.Sprintf("CREATE TEMPORARY TABLE %s AS SELECT * FROM %s LIMIT 0", + tempTableName, s.dstTableName) + if _, err := s.connector.database.ExecContext(s.connector.ctx, createTempTableCmd); err != nil { + return fmt.Errorf("failed to create temp table: %w", err) + } + s.connector.logger.Info("created temp table " + tempTableName) + + copyCmd := s.getCopyTransformation(tempTableName) + _, err = s.connector.database.ExecContext(s.connector.ctx, copyCmd) + if err != nil { + return fmt.Errorf("failed to run COPY INTO command: %w", err) + } + s.connector.logger.Info("copied file from stage " + s.stage + " to temp table " + tempTableName) + + mergeCmd := s.generateUpsertMergeCommand(tempTableName) + + startTime := time.Now() + rows, err := s.connector.database.ExecContext(s.connector.ctx, mergeCmd) + if err != nil { + return fmt.Errorf("failed to merge data into destination table '%s': %w", mergeCmd, err) + } + rowCount, err := rows.RowsAffected() + if err == nil { + totalRowsAtTarget, err := s.connector.getTableCounts([]string{s.dstTableName}) + if err != nil { + return err + } + s.connector.logger.Info(fmt.Sprintf("merged %d rows into destination table %s, total rows at target: %d", + rowCount, s.dstTableName, totalRowsAtTarget)) + } else { + s.connector.logger.Error("failed to get rows affected", slog.Any("error", err)) + } + + s.connector.logger.Info(fmt.Sprintf("merged data from temp table %s into destination table %s, time taken %v", + tempTableName, s.dstTableName, time.Since(startTime))) + return nil +} diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 9f81d254de..146876f8c6 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -18,27 +18,22 @@ import ( "go.temporal.io/sdk/activity" ) -type CopyInfo struct { - transformationSQL string - columnsSQL string -} - -type SnowflakeAvroSyncMethod struct { +type SnowflakeAvroSyncHandler struct { config *protos.QRepConfig connector *SnowflakeConnector } -func NewSnowflakeAvroSyncMethod( +func NewSnowflakeAvroSyncHandler( config *protos.QRepConfig, connector *SnowflakeConnector, -) *SnowflakeAvroSyncMethod { - return &SnowflakeAvroSyncMethod{ +) *SnowflakeAvroSyncHandler { + return &SnowflakeAvroSyncHandler{ config: config, connector: connector, } } -func (s *SnowflakeAvroSyncMethod) SyncRecords( +func (s *SnowflakeAvroSyncHandler) SyncRecords( dstTableSchema []*sql.ColumnType, stream *model.QRecordStream, flowJobName string, @@ -73,18 +68,14 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( } s.connector.logger.Info(fmt.Sprintf("Created stage %s", stage)) - colNames, _, err := s.connector.getColsFromTable(s.config.DestinationTableIdentifier) - if err != nil { - return 0, err - } - err = s.putFileToStage(avroFile, stage) if err != nil { return 0, err } s.connector.logger.Info("pushed avro file to stage", tableLog) - err = CopyStageToDestination(s.connector, s.config, s.config.DestinationTableIdentifier, stage, colNames) + writeHandler := NewSnowflakeAvroConsolidateHandler(s.connector, s.config, s.config.DestinationTableIdentifier, stage) + err = writeHandler.CopyStageToDestination() if err != nil { return 0, err } @@ -94,7 +85,7 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( return avroFile.NumRecords, nil } -func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( +func (s *SnowflakeAvroSyncHandler) SyncQRepRecords( config *protos.QRepConfig, partition *protos.QRepPartition, dstTableSchema []*sql.ColumnType, @@ -149,7 +140,7 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( return avroFile.NumRecords, nil } -func (s *SnowflakeAvroSyncMethod) addMissingColumns( +func (s *SnowflakeAvroSyncHandler) addMissingColumns( schema *model.QRecordSchema, dstTableSchema []*sql.ColumnType, dstTableName string, @@ -212,7 +203,7 @@ func (s *SnowflakeAvroSyncMethod) addMissingColumns( return nil } -func (s *SnowflakeAvroSyncMethod) getAvroSchema( +func (s *SnowflakeAvroSyncHandler) getAvroSchema( dstTableName string, schema *model.QRecordSchema, ) (*model.QRecordAvroSchemaDefinition, error) { @@ -225,7 +216,7 @@ func (s *SnowflakeAvroSyncMethod) getAvroSchema( return avroSchema, nil } -func (s *SnowflakeAvroSyncMethod) writeToAvroFile( +func (s *SnowflakeAvroSyncHandler) writeToAvroFile( stream *model.QRecordStream, avroSchema *model.QRecordAvroSchemaDefinition, partitionID string, @@ -270,7 +261,7 @@ func (s *SnowflakeAvroSyncMethod) writeToAvroFile( return nil, fmt.Errorf("unsupported staging path: %s", s.config.StagingPath) } -func (s *SnowflakeAvroSyncMethod) putFileToStage(avroFile *avro.AvroFile, stage string) error { +func (s *SnowflakeAvroSyncHandler) putFileToStage(avroFile *avro.AvroFile, stage string) error { if avroFile.StorageLocation != avro.AvroLocalStorage { s.connector.logger.Info("no file to put to stage") return nil @@ -292,106 +283,7 @@ func (s *SnowflakeAvroSyncMethod) putFileToStage(avroFile *avro.AvroFile, stage return nil } -func GetTransformSQL(colNames []string, colTypes []string, syncedAtCol string) (string, string) { - transformations := make([]string, 0, len(colNames)) - columnOrder := make([]string, 0, len(colNames)) - for idx, avroColName := range colNames { - colType := colTypes[idx] - normalizedColName := SnowflakeIdentifierNormalize(avroColName) - columnOrder = append(columnOrder, normalizedColName) - if avroColName == syncedAtCol { - transformations = append(transformations, fmt.Sprintf("CURRENT_TIMESTAMP AS %s", normalizedColName)) - continue - } - - if utils.IsUpper(avroColName) { - avroColName = strings.ToLower(avroColName) - } - // Avro files are written with lowercase in mind, so don't normalize it like everything else - switch colType { - case "GEOGRAPHY": - transformations = append(transformations, - fmt.Sprintf("TO_GEOGRAPHY($1:\"%s\"::string, true) AS %s", avroColName, normalizedColName)) - case "GEOMETRY": - transformations = append(transformations, - fmt.Sprintf("TO_GEOMETRY($1:\"%s\"::string, true) AS %s", avroColName, normalizedColName)) - case "NUMBER": - transformations = append(transformations, - fmt.Sprintf("$1:\"%s\" AS %s", avroColName, normalizedColName)) - case "VARIANT": - transformations = append(transformations, - fmt.Sprintf("PARSE_JSON($1:\"%s\") AS %s", avroColName, normalizedColName)) - - default: - transformations = append(transformations, - fmt.Sprintf("($1:\"%s\")::%s AS %s", avroColName, colType, normalizedColName)) - } - } - transformationSQL := strings.Join(transformations, ",") - columnsSQL := strings.Join(columnOrder, ",") - - return transformationSQL, columnsSQL -} - -func (c *SnowflakeConnector) GetCopyTransformation( - dstTableName string, - syncedAtCol string, -) (*CopyInfo, error) { - colNames, colTypes, colsErr := c.getColsFromTable(dstTableName) - if colsErr != nil { - return nil, fmt.Errorf("failed to get columns from destination table: %w", colsErr) - } - - transformationSQL, columnsSQL := GetTransformSQL(colNames, colTypes, syncedAtCol) - return &CopyInfo{transformationSQL, columnsSQL}, nil -} - -func CopyStageToDestination( - connector *SnowflakeConnector, - config *protos.QRepConfig, - dstTableName string, - stage string, - allCols []string, -) error { - connector.logger.Info("Copying stage to destination " + dstTableName) - copyOpts := []string{ - "FILE_FORMAT = (TYPE = AVRO)", - "PURGE = TRUE", - "ON_ERROR = 'CONTINUE'", - } - - writeHandler := NewSnowflakeAvroWriteHandler(connector, dstTableName, stage, copyOpts) - - appendMode := true - if config.WriteMode != nil { - writeType := config.WriteMode.WriteType - if writeType == protos.QRepWriteType_QREP_WRITE_MODE_UPSERT { - appendMode = false - } - } - - copyTransformation, err := connector.GetCopyTransformation(dstTableName, config.SyncedAtColName) - if err != nil { - return fmt.Errorf("failed to get copy transformation: %w", err) - } - if appendMode { - err := writeHandler.HandleAppendMode(copyTransformation) - if err != nil { - return fmt.Errorf("failed to handle append mode: %w", err) - } - } else { - upsertKeyCols := config.WriteMode.UpsertKeyColumns - err := writeHandler.HandleUpsertMode(allCols, upsertKeyCols, config.WatermarkColumn, - config.FlowJobName, copyTransformation) - if err != nil { - return fmt.Errorf("failed to handle upsert mode: %w", err) - } - } - - return nil -} - -func (s *SnowflakeAvroSyncMethod) insertMetadata( +func (s *SnowflakeAvroSyncHandler) insertMetadata( partition *protos.QRepPartition, flowJobName string, startTime time.Time, @@ -413,155 +305,3 @@ func (s *SnowflakeAvroSyncMethod) insertMetadata( s.connector.logger.Info("inserted metadata for partition", partitionLog) return nil } - -type SnowflakeAvroWriteHandler struct { - connector *SnowflakeConnector - dstTableName string - stage string - copyOpts []string -} - -// NewSnowflakeAvroWriteHandler creates a new SnowflakeAvroWriteHandler -func NewSnowflakeAvroWriteHandler( - connector *SnowflakeConnector, - dstTableName string, - stage string, - copyOpts []string, -) *SnowflakeAvroWriteHandler { - return &SnowflakeAvroWriteHandler{ - connector: connector, - dstTableName: dstTableName, - stage: stage, - copyOpts: copyOpts, - } -} - -func (s *SnowflakeAvroWriteHandler) HandleAppendMode( - copyInfo *CopyInfo, -) error { - parsedDstTable, _ := utils.ParseSchemaTable(s.dstTableName) - //nolint:gosec - copyCmd := fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", - snowflakeSchemaTableNormalize(parsedDstTable), copyInfo.columnsSQL, - copyInfo.transformationSQL, s.stage, strings.Join(s.copyOpts, ",")) - s.connector.logger.Info("running copy command: " + copyCmd) - _, err := s.connector.database.ExecContext(s.connector.ctx, copyCmd) - if err != nil { - return fmt.Errorf("failed to run COPY INTO command: %w", err) - } - - s.connector.logger.Info("copied file from stage " + s.stage + " to table " + s.dstTableName) - return nil -} - -func generateUpsertMergeCommand( - allCols []string, - upsertKeyCols []string, - tempTableName string, - dstTable string, -) string { - // all cols are acquired from snowflake schema, so let us try to make upsert key cols match the case - // and also the watermark col, then the quoting should be fine - caseMatchedCols := map[string]string{} - for _, col := range allCols { - caseMatchedCols[strings.ToLower(col)] = col - } - - for i, col := range upsertKeyCols { - upsertKeyCols[i] = caseMatchedCols[strings.ToLower(col)] - } - - upsertKeys := []string{} - partitionKeyCols := []string{} - for _, key := range upsertKeyCols { - quotedKey := utils.QuoteIdentifier(key) - upsertKeys = append(upsertKeys, fmt.Sprintf("dst.%s = src.%s", quotedKey, quotedKey)) - partitionKeyCols = append(partitionKeyCols, quotedKey) - } - upsertKeyClause := strings.Join(upsertKeys, " AND ") - - updateSetClauses := []string{} - insertColumnsClauses := []string{} - insertValuesClauses := []string{} - for _, column := range allCols { - quotedColumn := utils.QuoteIdentifier(column) - updateSetClauses = append(updateSetClauses, fmt.Sprintf("%s = src.%s", quotedColumn, quotedColumn)) - insertColumnsClauses = append(insertColumnsClauses, quotedColumn) - insertValuesClauses = append(insertValuesClauses, fmt.Sprintf("src.%s", quotedColumn)) - } - updateSetClause := strings.Join(updateSetClauses, ", ") - insertColumnsClause := strings.Join(insertColumnsClauses, ", ") - insertValuesClause := strings.Join(insertValuesClauses, ", ") - selectCmd := fmt.Sprintf(` - SELECT * - FROM %s - QUALIFY ROW_NUMBER() OVER (PARTITION BY %s ORDER BY %s DESC) = 1 - `, tempTableName, strings.Join(partitionKeyCols, ","), partitionKeyCols[0]) - - mergeCmd := fmt.Sprintf(` - MERGE INTO %s dst - USING (%s) src - ON %s - WHEN MATCHED THEN UPDATE SET %s - WHEN NOT MATCHED THEN INSERT (%s) VALUES (%s) - `, dstTable, selectCmd, upsertKeyClause, - updateSetClause, insertColumnsClause, insertValuesClause) - - return mergeCmd -} - -// HandleUpsertMode handles the upsert mode -func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( - allCols []string, - upsertKeyCols []string, - watermarkCol string, - flowJobName string, - copyInfo *CopyInfo, -) error { - runID, err := shared.RandomUInt64() - if err != nil { - return fmt.Errorf("failed to generate run ID: %w", err) - } - - tempTableName := fmt.Sprintf("%s_temp_%d", s.dstTableName, runID) - - //nolint:gosec - createTempTableCmd := fmt.Sprintf("CREATE TEMPORARY TABLE %s AS SELECT * FROM %s LIMIT 0", - tempTableName, s.dstTableName) - if _, err := s.connector.database.ExecContext(s.connector.ctx, createTempTableCmd); err != nil { - return fmt.Errorf("failed to create temp table: %w", err) - } - s.connector.logger.Info("created temp table " + tempTableName) - - //nolint:gosec - copyCmd := fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", - tempTableName, copyInfo.columnsSQL, copyInfo.transformationSQL, s.stage, strings.Join(s.copyOpts, ",")) - _, err = s.connector.database.ExecContext(s.connector.ctx, copyCmd) - if err != nil { - return fmt.Errorf("failed to run COPY INTO command: %w", err) - } - s.connector.logger.Info("copied file from stage " + s.stage + " to temp table " + tempTableName) - - mergeCmd := generateUpsertMergeCommand(allCols, upsertKeyCols, tempTableName, s.dstTableName) - - startTime := time.Now() - rows, err := s.connector.database.ExecContext(s.connector.ctx, mergeCmd) - if err != nil { - return fmt.Errorf("failed to merge data into destination table '%s': %w", mergeCmd, err) - } - rowCount, err := rows.RowsAffected() - if err == nil { - totalRowsAtTarget, err := s.connector.getTableCounts([]string{s.dstTableName}) - if err != nil { - return err - } - s.connector.logger.Info(fmt.Sprintf("merged %d rows into destination table %s, total rows at target: %d", - rowCount, s.dstTableName, totalRowsAtTarget)) - } else { - s.connector.logger.Error("failed to get rows affected", slog.Any("error", err)) - } - - s.connector.logger.Info(fmt.Sprintf("merged data from temp table %s into destination table %s, time taken %v", - tempTableName, s.dstTableName, time.Since(startTime))) - return nil -} diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 10bb93b22c..a1dae65da5 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -541,7 +541,7 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( DestinationTableIdentifier: strings.ToLower(fmt.Sprintf("%s.%s", c.metadataSchema, rawTableIdentifier)), } - avroSyncer := NewSnowflakeAvroSyncMethod(qrepConfig, c) + avroSyncer := NewSnowflakeAvroSyncHandler(qrepConfig, c) destinationTableSchema, err := c.getTableSchema(qrepConfig.DestinationTableIdentifier) if err != nil { return nil, err From 990cb5ba9d39b3eebaf4baf728ea45278b454112 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Sat, 20 Jan 2024 02:00:20 +0530 Subject: [PATCH 10/28] Validate Mirror: PostgreSQL Checks (#1110) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR adds a Validate mirror button and corresponding logic to check if the postgres source peer and selected tables are ready to be mirrored. Moves validate peer logic to separate file This PR covers: - Check if replication permissions are there for user - Check if we can do SELECT * FROM TABLE LIMIT 0. for all selected tables. - If initial copy is selected, they can’t enter the replication slot. - Check if publication they provide, is attached to all selected tables. - If no tables show up, remind them check GRANT permissions. - Validate peer: Don’t let you create < 12 version - Link to docs in CREATE PEER page. Screenshot 2024-01-19 at 7 12 21 PM Screenshot 2024-01-19 at 7 13 03 PM Screenshot 2024-01-19 at 7 55 41 PM --- flow/activities/flowable.go | 4 + flow/cmd/handler.go | 51 +------- flow/cmd/validate_mirror.go | 54 +++++++++ flow/cmd/validate_peer.go | 73 ++++++++++++ flow/connectors/postgres/client.go | 103 +++++++++++++++++ protos/route.proto | 10 ++ ui/app/api/mirrors/cdc/route.ts | 21 ++-- ui/app/api/mirrors/cdc/validate/route.ts | 31 +++++ ui/app/dto/MirrorsDTO.ts | 5 + ui/app/mirrors/create/cdc/cdc.tsx | 8 +- ui/app/mirrors/create/cdc/fields.tsx | 4 +- ui/app/mirrors/create/cdc/guide.tsx | 33 ++++++ ui/app/mirrors/create/cdc/tablemapping.tsx | 17 ++- ui/app/mirrors/create/handlers.ts | 95 +++++++++++---- ui/app/mirrors/create/helpers/cdc.ts | 6 +- ui/app/mirrors/create/page.tsx | 128 +++++++++++++-------- ui/app/mirrors/create/schema.ts | 13 ++- ui/app/mirrors/create/styles.tsx | 20 ++++ ui/app/peers/create/[peerType]/page.tsx | 2 + 19 files changed, 537 insertions(+), 141 deletions(-) create mode 100644 flow/cmd/validate_mirror.go create mode 100644 flow/cmd/validate_peer.go create mode 100644 ui/app/api/mirrors/cdc/validate/route.ts create mode 100644 ui/app/mirrors/create/cdc/guide.tsx create mode 100644 ui/app/mirrors/create/styles.tsx diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 8dffb9d710..a6224da626 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -642,6 +642,8 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config return err } + defer connectors.CloseConnector(dstConn) + shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { return fmt.Sprintf("consolidating partitions for job - %s", config.FlowJobName) }) @@ -665,6 +667,8 @@ func (a *FlowableActivity) CleanupQRepFlow(ctx context.Context, config *protos.Q return err } + defer dst.Close() + return dst.CleanupQRepFlow(config) } diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index c5ed5db1f6..92c9ace028 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -8,7 +8,6 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/connectors" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" @@ -122,6 +121,12 @@ func (h *FlowRequestHandler) CreateCDCFlow( ctx context.Context, req *protos.CreateCDCFlowRequest, ) (*protos.CreateCDCFlowResponse, error) { cfg := req.ConnectionConfigs + _, validateErr := h.ValidateCDCMirror(ctx, req) + if validateErr != nil { + slog.Error("validate mirror error", slog.Any("error", validateErr)) + return nil, fmt.Errorf("invalid mirror: %w", validateErr) + } + workflowID := fmt.Sprintf("%s-peerflow-%s", cfg.FlowJobName, uuid.New()) workflowOptions := client.StartWorkflowOptions{ ID: workflowID, @@ -556,50 +561,6 @@ func (h *FlowRequestHandler) handleWorkflowNotClosed(ctx context.Context, workfl return nil } -func (h *FlowRequestHandler) ValidatePeer( - ctx context.Context, - req *protos.ValidatePeerRequest, -) (*protos.ValidatePeerResponse, error) { - if req.Peer == nil { - return &protos.ValidatePeerResponse{ - Status: protos.ValidatePeerStatus_INVALID, - Message: "no peer provided", - }, nil - } - - if len(req.Peer.Name) == 0 { - return &protos.ValidatePeerResponse{ - Status: protos.ValidatePeerStatus_INVALID, - Message: "no peer name provided", - }, nil - } - - conn, err := connectors.GetConnector(ctx, req.Peer) - if err != nil { - return &protos.ValidatePeerResponse{ - Status: protos.ValidatePeerStatus_INVALID, - Message: fmt.Sprintf("peer type is missing or "+ - "your requested configuration for %s peer %s was invalidated: %s", - req.Peer.Type, req.Peer.Name, err), - }, nil - } - - connErr := conn.ConnectionActive() - if connErr != nil { - return &protos.ValidatePeerResponse{ - Status: protos.ValidatePeerStatus_INVALID, - Message: fmt.Sprintf("failed to establish active connection to %s peer %s: %v", - req.Peer.Type, req.Peer.Name, connErr), - }, nil - } - - return &protos.ValidatePeerResponse{ - Status: protos.ValidatePeerStatus_VALID, - Message: fmt.Sprintf("%s peer %s is valid", - req.Peer.Type, req.Peer.Name), - }, nil -} - func (h *FlowRequestHandler) CreatePeer( ctx context.Context, req *protos.CreatePeerRequest, diff --git a/flow/cmd/validate_mirror.go b/flow/cmd/validate_mirror.go new file mode 100644 index 0000000000..7f10020e58 --- /dev/null +++ b/flow/cmd/validate_mirror.go @@ -0,0 +1,54 @@ +package main + +import ( + "context" + "fmt" + "log/slog" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/generated/protos" +) + +func (h *FlowRequestHandler) ValidateCDCMirror( + ctx context.Context, req *protos.CreateCDCFlowRequest, +) (*protos.ValidateCDCMirrorResponse, error) { + pgPeer, err := connpostgres.NewPostgresConnector(ctx, req.ConnectionConfigs.Source.GetPostgresConfig()) + if err != nil { + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, fmt.Errorf("failed to create postgres connector: %v", err) + } + + defer pgPeer.Close() + + sourcePeerConfig := req.ConnectionConfigs.Source.GetPostgresConfig() + if sourcePeerConfig == nil { + slog.Error("/validatecdc source peer config is nil", slog.Any("peer", req.ConnectionConfigs.Source)) + return nil, fmt.Errorf("source peer config is nil") + } + + // Check permissions of postgres peer + err = pgPeer.CheckReplicationPermissions(sourcePeerConfig.User) + if err != nil { + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, fmt.Errorf("failed to check replication permissions: %v", err) + } + + // Check source tables + sourceTables := make([]string, 0, len(req.ConnectionConfigs.TableMappings)) + for _, tableMapping := range req.ConnectionConfigs.TableMappings { + sourceTables = append(sourceTables, tableMapping.SourceTableIdentifier) + } + + err = pgPeer.CheckSourceTables(sourceTables, req.ConnectionConfigs.PublicationName) + if err != nil { + return &protos.ValidateCDCMirrorResponse{ + Ok: false, + }, fmt.Errorf("provided source tables invalidated: %v", err) + } + + return &protos.ValidateCDCMirrorResponse{ + Ok: true, + }, nil +} diff --git a/flow/cmd/validate_peer.go b/flow/cmd/validate_peer.go new file mode 100644 index 0000000000..e3b04e88ba --- /dev/null +++ b/flow/cmd/validate_peer.go @@ -0,0 +1,73 @@ +package main + +import ( + "context" + "fmt" + "log/slog" + + "github.com/PeerDB-io/peer-flow/connectors" + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/generated/protos" +) + +func (h *FlowRequestHandler) ValidatePeer( + ctx context.Context, + req *protos.ValidatePeerRequest, +) (*protos.ValidatePeerResponse, error) { + if req.Peer == nil { + return &protos.ValidatePeerResponse{ + Status: protos.ValidatePeerStatus_INVALID, + Message: "no peer provided", + }, nil + } + + if len(req.Peer.Name) == 0 { + return &protos.ValidatePeerResponse{ + Status: protos.ValidatePeerStatus_INVALID, + Message: "no peer name provided", + }, nil + } + + conn, err := connectors.GetConnector(ctx, req.Peer) + if err != nil { + return &protos.ValidatePeerResponse{ + Status: protos.ValidatePeerStatus_INVALID, + Message: fmt.Sprintf("peer type is missing or "+ + "your requested configuration for %s peer %s was invalidated: %s", + req.Peer.Type, req.Peer.Name, err), + }, nil + } + + defer conn.Close() + + if req.Peer.Type == protos.DBType_POSTGRES { + version, err := conn.(*connpostgres.PostgresConnector).GetPostgresVersion() + if err != nil { + slog.Error("/peer/validate: pg version check", slog.Any("error", err)) + return nil, err + } + + if version < 12 { + return &protos.ValidatePeerResponse{ + Status: protos.ValidatePeerStatus_INVALID, + Message: fmt.Sprintf("%s peer %s must be of version 12 or above. Current version: %d", + req.Peer.Type, req.Peer.Name, version), + }, nil + } + } + + connErr := conn.ConnectionActive() + if connErr != nil { + return &protos.ValidatePeerResponse{ + Status: protos.ValidatePeerStatus_INVALID, + Message: fmt.Sprintf("failed to establish active connection to %s peer %s: %v", + req.Peer.Type, req.Peer.Name, connErr), + }, nil + } + + return &protos.ValidatePeerResponse{ + Status: protos.ValidatePeerStatus_VALID, + Message: fmt.Sprintf("%s peer %s is valid", + req.Peer.Type, req.Peer.Name), + }, nil +} diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index f18d38be52..371510adbb 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -5,6 +5,7 @@ import ( "fmt" "log" "regexp" + "strconv" "strings" "github.com/PeerDB-io/peer-flow/connectors/utils" @@ -572,3 +573,105 @@ func (c *PostgresConnector) getCurrentLSN() (pglogrepl.LSN, error) { func (c *PostgresConnector) getDefaultPublicationName(jobName string) string { return fmt.Sprintf("peerflow_pub_%s", jobName) } + +func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName string) error { + if c.pool == nil { + return fmt.Errorf("check tables: pool is nil") + } + + // Check that we can select from all tables + for _, tableName := range tableNames { + var row pgx.Row + err := c.pool.QueryRow(c.ctx, fmt.Sprintf("SELECT * FROM %s LIMIT 0;", tableName)).Scan(&row) + if err != nil && err != pgx.ErrNoRows { + return err + } + } + + // Check if tables belong to publication + tableArr := make([]string, 0, len(tableNames)) + for _, tableName := range tableNames { + tableArr = append(tableArr, fmt.Sprintf("'%s'", tableName)) + } + + tableStr := strings.Join(tableArr, ",") + + if pubName != "" { + var pubTableCount int + err := c.pool.QueryRow(c.ctx, fmt.Sprintf("select COUNT(DISTINCT(schemaname||'.'||tablename)) from pg_publication_tables "+ + "where schemaname||'.'||tablename in (%s) and pubname=$1;", tableStr), pubName).Scan(&pubTableCount) + if err != nil { + return err + } + + if pubTableCount != len(tableNames) { + return fmt.Errorf("not all tables belong to publication") + } + } + + return nil +} + +func (c *PostgresConnector) CheckReplicationPermissions(username string) error { + if c.pool == nil { + return fmt.Errorf("check replication permissions: pool is nil") + } + + var replicationRes bool + err := c.pool.QueryRow(c.ctx, "SELECT rolreplication FROM pg_roles WHERE rolname = $1;", username).Scan(&replicationRes) + if err != nil { + return err + } + + if !replicationRes { + return fmt.Errorf("postgres user does not have replication role") + } + + // check wal_level + var walLevel string + err = c.pool.QueryRow(c.ctx, "SHOW wal_level;").Scan(&walLevel) + if err != nil { + return err + } + + if walLevel != "logical" { + return fmt.Errorf("wal_level is not logical") + } + + // max_wal_senders must be at least 2 + var maxWalSendersRes string + err = c.pool.QueryRow(c.ctx, "SHOW max_wal_senders;").Scan(&maxWalSendersRes) + if err != nil { + return err + } + + maxWalSenders, err := strconv.Atoi(maxWalSendersRes) + if err != nil { + return err + } + + if maxWalSenders < 2 { + return fmt.Errorf("max_wal_senders must be at least 2") + } + + return nil +} + +func (c *PostgresConnector) GetPostgresVersion() (int, error) { + if c.pool == nil { + return -1, fmt.Errorf("version check: pool is nil") + } + + var versionRes string + err := c.pool.QueryRow(c.ctx, "SHOW server_version_num;").Scan(&versionRes) + if err != nil { + return -1, err + } + + version, err := strconv.Atoi(versionRes) + if err != nil { + return -1, err + } + + return version / 10000, nil +} diff --git a/protos/route.proto b/protos/route.proto index 6dcfbe14ad..046a63aca3 100644 --- a/protos/route.proto +++ b/protos/route.proto @@ -197,6 +197,10 @@ message MirrorStatusResponse { peerdb_flow.FlowStatus current_flow_state = 5; } +message ValidateCDCMirrorResponse{ + bool ok = 1; +} + message FlowStateChangeRequest { string flow_job_name = 1; peerdb_flow.FlowStatus requested_flow_state = 2; @@ -224,6 +228,12 @@ service FlowService { post: "/v1/peers/validate", body: "*" }; + } + rpc ValidateCDCMirror(CreateCDCFlowRequest) returns (ValidateCDCMirrorResponse) { + option (google.api.http) = { + post: "/v1/mirrors/cdc/validate", + body: "*" + }; } rpc CreatePeer(CreatePeerRequest) returns (CreatePeerResponse) { option (google.api.http) = { diff --git a/ui/app/api/mirrors/cdc/route.ts b/ui/app/api/mirrors/cdc/route.ts index 12efa4a826..0e48fec524 100644 --- a/ui/app/api/mirrors/cdc/route.ts +++ b/ui/app/api/mirrors/cdc/route.ts @@ -1,8 +1,5 @@ import { UCreateMirrorResponse } from '@/app/dto/MirrorsDTO'; -import { - CreateCDCFlowRequest, - CreateCDCFlowResponse, -} from '@/grpc_generated/route'; +import { CreateCDCFlowRequest } from '@/grpc_generated/route'; import { GetFlowHttpAddressFromEnv } from '@/rpc/http'; export async function POST(request: Request) { @@ -15,18 +12,18 @@ export async function POST(request: Request) { createCatalogEntry: true, }; try { - const createStatus: CreateCDCFlowResponse = await fetch( - `${flowServiceAddr}/v1/flows/cdc/create`, - { - method: 'POST', - body: JSON.stringify(req), - } - ).then((res) => { + const createStatus = await fetch(`${flowServiceAddr}/v1/flows/cdc/create`, { + method: 'POST', + body: JSON.stringify(req), + }).then((res) => { return res.json(); }); + if (!createStatus.worflowId) { + return new Response(JSON.stringify(createStatus)); + } let response: UCreateMirrorResponse = { - created: !!createStatus.worflowId, + created: true, }; return new Response(JSON.stringify(response)); diff --git a/ui/app/api/mirrors/cdc/validate/route.ts b/ui/app/api/mirrors/cdc/validate/route.ts new file mode 100644 index 0000000000..a0be93bcbd --- /dev/null +++ b/ui/app/api/mirrors/cdc/validate/route.ts @@ -0,0 +1,31 @@ +import { + CreateCDCFlowRequest, + ValidateCDCMirrorResponse, +} from '@/grpc_generated/route'; +import { GetFlowHttpAddressFromEnv } from '@/rpc/http'; + +export async function POST(request: Request) { + const body = await request.json(); + const { config } = body; + console.log('/mirrors/cdc/validate config: ', config); + const flowServiceAddr = GetFlowHttpAddressFromEnv(); + const req: CreateCDCFlowRequest = { + connectionConfigs: config, + createCatalogEntry: false, + }; + try { + const validateResponse: ValidateCDCMirrorResponse = await fetch( + `${flowServiceAddr}/v1/mirrors/cdc/validate`, + { + method: 'POST', + body: JSON.stringify(req), + } + ).then((res) => { + return res.json(); + }); + + return new Response(JSON.stringify(validateResponse)); + } catch (e) { + console.log(e); + } +} diff --git a/ui/app/dto/MirrorsDTO.ts b/ui/app/dto/MirrorsDTO.ts index e33890fe03..f76904b08d 100644 --- a/ui/app/dto/MirrorsDTO.ts +++ b/ui/app/dto/MirrorsDTO.ts @@ -5,6 +5,11 @@ export type UCreateMirrorResponse = { created: boolean; }; +export type UValidateMirrorResponse = { + ok: boolean; + errorMessage: string; +}; + export type UDropMirrorResponse = { dropped: boolean; errorMessage: string; diff --git a/ui/app/mirrors/create/cdc/cdc.tsx b/ui/app/mirrors/create/cdc/cdc.tsx index 6d89fe6c5c..9101703c20 100644 --- a/ui/app/mirrors/create/cdc/cdc.tsx +++ b/ui/app/mirrors/create/cdc/cdc.tsx @@ -6,7 +6,7 @@ import { Icon } from '@/lib/Icon'; import { Dispatch, SetStateAction, useMemo, useState } from 'react'; import { CDCConfig, MirrorSetter, TableMapRow } from '../../../dto/MirrorsDTO'; import { MirrorSetting } from '../helpers/common'; -import CDCFields from './fields'; +import CDCField from './fields'; import TableMapping from './tablemapping'; interface MirrorConfigProps { @@ -55,6 +55,8 @@ export default function CDCConfigForm({ const label = setting.label.toLowerCase(); if ( (label.includes('snapshot') && mirrorConfig.doInitialSnapshot !== true) || + (label === 'replication slot name' && + mirrorConfig.doInitialSnapshot === true) || (label.includes('staging path') && defaultSyncMode(mirrorConfig.destination?.type) !== 'AVRO') ) { @@ -69,7 +71,7 @@ export default function CDCConfigForm({ {normalSettings.map((setting, id) => { return ( paramDisplayCondition(setting) && ( - { return ( - void; } -const CDCFields = ({ setting, handleChange }: FieldProps) => { +const CDCField = ({ setting, handleChange }: FieldProps) => { return setting.type === 'switch' ? ( {setting.label}} @@ -67,4 +67,4 @@ const CDCFields = ({ setting, handleChange }: FieldProps) => { ); }; -export default CDCFields; +export default CDCField; diff --git a/ui/app/mirrors/create/cdc/guide.tsx b/ui/app/mirrors/create/cdc/guide.tsx new file mode 100644 index 0000000000..041632e378 --- /dev/null +++ b/ui/app/mirrors/create/cdc/guide.tsx @@ -0,0 +1,33 @@ +import { Label } from '@/lib/Label'; +import Link from 'next/link'; + +const GuideForDestinationSetup = ({ + dstPeerType: peerType, +}: { + dstPeerType: string; +}) => { + const linkForDst = () => { + switch (peerType) { + case 'SNOWFLAKE': + return 'https://docs.peerdb.io/connect/snowflake'; + case 'BIGQUERY': + return 'https://docs.peerdb.io/connect/bigquery'; + default: + return 'https://docs.peerdb.io/'; + } + }; + if (peerType != 'SNOWFLAKE' && peerType != 'BIGQUERY') { + return <>; + } + return ( + + ); +}; + +export default GuideForDestinationSetup; diff --git a/ui/app/mirrors/create/cdc/tablemapping.tsx b/ui/app/mirrors/create/cdc/tablemapping.tsx index d231812319..541ceaebbb 100644 --- a/ui/app/mirrors/create/cdc/tablemapping.tsx +++ b/ui/app/mirrors/create/cdc/tablemapping.tsx @@ -2,6 +2,7 @@ import { DBType } from '@/grpc_generated/peers'; import { Label } from '@/lib/Label'; import { SearchField } from '@/lib/SearchField'; +import Link from 'next/link'; import { Dispatch, SetStateAction, useEffect, useMemo, useState } from 'react'; import { BarLoader } from 'react-spinners/'; import { TableMapRow } from '../../../dto/MirrorsDTO'; @@ -39,9 +40,23 @@ const TableMapping = ({ return (
- + Date: Sat, 20 Jan 2024 11:16:32 +0000 Subject: [PATCH 11/28] Remove TotalSyncFlows (#1116) Only used by 1 test, convert it to WaitFor --- flow/cmd/handler.go | 1 - flow/e2e/s3/cdc_s3_test.go | 30 ++++++++++++------------------ flow/workflows/cdc_flow.go | 12 ++---------- 3 files changed, 14 insertions(+), 29 deletions(-) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 92c9ace028..1cee1fd5fc 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -142,7 +142,6 @@ func (h *FlowRequestHandler) CreateCDCFlow( } limits := &peerflow.CDCFlowLimits{ - TotalSyncFlows: 0, ExitAfterRecords: -1, MaxBatchSize: maxBatchSize, } diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index b26e44ad25..26eb42d38e 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -43,8 +43,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { flowConnConfig := connectionGen.GenerateFlowConnectionConfigs() limits := peerflow.CDCFlowLimits{ - TotalSyncFlows: 4, - ExitAfterRecords: 20, + ExitAfterRecords: -1, MaxBatchSize: 5, } @@ -60,23 +59,18 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { e2e.EnvNoError(s.t, env, err) } e2e.EnvNoError(s.t, env, err) - }() - - env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) - - // Verify workflow completes without error - require.True(s.t, env.IsWorkflowCompleted()) - err = env.GetWorkflowError() - // allow only continue as new error - require.Contains(s.t, err.Error(), "continue as new") + e2e.EnvWaitFor(s.t, env, time.Minute, "waiting for blobs", func() bool { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + files, err := s.s3Helper.ListAllFiles(ctx, flowJobName) + s.t.Logf("Files in Test_Complete_Simple_Flow_S3 %s: %d", flowJobName, len(files)) + e2e.EnvNoError(s.t, env, err) + return len(files) == 4 + }) - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - s.t.Logf("JobName: %s", flowJobName) - files, err := s.s3Helper.ListAllFiles(ctx, flowJobName) - s.t.Logf("Files in Test_Complete_Simple_Flow_S3: %d", len(files)) - require.NoError(s.t, err) + env.CancelWorkflow() + }() - require.Equal(s.t, 4, len(files)) + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) } diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index bc29eb412d..5f2d29b7e3 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -22,10 +22,6 @@ const ( ) type CDCFlowLimits struct { - // Number of sync flows to execute in total. - // If 0, the number of sync flows will be continuously executed until the peer flow is cancelled. - // This is typically non-zero for testing purposes. - TotalSyncFlows int // Maximum number of rows in a sync flow batch. MaxBatchSize uint32 // Rows synced after which we can say a test is done. @@ -364,10 +360,6 @@ func CDCFlowWorkflowWithConfig( } } - if limits.TotalSyncFlows == 0 { - limits.TotalSyncFlows = maxSyncFlowsPerCDCFlow - } - syncFlowOptions := &protos.SyncFlowOptions{ BatchSize: limits.MaxBatchSize, IdleTimeoutSeconds: 0, @@ -464,9 +456,9 @@ func CDCFlowWorkflowWithConfig( // check if total sync flows have been completed // since this happens immediately after we check for signals, the case of a signal being missed // due to a new workflow starting is vanishingly low, but possible - if limits.TotalSyncFlows != 0 && currentSyncFlowNum == limits.TotalSyncFlows { + if currentSyncFlowNum == maxSyncFlowsPerCDCFlow { w.logger.Info("All the syncflows have completed successfully, there was a"+ - " limit on the number of syncflows to be executed: ", limits.TotalSyncFlows) + " limit on the number of syncflows to be executed: ", currentSyncFlowNum) break } currentSyncFlowNum++ From 98581a4e18839b054d40a37496341cfa2f9a246b Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Sun, 21 Jan 2024 19:25:33 +0530 Subject: [PATCH 12/28] Add check for rds replication (#1118) --- flow/connectors/postgres/client.go | 7 ++++++- ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx | 8 +++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 371510adbb..0b87e84078 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -624,7 +624,12 @@ func (c *PostgresConnector) CheckReplicationPermissions(username string) error { } if !replicationRes { - return fmt.Errorf("postgres user does not have replication role") + // RDS case: check pg_settings for rds.logical_replication + var setting string + err := c.pool.QueryRow(c.ctx, "SELECT setting FROM pg_settings WHERE name = 'rds.logical_replication';").Scan(&setting) + if err != nil || setting != "on" { + return fmt.Errorf("postgres user does not have replication role") + } } // check wal_level diff --git a/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx b/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx index 5a635eee1e..24f4c2ac38 100644 --- a/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx +++ b/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx @@ -17,7 +17,13 @@ type props = { createdAt?: Date; }; function CdcDetails({ syncs, createdAt, mirrorConfig }: props) { - let lastSyncedAt = moment(syncs[0]?.endTime).fromNow(); + let lastSyncedAt = moment( + syncs.length > 1 + ? syncs[1]?.endTime + : syncs.length + ? syncs[0]?.startTime + : new Date() + ).fromNow(); let rowsSynced = syncs.reduce((acc, sync) => { if (sync.endTime !== null) { return acc + sync.numRows; From 48f5e555b90bcb2a7da721ebbc428f9d1b1ea928 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Mon, 22 Jan 2024 16:01:29 +0530 Subject: [PATCH 13/28] removed duplicate function and made version a type (#1120) --- flow/cmd/validate_peer.go | 4 ++-- flow/connectors/postgres/client.go | 35 ++++++++++------------------ flow/connectors/postgres/postgres.go | 2 +- 3 files changed, 15 insertions(+), 26 deletions(-) diff --git a/flow/cmd/validate_peer.go b/flow/cmd/validate_peer.go index e3b04e88ba..2de6345c14 100644 --- a/flow/cmd/validate_peer.go +++ b/flow/cmd/validate_peer.go @@ -41,13 +41,13 @@ func (h *FlowRequestHandler) ValidatePeer( defer conn.Close() if req.Peer.Type == protos.DBType_POSTGRES { - version, err := conn.(*connpostgres.PostgresConnector).GetPostgresVersion() + isValid, version, err := conn.(*connpostgres.PostgresConnector).MajorVersionCheck(connpostgres.POSTGRES_12) if err != nil { slog.Error("/peer/validate: pg version check", slog.Any("error", err)) return nil, err } - if version < 12 { + if isValid { return &protos.ValidatePeerResponse{ Status: protos.ValidatePeerStatus_INVALID, Message: fmt.Sprintf("%s peer %s must be of version 12 or above. Current version: %d", diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 0b87e84078..92bd3a1c40 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -17,6 +17,14 @@ import ( "github.com/lib/pq/oid" ) +type PGVersion int + +const ( + POSTGRES_12 PGVersion = 120000 + POSTGRES_13 PGVersion = 130000 + POSTGRES_15 PGVersion = 150000 +) + const ( mirrorJobsTableIdentifier = "peerdb_mirror_jobs" createMirrorJobsTableSQL = `CREATE TABLE IF NOT EXISTS %s.%s(mirror_job_name TEXT PRIMARY KEY, @@ -305,7 +313,7 @@ func (c *PostgresConnector) createSlotAndPublication( if !s.PublicationExists { // check and enable publish_via_partition_root - supportsPubViaRoot, err := c.majorVersionCheck(130000) + supportsPubViaRoot, _, err := c.MajorVersionCheck(POSTGRES_13) if err != nil { return fmt.Errorf("error checking Postgres version: %w", err) } @@ -475,14 +483,14 @@ func (c *PostgresConnector) jobMetadataExists(jobName string) (bool, error) { return result.Bool, nil } -func (c *PostgresConnector) majorVersionCheck(majorVersion int) (bool, error) { +func (c *PostgresConnector) MajorVersionCheck(majorVersion PGVersion) (bool, int64, error) { var version pgtype.Int8 err := c.pool.QueryRow(c.ctx, "SELECT current_setting('server_version_num')::INTEGER").Scan(&version) if err != nil { - return false, fmt.Errorf("failed to get server version: %w", err) + return false, 0, fmt.Errorf("failed to get server version: %w", err) } - return int(version.Int64) >= majorVersion, nil + return version.Int64 >= int64(majorVersion), version.Int64, nil } func (c *PostgresConnector) updateSyncMetadata(flowJobName string, lastCP int64, syncBatchID int64, @@ -661,22 +669,3 @@ func (c *PostgresConnector) CheckReplicationPermissions(username string) error { return nil } - -func (c *PostgresConnector) GetPostgresVersion() (int, error) { - if c.pool == nil { - return -1, fmt.Errorf("version check: pool is nil") - } - - var versionRes string - err := c.pool.QueryRow(c.ctx, "SHOW server_version_num;").Scan(&versionRes) - if err != nil { - return -1, err - } - - version, err := strconv.Atoi(versionRes) - if err != nil { - return -1, err - } - - return version / 10000, nil -} diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 48d67ecfaf..4af329d3b4 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -473,7 +473,7 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) } }() - supportsMerge, err := c.majorVersionCheck(150000) + supportsMerge, _, err := c.MajorVersionCheck(POSTGRES_15) if err != nil { return nil, err } From 8d4ed60e1b39e4b090da8d249f5311321d7d59f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 13:18:18 +0000 Subject: [PATCH 14/28] Update npm dependencies (#1117) --- .github/workflows/ui-lint.yml | 1 + ui/app/alert-config/new.tsx | 2 +- ui/app/mirrors/create/page.tsx | 5 +- .../mirrors/errors/[mirrorName]/ackbutton.tsx | 2 +- ui/package-lock.json | 1224 ++++++++--------- ui/package.json | 40 +- ui/tsconfig.json | 10 +- 7 files changed, 617 insertions(+), 667 deletions(-) diff --git a/.github/workflows/ui-lint.yml b/.github/workflows/ui-lint.yml index 5c462ca252..11ef0c0eca 100644 --- a/.github/workflows/ui-lint.yml +++ b/.github/workflows/ui-lint.yml @@ -43,3 +43,4 @@ jobs: prettier_dir: ui eslint_args: "--max-warnings 0" eslint_extensions: js,ts,jsx,tsx + prettier_extensions: js,ts,jsx,tsx,json diff --git a/ui/app/alert-config/new.tsx b/ui/app/alert-config/new.tsx index 17f94290b6..70f7fbbfd2 100644 --- a/ui/app/alert-config/new.tsx +++ b/ui/app/alert-config/new.tsx @@ -9,7 +9,7 @@ import 'react-toastify/dist/ReactToastify.css'; import { alertConfigReqSchema, alertConfigType } from './validation'; const notifyErr = (errMsg: string) => { toast.error(errMsg, { - position: toast.POSITION.BOTTOM_CENTER, + position: 'bottom-center', }); }; diff --git a/ui/app/mirrors/create/page.tsx b/ui/app/mirrors/create/page.tsx index 350e57d407..cf59891cb7 100644 --- a/ui/app/mirrors/create/page.tsx +++ b/ui/app/mirrors/create/page.tsx @@ -58,12 +58,11 @@ function getPeerLabel(peer: Peer) { const notifyErr = (msg: string, ok?: boolean) => { if (ok) { toast.success(msg, { - position: toast.POSITION.BOTTOM_CENTER, + position: 'bottom-center', }); - return; } else { toast.error(msg, { - position: toast.POSITION.BOTTOM_CENTER, + position: 'bottom-center', }); } }; diff --git a/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx b/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx index b6d14eed89..c6b23a6e72 100644 --- a/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx +++ b/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx @@ -7,7 +7,7 @@ import { toast } from 'react-toastify'; const notifyErr = (errMsg: string) => { toast.error(errMsg, { - position: toast.POSITION.BOTTOM_CENTER, + position: 'bottom-center', }); }; diff --git a/ui/package-lock.json b/ui/package-lock.json index 3bb8a35e2c..611c8af27b 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -8,9 +8,9 @@ "name": "peerdb-ui", "version": "0.1.0", "dependencies": { - "@grpc/grpc-js": "^1.9.13", + "@grpc/grpc-js": "^1.9.14", "@monaco-editor/react": "^4.6.0", - "@prisma/client": "^5.8.0", + "@prisma/client": "^5.8.1", "@radix-ui/react-checkbox": "^1.0.4", "@radix-ui/react-collapsible": "^1.0.3", "@radix-ui/react-dialog": "^1.0.5", @@ -25,52 +25,52 @@ "@radix-ui/react-toggle": "^1.0.3", "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", - "@tremor/react": "^3.13.1", - "@types/node": "^20.11.0", - "@types/react": "^18.2.47", + "@tremor/react": "^3.13.2", + "@types/node": "^20.11.5", + "@types/react": "^18.2.48", "@types/react-dom": "^18.2.18", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.309.0", + "lucide-react": "^0.312.0", "material-symbols": "^0.14.5", "moment": "^2.30.1", "moment-timezone": "^0.5.44", - "next": "^14.0.4", + "next": "^14.1.0", "next-auth": "^4.24.5", "prop-types": "^15.8.1", - "protobufjs": "^7.2.5", + "protobufjs": "^7.2.6", "react": "18.2.0", "react-dom": "18.2.0", "react-select": "^5.8.0", "react-spinners": "^0.13.8", - "react-toastify": "^9.1.3", + "react-toastify": "^10.0.3", "styled-components": "^6.1.8", "swr": "^2.2.4", - "usehooks-ts": "^2.9.2", + "usehooks-ts": "^2.9.5", "zod": "^3.22.4" }, "devDependencies": { - "@storybook/addon-essentials": "^7.6.8", - "@storybook/addon-interactions": "^7.6.8", - "@storybook/addon-links": "^7.6.8", + "@storybook/addon-essentials": "^7.6.10", + "@storybook/addon-interactions": "^7.6.10", + "@storybook/addon-links": "^7.6.10", "@storybook/addon-styling": "^1.3.7", "@storybook/blocks": "^7.3.0", - "@storybook/nextjs": "^7.6.8", + "@storybook/nextjs": "^7.6.10", "@storybook/react": "^7.3.0", "@storybook/testing-library": "^0.2.2", - "autoprefixer": "^10.4.16", - "copy-webpack-plugin": "^12.0.1", + "autoprefixer": "^10.4.17", + "copy-webpack-plugin": "^12.0.2", "eslint": "^8.56.0", - "eslint-config-next": "^14.0.4", + "eslint-config-next": "^14.1.0", "eslint-config-prettier": "^9.1.0", "eslint-plugin-storybook": "^0.6.15", "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.33", - "prettier": "^3.2.1", + "prettier": "^3.2.4", "prettier-plugin-organize-imports": "^3.2.4", - "prisma": "^5.8.0", - "storybook": "^7.6.8", + "prisma": "^5.8.1", + "storybook": "^7.6.10", "string-width": "^7.0.0", "tailwindcss": "^3.4.1", "tailwindcss-animate": "^1.0.7", @@ -334,9 +334,9 @@ } }, "node_modules/@babel/helper-define-polyfill-provider": { - "version": "0.4.4", - "resolved": "https://registry.npmjs.org/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.4.4.tgz", - "integrity": "sha512-QcJMILQCu2jm5TFPGA3lCpJJTeEP+mqeXooG/NZbg/h5FTFi6V0+99ahlRsW8/kRLyb24LZVCCiclDedhLKcBA==", + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.5.0.tgz", + "integrity": "sha512-NovQquuQLAQ5HuyjCz7WQP9MjRj7dx++yspwiyUiGl9ZyadHRSql1HZh5ogRd8W8w6YM6EQ/NTB8rgjLt5W65Q==", "dev": true, "dependencies": { "@babel/helper-compilation-targets": "^7.22.6", @@ -2907,9 +2907,9 @@ } }, "node_modules/@floating-ui/react-dom": { - "version": "2.0.5", - "resolved": "https://registry.npmjs.org/@floating-ui/react-dom/-/react-dom-2.0.5.tgz", - "integrity": "sha512-UsBK30Bg+s6+nsgblXtZmwHhgS2vmbuQK22qgt2pTQM6M3X6H1+cQcLXqgRY3ihVLcZJE6IvqDQozhsnIVqK/Q==", + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/@floating-ui/react-dom/-/react-dom-2.0.6.tgz", + "integrity": "sha512-IB8aCRFxr8nFkdYZgH+Otd9EVQPJoynxeFRGTB8voPoZMRWo8XjYuCRgpI1btvuKY69XMiLnW+ym7zoBHM90Rw==", "dependencies": { "@floating-ui/dom": "^1.5.4" }, @@ -2936,9 +2936,9 @@ "integrity": "sha512-9TANp6GPoMtYzQdt54kfAyMmz1+osLlXdg2ENroU7zzrtflTLrrC/lgrIfaSe+Wu0b89GKccT7vxXA0MoAIO+Q==" }, "node_modules/@grpc/grpc-js": { - "version": "1.9.13", - "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.9.13.tgz", - "integrity": "sha512-OEZZu9v9AA+7/tghMDE8o5DAMD5THVnwSqDWuh7PPYO5287rTyqy0xEHT6/e4pbqSrhyLPdQFsam4TwFQVVIIw==", + "version": "1.9.14", + "resolved": "https://registry.npmjs.org/@grpc/grpc-js/-/grpc-js-1.9.14.tgz", + "integrity": "sha512-nOpuzZ2G3IuMFN+UPPpKrC6NsLmWsTqSsm66IRfnBt1D4pwTqE27lmbpcPM+l2Ua4gE7PfjRHI6uedAy7hoXUw==", "dependencies": { "@grpc/proto-loader": "^0.7.8", "@types/node": ">=12.12.47" @@ -3258,9 +3258,9 @@ "integrity": "sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg==" }, "node_modules/@jridgewell/trace-mapping": { - "version": "0.3.21", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.21.tgz", - "integrity": "sha512-SRfKmRe1KvYnxjEMtxEr+J4HIeMX5YBg/qhRHpxEIGjhX1rshcHlnFUE9K0GazhVKWM7B+nARSkV8LuvJdJ5/g==", + "version": "0.3.22", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.22.tgz", + "integrity": "sha512-Wf963MzWtA2sjrNt+g18IAln9lKnlRp+K2eH4jjIoF1wYeq3aMREpG09xhlhdzS0EjwU7qmUJYangWa+151vZw==", "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", "@jridgewell/sourcemap-codec": "^1.4.14" @@ -3373,43 +3373,23 @@ } }, "node_modules/@next/env": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/env/-/env-14.0.4.tgz", - "integrity": "sha512-irQnbMLbUNQpP1wcE5NstJtbuA/69kRfzBrpAD7Gsn8zm/CY6YQYc3HQBz8QPxwISG26tIm5afvvVbu508oBeQ==" + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/env/-/env-14.1.0.tgz", + "integrity": "sha512-Py8zIo+02ht82brwwhTg36iogzFqGLPXlRGKQw5s+qP/kMNc4MAyDeEwBKDijk6zTIbegEgu8Qy7C1LboslQAw==" }, "node_modules/@next/eslint-plugin-next": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.0.4.tgz", - "integrity": "sha512-U3qMNHmEZoVmHA0j/57nRfi3AscXNvkOnxDmle/69Jz/G0o/gWjXTDdlgILZdrxQ0Lw/jv2mPW8PGy0EGIHXhQ==", - "dev": true, - "dependencies": { - "glob": "7.1.7" - } - }, - "node_modules/@next/eslint-plugin-next/node_modules/glob": { - "version": "7.1.7", - "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.7.tgz", - "integrity": "sha512-OvD9ENzPLbegENnYP5UUfJIirTg4+XwMWGaQfQTY0JenxNvvIKP3U3/tAQSPIu/lHxXYSZmpXlUHeqAIdKzBLQ==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/eslint-plugin-next/-/eslint-plugin-next-14.1.0.tgz", + "integrity": "sha512-x4FavbNEeXx/baD/zC/SdrvkjSby8nBn8KcCREqk6UuwvwoAPZmaV8TFCAuo/cpovBRTIY67mHhe86MQQm/68Q==", "dev": true, "dependencies": { - "fs.realpath": "^1.0.0", - "inflight": "^1.0.4", - "inherits": "2", - "minimatch": "^3.0.4", - "once": "^1.3.0", - "path-is-absolute": "^1.0.0" - }, - "engines": { - "node": "*" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" + "glob": "10.3.10" } }, "node_modules/@next/swc-darwin-arm64": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.0.4.tgz", - "integrity": "sha512-mF05E/5uPthWzyYDyptcwHptucf/jj09i2SXBPwNzbgBNc+XnwzrL0U6BmPjQeOL+FiB+iG1gwBeq7mlDjSRPg==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-arm64/-/swc-darwin-arm64-14.1.0.tgz", + "integrity": "sha512-nUDn7TOGcIeyQni6lZHfzNoo9S0euXnu0jhsbMOmMJUBfgsnESdjN97kM7cBqQxZa8L/bM9om/S5/1dzCrW6wQ==", "cpu": [ "arm64" ], @@ -3422,9 +3402,9 @@ } }, "node_modules/@next/swc-darwin-x64": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.0.4.tgz", - "integrity": "sha512-IZQ3C7Bx0k2rYtrZZxKKiusMTM9WWcK5ajyhOZkYYTCc8xytmwSzR1skU7qLgVT/EY9xtXDG0WhY6fyujnI3rw==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-darwin-x64/-/swc-darwin-x64-14.1.0.tgz", + "integrity": "sha512-1jgudN5haWxiAl3O1ljUS2GfupPmcftu2RYJqZiMJmmbBT5M1XDffjUtRUzP4W3cBHsrvkfOFdQ71hAreNQP6g==", "cpu": [ "x64" ], @@ -3437,9 +3417,9 @@ } }, "node_modules/@next/swc-linux-arm64-gnu": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.0.4.tgz", - "integrity": "sha512-VwwZKrBQo/MGb1VOrxJ6LrKvbpo7UbROuyMRvQKTFKhNaXjUmKTu7wxVkIuCARAfiI8JpaWAnKR+D6tzpCcM4w==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-gnu/-/swc-linux-arm64-gnu-14.1.0.tgz", + "integrity": "sha512-RHo7Tcj+jllXUbK7xk2NyIDod3YcCPDZxj1WLIYxd709BQ7WuRYl3OWUNG+WUfqeQBds6kvZYlc42NJJTNi4tQ==", "cpu": [ "arm64" ], @@ -3452,9 +3432,9 @@ } }, "node_modules/@next/swc-linux-arm64-musl": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.0.4.tgz", - "integrity": "sha512-8QftwPEW37XxXoAwsn+nXlodKWHfpMaSvt81W43Wh8dv0gkheD+30ezWMcFGHLI71KiWmHK5PSQbTQGUiidvLQ==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-linux-arm64-musl/-/swc-linux-arm64-musl-14.1.0.tgz", + "integrity": "sha512-v6kP8sHYxjO8RwHmWMJSq7VZP2nYCkRVQ0qolh2l6xroe9QjbgV8siTbduED4u0hlk0+tjS6/Tuy4n5XCp+l6g==", "cpu": [ "arm64" ], @@ -3467,9 +3447,9 @@ } }, "node_modules/@next/swc-linux-x64-gnu": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.0.4.tgz", - "integrity": "sha512-/s/Pme3VKfZAfISlYVq2hzFS8AcAIOTnoKupc/j4WlvF6GQ0VouS2Q2KEgPuO1eMBwakWPB1aYFIA4VNVh667A==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-gnu/-/swc-linux-x64-gnu-14.1.0.tgz", + "integrity": "sha512-zJ2pnoFYB1F4vmEVlb/eSe+VH679zT1VdXlZKX+pE66grOgjmKJHKacf82g/sWE4MQ4Rk2FMBCRnX+l6/TVYzQ==", "cpu": [ "x64" ], @@ -3482,9 +3462,9 @@ } }, "node_modules/@next/swc-linux-x64-musl": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.0.4.tgz", - "integrity": "sha512-m8z/6Fyal4L9Bnlxde5g2Mfa1Z7dasMQyhEhskDATpqr+Y0mjOBZcXQ7G5U+vgL22cI4T7MfvgtrM2jdopqWaw==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-linux-x64-musl/-/swc-linux-x64-musl-14.1.0.tgz", + "integrity": "sha512-rbaIYFt2X9YZBSbH/CwGAjbBG2/MrACCVu2X0+kSykHzHnYH5FjHxwXLkcoJ10cX0aWCEynpu+rP76x0914atg==", "cpu": [ "x64" ], @@ -3497,9 +3477,9 @@ } }, "node_modules/@next/swc-win32-arm64-msvc": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.0.4.tgz", - "integrity": "sha512-7Wv4PRiWIAWbm5XrGz3D8HUkCVDMMz9igffZG4NB1p4u1KoItwx9qjATHz88kwCEal/HXmbShucaslXCQXUM5w==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-win32-arm64-msvc/-/swc-win32-arm64-msvc-14.1.0.tgz", + "integrity": "sha512-o1N5TsYc8f/HpGt39OUQpQ9AKIGApd3QLueu7hXk//2xq5Z9OxmV6sQfNp8C7qYmiOlHYODOGqNNa0e9jvchGQ==", "cpu": [ "arm64" ], @@ -3512,9 +3492,9 @@ } }, "node_modules/@next/swc-win32-ia32-msvc": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.0.4.tgz", - "integrity": "sha512-zLeNEAPULsl0phfGb4kdzF/cAVIfaC7hY+kt0/d+y9mzcZHsMS3hAS829WbJ31DkSlVKQeHEjZHIdhN+Pg7Gyg==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-win32-ia32-msvc/-/swc-win32-ia32-msvc-14.1.0.tgz", + "integrity": "sha512-XXIuB1DBRCFwNO6EEzCTMHT5pauwaSj4SWs7CYnME57eaReAKBXCnkUE80p/pAZcewm7hs+vGvNqDPacEXHVkw==", "cpu": [ "ia32" ], @@ -3527,9 +3507,9 @@ } }, "node_modules/@next/swc-win32-x64-msvc": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.0.4.tgz", - "integrity": "sha512-yEh2+R8qDlDCjxVpzOTEpBLQTEFAcP2A8fUFLaWNap9GitYKkKv1//y2S6XY6zsR4rCOPRpU7plYDR+az2n30A==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/@next/swc-win32-x64-msvc/-/swc-win32-x64-msvc-14.1.0.tgz", + "integrity": "sha512-9WEbVRRAqJ3YFVqEZIxUqkiO8l1nool1LmNxygr5HWF8AcSYsEpneUDhmjUVJEzO2A04+oPtZdombzzPPkTtgg==", "cpu": [ "x64" ], @@ -3655,9 +3635,9 @@ } }, "node_modules/@prisma/client": { - "version": "5.8.0", - "resolved": "https://registry.npmjs.org/@prisma/client/-/client-5.8.0.tgz", - "integrity": "sha512-QxO6C4MaA/ysTIbC+EcAH1aX/YkpymhXtO6zPdk+FvA7+59tNibIYpd+7koPdViLg2iKES4ojsxWNUGNJaEcbA==", + "version": "5.8.1", + "resolved": "https://registry.npmjs.org/@prisma/client/-/client-5.8.1.tgz", + "integrity": "sha512-xQtMPfbIwLlbm0VVIVQY2yqQVOxPwRQhvIp7Z3m2900g1bu/zRHKhYZJQWELqmjl6d8YwBy0K2NvMqh47v1ubw==", "hasInstallScript": true, "engines": { "node": ">=16.13" @@ -3672,48 +3652,48 @@ } }, "node_modules/@prisma/debug": { - "version": "5.8.0", - "resolved": "https://registry.npmjs.org/@prisma/debug/-/debug-5.8.0.tgz", - "integrity": "sha512-ZqPpkvbovu/kQJ1bvy57NO4dw97fpQGcbQSCtsqlwSE1UNKJP75R3BKxdznk8ZPMY+GJdMRetWNv4oAvSbWn8Q==", + "version": "5.8.1", + "resolved": "https://registry.npmjs.org/@prisma/debug/-/debug-5.8.1.tgz", + "integrity": "sha512-tjuw7eA0Us3T42jx9AmAgL58rzwzpFGYc3R7Y4Ip75EBYrKMBA1YihuWMcBC92ILmjlQ/u3p8VxcIE0hr+fZfg==", "devOptional": true }, "node_modules/@prisma/engines": { - "version": "5.8.0", - "resolved": "https://registry.npmjs.org/@prisma/engines/-/engines-5.8.0.tgz", - "integrity": "sha512-Qhqm9WWLujNEC13AuZlUO14SQ15tNLe5puaz+tOk7UqINqJ3PtqMmuSuzomiw2diGVqZ+HYiSQzlR3+pPucVHA==", + "version": "5.8.1", + "resolved": "https://registry.npmjs.org/@prisma/engines/-/engines-5.8.1.tgz", + "integrity": "sha512-TJgYLRrZr56uhqcXO4GmP5be+zjCIHtLDK20Cnfg+o9d905hsN065QOL+3Z0zQAy6YD31Ol4u2kzSfRmbJv/uA==", "devOptional": true, "hasInstallScript": true, "dependencies": { - "@prisma/debug": "5.8.0", - "@prisma/engines-version": "5.8.0-37.0a83d8541752d7582de2ebc1ece46519ce72a848", - "@prisma/fetch-engine": "5.8.0", - "@prisma/get-platform": "5.8.0" + "@prisma/debug": "5.8.1", + "@prisma/engines-version": "5.8.1-1.78caf6feeaed953168c64e15a249c3e9a033ebe2", + "@prisma/fetch-engine": "5.8.1", + "@prisma/get-platform": "5.8.1" } }, "node_modules/@prisma/engines-version": { - "version": "5.8.0-37.0a83d8541752d7582de2ebc1ece46519ce72a848", - "resolved": "https://registry.npmjs.org/@prisma/engines-version/-/engines-version-5.8.0-37.0a83d8541752d7582de2ebc1ece46519ce72a848.tgz", - "integrity": "sha512-cXcoVweYbnv8xRfkWq9oj8BECOdzHUazrSpYCa0ehp5TNz4l5Spa8jbq/VROCTzj3ZncH5D9Q2TmySYTOUeKlw==", + "version": "5.8.1-1.78caf6feeaed953168c64e15a249c3e9a033ebe2", + "resolved": "https://registry.npmjs.org/@prisma/engines-version/-/engines-version-5.8.1-1.78caf6feeaed953168c64e15a249c3e9a033ebe2.tgz", + "integrity": "sha512-f5C3JM3l9yhGr3cr4FMqWloFaSCpNpMi58Om22rjD2DOz3owci2mFdFXMgnAGazFPKrCbbEhcxdsRfspEYRoFQ==", "devOptional": true }, "node_modules/@prisma/fetch-engine": { - "version": "5.8.0", - "resolved": "https://registry.npmjs.org/@prisma/fetch-engine/-/fetch-engine-5.8.0.tgz", - "integrity": "sha512-1CAuE+JoYsPNggMEn6qk0zos06Uc9bYZBJ0VBPHD6R7REL05614koAbOCmn52IaYz3nobb7f25hqW6AY7rLkIw==", + "version": "5.8.1", + "resolved": "https://registry.npmjs.org/@prisma/fetch-engine/-/fetch-engine-5.8.1.tgz", + "integrity": "sha512-+bgjjoSFa6uYEbAPlklfoVSStOEfcpheOjoBoNsNNSQdSzcwE2nM4Q0prun0+P8/0sCHo18JZ9xqa8gObvgOUw==", "devOptional": true, "dependencies": { - "@prisma/debug": "5.8.0", - "@prisma/engines-version": "5.8.0-37.0a83d8541752d7582de2ebc1ece46519ce72a848", - "@prisma/get-platform": "5.8.0" + "@prisma/debug": "5.8.1", + "@prisma/engines-version": "5.8.1-1.78caf6feeaed953168c64e15a249c3e9a033ebe2", + "@prisma/get-platform": "5.8.1" } }, "node_modules/@prisma/get-platform": { - "version": "5.8.0", - "resolved": "https://registry.npmjs.org/@prisma/get-platform/-/get-platform-5.8.0.tgz", - "integrity": "sha512-Nk3rhTFZ1LYkFZJnpSvQcLPCaBWgJQfteHII6UEENOOkYlmP0k3FuswND54tzzEr4qs39wOdV9pbXKX9U2lv7A==", + "version": "5.8.1", + "resolved": "https://registry.npmjs.org/@prisma/get-platform/-/get-platform-5.8.1.tgz", + "integrity": "sha512-wnA+6HTFcY+tkykMokix9GiAkaauPC5W/gg0O5JB0J8tCTNWrqpnQ7AsaGRfkYUbeOIioh6woDjQrGTTRf1Zag==", "devOptional": true, "dependencies": { - "@prisma/debug": "5.8.0" + "@prisma/debug": "5.8.1" } }, "node_modules/@protobufjs/aspromise": { @@ -4918,9 +4898,9 @@ } }, "node_modules/@rushstack/eslint-patch": { - "version": "1.6.1", - "resolved": "https://registry.npmjs.org/@rushstack/eslint-patch/-/eslint-patch-1.6.1.tgz", - "integrity": "sha512-UY+FGM/2jjMkzQLn8pxcHGMaVLh9aEitG3zY2CiY7XHdLiz3bZOwa6oDxNqEMv7zZkV+cj5DOdz0cQ1BP5Hjgw==", + "version": "1.7.0", + "resolved": "https://registry.npmjs.org/@rushstack/eslint-patch/-/eslint-patch-1.7.0.tgz", + "integrity": "sha512-Jh4t/593gxs0lJZ/z3NnasKlplXT2f+4y/LZYuaKZW5KAaiVFL/fThhs+17EbUd53jUVJ0QudYCBGbN/psvaqg==", "dev": true }, "node_modules/@sinclair/typebox": { @@ -4942,12 +4922,12 @@ } }, "node_modules/@storybook/addon-actions": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-actions/-/addon-actions-7.6.8.tgz", - "integrity": "sha512-/KQlr/nLsAazJuSVUoMjQdwAeeXkKEtElKdqXrqI1LVOi5a7kMgB+bmn9aKX+7VBQLfQ36Btyty+FaY7bRtehQ==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-actions/-/addon-actions-7.6.10.tgz", + "integrity": "sha512-pcKmf0H/caGzKDy8cz1adNSjv+KOBWLJ11RzGExrWm+Ad5ACifwlsQPykJ3TQ/21sTd9IXVrE9uuq4LldEnPbg==", "dev": true, "dependencies": { - "@storybook/core-events": "7.6.8", + "@storybook/core-events": "7.6.10", "@storybook/global": "^5.0.0", "@types/uuid": "^9.0.1", "dequal": "^2.0.2", @@ -4960,9 +4940,9 @@ } }, "node_modules/@storybook/addon-backgrounds": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-backgrounds/-/addon-backgrounds-7.6.8.tgz", - "integrity": "sha512-b+Oj41z2W/Pv6oCXmcjGdNkOStbVItrlDoIeUGyDKrngzH9Kpv5u2XZTHkZWGWusLhOVq8ENBDqj6ENRL6kDtw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-backgrounds/-/addon-backgrounds-7.6.10.tgz", + "integrity": "sha512-kGzsN1QkfyI8Cz7TErEx9OCB3PMzpCFGLd/iy7FreXwbMbeAQ3/9fYgKUsNOYgOhuTz7S09koZUWjS/WJuZGFA==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", @@ -4975,12 +4955,12 @@ } }, "node_modules/@storybook/addon-controls": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-controls/-/addon-controls-7.6.8.tgz", - "integrity": "sha512-vjBwO1KbjB3l74qOVvLvks4LJjAIStr2n4j7Grdhqf2eeQvj122gT51dXstndtMNFqNHD4y3eImwNAbuaYrrnw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-controls/-/addon-controls-7.6.10.tgz", + "integrity": "sha512-LjwCQRMWq1apLtFwDi6U8MI6ITUr+KhxJucZ60tfc58RgB2v8ayozyDAonFEONsx9YSR1dNIJ2Z/e2rWTBJeYA==", "dev": true, "dependencies": { - "@storybook/blocks": "7.6.8", + "@storybook/blocks": "7.6.10", "lodash": "^4.17.21", "ts-dedent": "^2.0.0" }, @@ -4990,26 +4970,26 @@ } }, "node_modules/@storybook/addon-docs": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-docs/-/addon-docs-7.6.8.tgz", - "integrity": "sha512-vl7jNKT8x8Hnwn38l5cUr6TQZFCmx09VxarGUrMEO4mwTOoVRL2ofoh9JKFXhCiCHlMI9R0lnupGB/LAplWgPg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-docs/-/addon-docs-7.6.10.tgz", + "integrity": "sha512-GtyQ9bMx1AOOtl6ZS9vwK104HFRK+tqzxddRRxhXkpyeKu3olm9aMgXp35atE/3fJSqyyDm2vFtxxH8mzBA20A==", "dev": true, "dependencies": { "@jest/transform": "^29.3.1", "@mdx-js/react": "^2.1.5", - "@storybook/blocks": "7.6.8", - "@storybook/client-logger": "7.6.8", - "@storybook/components": "7.6.8", - "@storybook/csf-plugin": "7.6.8", - "@storybook/csf-tools": "7.6.8", + "@storybook/blocks": "7.6.10", + "@storybook/client-logger": "7.6.10", + "@storybook/components": "7.6.10", + "@storybook/csf-plugin": "7.6.10", + "@storybook/csf-tools": "7.6.10", "@storybook/global": "^5.0.0", "@storybook/mdx2-csf": "^1.0.0", - "@storybook/node-logger": "7.6.8", - "@storybook/postinstall": "7.6.8", - "@storybook/preview-api": "7.6.8", - "@storybook/react-dom-shim": "7.6.8", - "@storybook/theming": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/node-logger": "7.6.10", + "@storybook/postinstall": "7.6.10", + "@storybook/preview-api": "7.6.10", + "@storybook/react-dom-shim": "7.6.10", + "@storybook/theming": "7.6.10", + "@storybook/types": "7.6.10", "fs-extra": "^11.1.0", "remark-external-links": "^8.0.0", "remark-slug": "^6.0.0", @@ -5025,24 +5005,24 @@ } }, "node_modules/@storybook/addon-essentials": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-essentials/-/addon-essentials-7.6.8.tgz", - "integrity": "sha512-UoRZWPkDYL/UWsfAJk4q4nn5nayYdOvPApVsF/ZDnGsiv1zB2RpqbkiD1bfxPlGEVCoB+NQIN2s867gEpf+DjA==", - "dev": true, - "dependencies": { - "@storybook/addon-actions": "7.6.8", - "@storybook/addon-backgrounds": "7.6.8", - "@storybook/addon-controls": "7.6.8", - "@storybook/addon-docs": "7.6.8", - "@storybook/addon-highlight": "7.6.8", - "@storybook/addon-measure": "7.6.8", - "@storybook/addon-outline": "7.6.8", - "@storybook/addon-toolbars": "7.6.8", - "@storybook/addon-viewport": "7.6.8", - "@storybook/core-common": "7.6.8", - "@storybook/manager-api": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/preview-api": "7.6.8", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-essentials/-/addon-essentials-7.6.10.tgz", + "integrity": "sha512-cjbuCCK/3dtUity0Uqi5LwbkgfxqCCE5x5mXZIk9lTMeDz5vB9q6M5nzncVDy8F8przF3NbDLLgxKlt8wjiICg==", + "dev": true, + "dependencies": { + "@storybook/addon-actions": "7.6.10", + "@storybook/addon-backgrounds": "7.6.10", + "@storybook/addon-controls": "7.6.10", + "@storybook/addon-docs": "7.6.10", + "@storybook/addon-highlight": "7.6.10", + "@storybook/addon-measure": "7.6.10", + "@storybook/addon-outline": "7.6.10", + "@storybook/addon-toolbars": "7.6.10", + "@storybook/addon-viewport": "7.6.10", + "@storybook/core-common": "7.6.10", + "@storybook/manager-api": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/preview-api": "7.6.10", "ts-dedent": "^2.0.0" }, "funding": { @@ -5055,9 +5035,9 @@ } }, "node_modules/@storybook/addon-highlight": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-highlight/-/addon-highlight-7.6.8.tgz", - "integrity": "sha512-3mUfdLxaegCKWSm0i245RhnmEgkE+uLnOkE7h2kiztrWGqYuzGBKjgfZuVrftqsEWWc7LlJ1xdDZsIgs5Z06gA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-highlight/-/addon-highlight-7.6.10.tgz", + "integrity": "sha512-dIuS5QmoT1R+gFOcf6CoBa6D9UR5/wHCfPqPRH8dNNcCLtIGSHWQ4v964mS5OCq1Huj7CghmR15lOUk7SaYwUA==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0" @@ -5068,13 +5048,13 @@ } }, "node_modules/@storybook/addon-interactions": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-interactions/-/addon-interactions-7.6.8.tgz", - "integrity": "sha512-E1ZMrJ/4larCPW92AFuY71I9s8Ri+DEdwNtVnU/WV55NA+E9oRKt5/qOrJLcjQorViwh9KOHeeuc8kagA2hjnA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-interactions/-/addon-interactions-7.6.10.tgz", + "integrity": "sha512-lEsAdP/PrOZK/KmRbZ/fU4RjEqDP+e/PBlVVVJT2QvHniWK/xxkjCD0axsHU/XuaeQRFhmg0/KR342PC/cIf9A==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", - "@storybook/types": "7.6.8", + "@storybook/types": "7.6.10", "jest-mock": "^27.0.6", "polished": "^4.2.2", "ts-dedent": "^2.2.0" @@ -5085,9 +5065,9 @@ } }, "node_modules/@storybook/addon-links": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-links/-/addon-links-7.6.8.tgz", - "integrity": "sha512-lw+xMvzfhyOR5I5792rGCf31OfVsiNG+uCc6CEewjKdC+e4GZDXzAkLIrLVUvbf6iUvHzERD63Y5nKz2bt5yZA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-links/-/addon-links-7.6.10.tgz", + "integrity": "sha512-s/WkSYHpr2pb9p57j6u/xDBg3TKJhBq55YMl0GB5gXgkRPIeuGbPhGJhm2yTGVFLvXgr/aHHnOxb/R/W8PiRhA==", "dev": true, "dependencies": { "@storybook/csf": "^0.1.2", @@ -5108,9 +5088,9 @@ } }, "node_modules/@storybook/addon-measure": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-measure/-/addon-measure-7.6.8.tgz", - "integrity": "sha512-76ItcwATq3BRPEtGV5Apby3E+7tOn6d5dtNpBYBZOdjUsj6E+uFtdmfHrc1Bt1ersJ7hRDCgsHArqOGXeLuDrw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-measure/-/addon-measure-7.6.10.tgz", + "integrity": "sha512-OVfTI56+kc4hLWfZ/YPV3WKj/aA9e4iKXYxZyPdhfX4Z8TgZdD1wv9Z6e8DKS0H5kuybYrHKHaID5ki6t7qz3w==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", @@ -5122,9 +5102,9 @@ } }, "node_modules/@storybook/addon-outline": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-outline/-/addon-outline-7.6.8.tgz", - "integrity": "sha512-eTHreyvxYLIPt5AbMyDO3CEgGClQFt+CtA/RgSjpyv9MgYXPsZp/h1ZHpYYhSPRYnRE4//YnPMuk7eLf4udaag==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-outline/-/addon-outline-7.6.10.tgz", + "integrity": "sha512-RVJrEoPArhI6zAIMNl1Gz0zrj84BTfEWYYz0yDWOTVgvN411ugsoIk1hw0671MOneXJ2RcQ9MFIeV/v6AVDQYg==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", @@ -5204,9 +5184,9 @@ } }, "node_modules/@storybook/addon-toolbars": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-toolbars/-/addon-toolbars-7.6.8.tgz", - "integrity": "sha512-Akr9Pfw+AzQBRPVdo8yjcdS4IiOyEIBPVn/OAcbLi6a2zLYBdn99yKi21P0o03TJjNy32A254iAQQ7zyjIwEtA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-toolbars/-/addon-toolbars-7.6.10.tgz", + "integrity": "sha512-PaXY/oj9yxF7/H0CNdQKcioincyCkfeHpISZriZbZqhyqsjn3vca7RFEmsB88Q+ou6rMeqyA9st+6e2cx/Ct6A==", "dev": true, "funding": { "type": "opencollective", @@ -5214,9 +5194,9 @@ } }, "node_modules/@storybook/addon-viewport": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/addon-viewport/-/addon-viewport-7.6.8.tgz", - "integrity": "sha512-9fvaTudqTA7HYygOWq8gnlmR5XLLjMgK4RoZqMP8OhzX0Vkkg72knPI8lyrnHwze/yMcR1e2lmbdLm55rPq6QA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/addon-viewport/-/addon-viewport-7.6.10.tgz", + "integrity": "sha512-+bA6juC/lH4vEhk+w0rXakaG8JgLG4MOYrIudk5vJKQaC6X58LIM9N4kzIS2KSExRhkExXBPrWsnMfCo7uxmKg==", "dev": true, "dependencies": { "memoizerific": "^1.11.3" @@ -5227,13 +5207,13 @@ } }, "node_modules/@storybook/api": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/api/-/api-7.6.8.tgz", - "integrity": "sha512-cuc4O75n3ZNnc6880hM1Tj3ieSP12v4WgmZbTv+HlEafp/5d7gqyMSLh8KusJpkbI7CHMCJ/7vv1qmLrvHK9iw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/api/-/api-7.6.10.tgz", + "integrity": "sha512-28qfs7sdzcnKfGBVnWuKgqAPe++NLrMjL6cp/rxuRlAtGyl0Q3eC9jj1ggOm4SFVTu6NoFzTuTXFUEbskZVk1Q==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.8", - "@storybook/manager-api": "7.6.8" + "@storybook/client-logger": "7.6.10", + "@storybook/manager-api": "7.6.10" }, "funding": { "type": "opencollective", @@ -5241,22 +5221,22 @@ } }, "node_modules/@storybook/blocks": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/blocks/-/blocks-7.6.8.tgz", - "integrity": "sha512-9cjwqj+VLmVHD8lU1xIGbZiu2xPQ3A+cAobmam045wvEB/wYhcrF0K0lBwHLqUWTcNdOzZy5uaoaCu/1G5AmDg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/blocks/-/blocks-7.6.10.tgz", + "integrity": "sha512-oSIukGC3yuF8pojABC/HLu5tv2axZvf60TaUs8eDg7+NiiKhzYSPoMQxs5uMrKngl+EJDB92ESgWT9vvsfvIPg==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.8", - "@storybook/client-logger": "7.6.8", - "@storybook/components": "7.6.8", - "@storybook/core-events": "7.6.8", + "@storybook/channels": "7.6.10", + "@storybook/client-logger": "7.6.10", + "@storybook/components": "7.6.10", + "@storybook/core-events": "7.6.10", "@storybook/csf": "^0.1.2", - "@storybook/docs-tools": "7.6.8", + "@storybook/docs-tools": "7.6.10", "@storybook/global": "^5.0.0", - "@storybook/manager-api": "7.6.8", - "@storybook/preview-api": "7.6.8", - "@storybook/theming": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/manager-api": "7.6.10", + "@storybook/preview-api": "7.6.10", + "@storybook/theming": "7.6.10", + "@storybook/types": "7.6.10", "@types/lodash": "^4.14.167", "color-convert": "^2.0.1", "dequal": "^2.0.2", @@ -5280,15 +5260,15 @@ } }, "node_modules/@storybook/builder-manager": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/builder-manager/-/builder-manager-7.6.8.tgz", - "integrity": "sha512-4CZo1RHPlDJA7G+lJoVdi+/3/L1ERxVxtvwuGgk8CxVDt6vFNpoc7fEGryNv3GRzKN1/luNYNU1MTnCUSn0B2g==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/builder-manager/-/builder-manager-7.6.10.tgz", + "integrity": "sha512-f+YrjZwohGzvfDtH8BHzqM3xW0p4vjjg9u7uzRorqUiNIAAKHpfNrZ/WvwPlPYmrpAHt4xX/nXRJae4rFSygPw==", "dev": true, "dependencies": { "@fal-works/esbuild-plugin-global-externals": "^2.1.2", - "@storybook/core-common": "7.6.8", - "@storybook/manager": "7.6.8", - "@storybook/node-logger": "7.6.8", + "@storybook/core-common": "7.6.10", + "@storybook/manager": "7.6.10", + "@storybook/node-logger": "7.6.10", "@types/ejs": "^3.1.1", "@types/find-cache-dir": "^3.2.1", "@yarnpkg/esbuild-plugin-pnp": "^3.0.0-rc.10", @@ -5308,26 +5288,27 @@ } }, "node_modules/@storybook/builder-webpack5": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/builder-webpack5/-/builder-webpack5-7.6.8.tgz", - "integrity": "sha512-g4gYcHrrV/8Xve4Q/DJfXk8Bxkq5cxzy7KIBkb8PK5h+MFUiS/xoZc5qXk/WuX256zj2JnZRV//2yf61OhNd6g==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/builder-webpack5/-/builder-webpack5-7.6.10.tgz", + "integrity": "sha512-ja47rdy75tAs37T+JLSqgUGJiba+74zM/8IpEZAzgJmGxLetnHuCWEDskZWh3NXemxYS2uCvsg5rNc+dL9z4RA==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", - "@storybook/channels": "7.6.8", - "@storybook/client-logger": "7.6.8", - "@storybook/core-common": "7.6.8", - "@storybook/core-events": "7.6.8", - "@storybook/core-webpack": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/preview": "7.6.8", - "@storybook/preview-api": "7.6.8", + "@storybook/channels": "7.6.10", + "@storybook/client-logger": "7.6.10", + "@storybook/core-common": "7.6.10", + "@storybook/core-events": "7.6.10", + "@storybook/core-webpack": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/preview": "7.6.10", + "@storybook/preview-api": "7.6.10", "@swc/core": "^1.3.82", "@types/node": "^18.0.0", "@types/semver": "^7.3.4", "babel-loader": "^9.0.0", "browser-assert": "^1.2.1", "case-sensitive-paths-webpack-plugin": "^2.4.0", + "cjs-module-lexer": "^1.2.3", "constants-browserify": "^1.0.0", "css-loader": "^6.7.1", "es-module-lexer": "^1.4.1", @@ -5362,9 +5343,9 @@ } }, "node_modules/@storybook/builder-webpack5/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -5404,13 +5385,13 @@ "dev": true }, "node_modules/@storybook/channels": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/channels/-/channels-7.6.8.tgz", - "integrity": "sha512-aPgQcSjeyZDhAfr/slCphVfYGCihxuFCaCVlZuJA4uTaGEUkn+kPW2jP0yLtlSN33J79wFXsMLPQYwIS3aQ4Ew==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/channels/-/channels-7.6.10.tgz", + "integrity": "sha512-ITCLhFuDBKgxetuKnWwYqMUWlU7zsfH3gEKZltTb+9/2OAWR7ez0iqU7H6bXP1ridm0DCKkt2UMWj2mmr9iQqg==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.8", - "@storybook/core-events": "7.6.8", + "@storybook/client-logger": "7.6.10", + "@storybook/core-events": "7.6.10", "@storybook/global": "^5.0.0", "qs": "^6.10.0", "telejson": "^7.2.0", @@ -5422,23 +5403,23 @@ } }, "node_modules/@storybook/cli": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/cli/-/cli-7.6.8.tgz", - "integrity": "sha512-Is8nkgsbIOu+Jk9Z7x5sgMPgGs9RTVDum3cz9eA4UspPiIBJsf7nGHAWOtc+mCIm6Z3eeNbT1YMOWxz9EuqboA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/cli/-/cli-7.6.10.tgz", + "integrity": "sha512-pK1MEseMm73OMO2OVoSz79QWX8ymxgIGM8IeZTCo9gImiVRChMNDFYcv8yPWkjuyesY8c15CoO48aR7pdA1OjQ==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", "@babel/preset-env": "^7.23.2", "@babel/types": "^7.23.0", "@ndelangen/get-tarball": "^3.0.7", - "@storybook/codemod": "7.6.8", - "@storybook/core-common": "7.6.8", - "@storybook/core-events": "7.6.8", - "@storybook/core-server": "7.6.8", - "@storybook/csf-tools": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/telemetry": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/codemod": "7.6.10", + "@storybook/core-common": "7.6.10", + "@storybook/core-events": "7.6.10", + "@storybook/core-server": "7.6.10", + "@storybook/csf-tools": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/telemetry": "7.6.10", + "@storybook/types": "7.6.10", "@types/semver": "^7.3.4", "@yarnpkg/fslib": "2.10.3", "@yarnpkg/libzip": "2.3.0", @@ -5463,7 +5444,6 @@ "puppeteer-core": "^2.1.1", "read-pkg-up": "^7.0.1", "semver": "^7.3.7", - "simple-update-notifier": "^2.0.0", "strip-json-comments": "^3.0.1", "tempy": "^1.0.1", "ts-dedent": "^2.0.0", @@ -5556,9 +5536,9 @@ "dev": true }, "node_modules/@storybook/client-logger": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/client-logger/-/client-logger-7.6.8.tgz", - "integrity": "sha512-WyK+RNSYk+sy0pxk8np1MnUXSWFdy54WqtT7u64vDFs9Jxfa1oMZ+Vl6XhaFQYR++tKC7VabLcI6vZ0pOoE9Jw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/client-logger/-/client-logger-7.6.10.tgz", + "integrity": "sha512-U7bbpu21ntgePMz/mKM18qvCSWCUGCUlYru8mgVlXLCKqFqfTeP887+CsPEQf29aoE3cLgDrxqbRJ1wxX9kL9A==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0" @@ -5569,18 +5549,18 @@ } }, "node_modules/@storybook/codemod": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/codemod/-/codemod-7.6.8.tgz", - "integrity": "sha512-3Gk+ZsD35DUgqbbRNdX547kzZK/ajIbgwynmR0FuPhZhhZuYI4+2eMNzdmI/Oe9Nov4R16senQuAZjw/Dc5LrA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/codemod/-/codemod-7.6.10.tgz", + "integrity": "sha512-pzFR0nocBb94vN9QCJLC3C3dP734ZigqyPmd0ZCDj9Xce2ytfHK3v1lKB6TZWzKAZT8zztauECYxrbo4LVuagw==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", "@babel/preset-env": "^7.23.2", "@babel/types": "^7.23.0", "@storybook/csf": "^0.1.2", - "@storybook/csf-tools": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/csf-tools": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/types": "7.6.10", "@types/cross-spawn": "^6.0.2", "cross-spawn": "^7.0.3", "globby": "^11.0.2", @@ -5630,18 +5610,18 @@ } }, "node_modules/@storybook/components": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/components/-/components-7.6.8.tgz", - "integrity": "sha512-ghrQkws7F2s9xwdiQq2ezQoOozCiYF9g/vnh+qttd4UgKqXDWoILb8LJGKtS7C0u0vV/Ui59EYUyDIVBT6wHlw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/components/-/components-7.6.10.tgz", + "integrity": "sha512-H5hF8pxwtbt0LxV24KMMsPlbYG9Oiui3ObvAQkvGu6q62EYxRPeNSrq3GBI5XEbI33OJY9bT24cVaZx18dXqwQ==", "dev": true, "dependencies": { "@radix-ui/react-select": "^1.2.2", "@radix-ui/react-toolbar": "^1.0.4", - "@storybook/client-logger": "7.6.8", + "@storybook/client-logger": "7.6.10", "@storybook/csf": "^0.1.2", "@storybook/global": "^5.0.0", - "@storybook/theming": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/theming": "7.6.10", + "@storybook/types": "7.6.10", "memoizerific": "^1.11.3", "use-resize-observer": "^9.1.0", "util-deprecate": "^1.0.2" @@ -5656,13 +5636,13 @@ } }, "node_modules/@storybook/core-client": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/core-client/-/core-client-7.6.8.tgz", - "integrity": "sha512-Avt0R0F9U+PEndPS23LHyIBxbwVCeF/VCIuIfD1eTYwE9nSLzvJXqlxARfFyhYV43LQcC5fIKjxfrsyUjM5vbQ==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/core-client/-/core-client-7.6.10.tgz", + "integrity": "sha512-DjnzSzSNDmZyxyg6TxugzWQwOsW+n/iWVv6sHNEvEd5STr0mjuJjIEELmv58LIr5Lsre5+LEddqHsyuLyt8ubg==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.8", - "@storybook/preview-api": "7.6.8" + "@storybook/client-logger": "7.6.10", + "@storybook/preview-api": "7.6.10" }, "funding": { "type": "opencollective", @@ -5670,14 +5650,14 @@ } }, "node_modules/@storybook/core-common": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/core-common/-/core-common-7.6.8.tgz", - "integrity": "sha512-TRbiv5AF2m88ixyh31yqn6FgWDYZO6e6IxbJolRvEKD4b9opfPJ5e1ocb/QPz9sBUmsrX59ghMjO8R6dDYzdwA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/core-common/-/core-common-7.6.10.tgz", + "integrity": "sha512-K3YWqjCKMnpvYsWNjOciwTH6zWbuuZzmOiipziZaVJ+sB1XYmH52Y3WGEm07TZI8AYK9DRgwA13dR/7W0nw72Q==", "dev": true, "dependencies": { - "@storybook/core-events": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/core-events": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/types": "7.6.10", "@types/find-cache-dir": "^3.2.1", "@types/node": "^18.0.0", "@types/node-fetch": "^2.6.4", @@ -5705,18 +5685,18 @@ } }, "node_modules/@storybook/core-common/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" } }, "node_modules/@storybook/core-events": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/core-events/-/core-events-7.6.8.tgz", - "integrity": "sha512-c1onJHG71JKbU4hMZC31rVTSbcfhcXaB0ikGnb7rJzlUZ1YkWnb0wf0/ikQR0seDOpR3HS+WQ0M3FIpqANyETg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/core-events/-/core-events-7.6.10.tgz", + "integrity": "sha512-yccDH67KoROrdZbRKwxgTswFMAco5nlCyxszCDASCLygGSV2Q2e+YuywrhchQl3U6joiWi3Ps1qWu56NeNafag==", "dev": true, "dependencies": { "ts-dedent": "^2.0.0" @@ -5727,26 +5707,26 @@ } }, "node_modules/@storybook/core-server": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/core-server/-/core-server-7.6.8.tgz", - "integrity": "sha512-/csAFNuAhF11f6D9neYNavmKPFK/ZxTskaktc4iDwBRgBM95kZ6DBFjg9ErRi5Q8Z/i92wk6qORkq4bkN/lI9w==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/core-server/-/core-server-7.6.10.tgz", + "integrity": "sha512-2icnqJkn3vwq0eJPP0rNaHd7IOvxYf5q4lSVl2AWTxo/Ae19KhokI6j/2vvS2XQJMGQszwshlIwrZUNsj5p0yw==", "dev": true, "dependencies": { "@aw-web-design/x-default-browser": "1.4.126", "@discoveryjs/json-ext": "^0.5.3", - "@storybook/builder-manager": "7.6.8", - "@storybook/channels": "7.6.8", - "@storybook/core-common": "7.6.8", - "@storybook/core-events": "7.6.8", + "@storybook/builder-manager": "7.6.10", + "@storybook/channels": "7.6.10", + "@storybook/core-common": "7.6.10", + "@storybook/core-events": "7.6.10", "@storybook/csf": "^0.1.2", - "@storybook/csf-tools": "7.6.8", + "@storybook/csf-tools": "7.6.10", "@storybook/docs-mdx": "^0.1.0", "@storybook/global": "^5.0.0", - "@storybook/manager": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/preview-api": "7.6.8", - "@storybook/telemetry": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/manager": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/preview-api": "7.6.10", + "@storybook/telemetry": "7.6.10", + "@storybook/types": "7.6.10", "@types/detect-port": "^1.3.0", "@types/node": "^18.0.0", "@types/pretty-hrtime": "^1.0.0", @@ -5780,9 +5760,9 @@ } }, "node_modules/@storybook/core-server/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -5842,14 +5822,14 @@ "dev": true }, "node_modules/@storybook/core-webpack": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/core-webpack/-/core-webpack-7.6.8.tgz", - "integrity": "sha512-UOTW2WhKmB8baCLc1eRssmz11sBv+iDRyS2WFK+WONkiGy3pQrpxfq2OVXXMFYkSHGXqj/jSKfKXSmNQBbkyAQ==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/core-webpack/-/core-webpack-7.6.10.tgz", + "integrity": "sha512-+GiCRp+2Hw0NO3NYRKamG/U5SyOQ8tOfRUxuAqWI7nduXwB3WWdjji3/ofjqOm/ryKesuQFtfhozaczvBJBvng==", "dev": true, "dependencies": { - "@storybook/core-common": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/core-common": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/types": "7.6.10", "@types/node": "^18.0.0", "ts-dedent": "^2.0.0" }, @@ -5859,9 +5839,9 @@ } }, "node_modules/@storybook/core-webpack/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -5877,12 +5857,12 @@ } }, "node_modules/@storybook/csf-plugin": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/csf-plugin/-/csf-plugin-7.6.8.tgz", - "integrity": "sha512-KYh7VwTHhXz/V9weuGY3pK9messE56TJHUD+0SO9dF2BVNKsKpAOVcjzrE6masiAFX35Dz/t9ywy8iFcfAo0dg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/csf-plugin/-/csf-plugin-7.6.10.tgz", + "integrity": "sha512-Sc+zZg/BnPH2X28tthNaQBnDiFfO0QmfjVoOx0fGYM9SvY3P5ehzWwp5hMRBim6a/twOTzePADtqYL+t6GMqqg==", "dev": true, "dependencies": { - "@storybook/csf-tools": "7.6.8", + "@storybook/csf-tools": "7.6.10", "unplugin": "^1.3.1" }, "funding": { @@ -5891,9 +5871,9 @@ } }, "node_modules/@storybook/csf-tools": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/csf-tools/-/csf-tools-7.6.8.tgz", - "integrity": "sha512-ea6QnQRvhPOpSUbfioLlJYRLpJldNZcocgUJwOJ/e3TM6M67BZBzeDnVOJkuUKejrp++KF22GEIkbGAWErIlnA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/csf-tools/-/csf-tools-7.6.10.tgz", + "integrity": "sha512-TnDNAwIALcN6SA4l00Cb67G02XMOrYU38bIpFJk5VMDX2dvgPjUtJNBuLmEbybGcOt7nPyyFIHzKcY5FCVGoWA==", "dev": true, "dependencies": { "@babel/generator": "^7.23.0", @@ -5901,7 +5881,7 @@ "@babel/traverse": "^7.23.2", "@babel/types": "^7.23.0", "@storybook/csf": "^0.1.2", - "@storybook/types": "7.6.8", + "@storybook/types": "7.6.10", "fs-extra": "^11.1.0", "recast": "^0.23.1", "ts-dedent": "^2.0.0" @@ -5918,14 +5898,14 @@ "dev": true }, "node_modules/@storybook/docs-tools": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/docs-tools/-/docs-tools-7.6.8.tgz", - "integrity": "sha512-zIbrje4JLFpfK05y3SkDNtIth/vTOEaJVa/zaHuwS1gUX73Pq3jwF2eMGVabeVWi6hvxGeZXhnIsymh/Hpbn5w==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/docs-tools/-/docs-tools-7.6.10.tgz", + "integrity": "sha512-UgbikducoXzqQHf2TozO0f2rshaeBNnShVbL5Ai4oW7pDymBmrfzdjGbF/milO7yxNKcoIByeoNmu384eBamgQ==", "dev": true, "dependencies": { - "@storybook/core-common": "7.6.8", - "@storybook/preview-api": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/core-common": "7.6.10", + "@storybook/preview-api": "7.6.10", + "@storybook/types": "7.6.10", "@types/doctrine": "^0.0.3", "assert": "^2.1.0", "doctrine": "^3.0.0", @@ -5943,9 +5923,9 @@ "dev": true }, "node_modules/@storybook/manager": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/manager/-/manager-7.6.8.tgz", - "integrity": "sha512-INoXXoHXyw9PPMJAOAhwf9u2GNDDNdv1JAI1fhrbCAECzDabHT9lRVUo6v8I5XMc+YdMHLM1Vz38DbB+w18hFw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/manager/-/manager-7.6.10.tgz", + "integrity": "sha512-Co3sLCbNYY6O4iH2ggmRDLCPWLj03JE5s/DOG8OVoXc6vBwTc/Qgiyrsxxp6BHQnPpM0mxL6aKAxE3UjsW/Nog==", "dev": true, "funding": { "type": "opencollective", @@ -5953,19 +5933,19 @@ } }, "node_modules/@storybook/manager-api": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/manager-api/-/manager-api-7.6.8.tgz", - "integrity": "sha512-BGVZb0wMTd8Hi8rUYPRzdIhWRw73qXlEupwEYyGtH63sg+aD67wyAo8/pMEpQBH4kVss7VheWY2JGpRJeFVUxw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/manager-api/-/manager-api-7.6.10.tgz", + "integrity": "sha512-8eGVpRlpunuFScDtc7nxpPJf/4kJBAAZlNdlhmX09j8M3voX6GpcxabBamSEX5pXZqhwxQCshD4IbqBmjvadlw==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.8", - "@storybook/client-logger": "7.6.8", - "@storybook/core-events": "7.6.8", + "@storybook/channels": "7.6.10", + "@storybook/client-logger": "7.6.10", + "@storybook/core-events": "7.6.10", "@storybook/csf": "^0.1.2", "@storybook/global": "^5.0.0", - "@storybook/router": "7.6.8", - "@storybook/theming": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/router": "7.6.10", + "@storybook/theming": "7.6.10", + "@storybook/types": "7.6.10", "dequal": "^2.0.2", "lodash": "^4.17.21", "memoizerific": "^1.11.3", @@ -5985,9 +5965,9 @@ "dev": true }, "node_modules/@storybook/nextjs": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/nextjs/-/nextjs-7.6.8.tgz", - "integrity": "sha512-17n2k7h5Eg6LGZpQbEpBXa949+QY5Zv0u5zvtXpMShpnrcuX2E3e9AQabwUmIvU7WiLaRw9rxNC0HrW6GUB3zg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/nextjs/-/nextjs-7.6.10.tgz", + "integrity": "sha512-YfUr4+2BlZJMW7bo2W8nNcUFPbf0wTAfhRvCtJDGDu6ne2vjlLyRmDXY5S11YQdO6pd4W8GWHyWTX+bKKIX3qg==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", @@ -6003,14 +5983,14 @@ "@babel/preset-react": "^7.22.15", "@babel/preset-typescript": "^7.23.2", "@babel/runtime": "^7.23.2", - "@storybook/addon-actions": "7.6.8", - "@storybook/builder-webpack5": "7.6.8", - "@storybook/core-common": "7.6.8", - "@storybook/core-events": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/preset-react-webpack": "7.6.8", - "@storybook/preview-api": "7.6.8", - "@storybook/react": "7.6.8", + "@storybook/addon-actions": "7.6.10", + "@storybook/builder-webpack5": "7.6.10", + "@storybook/core-common": "7.6.10", + "@storybook/core-events": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/preset-react-webpack": "7.6.10", + "@storybook/preview-api": "7.6.10", + "@storybook/react": "7.6.10", "@types/node": "^18.0.0", "@types/semver": "^7.3.4", "css-loader": "^6.7.3", @@ -6059,9 +6039,9 @@ } }, "node_modules/@storybook/nextjs/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -6139,9 +6119,9 @@ "dev": true }, "node_modules/@storybook/node-logger": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/node-logger/-/node-logger-7.6.8.tgz", - "integrity": "sha512-SVvwZAcOLdkstqnAbE5hVYsriXh6OXjLcwFEBpAYi1meQ0R70iNALVSPEfIDK1r7M163Jngsq2hRnHvbLoQNkg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/node-logger/-/node-logger-7.6.10.tgz", + "integrity": "sha512-ZBuqrv4bjJzKXyfRGFkVIi+z6ekn6rOPoQao4KmsfLNQAUUsEdR8Baw/zMnnU417zw5dSEaZdpuwx75SCQAeOA==", "dev": true, "funding": { "type": "opencollective", @@ -6149,9 +6129,9 @@ } }, "node_modules/@storybook/postinstall": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/postinstall/-/postinstall-7.6.8.tgz", - "integrity": "sha512-9ixyNpoT1w3WmSooCzndAWDnw4fENA1WUBcdqrzlcgaSBKiAHad1k/Yct/uBAU95l/uQ13NgXK3mx4+S6unx/g==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/postinstall/-/postinstall-7.6.10.tgz", + "integrity": "sha512-SMdXtednPCy3+SRJ7oN1OPN1oVFhj3ih+ChOEX8/kZ5J3nfmV3wLPtsZvFGUCf0KWQEP1xL+1Urv48mzMKcV/w==", "dev": true, "funding": { "type": "opencollective", @@ -6159,18 +6139,18 @@ } }, "node_modules/@storybook/preset-react-webpack": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/preset-react-webpack/-/preset-react-webpack-7.6.8.tgz", - "integrity": "sha512-S7z2IKonfZyvaETPwDHaOsw2hnG6Kny6aVnWj1/oAMHLRkAo08v/uxXc3of27HmCng3sKoPtEKypQa6yV863MA==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/preset-react-webpack/-/preset-react-webpack-7.6.10.tgz", + "integrity": "sha512-fUcr4dmXJdPIQdjkhA4bE8QF8Pavr4BSLxovtTRupbWxtRjZxJrH5hf+0HZycq1cp9umO/11Lsmw9Nx5Xg3Eww==", "dev": true, "dependencies": { "@babel/preset-flow": "^7.22.15", "@babel/preset-react": "^7.22.15", "@pmmmwh/react-refresh-webpack-plugin": "^0.5.11", - "@storybook/core-webpack": "7.6.8", - "@storybook/docs-tools": "7.6.8", - "@storybook/node-logger": "7.6.8", - "@storybook/react": "7.6.8", + "@storybook/core-webpack": "7.6.10", + "@storybook/docs-tools": "7.6.10", + "@storybook/node-logger": "7.6.10", + "@storybook/react": "7.6.10", "@storybook/react-docgen-typescript-plugin": "1.0.6--canary.9.0c3f3b7.0", "@types/node": "^18.0.0", "@types/semver": "^7.3.4", @@ -6204,9 +6184,9 @@ } }, "node_modules/@storybook/preset-react-webpack/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" @@ -6246,9 +6226,9 @@ "dev": true }, "node_modules/@storybook/preview": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/preview/-/preview-7.6.8.tgz", - "integrity": "sha512-f54EXmJcIkc5A7nQmtnCUtNFNfEOoTuPYFK7pDfcK/bVU+g63zzWhBAeIUZ8yioLKGqZPTzFEhXkpa+OqsT0Jg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/preview/-/preview-7.6.10.tgz", + "integrity": "sha512-F07BzVXTD3byq+KTWtvsw3pUu3fQbyiBNLFr2CnfU4XSdLKja5lDt8VqDQq70TayVQOf5qfUTzRd4M6pQkjw1w==", "dev": true, "funding": { "type": "opencollective", @@ -6256,17 +6236,17 @@ } }, "node_modules/@storybook/preview-api": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/preview-api/-/preview-api-7.6.8.tgz", - "integrity": "sha512-rtP9Yo8ZV1NWhtA3xCOAb1vU70KCV3D2U4E3rOb2prqJ2CEQ/MQbrB7KUTDRSQdT7VFbjsLQWVCTUcNo29U8JQ==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/preview-api/-/preview-api-7.6.10.tgz", + "integrity": "sha512-5A3etoIwZCx05yuv3KSTv1wynN4SR4rrzaIs/CTBp3BC4q1RBL+Or/tClk0IJPXQMlx/4Y134GtNIBbkiDofpw==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.8", - "@storybook/client-logger": "7.6.8", - "@storybook/core-events": "7.6.8", + "@storybook/channels": "7.6.10", + "@storybook/client-logger": "7.6.10", + "@storybook/core-events": "7.6.10", "@storybook/csf": "^0.1.2", "@storybook/global": "^5.0.0", - "@storybook/types": "7.6.8", + "@storybook/types": "7.6.10", "@types/qs": "^6.9.5", "dequal": "^2.0.2", "lodash": "^4.17.21", @@ -6282,18 +6262,18 @@ } }, "node_modules/@storybook/react": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/react/-/react-7.6.8.tgz", - "integrity": "sha512-yMqcCNskCxqoYSGWO1qu6Jdju9zhEEwd8tOC7AgIC8sAB7K8FTxZu0d6+QFpeg9fGq+hyAmRM4GrT9Fq9IKwwQ==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/react/-/react-7.6.10.tgz", + "integrity": "sha512-wwBn1cg2uZWW4peqqBjjU7XGmFq8HdkVUtWwh6dpfgmlY1Aopi+vPgZt7pY9KkWcTOq5+DerMdSfwxukpc3ajQ==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.8", - "@storybook/core-client": "7.6.8", - "@storybook/docs-tools": "7.6.8", + "@storybook/client-logger": "7.6.10", + "@storybook/core-client": "7.6.10", + "@storybook/docs-tools": "7.6.10", "@storybook/global": "^5.0.0", - "@storybook/preview-api": "7.6.8", - "@storybook/react-dom-shim": "7.6.8", - "@storybook/types": "7.6.8", + "@storybook/preview-api": "7.6.10", + "@storybook/react-dom-shim": "7.6.10", + "@storybook/types": "7.6.10", "@types/escodegen": "^0.0.6", "@types/estree": "^0.0.51", "@types/node": "^18.0.0", @@ -6347,9 +6327,9 @@ } }, "node_modules/@storybook/react-dom-shim": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/react-dom-shim/-/react-dom-shim-7.6.8.tgz", - "integrity": "sha512-NIvtjdXCTwd0VA/zCaCuCYv7L35nze7qDsFW6JhSHyqB7fKyIEMSbluktO2VISotHOSkgZ2zA+rGpk3O8yh6lg==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/react-dom-shim/-/react-dom-shim-7.6.10.tgz", + "integrity": "sha512-M+N/h6ximacaFdIDjMN2waNoWwApeVYTpFeoDppiFTvdBTXChyIuiPgYX9QSg7gDz92OaA52myGOot4wGvXVzg==", "dev": true, "funding": { "type": "opencollective", @@ -6361,21 +6341,21 @@ } }, "node_modules/@storybook/react/node_modules/@types/node": { - "version": "18.19.6", - "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.6.tgz", - "integrity": "sha512-X36s5CXMrrJOs2lQCdDF68apW4Rfx9ixYMawlepwmE4Anezv/AV2LSpKD1Ub8DAc+urp5bk0BGZ6NtmBitfnsg==", + "version": "18.19.8", + "resolved": "https://registry.npmjs.org/@types/node/-/node-18.19.8.tgz", + "integrity": "sha512-g1pZtPhsvGVTwmeVoexWZLTQaOvXwoSq//pTL0DHeNzUDrFnir4fgETdhjhIxjVnN+hKOuh98+E1eMLnUXstFg==", "dev": true, "dependencies": { "undici-types": "~5.26.4" } }, "node_modules/@storybook/router": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/router/-/router-7.6.8.tgz", - "integrity": "sha512-pFoq22w1kEwduqMpGX3FPSSukdWLMX6UQa2Cw4MDW+hzp3vhC7+3MVaBG5ShQAjGv46NNcSgsIUkyarlU5wd/A==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/router/-/router-7.6.10.tgz", + "integrity": "sha512-G/H4Jn2+y8PDe8Zbq4DVxF/TPn0/goSItdILts39JENucHiuGBCjKjSWGBe1rkwKi1tUbB3yhxJVrLagxFEPpQ==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.8", + "@storybook/client-logger": "7.6.10", "memoizerific": "^1.11.3", "qs": "^6.10.0" }, @@ -6385,14 +6365,14 @@ } }, "node_modules/@storybook/telemetry": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/telemetry/-/telemetry-7.6.8.tgz", - "integrity": "sha512-hHUS3fyHjKR3ZdbG+/OVI+pwXXKOmS8L8GMuWKlpUovvCYBLm0/Q0MUQ9XaLuByOCzvAurqB3Owp3ZV7GiY30Q==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/telemetry/-/telemetry-7.6.10.tgz", + "integrity": "sha512-p3mOSUtIyy2tF1z6pQXxNh1JzYFcAm97nUgkwLzF07GfEdVAPM+ftRSLFbD93zVvLEkmLTlsTiiKaDvOY/lQWg==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.8", - "@storybook/core-common": "7.6.8", - "@storybook/csf-tools": "7.6.8", + "@storybook/client-logger": "7.6.10", + "@storybook/core-common": "7.6.10", + "@storybook/csf-tools": "7.6.10", "chalk": "^4.1.0", "detect-package-manager": "^2.0.1", "fetch-retry": "^5.0.2", @@ -6416,13 +6396,13 @@ } }, "node_modules/@storybook/theming": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/theming/-/theming-7.6.8.tgz", - "integrity": "sha512-0ervBgeYGieifjISlFS7x5QZF9vNgLtHHlYKdkrAsACTK+VfB0JglVwFdLrgzAKxQRlVompaxl3TecFGWlvhtw==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/theming/-/theming-7.6.10.tgz", + "integrity": "sha512-f5tuy7yV3TOP3fIboSqpgLHy0wKayAw/M8HxX0jVET4Z4fWlFK0BiHJabQ+XEdAfQM97XhPFHB2IPbwsqhCEcQ==", "dev": true, "dependencies": { "@emotion/use-insertion-effect-with-fallbacks": "^1.0.0", - "@storybook/client-logger": "7.6.8", + "@storybook/client-logger": "7.6.10", "@storybook/global": "^5.0.0", "memoizerific": "^1.11.3" }, @@ -6436,12 +6416,12 @@ } }, "node_modules/@storybook/types": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/@storybook/types/-/types-7.6.8.tgz", - "integrity": "sha512-+mABX20OhwJjqULocG5Betfidwrlk+Kq+grti+LAYwYsdBwxctBNSrqK8P9r8XDFL6PbppZeExGiHKwGu6WsKQ==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/@storybook/types/-/types-7.6.10.tgz", + "integrity": "sha512-hcS2HloJblaMpCAj2axgGV+53kgSRYPT0a1PG1IHsZaYQILfHSMmBqM8XzXXYTsgf9250kz3dqFX1l0n3EqMlQ==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.8", + "@storybook/channels": "7.6.10", "@types/babel__core": "^7.0.0", "@types/express": "^4.7.0", "file-system-cache": "2.3.0" @@ -6452,9 +6432,9 @@ } }, "node_modules/@swc/core": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core/-/core-1.3.102.tgz", - "integrity": "sha512-OAjNLY/f6QWKSDzaM3bk31A+OYHu6cPa9P/rFIx8X5d24tHXUpRiiq6/PYI6SQRjUPlB72GjsjoEU8F+ALadHg==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core/-/core-1.3.104.tgz", + "integrity": "sha512-9LWH/qzR/Pmyco+XwPiPfz59T1sryI7o5dmqb593MfCkaX5Fzl9KhwQTI47i21/bXYuCdfa9ySZuVkzXMirYxA==", "dev": true, "hasInstallScript": true, "dependencies": { @@ -6469,16 +6449,16 @@ "url": "https://opencollective.com/swc" }, "optionalDependencies": { - "@swc/core-darwin-arm64": "1.3.102", - "@swc/core-darwin-x64": "1.3.102", - "@swc/core-linux-arm-gnueabihf": "1.3.102", - "@swc/core-linux-arm64-gnu": "1.3.102", - "@swc/core-linux-arm64-musl": "1.3.102", - "@swc/core-linux-x64-gnu": "1.3.102", - "@swc/core-linux-x64-musl": "1.3.102", - "@swc/core-win32-arm64-msvc": "1.3.102", - "@swc/core-win32-ia32-msvc": "1.3.102", - "@swc/core-win32-x64-msvc": "1.3.102" + "@swc/core-darwin-arm64": "1.3.104", + "@swc/core-darwin-x64": "1.3.104", + "@swc/core-linux-arm-gnueabihf": "1.3.104", + "@swc/core-linux-arm64-gnu": "1.3.104", + "@swc/core-linux-arm64-musl": "1.3.104", + "@swc/core-linux-x64-gnu": "1.3.104", + "@swc/core-linux-x64-musl": "1.3.104", + "@swc/core-win32-arm64-msvc": "1.3.104", + "@swc/core-win32-ia32-msvc": "1.3.104", + "@swc/core-win32-x64-msvc": "1.3.104" }, "peerDependencies": { "@swc/helpers": "^0.5.0" @@ -6490,9 +6470,9 @@ } }, "node_modules/@swc/core-darwin-arm64": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-darwin-arm64/-/core-darwin-arm64-1.3.102.tgz", - "integrity": "sha512-CJDxA5Wd2cUMULj3bjx4GEoiYyyiyL8oIOu4Nhrs9X+tlg8DnkCm4nI57RJGP8Mf6BaXPIJkHX8yjcefK2RlDA==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-darwin-arm64/-/core-darwin-arm64-1.3.104.tgz", + "integrity": "sha512-rCnVj8x3kn6s914Adddu+zROHUn6mUEMkNKUckofs3W9OthNlZXJA3C5bS2MMTRFXCWamJ0Zmh6INFpz+f4Tfg==", "cpu": [ "arm64" ], @@ -6506,9 +6486,9 @@ } }, "node_modules/@swc/core-darwin-x64": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-darwin-x64/-/core-darwin-x64-1.3.102.tgz", - "integrity": "sha512-X5akDkHwk6oAer49oER0qZMjNMkLH3IOZaV1m98uXIasAGyjo5WH1MKPeMLY1sY6V6TrufzwiSwD4ds571ytcg==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-darwin-x64/-/core-darwin-x64-1.3.104.tgz", + "integrity": "sha512-LBCWGTYkn1UjyxrmcLS3vZgtCDVhwxsQMV7jz5duc7Gas8SRWh6ZYqvUkjlXMDX1yx0uvzHrkaRw445+zDRj7Q==", "cpu": [ "x64" ], @@ -6522,9 +6502,9 @@ } }, "node_modules/@swc/core-linux-arm-gnueabihf": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-linux-arm-gnueabihf/-/core-linux-arm-gnueabihf-1.3.102.tgz", - "integrity": "sha512-kJH3XtZP9YQdjq/wYVBeFuiVQl4HaC4WwRrIxAHwe2OyvrwUI43dpW3LpxSggBnxXcVCXYWf36sTnv8S75o2Gw==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-linux-arm-gnueabihf/-/core-linux-arm-gnueabihf-1.3.104.tgz", + "integrity": "sha512-iFbsWcx0TKHWnFBNCuUstYqRtfkyBx7FKv5To1Hx14EMuvvoCD/qUoJEiNfDQN5n/xU9g5xq4RdbjEWCFLhAbA==", "cpu": [ "arm" ], @@ -6538,9 +6518,9 @@ } }, "node_modules/@swc/core-linux-arm64-gnu": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-linux-arm64-gnu/-/core-linux-arm64-gnu-1.3.102.tgz", - "integrity": "sha512-flQP2WDyCgO24WmKA1wjjTx+xfCmavUete2Kp6yrM+631IHLGnr17eu7rYJ/d4EnDBId/ytMyrnWbTVkaVrpbQ==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-linux-arm64-gnu/-/core-linux-arm64-gnu-1.3.104.tgz", + "integrity": "sha512-1BIIp+nUPrRHHaJ35YJqrwXPwYSITp5robqqjyTwoKGw2kq0x+A964kpWul6v0d7A9Ial8fyH4m13eSWBodD2A==", "cpu": [ "arm64" ], @@ -6554,9 +6534,9 @@ } }, "node_modules/@swc/core-linux-arm64-musl": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-linux-arm64-musl/-/core-linux-arm64-musl-1.3.102.tgz", - "integrity": "sha512-bQEQSnC44DyoIGLw1+fNXKVGoCHi7eJOHr8BdH0y1ooy9ArskMjwobBFae3GX4T1AfnrTaejyr0FvLYIb0Zkog==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-linux-arm64-musl/-/core-linux-arm64-musl-1.3.104.tgz", + "integrity": "sha512-IyDNkzpKwvLqmRwTW+s8f8OsOSSj1N6juZKbvNHpZRfWZkz3T70q3vJlDBWQwy8z8cm7ckd7YUT3eKcSBPPowg==", "cpu": [ "arm64" ], @@ -6570,9 +6550,9 @@ } }, "node_modules/@swc/core-linux-x64-gnu": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-linux-x64-gnu/-/core-linux-x64-gnu-1.3.102.tgz", - "integrity": "sha512-dFvnhpI478svQSxqISMt00MKTDS0e4YtIr+ioZDG/uJ/q+RpcNy3QI2KMm05Fsc8Y0d4krVtvCKWgfUMsJZXAg==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-linux-x64-gnu/-/core-linux-x64-gnu-1.3.104.tgz", + "integrity": "sha512-MfX/wiRdTjE5uXHTDnaX69xI4UBfxIhcxbVlMj//N+7AX/G2pl2UFityfVMU2HpM12BRckrCxVI8F/Zy3DZkYQ==", "cpu": [ "x64" ], @@ -6586,9 +6566,9 @@ } }, "node_modules/@swc/core-linux-x64-musl": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-linux-x64-musl/-/core-linux-x64-musl-1.3.102.tgz", - "integrity": "sha512-+a0M3CvjeIRNA/jTCzWEDh2V+mhKGvLreHOL7J97oULZy5yg4gf7h8lQX9J8t9QLbf6fsk+0F8bVH1Ie/PbXjA==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-linux-x64-musl/-/core-linux-x64-musl-1.3.104.tgz", + "integrity": "sha512-5yeILaxA31gGEmquErO8yxlq1xu0XVt+fz5mbbKXKZMRRILxYxNzAGb5mzV41r0oHz6Vhv4AXX/WMCmeWl+HkQ==", "cpu": [ "x64" ], @@ -6602,9 +6582,9 @@ } }, "node_modules/@swc/core-win32-arm64-msvc": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-win32-arm64-msvc/-/core-win32-arm64-msvc-1.3.102.tgz", - "integrity": "sha512-w76JWLjkZNOfkB25nqdWUNCbt0zJ41CnWrJPZ+LxEai3zAnb2YtgB/cCIrwxDebRuMgE9EJXRj7gDDaTEAMOOQ==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-win32-arm64-msvc/-/core-win32-arm64-msvc-1.3.104.tgz", + "integrity": "sha512-rwcImsYnWDWGmeESG0XdGGOql5s3cG5wA8C4hHHKdH76zamPfDKKQFBsjmoNi0f1IsxaI9AJPeOmD4bAhT1ZoQ==", "cpu": [ "arm64" ], @@ -6618,9 +6598,9 @@ } }, "node_modules/@swc/core-win32-ia32-msvc": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-win32-ia32-msvc/-/core-win32-ia32-msvc-1.3.102.tgz", - "integrity": "sha512-vlDb09HiGqKwz+2cxDS9T5/461ipUQBplvuhW+cCbzzGuPq8lll2xeyZU0N1E4Sz3MVdSPx1tJREuRvlQjrwNg==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-win32-ia32-msvc/-/core-win32-ia32-msvc-1.3.104.tgz", + "integrity": "sha512-ICDA+CJLYC7NkePnrbh/MvXwDQfy3rZSFgrVdrqRosv9DKHdFjYDnA9++7ozjrIdFdBrFW2NR7pyUcidlwhNzA==", "cpu": [ "ia32" ], @@ -6634,9 +6614,9 @@ } }, "node_modules/@swc/core-win32-x64-msvc": { - "version": "1.3.102", - "resolved": "https://registry.npmjs.org/@swc/core-win32-x64-msvc/-/core-win32-x64-msvc-1.3.102.tgz", - "integrity": "sha512-E/jfSD7sShllxBwwgDPeXp1UxvIqehj/ShSUqq1pjR/IDRXngcRSXKJK92mJkNFY7suH6BcCWwzrxZgkO7sWmw==", + "version": "1.3.104", + "resolved": "https://registry.npmjs.org/@swc/core-win32-x64-msvc/-/core-win32-x64-msvc-1.3.104.tgz", + "integrity": "sha512-fZJ1Ju62U4lMZVU+nHxLkFNcu0hG5Y0Yj/5zjrlbuX5N8J5eDndWAFsVnQhxRTZqKhZB53pvWRQs5FItSDqgXg==", "cpu": [ "x64" ], @@ -6670,9 +6650,9 @@ "dev": true }, "node_modules/@tanstack/react-virtual": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.0.1.tgz", - "integrity": "sha512-IFOFuRUTaiM/yibty9qQ9BfycQnYXIDHGP2+cU+0LrFFGNhVxCXSQnaY6wkX8uJVteFEBjUondX0Hmpp7TNcag==", + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/@tanstack/react-virtual/-/react-virtual-3.0.2.tgz", + "integrity": "sha512-9XbRLPKgnhMwwmuQMnJMv+5a9sitGNCSEtf/AZXzmJdesYk7XsjYHaEDny+IrJzvPNwZliIIDwCRiaUqR3zzCA==", "dependencies": { "@tanstack/virtual-core": "3.0.0" }, @@ -6727,9 +6707,9 @@ } }, "node_modules/@tremor/react": { - "version": "3.13.1", - "resolved": "https://registry.npmjs.org/@tremor/react/-/react-3.13.1.tgz", - "integrity": "sha512-vWUjWO5cRKaS4jocwmDM2wkVKi8biXyJJtHOmXKAfH7KFLDRcHEmCKjjr5l34wOp97TtSuG6L3gDzAeV/VRzhw==", + "version": "3.13.2", + "resolved": "https://registry.npmjs.org/@tremor/react/-/react-3.13.2.tgz", + "integrity": "sha512-Imw5wei9A8fOUIKkqt8+XDzY8NCh88cO/hmzhv0HV+SMydj/Km0WpMvQqeo1Ft6GUIVNhcIo1L/T3/02+OwHWA==", "dependencies": { "@floating-ui/react": "^0.19.2", "@headlessui/react": "^1.7.18", @@ -7042,17 +7022,17 @@ "dev": true }, "node_modules/@types/node": { - "version": "20.11.0", - "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.0.tgz", - "integrity": "sha512-o9bjXmDNcF7GbM4CNQpmi+TutCgap/K3w1JyKgxAjqx41zp9qlIAVFi0IhCNsJcXolEqLWhbFbEeL0PvYm4pcQ==", + "version": "20.11.5", + "resolved": "https://registry.npmjs.org/@types/node/-/node-20.11.5.tgz", + "integrity": "sha512-g557vgQjUUfN76MZAN/dt1z3dzcUsimuysco0KeluHgrPdJXkP/XdAURgyO2W9fZWHRtRBiVKzKn8vyOAwlG+w==", "dependencies": { "undici-types": "~5.26.4" } }, "node_modules/@types/node-fetch": { - "version": "2.6.10", - "resolved": "https://registry.npmjs.org/@types/node-fetch/-/node-fetch-2.6.10.tgz", - "integrity": "sha512-PPpPK6F9ALFTn59Ka3BaL+qGuipRfxNE8qVgkp0bVixeiR2c2/L+IVOiBdu9JhhT22sWnQEp6YyHGI2b2+CMcA==", + "version": "2.6.11", + "resolved": "https://registry.npmjs.org/@types/node-fetch/-/node-fetch-2.6.11.tgz", + "integrity": "sha512-24xFj9R5+rfQJLRyM56qh+wnVSYhyXC2tkoBndtY0U+vubqNsYXGjufB2nn8Q6gt0LrARwL6UBtMCSVCwl4B1g==", "dev": true, "dependencies": { "@types/node": "*", @@ -7094,9 +7074,9 @@ "dev": true }, "node_modules/@types/react": { - "version": "18.2.47", - "resolved": "https://registry.npmjs.org/@types/react/-/react-18.2.47.tgz", - "integrity": "sha512-xquNkkOirwyCgoClNk85BjP+aqnIS+ckAJ8i37gAbDs14jfW/J23f2GItAf33oiUPQnqNMALiFeoM9Y5mbjpVQ==", + "version": "18.2.48", + "resolved": "https://registry.npmjs.org/@types/react/-/react-18.2.48.tgz", + "integrity": "sha512-qboRCl6Ie70DQQG9hhNREz81jqC1cs9EVNcjQ1AU+jH6NFfSAhVVbrrY/+nSF+Bsk4AOwm9Qa61InvMCyV+H3w==", "dependencies": { "@types/prop-types": "*", "@types/scheduler": "*", @@ -7190,15 +7170,15 @@ "dev": true }, "node_modules/@typescript-eslint/parser": { - "version": "6.18.1", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-6.18.1.tgz", - "integrity": "sha512-zct/MdJnVaRRNy9e84XnVtRv9Vf91/qqe+hZJtKanjojud4wAVy/7lXxJmMyX6X6J+xc6c//YEWvpeif8cAhWA==", + "version": "6.19.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-6.19.0.tgz", + "integrity": "sha512-1DyBLG5SH7PYCd00QlroiW60YJ4rWMuUGa/JBV0iZuqi4l4IK3twKPq5ZkEebmGqRjXWVgsUzfd3+nZveewgow==", "dev": true, "dependencies": { - "@typescript-eslint/scope-manager": "6.18.1", - "@typescript-eslint/types": "6.18.1", - "@typescript-eslint/typescript-estree": "6.18.1", - "@typescript-eslint/visitor-keys": "6.18.1", + "@typescript-eslint/scope-manager": "6.19.0", + "@typescript-eslint/types": "6.19.0", + "@typescript-eslint/typescript-estree": "6.19.0", + "@typescript-eslint/visitor-keys": "6.19.0", "debug": "^4.3.4" }, "engines": { @@ -7218,13 +7198,13 @@ } }, "node_modules/@typescript-eslint/scope-manager": { - "version": "6.18.1", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-6.18.1.tgz", - "integrity": "sha512-BgdBwXPFmZzaZUuw6wKiHKIovms97a7eTImjkXCZE04TGHysG+0hDQPmygyvgtkoB/aOQwSM/nWv3LzrOIQOBw==", + "version": "6.19.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-6.19.0.tgz", + "integrity": "sha512-dO1XMhV2ehBI6QN8Ufi7I10wmUovmLU0Oru3n5LVlM2JuzB4M+dVphCPLkVpKvGij2j/pHBWuJ9piuXx+BhzxQ==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.18.1", - "@typescript-eslint/visitor-keys": "6.18.1" + "@typescript-eslint/types": "6.19.0", + "@typescript-eslint/visitor-keys": "6.19.0" }, "engines": { "node": "^16.0.0 || >=18.0.0" @@ -7235,9 +7215,9 @@ } }, "node_modules/@typescript-eslint/types": { - "version": "6.18.1", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-6.18.1.tgz", - "integrity": "sha512-4TuMAe+tc5oA7wwfqMtB0Y5OrREPF1GeJBAjqwgZh1lEMH5PJQgWgHGfYufVB51LtjD+peZylmeyxUXPfENLCw==", + "version": "6.19.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-6.19.0.tgz", + "integrity": "sha512-lFviGV/vYhOy3m8BJ/nAKoAyNhInTdXpftonhWle66XHAtT1ouBlkjL496b5H5hb8dWXHwtypTqgtb/DEa+j5A==", "dev": true, "engines": { "node": "^16.0.0 || >=18.0.0" @@ -7248,13 +7228,13 @@ } }, "node_modules/@typescript-eslint/typescript-estree": { - "version": "6.18.1", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-6.18.1.tgz", - "integrity": "sha512-fv9B94UAhywPRhUeeV/v+3SBDvcPiLxRZJw/xZeeGgRLQZ6rLMG+8krrJUyIf6s1ecWTzlsbp0rlw7n9sjufHA==", + "version": "6.19.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-6.19.0.tgz", + "integrity": "sha512-o/zefXIbbLBZ8YJ51NlkSAt2BamrK6XOmuxSR3hynMIzzyMY33KuJ9vuMdFSXW+H0tVvdF9qBPTHA91HDb4BIQ==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.18.1", - "@typescript-eslint/visitor-keys": "6.18.1", + "@typescript-eslint/types": "6.19.0", + "@typescript-eslint/visitor-keys": "6.19.0", "debug": "^4.3.4", "globby": "^11.1.0", "is-glob": "^4.0.3", @@ -7528,12 +7508,12 @@ "dev": true }, "node_modules/@typescript-eslint/visitor-keys": { - "version": "6.18.1", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-6.18.1.tgz", - "integrity": "sha512-/kvt0C5lRqGoCfsbmm7/CwMqoSkY3zzHLIjdhHZQW3VFrnz7ATecOHR7nb7V+xn4286MBxfnQfQhAmCI0u+bJA==", + "version": "6.19.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-6.19.0.tgz", + "integrity": "sha512-hZaUCORLgubBvtGpp1JEFEazcuEdfxta9j4iUwdSAr7mEsYYAp3EAUyCZk3VEEqGj6W+AV4uWyrDGtrlawAsgQ==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.18.1", + "@typescript-eslint/types": "6.19.0", "eslint-visitor-keys": "^3.4.1" }, "engines": { @@ -8251,9 +8231,9 @@ "dev": true }, "node_modules/autoprefixer": { - "version": "10.4.16", - "resolved": "https://registry.npmjs.org/autoprefixer/-/autoprefixer-10.4.16.tgz", - "integrity": "sha512-7vd3UC6xKp0HLfua5IjZlcXvGAGy7cBAXTg2lyQ/8WpNhd6SiZ8Be+xm3FyBSYJx5GKcpRCzBh7RH4/0dnY+uQ==", + "version": "10.4.17", + "resolved": "https://registry.npmjs.org/autoprefixer/-/autoprefixer-10.4.17.tgz", + "integrity": "sha512-/cpVNRLSfhOtcGflT13P2794gVSgmPgTR+erw5ifnMLZb0UnSlkK4tquLmkd3BhA+nLo5tX8Cu0upUsGKvKbmg==", "dev": true, "funding": [ { @@ -8270,9 +8250,9 @@ } ], "dependencies": { - "browserslist": "^4.21.10", - "caniuse-lite": "^1.0.30001538", - "fraction.js": "^4.3.6", + "browserslist": "^4.22.2", + "caniuse-lite": "^1.0.30001578", + "fraction.js": "^4.3.7", "normalize-range": "^0.1.2", "picocolors": "^1.0.0", "postcss-value-parser": "^4.2.0" @@ -8552,13 +8532,13 @@ } }, "node_modules/babel-plugin-polyfill-corejs2": { - "version": "0.4.7", - "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.4.7.tgz", - "integrity": "sha512-LidDk/tEGDfuHW2DWh/Hgo4rmnw3cduK6ZkOI1NPFceSK3n/yAGeOsNT7FLnSGHkXj3RHGSEVkN3FsCTY6w2CQ==", + "version": "0.4.8", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-corejs2/-/babel-plugin-polyfill-corejs2-0.4.8.tgz", + "integrity": "sha512-OtIuQfafSzpo/LhnJaykc0R/MMnuLSSVjVYy9mHArIZ9qTCSZ6TpWCuEKZYVoN//t8HqBNScHrOtCrIK5IaGLg==", "dev": true, "dependencies": { "@babel/compat-data": "^7.22.6", - "@babel/helper-define-polyfill-provider": "^0.4.4", + "@babel/helper-define-polyfill-provider": "^0.5.0", "semver": "^6.3.1" }, "peerDependencies": { @@ -8578,13 +8558,29 @@ "@babel/core": "^7.4.0 || ^8.0.0-0 <8.0.0" } }, + "node_modules/babel-plugin-polyfill-corejs3/node_modules/@babel/helper-define-polyfill-provider": { + "version": "0.4.4", + "resolved": "https://registry.npmjs.org/@babel/helper-define-polyfill-provider/-/helper-define-polyfill-provider-0.4.4.tgz", + "integrity": "sha512-QcJMILQCu2jm5TFPGA3lCpJJTeEP+mqeXooG/NZbg/h5FTFi6V0+99ahlRsW8/kRLyb24LZVCCiclDedhLKcBA==", + "dev": true, + "dependencies": { + "@babel/helper-compilation-targets": "^7.22.6", + "@babel/helper-plugin-utils": "^7.22.5", + "debug": "^4.1.1", + "lodash.debounce": "^4.0.8", + "resolve": "^1.14.2" + }, + "peerDependencies": { + "@babel/core": "^7.4.0 || ^8.0.0-0 <8.0.0" + } + }, "node_modules/babel-plugin-polyfill-regenerator": { - "version": "0.5.4", - "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.5.4.tgz", - "integrity": "sha512-S/x2iOCvDaCASLYsOOgWOq4bCfKYVqvO/uxjkaYyZ3rVsVE3CeAI/c84NpyuBBymEgNvHgjEot3a9/Z/kXvqsg==", + "version": "0.5.5", + "resolved": "https://registry.npmjs.org/babel-plugin-polyfill-regenerator/-/babel-plugin-polyfill-regenerator-0.5.5.tgz", + "integrity": "sha512-OJGYZlhLqBh2DDHeqAxWB1XIvr49CxiJ2gIt61/PU55CQK4Z58OzMqjDe1zwQdQk+rBYsRc+1rJmdajM3gimHg==", "dev": true, "dependencies": { - "@babel/helper-define-polyfill-provider": "^0.4.4" + "@babel/helper-define-polyfill-provider": "^0.5.0" }, "peerDependencies": { "@babel/core": "^7.4.0 || ^8.0.0-0 <8.0.0" @@ -9073,9 +9069,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001576", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001576.tgz", - "integrity": "sha512-ff5BdakGe2P3SQsMsiqmt1Lc8221NR1VzHj5jXN5vBny9A6fpze94HiVV/n7XRosOlsShJcvMv5mdnpjOGCEgg==", + "version": "1.0.30001579", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001579.tgz", + "integrity": "sha512-u5AUVkixruKHJjw/pj9wISlcMpgFWzSrczLZbrqBSxukQixmg0SJ5sZTpvaFvxU0HoQKd4yoyAogyrAz9pzJnA==", "funding": [ { "type": "opencollective", @@ -9205,6 +9201,12 @@ "consola": "^3.2.3" } }, + "node_modules/cjs-module-lexer": { + "version": "1.2.3", + "resolved": "https://registry.npmjs.org/cjs-module-lexer/-/cjs-module-lexer-1.2.3.tgz", + "integrity": "sha512-0TNiGstbQmCFwt4akjjBg5pLRTSyj/PkWQ1ZoO2zntmg9yLqSRxwEa4iCfQLGjqhiqBfOJa7W/E8wfGrTDmlZQ==", + "dev": true + }, "node_modules/classnames": { "version": "2.5.1", "resolved": "https://registry.npmjs.org/classnames/-/classnames-2.5.1.tgz", @@ -9387,9 +9389,9 @@ } }, "node_modules/clsx": { - "version": "1.2.1", - "resolved": "https://registry.npmjs.org/clsx/-/clsx-1.2.1.tgz", - "integrity": "sha512-EcR6r5a8bj6pu3ycsa/E/cKVGuTgZJZdsyUYHOksG/UHIiKfjxzRxYJpyVBwYaQeOvghal9fcc4PidlgzugAQg==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/clsx/-/clsx-2.1.0.tgz", + "integrity": "sha512-m3iNNWpd9rl3jvvcBnu70ylMdrXt8Vlq4HYadnU5fwcOtvkSQWPmj7amUcDT2qYI7risszBjI5AUIUox9D16pg==", "engines": { "node": ">=6" } @@ -9664,9 +9666,9 @@ } }, "node_modules/copy-webpack-plugin": { - "version": "12.0.1", - "resolved": "https://registry.npmjs.org/copy-webpack-plugin/-/copy-webpack-plugin-12.0.1.tgz", - "integrity": "sha512-dhMfjJMYKDmmbG6Yn2pRSs1g8FgeQRtbE/JM6VAM9Xouk3KO1UVrwlLHLXxaI5F+o9WgnRfhFZzY9eV34O2gZQ==", + "version": "12.0.2", + "resolved": "https://registry.npmjs.org/copy-webpack-plugin/-/copy-webpack-plugin-12.0.2.tgz", + "integrity": "sha512-SNwdBeHyII+rWvee/bTnAYyO8vfVdcSTud4EIb6jcZ8inLeWucJE0DnxXQBjlQ5zlteuuvooGQy3LIyGxhvlOA==", "dev": true, "dependencies": { "fast-glob": "^3.3.2", @@ -9881,16 +9883,16 @@ } }, "node_modules/css-loader": { - "version": "6.9.0", - "resolved": "https://registry.npmjs.org/css-loader/-/css-loader-6.9.0.tgz", - "integrity": "sha512-3I5Nu4ytWlHvOP6zItjiHlefBNtrH+oehq8tnQa2kO305qpVyx9XNIT1CXIj5bgCJs7qICBCkgCYxQLKPANoLA==", + "version": "6.9.1", + "resolved": "https://registry.npmjs.org/css-loader/-/css-loader-6.9.1.tgz", + "integrity": "sha512-OzABOh0+26JKFdMzlK6PY1u5Zx8+Ck7CVRlcGNZoY9qwJjdfu2VWFuprTIpPW+Av5TZTVViYWcFQaEEQURLknQ==", "dev": true, "dependencies": { "icss-utils": "^5.1.0", - "postcss": "^8.4.31", + "postcss": "^8.4.33", "postcss-modules-extract-imports": "^3.0.0", - "postcss-modules-local-by-default": "^4.0.3", - "postcss-modules-scope": "^3.1.0", + "postcss-modules-local-by-default": "^4.0.4", + "postcss-modules-scope": "^3.1.1", "postcss-modules-values": "^4.0.0", "postcss-value-parser": "^4.2.0", "semver": "^7.5.4" @@ -10588,9 +10590,9 @@ } }, "node_modules/dotenv": { - "version": "16.3.1", - "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-16.3.1.tgz", - "integrity": "sha512-IPzF4w4/Rd94bA9imS68tZBaYyBWSCE47V1RGuMrB94iyTOIEwRmVL2x/4An+6mETpLrKJ5hQkB8W4kFAadeIQ==", + "version": "16.3.2", + "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-16.3.2.tgz", + "integrity": "sha512-HTlk5nmhkm8F6JcdXvHIzaorzCoziNQT9mGxLPVXW8wJF1TiGSL60ZGB4gHWabHOaMmWmhvk2/lPHfnBiT78AQ==", "dev": true, "engines": { "node": ">=12" @@ -10683,9 +10685,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.630", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.630.tgz", - "integrity": "sha512-osHqhtjojpCsACVnuD11xO5g9xaCyw7Qqn/C2KParkMv42i8jrJJgx3g7mkHfpxwhy9MnOJr8+pKOdZ7qzgizg==", + "version": "1.4.640", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.640.tgz", + "integrity": "sha512-z/6oZ/Muqk4BaE7P69bXhUhpJbUM9ZJeka43ZwxsDshKtePns4mhBlh8bU5+yrnOnz3fhG82XLzGUXazOmsWnA==", "dev": true }, "node_modules/elliptic": { @@ -11130,12 +11132,12 @@ } }, "node_modules/eslint-config-next": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.0.4.tgz", - "integrity": "sha512-9/xbOHEQOmQtqvQ1UsTQZpnA7SlDMBtuKJ//S4JnoyK3oGLhILKXdBgu/UO7lQo/2xOykQULS1qQ6p2+EpHgAQ==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/eslint-config-next/-/eslint-config-next-14.1.0.tgz", + "integrity": "sha512-SBX2ed7DoRFXC6CQSLc/SbLY9Ut6HxNB2wPTcoIWjUMd7aF7O/SIE7111L8FdZ9TXsNV4pulUDnfthpyPtbFUg==", "dev": true, "dependencies": { - "@next/eslint-plugin-next": "14.0.4", + "@next/eslint-plugin-next": "14.1.0", "@rushstack/eslint-patch": "^1.3.3", "@typescript-eslint/parser": "^5.4.2 || ^6.0.0", "eslint-import-resolver-node": "^0.3.6", @@ -12160,9 +12162,9 @@ "dev": true }, "node_modules/flow-parser": { - "version": "0.226.0", - "resolved": "https://registry.npmjs.org/flow-parser/-/flow-parser-0.226.0.tgz", - "integrity": "sha512-YlH+Y/P/5s0S7Vg14RwXlJMF/JsGfkG7gcKB/zljyoqaPNX9YVsGzx+g6MLTbhZaWbPhs4347aTpmSb9GgiPtw==", + "version": "0.227.0", + "resolved": "https://registry.npmjs.org/flow-parser/-/flow-parser-0.227.0.tgz", + "integrity": "sha512-nOygtGKcX/siZK/lFzpfdHEfOkfGcTW7rNroR1Zsz6T/JxSahPALXVt5qVHq/fgvMJuv096BTKbgxN3PzVBaDA==", "dev": true, "engines": { "node": ">=0.4.0" @@ -12680,7 +12682,8 @@ "node_modules/glob-to-regexp": { "version": "0.4.1", "resolved": "https://registry.npmjs.org/glob-to-regexp/-/glob-to-regexp-0.4.1.tgz", - "integrity": "sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw==" + "integrity": "sha512-lkX1HJXwyMcprw/5YUZc2s7DrpAiHB21/V+E1rHUrVNokkvB6bqMzT0VfV6/86ZNabt1k14YOIaT7nDvOX3Iiw==", + "dev": true }, "node_modules/glob/node_modules/brace-expansion": { "version": "2.0.1", @@ -14502,9 +14505,9 @@ } }, "node_modules/lucide-react": { - "version": "0.309.0", - "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.309.0.tgz", - "integrity": "sha512-zNVPczuwFrCfksZH3zbd1UDE6/WYhYAdbe2k7CImVyPAkXLgIwbs6eXQ4loigqDnUFjyFYCI5jZ1y10Kqal0dg==", + "version": "0.312.0", + "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.312.0.tgz", + "integrity": "sha512-3UZsqyswRXjW4t+nw+InICewSimjPKHuSxiFYqTshv9xkK3tPPntXk/lvXc9pKlXIxm3v9WKyoxcrB6YHhP+dg==", "peerDependencies": { "react": "^16.5.1 || ^17.0.0 || ^18.0.0" } @@ -14975,18 +14978,17 @@ "dev": true }, "node_modules/next": { - "version": "14.0.4", - "resolved": "https://registry.npmjs.org/next/-/next-14.0.4.tgz", - "integrity": "sha512-qbwypnM7327SadwFtxXnQdGiKpkuhaRLE2uq62/nRul9cj9KhQ5LhHmlziTNqUidZotw/Q1I9OjirBROdUJNgA==", + "version": "14.1.0", + "resolved": "https://registry.npmjs.org/next/-/next-14.1.0.tgz", + "integrity": "sha512-wlzrsbfeSU48YQBjZhDzOwhWhGsy+uQycR8bHAOt1LY1bn3zZEcDyHQOEoN3aWzQ8LHCAJ1nqrWCc9XF2+O45Q==", "dependencies": { - "@next/env": "14.0.4", + "@next/env": "14.1.0", "@swc/helpers": "0.5.2", "busboy": "1.6.0", - "caniuse-lite": "^1.0.30001406", + "caniuse-lite": "^1.0.30001579", "graceful-fs": "^4.2.11", "postcss": "8.4.31", - "styled-jsx": "5.1.1", - "watchpack": "2.4.0" + "styled-jsx": "5.1.1" }, "bin": { "next": "dist/bin/next" @@ -14995,15 +14997,15 @@ "node": ">=18.17.0" }, "optionalDependencies": { - "@next/swc-darwin-arm64": "14.0.4", - "@next/swc-darwin-x64": "14.0.4", - "@next/swc-linux-arm64-gnu": "14.0.4", - "@next/swc-linux-arm64-musl": "14.0.4", - "@next/swc-linux-x64-gnu": "14.0.4", - "@next/swc-linux-x64-musl": "14.0.4", - "@next/swc-win32-arm64-msvc": "14.0.4", - "@next/swc-win32-ia32-msvc": "14.0.4", - "@next/swc-win32-x64-msvc": "14.0.4" + "@next/swc-darwin-arm64": "14.1.0", + "@next/swc-darwin-x64": "14.1.0", + "@next/swc-linux-arm64-gnu": "14.1.0", + "@next/swc-linux-arm64-musl": "14.1.0", + "@next/swc-linux-x64-gnu": "14.1.0", + "@next/swc-linux-x64-musl": "14.1.0", + "@next/swc-win32-arm64-msvc": "14.1.0", + "@next/swc-win32-ia32-msvc": "14.1.0", + "@next/swc-win32-x64-msvc": "14.1.0" }, "peerDependencies": { "@opentelemetry/api": "^1.1.0", @@ -16308,9 +16310,9 @@ } }, "node_modules/postcss-modules-local-by-default": { - "version": "4.0.3", - "resolved": "https://registry.npmjs.org/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.3.tgz", - "integrity": "sha512-2/u2zraspoACtrbFRnTijMiQtb4GW4BvatjaG/bCjYQo8kLTdevCUlwuBHx2sCnSyrI3x3qj4ZK1j5LQBgzmwA==", + "version": "4.0.4", + "resolved": "https://registry.npmjs.org/postcss-modules-local-by-default/-/postcss-modules-local-by-default-4.0.4.tgz", + "integrity": "sha512-L4QzMnOdVwRm1Qb8m4x8jsZzKAaPAgrUF1r/hjDR2Xj7R+8Zsf97jAlSQzWtKx5YNiNGN8QxmPFIc/sh+RQl+Q==", "dev": true, "dependencies": { "icss-utils": "^5.0.0", @@ -16325,9 +16327,9 @@ } }, "node_modules/postcss-modules-scope": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-3.1.0.tgz", - "integrity": "sha512-SaIbK8XW+MZbd0xHPf7kdfA/3eOt7vxJ72IRecn3EzuZVLr1r0orzf0MX/pN8m+NMDoo6X/SQd8oeKqGZd8PXg==", + "version": "3.1.1", + "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-3.1.1.tgz", + "integrity": "sha512-uZgqzdTleelWjzJY+Fhti6F3C9iF1JR/dODLs/JDefozYcKTBCdD8BIl6nNPbTbcLnGrk56hzwZC2DaGNvYjzA==", "dev": true, "dependencies": { "postcss-selector-parser": "^6.0.4" @@ -16498,9 +16500,9 @@ } }, "node_modules/prettier": { - "version": "3.2.1", - "resolved": "https://registry.npmjs.org/prettier/-/prettier-3.2.1.tgz", - "integrity": "sha512-qSUWshj1IobVbKc226Gw2pync27t0Kf0EdufZa9j7uBSJay1CC+B3K5lAAZoqgX3ASiKuWsk6OmzKRetXNObWg==", + "version": "3.2.4", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-3.2.4.tgz", + "integrity": "sha512-FWu1oLHKCrtpO1ypU6J0SbK2d9Ckwysq6bHj/uaCP26DxrPpppCLQRGVuqAxSTvhF00AcvDRyYrLNW7ocBhFFQ==", "dev": true, "bin": { "prettier": "bin/prettier.cjs" @@ -16578,13 +16580,13 @@ } }, "node_modules/prisma": { - "version": "5.8.0", - "resolved": "https://registry.npmjs.org/prisma/-/prisma-5.8.0.tgz", - "integrity": "sha512-hDKoEqPt2qEUTH5yGO3l27CBnPtwvte0CGMKrpCr9+/A919JghfqJ3qgCGgMbOwdkXUOzdho0RH9tyUF3UhpMw==", + "version": "5.8.1", + "resolved": "https://registry.npmjs.org/prisma/-/prisma-5.8.1.tgz", + "integrity": "sha512-N6CpjzECnUHZ5beeYpDzkt2rYpEdAeqXX2dweu6BoQaeYkNZrC/WJHM+5MO/uidFHTak8QhkPKBWck1o/4MD4A==", "devOptional": true, "hasInstallScript": true, "dependencies": { - "@prisma/engines": "5.8.0" + "@prisma/engines": "5.8.1" }, "bin": { "prisma": "build/index.js" @@ -16646,9 +16648,9 @@ "integrity": "sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ==" }, "node_modules/protobufjs": { - "version": "7.2.5", - "resolved": "https://registry.npmjs.org/protobufjs/-/protobufjs-7.2.5.tgz", - "integrity": "sha512-gGXRSXvxQ7UiPgfw8gevrfRWcTlSbOFg+p/N+JVJEK5VhueL2miT6qTymqAmjr1Q5WbOCyJbyrk6JfWKwlFn6A==", + "version": "7.2.6", + "resolved": "https://registry.npmjs.org/protobufjs/-/protobufjs-7.2.6.tgz", + "integrity": "sha512-dgJaEDDL6x8ASUZ1YqWciTRrdOuYNzoOf27oHNfdyvKqHr5i0FV7FSLU+aIeFjyFgVxrpTOtQUi0BLLBymZaBw==", "hasInstallScript": true, "dependencies": { "@protobufjs/aspromise": "^1.1.2", @@ -17220,11 +17222,11 @@ } }, "node_modules/react-toastify": { - "version": "9.1.3", - "resolved": "https://registry.npmjs.org/react-toastify/-/react-toastify-9.1.3.tgz", - "integrity": "sha512-fPfb8ghtn/XMxw3LkxQBk3IyagNpF/LIKjOBflbexr2AWxAH1MJgvnESwEwBn9liLFXgTKWgBSdZpw9m4OTHTg==", + "version": "10.0.3", + "resolved": "https://registry.npmjs.org/react-toastify/-/react-toastify-10.0.3.tgz", + "integrity": "sha512-PBJwXjFKKM73tgb6iSld4GMs9ShBWGUvc9zPHmdDgT4CdSr32iqSNh6y/fFN/tosvkTS6/tBLptDxXiXgcjvuw==", "dependencies": { - "clsx": "^1.1.1" + "clsx": "^2.1.0" }, "peerDependencies": { "react": ">=16", @@ -17439,14 +17441,6 @@ "decimal.js-light": "^2.4.1" } }, - "node_modules/recharts/node_modules/clsx": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/clsx/-/clsx-2.1.0.tgz", - "integrity": "sha512-m3iNNWpd9rl3jvvcBnu70ylMdrXt8Vlq4HYadnU5fwcOtvkSQWPmj7amUcDT2qYI7risszBjI5AUIUox9D16pg==", - "engines": { - "node": ">=6" - } - }, "node_modules/recharts/node_modules/react-is": { "version": "16.13.1", "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.13.1.tgz", @@ -17820,13 +17814,13 @@ } }, "node_modules/safe-array-concat": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/safe-array-concat/-/safe-array-concat-1.0.1.tgz", - "integrity": "sha512-6XbUAseYE2KtOuGueyeobCySj9L4+66Tn6KQMOPQJrAJEowYKW/YR/MGJZl7FdydUdaFu4LYyDZjxf4/Nmo23Q==", + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/safe-array-concat/-/safe-array-concat-1.1.0.tgz", + "integrity": "sha512-ZdQ0Jeb9Ofti4hbt5lX3T2JcAamT9hfzYU1MNB+z/jaEbB6wfFfPIR/zEORmZqobkCCJhSjodobH6WHNmJ97dg==", "dev": true, "dependencies": { - "call-bind": "^1.0.2", - "get-intrinsic": "^1.2.1", + "call-bind": "^1.0.5", + "get-intrinsic": "^1.2.2", "has-symbols": "^1.0.3", "isarray": "^2.0.5" }, @@ -18029,15 +18023,16 @@ } }, "node_modules/set-function-length": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/set-function-length/-/set-function-length-1.1.1.tgz", - "integrity": "sha512-VoaqjbBJKiWtg4yRcKBQ7g7wnGnLV3M8oLvVWwOk2PdYY6PEFegR1vezXR0tw6fZGF9csVakIRjrJiy2veSBFQ==", + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/set-function-length/-/set-function-length-1.2.0.tgz", + "integrity": "sha512-4DBHDoyHlM1IRPGYcoxexgh67y4ueR53FKV1yyxwFMY7aCqcN/38M1+SwZ/qJQ8iLv7+ck385ot4CcisOAPT9w==", "dev": true, "dependencies": { "define-data-property": "^1.1.1", - "get-intrinsic": "^1.2.1", + "function-bind": "^1.1.2", + "get-intrinsic": "^1.2.2", "gopd": "^1.0.1", - "has-property-descriptors": "^1.0.0" + "has-property-descriptors": "^1.0.1" }, "engines": { "node": ">= 0.4" @@ -18259,51 +18254,6 @@ "integrity": "sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ==", "dev": true }, - "node_modules/simple-update-notifier": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/simple-update-notifier/-/simple-update-notifier-2.0.0.tgz", - "integrity": "sha512-a2B9Y0KlNXl9u/vsW6sTIu9vGEpfKu2wRV6l1H3XEas/0gUIzGzBoP/IouTcUQbm9JWZLH3COxyn03TYlFax6w==", - "dev": true, - "dependencies": { - "semver": "^7.5.3" - }, - "engines": { - "node": ">=10" - } - }, - "node_modules/simple-update-notifier/node_modules/lru-cache": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz", - "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==", - "dev": true, - "dependencies": { - "yallist": "^4.0.0" - }, - "engines": { - "node": ">=10" - } - }, - "node_modules/simple-update-notifier/node_modules/semver": { - "version": "7.5.4", - "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.4.tgz", - "integrity": "sha512-1bCSESV6Pv+i21Hvpxp3Dx+pSD8lIPt8uVjRrxAUt/nbswYc+tK6Y2btiULjd4+fnq15PX+nqQDC7Oft7WkwcA==", - "dev": true, - "dependencies": { - "lru-cache": "^6.0.0" - }, - "bin": { - "semver": "bin/semver.js" - }, - "engines": { - "node": ">=10" - } - }, - "node_modules/simple-update-notifier/node_modules/yallist": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz", - "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==", - "dev": true - }, "node_modules/sisteransi": { "version": "1.0.5", "resolved": "https://registry.npmjs.org/sisteransi/-/sisteransi-1.0.5.tgz", @@ -18442,12 +18392,12 @@ "dev": true }, "node_modules/storybook": { - "version": "7.6.8", - "resolved": "https://registry.npmjs.org/storybook/-/storybook-7.6.8.tgz", - "integrity": "sha512-ugRtDSs2eTgHMOZ3wKXbUEbPnlJ2XImPbnvxNssK14py2mHKwPnhSqLNrjlQMkmkO13GdjalLDyj4lZtoYdo0Q==", + "version": "7.6.10", + "resolved": "https://registry.npmjs.org/storybook/-/storybook-7.6.10.tgz", + "integrity": "sha512-ypFeGhQTUBBfqSUVZYh7wS5ghn3O2wILCiQc4459SeUpvUn+skcqw/TlrwGSoF5EWjDA7gtRrWDxO3mnlPt5Cw==", "dev": true, "dependencies": { - "@storybook/cli": "7.6.8" + "@storybook/cli": "7.6.10" }, "bin": { "sb": "index.js", @@ -18509,9 +18459,9 @@ } }, "node_modules/stream-shift": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/stream-shift/-/stream-shift-1.0.2.tgz", - "integrity": "sha512-rV4Bovi9xx0BFzOb/X0B2GqoIjvqPCttZdu0Wgtx2Dxkj7ETyWl9gmqJ4EutWRLvtZWm8dxE+InQZX1IryZn/w==", + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/stream-shift/-/stream-shift-1.0.3.tgz", + "integrity": "sha512-76ORR0DO1o1hlKwTbi/DM3EXWGf3ZJYO8cXX5RJwnul2DEg2oyoZyjLNoQM8WsvZiFKCRfC1O0J7iCvie3RZmQ==", "dev": true }, "node_modules/streamsearch": { @@ -19066,9 +19016,9 @@ } }, "node_modules/tar-stream": { - "version": "3.1.6", - "resolved": "https://registry.npmjs.org/tar-stream/-/tar-stream-3.1.6.tgz", - "integrity": "sha512-B/UyjYwPpMBv+PaFSWAmtYjwdrlEaZQEhMIBFNC5oEG8lpiW8XjcSdmEaClj28ArfKScKHs2nshz3k2le6crsg==", + "version": "3.1.7", + "resolved": "https://registry.npmjs.org/tar-stream/-/tar-stream-3.1.7.tgz", + "integrity": "sha512-qJj60CXt7IU1Ffyc3NJMjh6EkuCFej46zUqJ4J7pqYlThyd9bO0XBTmcOIhSzZJVWfsLks0+nle/j538YAW9RQ==", "dev": true, "dependencies": { "b4a": "^1.6.4", @@ -19197,9 +19147,9 @@ } }, "node_modules/terser": { - "version": "5.26.0", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.26.0.tgz", - "integrity": "sha512-dytTGoE2oHgbNV9nTzgBEPaqAWvcJNl66VZ0BkJqlvp71IjO8CxdBx/ykCNb47cLnCmCvRZ6ZR0tLkqvZCdVBQ==", + "version": "5.27.0", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.27.0.tgz", + "integrity": "sha512-bi1HRwVRskAjheeYl291n3JC4GgO/Ty4z1nVs5AAsmonJulGxpSektecnNedrwK9C7vpvVtcX3cw00VSLt7U2A==", "dev": true, "dependencies": { "@jridgewell/source-map": "^0.3.3", @@ -20047,10 +19997,9 @@ } }, "node_modules/usehooks-ts": { - "version": "2.9.2", - "resolved": "https://registry.npmjs.org/usehooks-ts/-/usehooks-ts-2.9.2.tgz", - "integrity": "sha512-fOzPeG01rs51CGYzqgioP/zs9v1Cgpe+zcXeqJPlDHYfdfG/wjsdjBWHJi+Ph1JgQAGUrDo5sJbPlaZd+Z9lxw==", - "hasInstallScript": true, + "version": "2.9.5", + "resolved": "https://registry.npmjs.org/usehooks-ts/-/usehooks-ts-2.9.5.tgz", + "integrity": "sha512-M5LrjX9Wo4XRsof0zCCsq81x5+so/b89aUO53XPTjUehKAbOFhRq/DDRyv7qsRK4AEaJe1rhReFZqesPGnREhQ==", "engines": { "node": ">=16.15.0" }, @@ -20125,9 +20074,9 @@ } }, "node_modules/victory-vendor": { - "version": "36.8.1", - "resolved": "https://registry.npmjs.org/victory-vendor/-/victory-vendor-36.8.1.tgz", - "integrity": "sha512-T8cXN8D6J9wEtDEHLiXcgrOE5gyKR39s9fCFTGmcOfqDrT8m2XQLt+2p/n007uxEMRvCDH7GYYqy4vV7GIcGhw==", + "version": "36.8.2", + "resolved": "https://registry.npmjs.org/victory-vendor/-/victory-vendor-36.8.2.tgz", + "integrity": "sha512-NfSQi7ISCdBbDpn3b6rg+8RpFZmWIM9mcks48BbogHE2F6h1XKdA34oiCKP5hP1OGvTotDRzsexiJKzrK4Exuw==", "dependencies": { "@types/d3-array": "^3.0.3", "@types/d3-ease": "^3.0.0", @@ -20164,6 +20113,7 @@ "version": "2.4.0", "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.0.tgz", "integrity": "sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg==", + "dev": true, "dependencies": { "glob-to-regexp": "^0.4.1", "graceful-fs": "^4.1.2" diff --git a/ui/package.json b/ui/package.json index 925f0edd8c..0aba44b6ae 100644 --- a/ui/package.json +++ b/ui/package.json @@ -14,9 +14,9 @@ "storybook:deploy": "storybook build && touch storybook-static/.nojekyll && rm storybook-static/.gitignore && gh-pages -t -d storybook-static && rm -rf ./storybook-static" }, "dependencies": { - "@grpc/grpc-js": "^1.9.13", + "@grpc/grpc-js": "^1.9.14", "@monaco-editor/react": "^4.6.0", - "@prisma/client": "^5.8.0", + "@prisma/client": "^5.8.1", "@radix-ui/react-checkbox": "^1.0.4", "@radix-ui/react-collapsible": "^1.0.3", "@radix-ui/react-dialog": "^1.0.5", @@ -31,52 +31,52 @@ "@radix-ui/react-toggle": "^1.0.3", "@radix-ui/react-toggle-group": "^1.0.4", "@radix-ui/react-tooltip": "^1.0.7", - "@tremor/react": "^3.13.1", - "@types/node": "^20.11.0", - "@types/react": "^18.2.47", + "@tremor/react": "^3.13.2", + "@types/node": "^20.11.5", + "@types/react": "^18.2.48", "@types/react-dom": "^18.2.18", "classnames": "^2.5.1", "long": "^5.2.3", - "lucide-react": "^0.309.0", + "lucide-react": "^0.312.0", "material-symbols": "^0.14.5", "moment": "^2.30.1", "moment-timezone": "^0.5.44", - "next": "^14.0.4", + "next": "^14.1.0", "next-auth": "^4.24.5", "prop-types": "^15.8.1", - "protobufjs": "^7.2.5", + "protobufjs": "^7.2.6", "react": "18.2.0", "react-dom": "18.2.0", "react-select": "^5.8.0", "react-spinners": "^0.13.8", - "react-toastify": "^9.1.3", + "react-toastify": "^10.0.3", "styled-components": "^6.1.8", "swr": "^2.2.4", - "usehooks-ts": "^2.9.2", + "usehooks-ts": "^2.9.5", "zod": "^3.22.4" }, "devDependencies": { - "@storybook/addon-essentials": "^7.6.8", - "@storybook/addon-interactions": "^7.6.8", - "@storybook/addon-links": "^7.6.8", + "@storybook/addon-essentials": "^7.6.10", + "@storybook/addon-interactions": "^7.6.10", + "@storybook/addon-links": "^7.6.10", "@storybook/addon-styling": "^1.3.7", "@storybook/blocks": "^7.3.0", - "@storybook/nextjs": "^7.6.8", + "@storybook/nextjs": "^7.6.10", "@storybook/react": "^7.3.0", "@storybook/testing-library": "^0.2.2", - "autoprefixer": "^10.4.16", - "copy-webpack-plugin": "^12.0.1", + "autoprefixer": "^10.4.17", + "copy-webpack-plugin": "^12.0.2", "eslint": "^8.56.0", - "eslint-config-next": "^14.0.4", + "eslint-config-next": "^14.1.0", "eslint-config-prettier": "^9.1.0", "eslint-plugin-storybook": "^0.6.15", "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.33", - "prettier": "^3.2.1", + "prettier": "^3.2.4", "prettier-plugin-organize-imports": "^3.2.4", - "prisma": "^5.8.0", - "storybook": "^7.6.8", + "prisma": "^5.8.1", + "storybook": "^7.6.10", "string-width": "^7.0.0", "tailwindcss": "^3.4.1", "tailwindcss-animate": "^1.0.7", diff --git a/ui/tsconfig.json b/ui/tsconfig.json index c443fefcce..ad4b679af6 100644 --- a/ui/tsconfig.json +++ b/ui/tsconfig.json @@ -16,13 +16,13 @@ "incremental": true, "plugins": [ { - "name": "next" - } + "name": "next", + }, ], "paths": { - "@/*": ["./*"] - } + "@/*": ["./*"], + }, }, "include": ["next-env.d.ts", "**/*.ts", "**/*.tsx", ".next/types/**/*.ts"], - "exclude": ["node_modules"] + "exclude": ["node_modules"], } From baf6d23fa87534d036d5dffadf21d5c9d1270203 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Mon, 22 Jan 2024 18:49:03 +0530 Subject: [PATCH 15/28] Nexus bq: refactor run tracked (#1114) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We close the tracker after getting the result set but if we error while obtaining the result set, the token wouldn't get closed. This PR fixes that. --------- Co-authored-by: Philip Dubé --- nexus/peer-bigquery/src/lib.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/nexus/peer-bigquery/src/lib.rs b/nexus/peer-bigquery/src/lib.rs index e0f9fa99f3..880219661c 100644 --- a/nexus/peer-bigquery/src/lib.rs +++ b/nexus/peer-bigquery/src/lib.rs @@ -82,18 +82,17 @@ impl BigQueryQueryExecutor { .client .job() .query(&self.project_id, query_req) - .await - .map_err(|err| { - tracing::error!("error running query: {}", err); - PgWireError::ApiError(err.into()) - })?; + .await; token.end().await.map_err(|err| { tracing::error!("error closing tracking token: {}", err); PgWireError::ApiError(err.into()) })?; - Ok(result_set) + result_set.map_err(|err| { + tracing::error!("error running query: {}", err); + PgWireError::ApiError(err.into()) + }) } } From 7783fa07c3ab3727204eaa213a05ca731b9b529e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 13:29:17 +0000 Subject: [PATCH 16/28] Make all heartbeats 15 seconds, make all heartbeat timeouts 1 minute (#1068) Recording heartbeat is the time for context to be canceled, effectively making pulse rate the expected p99 cancel latency --- flow/activities/flowable.go | 41 +++++++++++++------ flow/connectors/bigquery/bigquery.go | 1 + flow/connectors/bigquery/qrep_avro_sync.go | 14 +++---- flow/connectors/eventhub/eventhub.go | 7 +--- flow/connectors/postgres/cdc.go | 2 +- .../postgres/qrep_query_executor.go | 3 +- flow/connectors/snowflake/qrep_avro_sync.go | 2 +- flow/connectors/snowflake/snowflake.go | 1 + flow/connectors/utils/avro/avro_writer.go | 3 +- flow/connectors/utils/heartbeat.go | 3 +- flow/workflows/cdc_flow.go | 2 +- flow/workflows/normalize_flow.go | 2 +- flow/workflows/qrep_flow.go | 13 +++--- flow/workflows/setup_flow.go | 2 +- flow/workflows/snapshot_flow.go | 2 +- flow/workflows/sync_flow.go | 2 +- flow/workflows/xmin_flow.go | 2 +- 17 files changed, 56 insertions(+), 46 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index a6224da626..7fdde3ef43 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -218,7 +218,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, go a.recordSlotSizePeriodically(errCtx, srcConn, slotNameForMetrics, input.FlowConnectionConfigs.Source.Name) - shutdown := utils.HeartbeatRoutine(ctx, 10*time.Second, func() string { + shutdown := utils.HeartbeatRoutine(ctx, func() string { jobName := input.FlowConnectionConfigs.FlowJobName return fmt.Sprintf("transferring records for job - %s", jobName) }) @@ -392,7 +392,7 @@ func (a *FlowableActivity) StartNormalize( } defer connectors.CloseConnector(dstConn) - shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { + shutdown := utils.HeartbeatRoutine(ctx, func() string { return fmt.Sprintf("normalizing records from batch for job - %s", input.FlowConnectionConfigs.FlowJobName) }) defer shutdown() @@ -461,7 +461,7 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, } defer connectors.CloseConnector(srcConn) - shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { + shutdown := utils.HeartbeatRoutine(ctx, func() string { return fmt.Sprintf("getting partitions for job - %s", config.FlowJobName) }) defer shutdown() @@ -550,6 +550,10 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, defer connectors.CloseConnector(dstConn) slog.InfoContext(ctx, fmt.Sprintf("replicating partition %s\n", partition.PartitionId)) + shutdown := utils.HeartbeatRoutine(ctx, func() string { + return fmt.Sprintf("syncing partition - %s: %d of %d total.", partition.PartitionId, idx, total) + }) + defer shutdown() var stream *model.QRecordStream bufferSize := shared.FetchAndChannelSize @@ -599,11 +603,6 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, } } - shutdown := utils.HeartbeatRoutine(ctx, 1*time.Minute, func() string { - return fmt.Sprintf("syncing partition - %s: %d of %d total.", partition.PartitionId, idx, total) - }) - defer shutdown() - rowsSynced, err := dstConn.SyncQRepRecords(config, partition, stream) if err != nil { a.Alerter.LogFlowError(ctx, config.FlowJobName, err) @@ -641,10 +640,9 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config } else if err != nil { return err } - defer connectors.CloseConnector(dstConn) - shutdown := utils.HeartbeatRoutine(ctx, 2*time.Minute, func() string { + shutdown := utils.HeartbeatRoutine(ctx, func() string { return fmt.Sprintf("consolidating partitions for job - %s", config.FlowJobName) }) defer shutdown() @@ -808,7 +806,21 @@ func (a *FlowableActivity) QRepWaitUntilNewRows(ctx context.Context, attemptCount := 1 for { activity.RecordHeartbeat(ctx, fmt.Sprintf("no new rows yet, attempt #%d", attemptCount)) - time.Sleep(waitBetweenBatches) + waitUntil := time.Now().Add(waitBetweenBatches) + for { + sleep := time.Until(waitUntil) + if sleep > 15*time.Second { + sleep = 15 * time.Second + } + time.Sleep(sleep) + + activity.RecordHeartbeat(ctx, "heartbeat while waiting before next batch") + if err := ctx.Err(); err != nil { + return fmt.Errorf("cancelled while waiting for new rows: %w", err) + } else if time.Now().After(waitUntil) { + break + } + } result, err := pgSrcConn.CheckForUpdatedMaxValue(config, last) if err != nil { @@ -836,6 +848,11 @@ func (a *FlowableActivity) RenameTables(ctx context.Context, config *protos.Rena } defer connectors.CloseConnector(dstConn) + shutdown := utils.HeartbeatRoutine(ctx, func() string { + return fmt.Sprintf("renaming tables for job - %s", config.FlowJobName) + }) + defer shutdown() + if config.Peer.Type == protos.DBType_SNOWFLAKE { sfConn, ok := dstConn.(*connsnowflake.SnowflakeConnector) if !ok { @@ -955,7 +972,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, return nil }) - shutdown := utils.HeartbeatRoutine(ctx, 5*time.Minute, func() string { + shutdown := utils.HeartbeatRoutine(ctx, func() string { return "syncing xmin." }) defer shutdown() diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 7177e4bb42..3d7e27d863 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -834,6 +834,7 @@ func (c *BigQueryConnector) SetupNormalizedTables( datasetTablesSet[*datasetTable] = struct{}{} // log that table was created c.logger.Info(fmt.Sprintf("created table %s", tableIdentifier)) + utils.RecordHeartbeatWithRecover(c.ctx, fmt.Sprintf("created table %s", tableIdentifier)) } return &protos.SetupNormalizedTableBatchOutput{ diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index ac609002d8..c9da0d6634 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -42,7 +42,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( stream *model.QRecordStream, tableNameRowsMapping map[string]uint32, ) (*model.SyncResponse, error) { - activity.RecordHeartbeat(s.connector.ctx, time.Minute, + activity.RecordHeartbeat(s.connector.ctx, fmt.Sprintf("Flow job %s: Obtaining Avro schema"+ " for destination table %s and sync batch ID %d", req.FlowJobName, rawTableName, syncBatchID), @@ -77,7 +77,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( return nil, fmt.Errorf("failed to update metadata: %v", err) } - activity.RecordHeartbeat(s.connector.ctx, time.Minute, + activity.RecordHeartbeat(s.connector.ctx, fmt.Sprintf("Flow job %s: performing insert and update transaction"+ " for destination table %s and sync batch ID %d", req.FlowJobName, rawTableName, syncBatchID), @@ -392,12 +392,10 @@ func (s *QRepAvroSyncMethod) writeToStage( stream *model.QRecordStream, flowName string, ) (int, error) { - shutdown := utils.HeartbeatRoutine(s.connector.ctx, time.Minute, - func() string { - return fmt.Sprintf("writing to avro stage for objectFolder %s and staging table %s", - objectFolder, stagingTable) - }, - ) + shutdown := utils.HeartbeatRoutine(s.connector.ctx, func() string { + return fmt.Sprintf("writing to avro stage for objectFolder %s and staging table %s", + objectFolder, stagingTable) + }) defer shutdown() var avroFile *avro.AvroFile diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 9250943918..a0d900db75 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -132,11 +132,8 @@ func (c *EventHubConnector) processBatch( lastUpdatedOffset := int64(0) numRecords := atomic.Uint32{} - shutdown := utils.HeartbeatRoutine(c.ctx, 10*time.Second, func() string { - return fmt.Sprintf( - "processed %d records for flow %s", - numRecords.Load(), flowJobName, - ) + shutdown := utils.HeartbeatRoutine(c.ctx, func() string { + return fmt.Sprintf("processed %d records for flow %s", numRecords.Load(), flowJobName) }) defer shutdown() diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 42baaf3199..8de328d0f6 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -244,7 +244,7 @@ func (p *PostgresCDCSource) consumeStream( } }() - shutdown := utils.HeartbeatRoutine(p.ctx, 10*time.Second, func() string { + shutdown := utils.HeartbeatRoutine(p.ctx, func() string { jobName := p.flowJobName currRecords := cdcRecordsStorage.Len() return fmt.Sprintf("pulling records for job - %s, currently have %d records", jobName, currRecords) diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index 44551a124a..506668c9be 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "log/slog" - "time" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/geo" @@ -84,7 +83,7 @@ func (qe *QRepQueryExecutor) executeQueryInTx(tx pgx.Tx, cursorName string, fetc q := fmt.Sprintf("FETCH %d FROM %s", fetchSize, cursorName) if !qe.testEnv { - shutdown := utils.HeartbeatRoutine(qe.ctx, 1*time.Minute, func() string { + shutdown := utils.HeartbeatRoutine(qe.ctx, func() string { qe.logger.Info(fmt.Sprintf("still running '%s'...", q)) return fmt.Sprintf("running '%s'", q) }) diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 146876f8c6..8355ab8bb2 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -270,7 +270,7 @@ func (s *SnowflakeAvroSyncHandler) putFileToStage(avroFile *avro.AvroFile, stage activity.RecordHeartbeat(s.connector.ctx, "putting file to stage") putCmd := fmt.Sprintf("PUT file://%s @%s", avroFile.FilePath, stage) - shutdown := utils.HeartbeatRoutine(s.connector.ctx, 10*time.Second, func() string { + shutdown := utils.HeartbeatRoutine(s.connector.ctx, func() string { return fmt.Sprintf("putting file to stage %s", stage) }) defer shutdown() diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index a1dae65da5..d2fc3a477d 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -418,6 +418,7 @@ func (c *SnowflakeConnector) SetupNormalizedTables( return nil, fmt.Errorf("[sf] error while creating normalized table: %w", err) } tableExistsMapping[tableIdentifier] = false + utils.RecordHeartbeatWithRecover(c.ctx, fmt.Sprintf("created table %s", tableIdentifier)) } return &protos.SetupNormalizedTableBatchOutput{ diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index 743dcb6419..c2f3af94dd 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -8,7 +8,6 @@ import ( "log/slog" "os" "sync/atomic" - "time" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/model" @@ -131,7 +130,7 @@ func (p *peerDBOCFWriter) writeRecordsToOCFWriter(ocfWriter *goavro.OCFWriter) ( numRows := atomic.Uint32{} if p.ctx != nil { - shutdown := utils.HeartbeatRoutine(p.ctx, 30*time.Second, func() string { + shutdown := utils.HeartbeatRoutine(p.ctx, func() string { written := numRows.Load() return fmt.Sprintf("[avro] written %d rows to OCF", written) }) diff --git a/flow/connectors/utils/heartbeat.go b/flow/connectors/utils/heartbeat.go index fda4ea06d2..2ff8e007de 100644 --- a/flow/connectors/utils/heartbeat.go +++ b/flow/connectors/utils/heartbeat.go @@ -11,7 +11,6 @@ import ( func HeartbeatRoutine( ctx context.Context, - interval time.Duration, message func() string, ) func() { shutdown := make(chan struct{}) @@ -26,7 +25,7 @@ func HeartbeatRoutine( return case <-ctx.Done(): return - case <-time.After(interval): + case <-time.After(15 * time.Second): } } }() diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 5f2d29b7e3..1dd453931b 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -343,7 +343,7 @@ func CDCFlowWorkflowWithConfig( state.TableNameSchemaMapping = correctedTableNameSchemaMapping renameTablesCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 12 * time.Hour, - HeartbeatTimeout: 1 * time.Hour, + HeartbeatTimeout: time.Minute, }) renameTablesFuture := workflow.ExecuteActivity(renameTablesCtx, flowable.RenameTables, renameOpts) if err := renameTablesFuture.Get(renameTablesCtx, nil); err != nil { diff --git a/flow/workflows/normalize_flow.go b/flow/workflows/normalize_flow.go index ebf23051f7..720cbba06f 100644 --- a/flow/workflows/normalize_flow.go +++ b/flow/workflows/normalize_flow.go @@ -50,7 +50,7 @@ func (s *NormalizeFlowExecution) executeNormalizeFlow( normalizeFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 7 * 24 * time.Hour, - HeartbeatTimeout: 5 * time.Minute, + HeartbeatTimeout: time.Minute, }) startNormalizeInput := &protos.StartNormalizeInput{ diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 2fadcbbe62..47354a148b 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -152,7 +152,7 @@ func (q *QRepFlowExecution) GetPartitions( ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 5 * time.Hour, - HeartbeatTimeout: 5 * time.Minute, + HeartbeatTimeout: time.Minute, }) partitionsFuture := workflow.ExecuteActivity(ctx, flowable.GetQRepPartitions, q.config, last, q.runUUID) @@ -171,7 +171,7 @@ func (q *QRepPartitionFlowExecution) ReplicatePartitions(ctx workflow.Context, ) error { ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * 5 * time.Hour, - HeartbeatTimeout: 5 * time.Minute, + HeartbeatTimeout: time.Minute, }) msg := fmt.Sprintf("replicating partition batch - %d", partitions.BatchId) @@ -280,7 +280,7 @@ func (q *QRepFlowExecution) consolidatePartitions(ctx workflow.Context) error { // only an operation for Snowflake currently. ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * time.Hour, - HeartbeatTimeout: 10 * time.Minute, + HeartbeatTimeout: time.Minute, }) if err := workflow.ExecuteActivity(ctx, flowable.ConsolidateQRepPartitions, q.config, @@ -303,10 +303,9 @@ func (q *QRepFlowExecution) consolidatePartitions(ctx workflow.Context) error { func (q *QRepFlowExecution) waitForNewRows(ctx workflow.Context, lastPartition *protos.QRepPartition) error { q.logger.Info("idling until new rows are detected") - waitActivityTimeout := time.Duration(q.config.WaitBetweenBatchesSeconds+60) * time.Second ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 16 * 365 * 24 * time.Hour, // 16 years - HeartbeatTimeout: waitActivityTimeout, + HeartbeatTimeout: time.Minute, }) if err := workflow.ExecuteActivity(ctx, flowable.QRepWaitUntilNewRows, q.config, @@ -322,7 +321,7 @@ func (q *QRepFlowExecution) handleTableCreationForResync(ctx workflow.Context, s renamedTableIdentifier := fmt.Sprintf("%s_peerdb_resync", q.config.DestinationTableIdentifier) createTablesFromExistingCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 10 * time.Minute, - HeartbeatTimeout: 2 * time.Minute, + HeartbeatTimeout: time.Minute, }) createTablesFromExistingFuture := workflow.ExecuteActivity( createTablesFromExistingCtx, flowable.CreateTablesFromExisting, &protos.CreateTablesFromExistingInput{ @@ -355,7 +354,7 @@ func (q *QRepFlowExecution) handleTableRenameForResync(ctx workflow.Context, sta renameTablesCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 30 * time.Minute, - HeartbeatTimeout: 5 * time.Minute, + HeartbeatTimeout: time.Minute, }) renameTablesFuture := workflow.ExecuteActivity(renameTablesCtx, flowable.RenameTables, renameOpts) if err := renameTablesFuture.Get(renameTablesCtx, nil); err != nil { diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index d5820a5cb0..9b9b168a89 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -175,7 +175,7 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Hour, - HeartbeatTimeout: 5 * time.Minute, + HeartbeatTimeout: time.Minute, }) sourceTables := maps.Keys(s.tableNameMapping) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 00e8f778e5..8830d0feec 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -288,7 +288,7 @@ func SnapshotFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionCon sessionOpts := &workflow.SessionOptions{ CreationTimeout: 5 * time.Minute, ExecutionTimeout: time.Hour * 24 * 365 * 100, // 100 years - HeartbeatTimeout: time.Hour * 24 * 365 * 100, // 100 years + HeartbeatTimeout: time.Hour, } sessionCtx, err := workflow.CreateSession(ctx, sessionOpts) diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 38a22f1961..d2b0e72136 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -64,7 +64,7 @@ func (s *SyncFlowExecution) executeSyncFlow( startFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 72 * time.Hour, - HeartbeatTimeout: 30 * time.Second, + HeartbeatTimeout: time.Minute, WaitForCancellation: true, }) diff --git a/flow/workflows/xmin_flow.go b/flow/workflows/xmin_flow.go index 387ab2e0a7..32b276ee96 100644 --- a/flow/workflows/xmin_flow.go +++ b/flow/workflows/xmin_flow.go @@ -78,7 +78,7 @@ func XminFlowWorkflow( var lastPartition int64 replicateXminPartitionCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * 5 * time.Hour, - HeartbeatTimeout: 5 * time.Minute, + HeartbeatTimeout: time.Minute, }) err = workflow.ExecuteActivity( replicateXminPartitionCtx, From a7cbc8f28fb42e83b74dcb063671c7be55c5e21b Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Mon, 22 Jan 2024 18:59:43 +0530 Subject: [PATCH 17/28] Validate Peer: Fix check condition (#1121) Return error on invalid PG version instead of a valid one --- flow/cmd/validate_peer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/cmd/validate_peer.go b/flow/cmd/validate_peer.go index 2de6345c14..b3eb16c941 100644 --- a/flow/cmd/validate_peer.go +++ b/flow/cmd/validate_peer.go @@ -47,7 +47,7 @@ func (h *FlowRequestHandler) ValidatePeer( return nil, err } - if isValid { + if !isValid { return &protos.ValidatePeerResponse{ Status: protos.ValidatePeerStatus_INVALID, Message: fmt.Sprintf("%s peer %s must be of version 12 or above. Current version: %d", From 0231c12910c9b4dc72f8d7ccc70328861f582a22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 15:01:28 +0000 Subject: [PATCH 18/28] flow lints: add testifylint & wastedassign (#1122) Also fix %v/%w flagged by errorlint, but don't enable for now since it had more complaints --- flow/.golangci.yml | 12 +++++++----- flow/connectors/postgres/client.go | 2 +- .../postgres/postgres_schema_delta_test.go | 4 ++-- flow/connectors/postgres/qrep_partition_test.go | 2 +- .../utils/cdc_records/cdc_records_storage_test.go | 6 +++--- flow/e2e/s3/qrep_flow_s3_test.go | 4 ++-- flow/e2e/snowflake/peer_flow_sf_test.go | 6 +++--- flow/model/conversion_avro.go | 4 ++-- flow/model/qrecord_test.go | 2 +- flow/model/qvalue/avro_converter.go | 2 +- 10 files changed, 23 insertions(+), 21 deletions(-) diff --git a/flow/.golangci.yml b/flow/.golangci.yml index f8a051b2c6..9897773d62 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -10,21 +10,23 @@ linters: - gofumpt - gosec - gosimple + - ineffassign + - lll - misspell - nakedret - nolintlint + - prealloc - staticcheck - stylecheck - sqlclosecheck + - testifylint + - thelper - unconvert - unparam - - whitespace - - prealloc - - thelper - - ineffassign - unparam - unused - - lll + - wastedassign + - whitespace linters-settings: stylecheck: checks: diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 92bd3a1c40..bf5bb98c46 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -246,7 +246,7 @@ func (c *PostgresConnector) checkSlotAndPublication(slot string, publication str // If slotName input is empty, all slot info rows are returned - this is for UI. // Else, only the row pertaining to that slotName will be returned. func (c *PostgresConnector) GetSlotInfo(slotName string) ([]*protos.SlotInfo, error) { - whereClause := "" + var whereClause string if slotName != "" { whereClause = fmt.Sprintf(" WHERE slot_name = %s", QuoteLiteral(slotName)) } else { diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 8b72b46f20..e912201533 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -250,9 +250,9 @@ func TestPostgresSchemaDeltaTestSuite(t *testing.T) { err = teardownTx.Commit(context.Background()) require.NoError(s.t, err) - require.True(s.t, s.connector.ConnectionActive() == nil) + require.NoError(s.t, s.connector.ConnectionActive()) err = s.connector.Close() require.NoError(s.t, err) - require.False(s.t, s.connector.ConnectionActive() == nil) + require.Error(s.t, s.connector.ConnectionActive()) }) } diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index 73278ce531..b18f9c2574 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -184,7 +184,7 @@ func TestGetQRepPartitions(t *testing.T) { // for now, but ideally we should check that the partition ranges // are correct as well. if tc.expectedNumPartitions != 0 { - assert.Equal(t, tc.expectedNumPartitions, len(got)) + assert.Len(t, got, tc.expectedNumPartitions) return } diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go index 9aea00ae18..1d7afae08c 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go @@ -83,7 +83,7 @@ func TestSingleRecord(t *testing.T) { err := cdcRecordsStore.Set(key, rec) require.NoError(t, err) // should not spill into DB - require.Equal(t, 1, len(cdcRecordsStore.inMemoryRecords)) + require.Len(t, cdcRecordsStore.inMemoryRecords, 1) require.Nil(t, cdcRecordsStore.pebbleDB) reck, ok, err := cdcRecordsStore.Get(key) @@ -100,11 +100,11 @@ func TestRecordsTillSpill(t *testing.T) { cdcRecordsStore.numRecordsSwitchThreshold = 10 // add records upto set limit - for i := 0; i < 10; i++ { + for i := 1; i <= 10; i++ { key, rec := genKeyAndRec(t) err := cdcRecordsStore.Set(key, rec) require.NoError(t, err) - require.Equal(t, i+1, len(cdcRecordsStore.inMemoryRecords)) + require.Len(t, cdcRecordsStore.inMemoryRecords, i) require.Nil(t, cdcRecordsStore.pebbleDB) } diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 46dd16ef4c..c321f268af 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -143,7 +143,7 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { require.NoError(s.t, err) - require.Equal(s.t, 1, len(files)) + require.Len(s.t, files, 1) } func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { @@ -192,5 +192,5 @@ func (s PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { require.NoError(s.t, err) - require.Equal(s.t, 10, len(files)) + require.Len(s.t, files, 10) } diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index bb4d64636a..95340c74f3 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -754,7 +754,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { require.NoError(s.t, err) // Make sure that there are no nulls - require.Equal(s.t, noNulls, true) + require.True(s.t, noNulls) } func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { @@ -1259,9 +1259,9 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { require.NoError(s.t, err) for _, field := range sfRows.Schema.Fields { - require.NotEqual(s.t, field.Name, "c2") + require.NotEqual(s.t, "c2", field.Name) } - require.Equal(s.t, 5, len(sfRows.Schema.Fields)) + require.Len(s.t, sfRows.Schema.Fields, 5) } func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { diff --git a/flow/model/conversion_avro.go b/flow/model/conversion_avro.go index 6399e54b5d..9b3e97fff9 100644 --- a/flow/model/conversion_avro.go +++ b/flow/model/conversion_avro.go @@ -42,7 +42,7 @@ func (qac *QRecordAvroConverter) Convert() (map[string]interface{}, error) { ) avroVal, err := avroConverter.ToAvroValue() if err != nil { - return nil, fmt.Errorf("failed to convert QValue to Avro-compatible value: %v", err) + return nil, fmt.Errorf("failed to convert QValue to Avro-compatible value: %w", err) } m[key] = avroVal @@ -100,7 +100,7 @@ func GetAvroSchemaDefinition( avroSchemaJSON, err := json.Marshal(avroSchema) if err != nil { - return nil, fmt.Errorf("failed to marshal Avro schema to JSON: %v", err) + return nil, fmt.Errorf("failed to marshal Avro schema to JSON: %w", err) } return &QRecordAvroSchemaDefinition{ diff --git a/flow/model/qrecord_test.go b/flow/model/qrecord_test.go index 0cd6022e52..40aead66c9 100644 --- a/flow/model/qrecord_test.go +++ b/flow/model/qrecord_test.go @@ -79,7 +79,7 @@ func TestEquals(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - assert.Equal(t, e2eshared.CheckQRecordEquality(t, tt.q1, tt.q2), tt.want) + assert.Equal(t, tt.want, e2eshared.CheckQRecordEquality(t, tt.q1, tt.q2)) }) } } diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index b682d277f5..3ab06047c8 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -505,7 +505,7 @@ func (c *QValueAvroConverter) processUUID() (interface{}, error) { u, err := uuid.FromBytes(byteData[:]) if err != nil { - return nil, fmt.Errorf("[conversion] conversion of invalid UUID value: %v", err) + return nil, fmt.Errorf("[conversion] conversion of invalid UUID value: %w", err) } uuidString := u.String() From f0763be5ceaedee4a80a91950c38e347f216dfbd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 16:23:44 +0000 Subject: [PATCH 19/28] Replace postgres-openssl with tokio-postgres-rustls (#1105) Removes our openssl dependency --- nexus/Cargo.lock | 195 +++++++++------------------ nexus/postgres-connection/Cargo.toml | 4 +- nexus/postgres-connection/src/lib.rs | 69 ++++++++-- stacks/peerdb-server.Dockerfile | 2 +- 4 files changed, 127 insertions(+), 143 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 76629b157c..3685c4773e 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -79,9 +79,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.7" +version = "0.6.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd2405b3ac1faab2990b74d728624cd9fd115651fcecc7c2d8daf01376275ba" +checksum = "6e2e1ebcb11de5c03c67de28a7df593d32191b44939c482e97702baaaa6ab6a5" dependencies = [ "anstyle", "anstyle-parse", @@ -151,9 +151,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "async-compression" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc2d0cfb2a7388d34f590e76686704c494ed7aaceed62ee1ba35cbf363abc2a5" +checksum = "a116f46a969224200a0a97f29cfd4c50e7534e4b4826bd23ea2c3c533039c82c" dependencies = [ "flate2", "futures-core", @@ -302,9 +302,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.4.1" +version = "2.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" +checksum = "ed570934406eb16438a4e976b1b4500774099c13b8cb96eec99f620f05090ddf" [[package]] name = "bitvec" @@ -402,9 +402,9 @@ checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" [[package]] name = "cargo-deb" -version = "2.0.3" +version = "2.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03ffcdb035e3a2d990cb4c8b37a3f845ba309b9c090dacd66dbc4aec7952a5be" +checksum = "d83090b6ad2812ddd045728406032cb6f6f07e766b66b4748a1becd0a7c07e4e" dependencies = [ "ar", "cargo_toml", @@ -430,9 +430,9 @@ dependencies = [ [[package]] name = "cargo_toml" -version = "0.17.2" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a969e13a7589e9e3e4207e153bae624ade2b5622fb4684a4923b23ec3d57719" +checksum = "802b755090e39835a4b0440fb0bbee0df7495a8b337f63db21e616f7821c7e8c" dependencies = [ "serde", "toml 0.8.8", @@ -513,9 +513,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.4.16" +version = "4.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "58e54881c004cec7895b0068a0a954cd5d62da01aef83fa35b1e594497bf5445" +checksum = "1e578d6ec4194633722ccf9544794b71b1385c3c027efe0c55db226fc880865c" dependencies = [ "clap_builder", "clap_derive", @@ -523,9 +523,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.4.16" +version = "4.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59cb82d7f531603d2fd1f507441cdd35184fa81beff7bd489570de7f773460bb" +checksum = "4df4df40ec50c46000231c914968278b1eb05098cf8f1b3a518a95030e71d1c7" dependencies = [ "anstream", "anstyle", @@ -797,9 +797,9 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.10.1" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95b3f3e67048839cb0d0781f445682a35113da7121f7c949db0e2be96a4fbece" +checksum = "4cd405aab171cb85d6735e5c8d9db038c17d3ca007a4d2c25f337935c3d90580" dependencies = [ "humantime", "is-terminal", @@ -889,21 +889,6 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" -[[package]] -name = "foreign-types" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" -dependencies = [ - "foreign-types-shared", -] - -[[package]] -name = "foreign-types-shared" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" - [[package]] name = "form_urlencoded" version = "1.2.1" @@ -1077,9 +1062,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.23" +version = "0.3.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b553656127a00601c8ae5590fcfdc118e4083a7924b6cf4ffc1ea4b99dc429d7" +checksum = "bb2c4422095b67ee78da96fbb51a4cc413b3b25883c7717ff7ca1ab31022c9c9" dependencies = [ "bytes", "fnv", @@ -1130,9 +1115,9 @@ checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" [[package]] name = "hermit-abi" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" +checksum = "5d3d0e0f38255e7fa3cf31335b3a56f05febd18025f4db5ef7a0cfb4f8da651f" [[package]] name = "hex" @@ -1401,9 +1386,9 @@ checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" [[package]] name = "linux-raw-sys" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4cd1a83af159aa67994778be9070f0ae1bd732942279cabb14f86f986a21456" +checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c" [[package]] name = "lock_api" @@ -1620,50 +1605,12 @@ version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" -[[package]] -name = "openssl" -version = "0.10.62" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cde4d2d9200ad5909f8dac647e29482e07c3a35de8a13fce7c9c7747ad9f671" -dependencies = [ - "bitflags 2.4.1", - "cfg-if", - "foreign-types", - "libc", - "once_cell", - "openssl-macros", - "openssl-sys", -] - -[[package]] -name = "openssl-macros" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.48", -] - [[package]] name = "openssl-probe" version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" -[[package]] -name = "openssl-sys" -version = "0.9.98" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1665caf8ab2dc9aef43d1c0023bd904633a6a05cb30b0ad59bec2ae986e57a7" -dependencies = [ - "cc", - "libc", - "pkg-config", - "vcpkg", -] - [[package]] name = "overload" version = "0.1.1" @@ -2056,9 +2003,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" +checksum = "2900ede94e305130c13ddd391e0ab7cbaeb783945ae07a279c268cb05109c6cb" [[package]] name = "postgres" @@ -2079,11 +2026,11 @@ name = "postgres-connection" version = "0.1.0" dependencies = [ "anyhow", - "openssl", - "postgres-openssl", "pt", + "rustls 0.22.2", "tokio", "tokio-postgres", + "tokio-postgres-rustls", "tracing", "urlencoding", ] @@ -2098,19 +2045,6 @@ dependencies = [ "postgres-types", ] -[[package]] -name = "postgres-openssl" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1de0ea6504e07ca78355a6fb88ad0f36cafe9e696cbc6717f16a207f3a60be72" -dependencies = [ - "futures", - "openssl", - "tokio", - "tokio-openssl", - "tokio-postgres", -] - [[package]] name = "postgres-protocol" version = "0.6.6" @@ -2169,9 +2103,9 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "3.0.0" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b2685dd208a3771337d8d386a89840f0f43cd68be8dae90a5f8c2384effc9cd" +checksum = "6d37c51ca738a55da99dc0c4a34860fd675453b8b36209178c2249bb13651284" dependencies = [ "toml_edit 0.21.0", ] @@ -2201,9 +2135,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.76" +version = "1.0.78" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95fc56cda0b5c3325f5fbbd7ff9fda9e02bb00bb3dac51252d2f1bfa1cb8cc8c" +checksum = "e2422ad645d89c99f8f3e6b88a9fdeca7fabeac836b1002371c4367c8f984aae" dependencies = [ "unicode-ident", ] @@ -2351,9 +2285,9 @@ dependencies = [ [[package]] name = "rayon" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c27db03db7734835b3f53954b534c91069375ce6ccaa2e065441e07d9b6cdb1" +checksum = "fa7237101a77a10773db45d62004a272517633fbcc3df19d96455ede1122e051" dependencies = [ "either", "rayon-core", @@ -2361,9 +2295,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.12.0" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ce3fb6ad83f861aac485e76e1985cd109d9a3713802152be56c3b1f0e0658ed" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" dependencies = [ "crossbeam-deque", "crossbeam-utils", @@ -2425,13 +2359,13 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.2" +version = "1.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" +checksum = "b62dbe01f0b06f9d8dc7d49e05a0785f153b00b2c227856282f671e0318c9b15" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.3", + "regex-automata 0.4.4", "regex-syntax 0.8.2", ] @@ -2446,9 +2380,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" +checksum = "3b7fa1134405e2ec9353fd416b17f8dacd46c473d7d3fd1cf202706a14eb792a" dependencies = [ "aho-corasick", "memchr", @@ -2611,7 +2545,7 @@ version = "0.38.30" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "322394588aaf33c24007e8bb3238ee3e4c5c09c084ab32bc73890b99ff326bca" dependencies = [ - "bitflags 2.4.1", + "bitflags 2.4.2", "errno", "libc", "linux-raw-sys", @@ -2956,9 +2890,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.11.2" +version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" +checksum = "e6ecd384b10a64542d77071bd64bd7b231f4ed5940fba55e98c3de13824cf3d7" [[package]] name = "socket2" @@ -3251,18 +3185,6 @@ dependencies = [ "syn 2.0.48", ] -[[package]] -name = "tokio-openssl" -version = "0.6.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ffab79df67727f6acf57f1ff743091873c24c579b1e2ce4d8f53e47ded4d63d" -dependencies = [ - "futures-util", - "openssl", - "openssl-sys", - "tokio", -] - [[package]] name = "tokio-postgres" version = "0.7.10" @@ -3289,6 +3211,21 @@ dependencies = [ "whoami", ] +[[package]] +name = "tokio-postgres-rustls" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ea13f22eda7127c827983bdaf0d7fff9df21c8817bab02815ac277a21143677" +dependencies = [ + "futures", + "ring", + "rustls 0.22.2", + "tokio", + "tokio-postgres", + "tokio-rustls 0.25.0", + "x509-certificate", +] + [[package]] name = "tokio-rustls" version = "0.24.1" @@ -3572,9 +3509,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" -version = "0.3.14" +version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f2528f27a9eb2b21e69c95319b30bd0efd85d09c379741b0f78ea1d86be2416" +checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" [[package]] name = "unicode-ident" @@ -3647,9 +3584,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e395fcf16a7a3d8127ec99782007af141946b4795001f876d54fb0d55978560" +checksum = "f00cc9702ca12d3c81455259621e676d0f7251cec66a21e98fe2e9a37db93b2a" dependencies = [ "getrandom", "serde", @@ -3679,12 +3616,6 @@ dependencies = [ "uuid", ] -[[package]] -name = "vcpkg" -version = "0.2.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" - [[package]] name = "version_check" version = "0.9.4" @@ -4041,9 +3972,9 @@ dependencies = [ [[package]] name = "xattr" -version = "1.2.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "914566e6413e7fa959cc394fb30e563ba80f3541fbd40816d4c05a0fc3f2a0f1" +checksum = "8da84f1a25939b27f6820d92aed108f83ff920fdf11a7b19366c27c4cda81d4f" dependencies = [ "libc", "linux-raw-sys", diff --git a/nexus/postgres-connection/Cargo.toml b/nexus/postgres-connection/Cargo.toml index a7110f07be..56632c7d78 100644 --- a/nexus/postgres-connection/Cargo.toml +++ b/nexus/postgres-connection/Cargo.toml @@ -8,9 +8,9 @@ edition = "2021" [dependencies] anyhow = "1" pt = { path = "../pt" } -openssl = "0.10.55" -postgres-openssl = "0.5.0" +rustls = "0.22" urlencoding = "2" tokio-postgres = "0.7.2" +tokio-postgres-rustls = "0.11.1" tokio = { version = "1", features = ["full"] } tracing = "0.1" diff --git a/nexus/postgres-connection/src/lib.rs b/nexus/postgres-connection/src/lib.rs index dfa165b7dc..b4a6c82bdd 100644 --- a/nexus/postgres-connection/src/lib.rs +++ b/nexus/postgres-connection/src/lib.rs @@ -1,7 +1,59 @@ -use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; -use postgres_openssl::MakeTlsConnector; use pt::peerdb_peers::PostgresConfig; +use rustls::pki_types::{CertificateDer, ServerName, UnixTime}; +use rustls::{ClientConfig, DigitallySignedStruct, RootCertStore, SignatureScheme}; use std::fmt::Write; +use std::sync::Arc; +use tokio_postgres_rustls::MakeRustlsConnect; + +#[derive(Copy, Clone, Debug)] +struct NoCertificateVerification; + +impl rustls::client::danger::ServerCertVerifier for NoCertificateVerification { + fn verify_server_cert( + &self, + _end_entity: &CertificateDer<'_>, + _intermediates: &[CertificateDer<'_>], + _server_name: &ServerName<'_>, + _ocsp_response: &[u8], + _now: UnixTime, + ) -> std::result::Result { + Ok(rustls::client::danger::ServerCertVerified::assertion()) + } + + fn verify_tls12_signature( + &self, + message: &[u8], + cert: &CertificateDer<'_>, + dss: &DigitallySignedStruct, + ) -> Result { + rustls::crypto::verify_tls12_signature( + message, + cert, + dss, + &rustls::crypto::ring::default_provider().signature_verification_algorithms, + ) + } + + fn verify_tls13_signature( + &self, + message: &[u8], + cert: &CertificateDer<'_>, + dss: &DigitallySignedStruct, + ) -> Result { + rustls::crypto::verify_tls13_signature( + message, + cert, + dss, + &rustls::crypto::ring::default_provider().signature_verification_algorithms, + ) + } + + fn supported_verify_schemes(&self) -> Vec { + rustls::crypto::ring::default_provider() + .signature_verification_algorithms + .supported_schemes() + } +} pub fn get_pg_connection_string(config: &PostgresConfig) -> String { let mut connection_string = String::from("postgres://"); @@ -26,12 +78,13 @@ pub fn get_pg_connection_string(config: &PostgresConfig) -> String { pub async fn connect_postgres(config: &PostgresConfig) -> anyhow::Result { let connection_string = get_pg_connection_string(config); - let mut builder = SslConnector::builder(SslMethod::tls())?; - // NONE seems roughly equivalent to the guarantees of sslmode=prefer or sslmode=require - // PEER seems to be equivalent to sslmode=verify-ca or sslmode=verify-full, requires presence of root certs. - builder.set_verify(SslVerifyMode::NONE); - - let tls_connector = MakeTlsConnector::new(builder.build()); + let mut config = ClientConfig::builder() + .with_root_certificates(RootCertStore::empty()) + .with_no_client_auth(); + config + .dangerous() + .set_certificate_verifier(Arc::new(NoCertificateVerification)); + let tls_connector = MakeRustlsConnect::new(config); let (client, connection) = tokio_postgres::connect(&connection_string, tls_connector) .await .map_err(|e| anyhow::anyhow!("error encountered while connecting to postgres {:?}", e))?; diff --git a/stacks/peerdb-server.Dockerfile b/stacks/peerdb-server.Dockerfile index b83f846de6..789f1e8883 100644 --- a/stacks/peerdb-server.Dockerfile +++ b/stacks/peerdb-server.Dockerfile @@ -12,7 +12,7 @@ FROM chef as builder RUN apt-get update \ && DEBIAN_FRONTEND=noninteractive \ apt-get install --assume-yes --no-install-recommends \ - build-essential libssl-dev pkg-config curl unzip + build-essential pkg-config curl unzip WORKDIR /root/nexus COPY scripts /root/scripts RUN /root/scripts/install-protobuf.sh From 6ef3707bcca20721ba7be522ca2c9e150090fa90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 16:30:29 +0000 Subject: [PATCH 20/28] flow lints: enable gocritic & nonamedreturns (#1124) --- flow/.golangci.yml | 5 +++++ flow/connectors/clickhouse/clickhouse.go | 4 ++-- flow/connectors/clickhouse/qrep.go | 4 ++-- flow/connectors/postgres/client.go | 5 ++--- flow/connectors/postgres/escape.go | 6 +++--- flow/connectors/postgres/ssh_wrapped_pool.go | 13 ++++++++----- flow/connectors/snowflake/snowflake.go | 5 ++--- flow/connectors/utils/partition/partition.go | 12 +++++------- flow/e2e/bigquery/bigquery_helper.go | 10 +++++----- flow/hstore/hstore.go | 4 ++-- flow/shared/signals.go | 7 ++++--- 11 files changed, 40 insertions(+), 35 deletions(-) diff --git a/flow/.golangci.yml b/flow/.golangci.yml index 9897773d62..eee69b73a1 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -7,6 +7,7 @@ linters: - durationcheck - errcheck - forbidigo + - gocritic - gofumpt - gosec - gosimple @@ -15,6 +16,7 @@ linters: - misspell - nakedret - nolintlint + - nonamedreturns - prealloc - staticcheck - stylecheck @@ -28,6 +30,9 @@ linters: - wastedassign - whitespace linters-settings: + gocritic: + disabled-checks: + - ifElseChain stylecheck: checks: - all diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index de51b0feb9..59f58b1379 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -37,8 +37,8 @@ func NewClickhouseConnector(ctx context.Context, } func connect(ctx context.Context, config *protos.ClickhouseConfig) (*sql.DB, error) { - dsn := fmt.Sprintf("tcp://%s:%d?username=%s&password=%s", //&database=%s" - config.Host, config.Port, config.User, config.Password) //, config.Database + dsn := fmt.Sprintf("tcp://%s:%d?username=%s&password=%s", // TODO &database=%s" + config.Host, config.Port, config.User, config.Password) // TODO , config.Database conn, err := sql.Open("clickhouse", dsn) if err != nil { diff --git a/flow/connectors/clickhouse/qrep.go b/flow/connectors/clickhouse/qrep.go index 74ffe26524..77ca30035b 100644 --- a/flow/connectors/clickhouse/qrep.go +++ b/flow/connectors/clickhouse/qrep.go @@ -107,7 +107,7 @@ func (c *ClickhouseConnector) isPartitionSynced(partitionID string) (bool, error } func (c *ClickhouseConnector) SetupQRepMetadataTables(config *protos.QRepConfig) error { - err := c.createQRepMetadataTable() //(createMetadataTablesTx) + err := c.createQRepMetadataTable() if err != nil { return err } @@ -122,7 +122,7 @@ func (c *ClickhouseConnector) SetupQRepMetadataTables(config *protos.QRepConfig) return nil } -func (c *ClickhouseConnector) createQRepMetadataTable() error { // createMetadataTableTx *sql.Tx +func (c *ClickhouseConnector) createQRepMetadataTable() error { // Define the schema schemaStatement := ` CREATE TABLE IF NOT EXISTS %s ( diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index bf5bb98c46..51f5d025f5 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -3,7 +3,6 @@ package connpostgres import ( "errors" "fmt" - "log" "regexp" "strconv" "strings" @@ -557,12 +556,12 @@ func (c *PostgresConnector) getTableNametoUnchangedCols(flowJobName string, sync for rows.Next() { err := rows.Scan(&destinationTableName, &unchangedToastColumns) if err != nil { - log.Fatalf("Failed to scan row: %v", err) + return nil, fmt.Errorf("failed to scan row: %w", err) } resultMap[destinationTableName.String] = unchangedToastColumns } if err := rows.Err(); err != nil { - log.Fatalf("Error iterating over rows: %v", err) + return nil, fmt.Errorf("error iterating over rows: %w", err) } return resultMap, nil } diff --git a/flow/connectors/postgres/escape.go b/flow/connectors/postgres/escape.go index 280d108338..783c0cd7c7 100644 --- a/flow/connectors/postgres/escape.go +++ b/flow/connectors/postgres/escape.go @@ -21,14 +21,14 @@ func QuoteLiteral(literal string) string { // https://git.postgresql.org/gitweb/?p=postgresql.git;a=blob;f=src/interfaces/libpq/fe-exec.c // // substitute any single-quotes (') with two single-quotes ('') - literal = strings.Replace(literal, `'`, `''`, -1) + literal = strings.ReplaceAll(literal, `'`, `''`) // determine if the string has any backslashes (\) in it. // if it does, replace any backslashes (\) with two backslashes (\\) // then, we need to wrap the entire string with a PostgreSQL // C-style escape. Per how "PQEscapeStringInternal" handles this case, we // also add a space before the "E" if strings.Contains(literal, `\`) { - literal = strings.Replace(literal, `\`, `\\`, -1) + literal = strings.ReplaceAll(literal, `\`, `\\`) literal = ` E'` + literal + `'` } else { // otherwise, we can just wrap the literal with a pair of single quotes @@ -53,5 +53,5 @@ func QuoteIdentifier(name string) string { if end > -1 { name = name[:end] } - return `"` + strings.Replace(name, `"`, `""`, -1) + `"` + return `"` + strings.ReplaceAll(name, `"`, `""`) + `"` } diff --git a/flow/connectors/postgres/ssh_wrapped_pool.go b/flow/connectors/postgres/ssh_wrapped_pool.go index a82356a721..9922c41675 100644 --- a/flow/connectors/postgres/ssh_wrapped_pool.go +++ b/flow/connectors/postgres/ssh_wrapped_pool.go @@ -137,18 +137,21 @@ func (swpp *SSHWrappedPostgresPool) Close() { type retryFunc func() error -func retryWithBackoff(fn retryFunc, maxRetries int, backoff time.Duration) (err error) { - for i := 0; i < maxRetries; i++ { - err = fn() +func retryWithBackoff(fn retryFunc, maxRetries int, backoff time.Duration) error { + i := 0 + for { + err := fn() if err == nil { return nil } - if i < maxRetries-1 { + i += 1 + if i < maxRetries { slog.Info(fmt.Sprintf("Attempt #%d failed, retrying in %s", i+1, backoff)) time.Sleep(backoff) + } else { + return err } } - return err } // see: https://github.com/jackc/pgx/issues/382#issuecomment-1496586216 diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index d2fc3a477d..29f41d5995 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -6,7 +6,6 @@ import ( "encoding/json" "errors" "fmt" - "log" "log/slog" "regexp" "strings" @@ -383,12 +382,12 @@ func (c *SnowflakeConnector) getTableNametoUnchangedCols(flowJobName string, syn var r UnchangedToastColumnResult err := rows.Scan(&r.TableName, &r.UnchangedToastColumns) if err != nil { - log.Fatalf("Failed to scan row: %v", err) + return nil, fmt.Errorf("failed to scan row: %w", err) } resultMap[r.TableName] = r.UnchangedToastColumns } if err := rows.Err(); err != nil { - log.Fatalf("Error iterating over rows: %v", err) + return nil, fmt.Errorf("error iterating over rows: %w", err) } return resultMap, nil } diff --git a/flow/connectors/utils/partition/partition.go b/flow/connectors/utils/partition/partition.go index cb2f326a66..37ecd15f37 100644 --- a/flow/connectors/utils/partition/partition.go +++ b/flow/connectors/utils/partition/partition.go @@ -45,14 +45,12 @@ func compareValues(prevEnd interface{}, start interface{}) int { return -1 } else if pe.BlockNumber > v.BlockNumber { return 1 + } else if pe.OffsetNumber < v.OffsetNumber { + return -1 + } else if pe.OffsetNumber > v.OffsetNumber { + return 1 } else { - if pe.OffsetNumber < v.OffsetNumber { - return -1 - } else if pe.OffsetNumber > v.OffsetNumber { - return 1 - } else { - return 0 - } + return 0 } case uint32: // xmin if prevEnd.(uint32) < v { diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 445488266e..7d2c5bce6b 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -382,11 +382,11 @@ func (b *BigQueryTestHelper) ExecuteAndProcessQuery(query string) (*model.QRecor } // returns whether the function errors or there are nulls -func (b *BigQueryTestHelper) CheckNull(tableName string, ColName []string) (bool, error) { - if len(ColName) == 0 { +func (b *BigQueryTestHelper) CheckNull(tableName string, colName []string) (bool, error) { + if len(colName) == 0 { return true, nil } - joinedString := strings.Join(ColName, " is null or ") + " is null" + joinedString := strings.Join(colName, " is null or ") + " is null" command := fmt.Sprintf("SELECT COUNT(*) FROM `%s.%s` WHERE %s", b.Config.DatasetId, tableName, joinedString) q := b.client.Query(command) @@ -419,8 +419,8 @@ func (b *BigQueryTestHelper) CheckNull(tableName string, ColName []string) (bool } // check if NaN, Inf double values are null -func (b *BigQueryTestHelper) CheckDoubleValues(tableName string, ColName []string) (bool, error) { - csep := strings.Join(ColName, ",") +func (b *BigQueryTestHelper) CheckDoubleValues(tableName string, colName []string) (bool, error) { + csep := strings.Join(colName, ",") command := fmt.Sprintf("SELECT %s FROM `%s.%s`", csep, b.Config.DatasetId, tableName) q := b.client.Query(command) diff --git a/flow/hstore/hstore.go b/flow/hstore/hstore.go index 0253fef2b1..cbb7d60c8a 100644 --- a/flow/hstore/hstore.go +++ b/flow/hstore/hstore.go @@ -40,11 +40,11 @@ func (p *hstoreParser) atEnd() bool { } // consume returns the next byte of the string, or end if the string is done. -func (p *hstoreParser) consume() (b byte, end bool) { +func (p *hstoreParser) consume() (byte, bool) { if p.pos >= len(p.str) { return 0, true } - b = p.str[p.pos] + b := p.str[p.pos] p.pos++ return b, false } diff --git a/flow/shared/signals.go b/flow/shared/signals.go index 2097ba95c5..5e441d0110 100644 --- a/flow/shared/signals.go +++ b/flow/shared/signals.go @@ -7,16 +7,17 @@ import ( func FlowSignalHandler(activeSignal CDCFlowSignal, v CDCFlowSignal, logger log.Logger, ) CDCFlowSignal { - if v == ShutdownSignal { + switch v { + case ShutdownSignal: logger.Info("received shutdown signal") return v - } else if v == PauseSignal { + case PauseSignal: logger.Info("received pause signal") if activeSignal == NoopSignal { logger.Info("workflow was running, pausing it") return v } - } else if v == NoopSignal { + case NoopSignal: logger.Info("received resume signal") if activeSignal == PauseSignal { logger.Info("workflow was paused, resuming it") From 785facf3a02622860aff495f1b493e1d665e7931 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Mon, 22 Jan 2024 23:18:34 +0530 Subject: [PATCH 21/28] Validate mirror: account for mixed case (#1123) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Forgot to quote the source table identifiers we use in validate mirror's source table check --------- Co-authored-by: Philip Dubé --- flow/connectors/postgres/client.go | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 51f5d025f5..2527a560c8 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -587,26 +587,31 @@ func (c *PostgresConnector) CheckSourceTables(tableNames []string, pubName strin } // Check that we can select from all tables - for _, tableName := range tableNames { + tableArr := make([]string, 0, len(tableNames)) + for _, table := range tableNames { var row pgx.Row - err := c.pool.QueryRow(c.ctx, fmt.Sprintf("SELECT * FROM %s LIMIT 0;", tableName)).Scan(&row) + schemaName, tableName, found := strings.Cut(table, ".") + if !found { + return fmt.Errorf("invalid source table identifier: %s", table) + } + + tableArr = append(tableArr, fmt.Sprintf(`(%s::text, %s::text)`, QuoteLiteral(schemaName), QuoteLiteral(tableName))) + err := c.pool.QueryRow(c.ctx, + fmt.Sprintf("SELECT * FROM %s.%s LIMIT 0;", QuoteIdentifier(schemaName), QuoteIdentifier(tableName))).Scan(&row) if err != nil && err != pgx.ErrNoRows { return err } } // Check if tables belong to publication - tableArr := make([]string, 0, len(tableNames)) - for _, tableName := range tableNames { - tableArr = append(tableArr, fmt.Sprintf("'%s'", tableName)) - } - tableStr := strings.Join(tableArr, ",") - if pubName != "" { var pubTableCount int - err := c.pool.QueryRow(c.ctx, fmt.Sprintf("select COUNT(DISTINCT(schemaname||'.'||tablename)) from pg_publication_tables "+ - "where schemaname||'.'||tablename in (%s) and pubname=$1;", tableStr), pubName).Scan(&pubTableCount) + err := c.pool.QueryRow(c.ctx, fmt.Sprintf(` + with source_table_components (sname, tname) as (values %s) + select COUNT(DISTINCT(schemaname,tablename)) from pg_publication_tables + INNER JOIN source_table_components stc + ON schemaname=stc.sname and tablename=stc.tname where pubname=$1;`, tableStr), pubName).Scan(&pubTableCount) if err != nil { return err } From 6997d5e8daceb18173678cc4e6a843d408092002 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 18:02:36 +0000 Subject: [PATCH 22/28] flow: replace aws-sdk-go with aws-sdk-go-v2 (#1115) s3manager is removed: https://github.com/aws/aws-sdk-go-v2/issues/1463 We already had an indirect dependency on aws-sdk-go-v2 https://aws.amazon.com/blogs/developer/announcing-end-of-support-for-aws-sdk-for-go-v1-on-july-31-2025 --- flow/connectors/clickhouse/qrep.go | 30 ++++-- flow/connectors/s3/s3.go | 18 ++-- flow/connectors/snowflake/qrep.go | 30 ++++-- flow/connectors/utils/avro/avro_writer.go | 13 +-- flow/connectors/utils/aws.go | 74 +++++++++++--- flow/e2e/s3/qrep_flow_s3_test.go | 2 +- flow/e2e/s3/s3_helper.go | 18 ++-- flow/go.mod | 75 +++++++------- flow/go.sum | 114 +++++++++++----------- 9 files changed, 216 insertions(+), 158 deletions(-) diff --git a/flow/connectors/clickhouse/qrep.go b/flow/connectors/clickhouse/qrep.go index 77ca30035b..57f2f94f26 100644 --- a/flow/connectors/clickhouse/qrep.go +++ b/flow/connectors/clickhouse/qrep.go @@ -11,9 +11,8 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" - "github.com/aws/aws-sdk-go/service/s3/s3manager" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" "google.golang.org/protobuf/encoding/protojson" ) @@ -177,16 +176,27 @@ func (c *ClickhouseConnector) dropStage(stagingPath string, job string) error { } // Create a list of all objects with the defined prefix in the bucket - iter := s3manager.NewDeleteListIterator(s3svc, &s3.ListObjectsInput{ + pages := s3.NewListObjectsV2Paginator(s3svc, &s3.ListObjectsV2Input{ Bucket: aws.String(s3o.Bucket), Prefix: aws.String(fmt.Sprintf("%s/%s", s3o.Prefix, job)), }) - - // Iterate through the objects in the bucket with the prefix and delete them - s3Client := s3manager.NewBatchDeleteWithClient(s3svc) - if err := s3Client.Delete(aws.BackgroundContext(), iter); err != nil { - c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) - return fmt.Errorf("failed to delete objects from bucket: %w", err) + for pages.HasMorePages() { + page, err := pages.NextPage(c.ctx) + if err != nil { + c.logger.Error("failed to list objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to list objects from bucket: %w", err) + } + + for _, object := range page.Contents { + _, err = s3svc.DeleteObject(c.ctx, &s3.DeleteObjectInput{ + Bucket: aws.String(s3o.Bucket), + Key: object.Key, + }) + if err != nil { + c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to delete objects from bucket: %w", err) + } + } } c.logger.Info(fmt.Sprintf("Deleted contents of bucket %s with prefix %s/%s", s3o.Bucket, s3o.Prefix, job)) diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 7933936bd7..0ddb8abf5c 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -12,8 +12,8 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" ) const ( @@ -24,7 +24,7 @@ type S3Connector struct { ctx context.Context url string pgMetadata *metadataStore.PostgresMetadataStore - client s3.S3 + client s3.Client creds utils.S3PeerCredentials logger slog.Logger } @@ -91,8 +91,8 @@ func (c *S3Connector) Close() error { return c.pgMetadata.Close() } -func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.PostgresMetadataStore) error { - _, listErr := s3Client.ListBuckets(nil) +func ValidCheck(ctx context.Context, s3Client *s3.Client, bucketURL string, metadataDB *metadataStore.PostgresMetadataStore) error { + _, listErr := s3Client.ListBuckets(ctx, nil) if listErr != nil { return fmt.Errorf("failed to list buckets: %w", listErr) } @@ -107,7 +107,7 @@ func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.Pos // Write an empty file and then delete it // to check if we have write permissions bucketName := aws.String(bucketPrefix.Bucket) - _, putErr := s3Client.PutObject(&s3.PutObjectInput{ + _, putErr := s3Client.PutObject(ctx, &s3.PutObjectInput{ Bucket: bucketName, Key: aws.String(_peerDBCheck), Body: reader, @@ -116,7 +116,7 @@ func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.Pos return fmt.Errorf("failed to write to bucket: %w", putErr) } - _, delErr := s3Client.DeleteObject(&s3.DeleteObjectInput{ + _, delErr := s3Client.DeleteObject(ctx, &s3.DeleteObjectInput{ Bucket: bucketName, Key: aws.String(_peerDBCheck), }) @@ -134,12 +134,12 @@ func ValidCheck(s3Client *s3.S3, bucketURL string, metadataDB *metadataStore.Pos } func (c *S3Connector) ConnectionActive() error { - _, listErr := c.client.ListBuckets(nil) + _, listErr := c.client.ListBuckets(c.ctx, nil) if listErr != nil { return listErr } - validErr := ValidCheck(&c.client, c.url, c.pgMetadata) + validErr := ValidCheck(c.ctx, &c.client, c.url, c.pgMetadata) if validErr != nil { c.logger.Error("failed to validate s3 connector:", slog.Any("error", validErr)) return validErr diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 264cd3ea0e..215fdeab41 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -11,9 +11,8 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" - "github.com/aws/aws-sdk-go/service/s3/s3manager" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/jackc/pgx/v5/pgtype" "google.golang.org/protobuf/encoding/protojson" ) @@ -338,16 +337,27 @@ func (c *SnowflakeConnector) dropStage(stagingPath string, job string) error { } // Create a list of all objects with the defined prefix in the bucket - iter := s3manager.NewDeleteListIterator(s3svc, &s3.ListObjectsInput{ + pages := s3.NewListObjectsV2Paginator(s3svc, &s3.ListObjectsV2Input{ Bucket: aws.String(s3o.Bucket), Prefix: aws.String(fmt.Sprintf("%s/%s", s3o.Prefix, job)), }) - - // Iterate through the objects in the bucket with the prefix and delete them - s3Client := s3manager.NewBatchDeleteWithClient(s3svc) - if err := s3Client.Delete(aws.BackgroundContext(), iter); err != nil { - c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) - return fmt.Errorf("failed to delete objects from bucket: %w", err) + for pages.HasMorePages() { + page, err := pages.NextPage(c.ctx) + if err != nil { + c.logger.Error("failed to list objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to list objects from bucket: %w", err) + } + + for _, object := range page.Contents { + _, err = s3svc.DeleteObject(c.ctx, &s3.DeleteObjectInput{ + Bucket: aws.String(s3o.Bucket), + Key: object.Key, + }) + if err != nil { + c.logger.Error("failed to delete objects from bucket", slog.Any("error", err)) + return fmt.Errorf("failed to delete objects from bucket: %w", err) + } + } } c.logger.Info(fmt.Sprintf("Deleted contents of bucket %s with prefix %s/%s", s3o.Bucket, s3o.Prefix, job)) diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index c2f3af94dd..e96a642dae 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -13,8 +13,9 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3/s3manager" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/feature/s3/manager" + "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/klauspost/compress/flate" "github.com/klauspost/compress/snappy" "github.com/klauspost/compress/zstd" @@ -201,11 +202,7 @@ func (p *peerDBOCFWriter) WriteRecordsToS3(bucketName, key string, s3Creds utils return nil, fmt.Errorf("failed to create S3 client: %w", err) } - // Create an uploader with the session and default options - uploader := s3manager.NewUploaderWithClient(s3svc) - - // Upload the file to S3. - result, err := uploader.Upload(&s3manager.UploadInput{ + _, err = manager.NewUploader(s3svc).Upload(p.ctx, &s3.PutObjectInput{ Bucket: aws.String(bucketName), Key: aws.String(key), Body: r, @@ -216,7 +213,7 @@ func (p *peerDBOCFWriter) WriteRecordsToS3(bucketName, key string, s3Creds utils return nil, fmt.Errorf("failed to upload file to path %s: %w", s3Path, err) } - slog.Info("file uploaded to" + result.Location) + slog.Info("file uploaded to " + fmt.Sprintf("%s/%s", bucketName, key)) return &AvroFile{ NumRecords: <-numRowsWritten, diff --git a/flow/connectors/utils/aws.go b/flow/connectors/utils/aws.go index 473efd0ce5..6e6cbed022 100644 --- a/flow/connectors/utils/aws.go +++ b/flow/connectors/utils/aws.go @@ -2,13 +2,15 @@ package utils import ( "fmt" + "net/http" "os" "strings" + "time" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/aws/credentials" - "github.com/aws/aws-sdk-go/aws/session" - "github.com/aws/aws-sdk-go/service/s3" + "github.com/aws/aws-sdk-go-v2/aws" + v4 "github.com/aws/aws-sdk-go-v2/aws/signer/v4" + "github.com/aws/aws-sdk-go-v2/credentials" + "github.com/aws/aws-sdk-go-v2/service/s3" ) type AWSSecrets struct { @@ -89,23 +91,65 @@ func NewS3BucketAndPrefix(s3Path string) (*S3BucketAndPrefix, error) { }, nil } -func CreateS3Client(s3Creds S3PeerCredentials) (*s3.S3, error) { +func CreateS3Client(s3Creds S3PeerCredentials) (*s3.Client, error) { awsSecrets, err := GetAWSSecrets(s3Creds) if err != nil { return nil, fmt.Errorf("failed to get AWS secrets: %w", err) } - - config := &aws.Config{ - Region: aws.String(awsSecrets.Region), - Endpoint: aws.String(awsSecrets.Endpoint), + options := s3.Options{ + Region: awsSecrets.Region, + Credentials: credentials.NewStaticCredentialsProvider(awsSecrets.AccessKeyID, awsSecrets.SecretAccessKey, ""), } - - if s3Creds.AccessKeyID != "" && s3Creds.SecretAccessKey != "" { - config.Credentials = credentials.NewStaticCredentials(s3Creds.AccessKeyID, s3Creds.SecretAccessKey, "") + if awsSecrets.Endpoint != "" { + options.BaseEndpoint = &awsSecrets.Endpoint + if strings.Contains(awsSecrets.Endpoint, "storage.googleapis.com") { + // Assign custom client with our own transport + options.HTTPClient = &http.Client{ + Transport: &RecalculateV4Signature{ + next: http.DefaultTransport, + signer: v4.NewSigner(), + credentials: options.Credentials, + region: options.Region, + }, + } + } } - sess := session.Must(session.NewSession(config)) + return s3.New(options), nil +} + +// RecalculateV4Signature allow GCS over S3, removing Accept-Encoding header from sign +// https://stackoverflow.com/a/74382598/1204665 +// https://github.com/aws/aws-sdk-go-v2/issues/1816 +type RecalculateV4Signature struct { + next http.RoundTripper + signer *v4.Signer + credentials aws.CredentialsProvider + region string +} + +func (lt *RecalculateV4Signature) RoundTrip(req *http.Request) (*http.Response, error) { + // store for later use + acceptEncodingValue := req.Header.Get("Accept-Encoding") + + // delete the header so the header doesn't account for in the signature + req.Header.Del("Accept-Encoding") + + // sign with the same date + timeString := req.Header.Get("X-Amz-Date") + timeDate, _ := time.Parse("20060102T150405Z", timeString) + + creds, err := lt.credentials.Retrieve(req.Context()) + if err != nil { + return nil, err + } + err = lt.signer.SignHTTP(req.Context(), creds, req, v4.GetPayloadHash(req.Context()), "s3", lt.region, timeDate) + if err != nil { + return nil, err + } + // Reset Accept-Encoding if desired + req.Header.Set("Accept-Encoding", acceptEncodingValue) - s3svc := s3.New(sess) - return s3svc, nil + // follows up the original round tripper + return lt.next.RoundTrip(req) } diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index c321f268af..5a2f266ab7 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -39,7 +39,7 @@ func (s PeerFlowE2ETestSuiteS3) Suffix() string { func tearDownSuite(s PeerFlowE2ETestSuiteS3) { e2e.TearDownPostgres(s) - err := s.s3Helper.CleanUp() + err := s.s3Helper.CleanUp(context.Background()) if err != nil { require.Fail(s.t, "failed to clean up s3", err) } diff --git a/flow/e2e/s3/s3_helper.go b/flow/e2e/s3/s3_helper.go index ef9020e278..ecca066ea5 100644 --- a/flow/e2e/s3/s3_helper.go +++ b/flow/e2e/s3/s3_helper.go @@ -12,8 +12,8 @@ import ( "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go/aws" - "github.com/aws/aws-sdk-go/service/s3" + "github.com/aws/aws-sdk-go-v2/service/s3" + s3types "github.com/aws/aws-sdk-go-v2/service/s3/types" ) const ( @@ -21,7 +21,7 @@ const ( ) type S3TestHelper struct { - client *s3.S3 + client *s3.Client s3Config *protos.S3Config bucketName string prefix string @@ -90,10 +90,10 @@ func (h *S3TestHelper) GetPeer() *protos.Peer { func (h *S3TestHelper) ListAllFiles( ctx context.Context, jobName string, -) ([]*s3.Object, error) { +) ([]s3types.Object, error) { Bucket := h.bucketName Prefix := fmt.Sprintf("%s/%s/", h.prefix, jobName) - files, err := h.client.ListObjects(&s3.ListObjectsInput{ + files, err := h.client.ListObjects(ctx, &s3.ListObjectsInput{ Bucket: &Bucket, Prefix: &Prefix, }) @@ -106,10 +106,10 @@ func (h *S3TestHelper) ListAllFiles( } // Delete all generated objects during the test -func (h *S3TestHelper) CleanUp() error { +func (h *S3TestHelper) CleanUp(ctx context.Context) error { Bucket := h.bucketName Prefix := h.prefix - files, err := h.client.ListObjects(&s3.ListObjectsInput{ + files, err := h.client.ListObjects(ctx, &s3.ListObjectsInput{ Bucket: &Bucket, Prefix: &Prefix, }) @@ -121,11 +121,11 @@ func (h *S3TestHelper) CleanUp() error { // Delete each object for _, obj := range files.Contents { deleteInput := &s3.DeleteObjectInput{ - Bucket: aws.String(Bucket), + Bucket: &Bucket, Key: obj.Key, } - _, err := h.client.DeleteObject(deleteInput) + _, err := h.client.DeleteObject(ctx, deleteInput) if err != nil { return err } diff --git a/flow/go.mod b/flow/go.mod index 5239fe48b1..8548882df5 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -4,13 +4,16 @@ go 1.21 require ( cloud.google.com/go v0.112.0 - cloud.google.com/go/bigquery v1.57.1 + cloud.google.com/go/bigquery v1.58.0 cloud.google.com/go/storage v1.36.0 - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 - github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2 + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 + github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.3 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 github.com/ClickHouse/clickhouse-go/v2 v2.17.1 - github.com/aws/aws-sdk-go v1.49.20 + github.com/aws/aws-sdk-go-v2 v1.24.1 + github.com/aws/aws-sdk-go-v2/credentials v1.16.16 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.13 + github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cockroachdb/pebble v0.0.0-20231210175914-b4d301aeb46a github.com/google/uuid v1.5.0 @@ -18,7 +21,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.0 github.com/jackc/pgerrcode v0.0.0-20220416144525-469b46aa5efa github.com/jackc/pglogrepl v0.0.0-20231111135425-1627ab1b5780 - github.com/jackc/pgx/v5 v5.5.1 + github.com/jackc/pgx/v5 v5.5.2 github.com/jmoiron/sqlx v1.3.5 github.com/joho/godotenv v1.5.1 github.com/klauspost/compress v1.17.4 @@ -27,77 +30,82 @@ require ( github.com/microsoft/go-mssqldb v1.6.0 github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/slack-go/slack v0.12.3 - github.com/snowflakedb/gosnowflake v1.7.1 + github.com/snowflakedb/gosnowflake v1.7.2 github.com/stretchr/testify v1.8.4 - github.com/twpayne/go-geos v0.15.0 + github.com/twpayne/go-geos v0.16.1 github.com/urfave/cli/v3 v3.0.0-alpha8 github.com/youmark/pkcs8 v0.0.0-20201027041543-1326539a0a0a go.temporal.io/api v1.26.0 go.temporal.io/sdk v1.25.1 go.uber.org/automaxprocs v1.5.3 + golang.org/x/crypto v0.18.0 golang.org/x/sync v0.6.0 - google.golang.org/api v0.156.0 - google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 + google.golang.org/api v0.157.0 + google.golang.org/genproto/googleapis/api v0.0.0-20240116215550-a9fa1716bcac google.golang.org/grpc v1.60.1 google.golang.org/protobuf v1.32.0 ) require ( - github.com/ClickHouse/ch-go v0.58.2 // indirect + github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect + github.com/99designs/keyring v1.2.2 // indirect + github.com/ClickHouse/ch-go v0.61.1 // indirect github.com/DataDog/zstd v1.5.5 // indirect + github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect + github.com/apache/arrow/go/v14 v14.0.2 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cockroachdb/errors v1.11.1 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect github.com/cockroachdb/redact v1.1.5 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect + github.com/danieljoos/wincred v1.2.1 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect + github.com/gabriel-vasile/mimetype v1.4.3 // indirect github.com/getsentry/sentry-go v0.26.0 // indirect github.com/go-faster/city v1.0.1 // indirect - github.com/go-faster/errors v0.6.1 // indirect + github.com/go-faster/errors v0.7.1 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect + github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/gorilla/websocket v1.5.1 // indirect + github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect - github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect - github.com/paulmach/orb v0.10.0 // indirect + github.com/mtibben/percent v0.2.1 // indirect + github.com/paulmach/orb v0.11.0 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/prometheus/client_golang v1.18.0 // indirect github.com/prometheus/client_model v0.5.0 // indirect - github.com/prometheus/common v0.45.0 // indirect + github.com/prometheus/common v0.46.0 // indirect github.com/prometheus/procfs v0.12.0 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/segmentio/asm v1.2.0 // indirect github.com/shopspring/decimal v1.3.1 // indirect github.com/sirupsen/logrus v1.9.3 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 // indirect - go.opentelemetry.io/otel v1.21.0 // indirect - go.opentelemetry.io/otel/metric v1.21.0 // indirect - go.opentelemetry.io/otel/trace v1.21.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.47.0 // indirect + go.opentelemetry.io/otel v1.22.0 // indirect + go.opentelemetry.io/otel/metric v1.22.0 // indirect + go.opentelemetry.io/otel/trace v1.22.0 // indirect go.uber.org/atomic v1.11.0 // indirect + golang.org/x/term v0.16.0 // indirect ) require ( cloud.google.com/go/compute v1.23.3 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.5 // indirect - github.com/99designs/go-keychain v0.0.0-20191008050251-8e49817e8af4 // indirect - github.com/99designs/keyring v1.2.2 // indirect github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.1 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.1 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.2.1 // indirect - github.com/Azure/go-amqp v1.0.3 // indirect + github.com/Azure/go-amqp v1.0.4 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 // indirect - github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c // indirect github.com/andybalholm/brotli v1.1.0 // indirect github.com/apache/arrow/go/v12 v12.0.1 // indirect github.com/apache/thrift v0.19.0 // indirect - github.com/aws/aws-sdk-go-v2 v1.24.1 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.16.14 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.11 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.10 // indirect @@ -105,16 +113,11 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.10 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.10 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0 // indirect github.com/aws/smithy-go v1.19.0 // indirect - github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect - github.com/gabriel-vasile/mimetype v1.4.3 // indirect github.com/goccy/go-json v0.10.2 // indirect - github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 // indirect github.com/gogo/googleapis v1.4.1 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/gogo/status v1.1.1 // indirect @@ -132,7 +135,6 @@ require ( github.com/grafana/pyroscope-go/godeltaprof v0.1.6 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect - github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect github.com/jackc/pgio v1.0.0 // indirect github.com/jackc/pgpassfile v1.0.0 // indirect github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9 // indirect @@ -143,7 +145,6 @@ require ( github.com/kylelemons/godebug v1.1.0 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect - github.com/mtibben/percent v0.2.1 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pierrec/lz4/v4 v4.1.21 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect @@ -153,19 +154,17 @@ require ( github.com/xrash/smetrics v0.0.0-20231213231151-1d8dd44e695e // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.18.0 - golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3 + golang.org/x/exp v0.0.0-20240119083558-1b970713d09a golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.20.0 // indirect golang.org/x/oauth2 v0.16.0 // indirect golang.org/x/sys v0.16.0 // indirect - golang.org/x/term v0.16.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.17.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto v0.0.0-20240108191215-35c7eff3a6b1 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240108191215-35c7eff3a6b1 // indirect + google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240116215550-a9fa1716bcac // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/flow/go.sum b/flow/go.sum index 7e8d745158..911bb2e9b2 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -2,8 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.0 h1:tpFCD7hpHFlQ8yPwT3x+QeXqc2T6+n6T+hmABHfDUSM= cloud.google.com/go v0.112.0/go.mod h1:3jEEVwZ/MHU4djK5t5RHuKOA/GbLddgTdVubX1qnPD4= -cloud.google.com/go/bigquery v1.57.1 h1:FiULdbbzUxWD0Y4ZGPSVCDLvqRSyCIO6zKV7E2nf5uA= -cloud.google.com/go/bigquery v1.57.1/go.mod h1:iYzC0tGVWt1jqSzBHqCr3lrRn0u13E8e+AqowBsDgug= +cloud.google.com/go/bigquery v1.58.0 h1:drSd9RcPVLJP2iFMimvOB9SCSIrcl+9HD4II03Oy7A0= +cloud.google.com/go/bigquery v1.58.0/go.mod h1:0eh4mWNY0KrBTjUzLjoYImapGORq9gEPT7MWjCy9lik= cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= @@ -22,12 +22,12 @@ 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/Azure/azure-sdk-for-go/sdk/azcore v1.9.1 h1:lGlwhPtrX6EVml1hO0ivjkUxsSyl4dsiw9qcA1k/3IQ= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.9.1/go.mod h1:RKUqNu35KJYcVG/fqTRqmuXJZYNhYkBrnC/hX7yGbTA= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 h1:BMAjVKJM0U/CYF27gA0ZMmXGkOcvfFtD0oHVZ1TIPRI= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0/go.mod h1:1fXstnBMas5kzG+S3q8UoJcmyU6nUeunJcMDHcRYHhs= +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/internal v1.5.1 h1:6oNBlSdi1QqM1PNW7FPA6xOGA5UNsXnkaYZz9vdPGhA= github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.1/go.mod h1:s4kgfzA0covAXNicZHDMN58jExvcng2mC/DepXiF1EI= -github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2 h1:ujuMdFIUqhfohvpjjt7YmWn6Wk5Vlw9cwtGC0/BEwLU= -github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2/go.mod h1:P39PnDHXbDhUV+BVw/8Nb7wQnM76jKUA7qx5T7eS+BU= +github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.3 h1:zkAs5JZZm1Yr4lxLUj3xt2FLgKmvcwGt3a94iJ8rgew= +github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.3/go.mod h1:P39PnDHXbDhUV+BVw/8Nb7wQnM76jKUA7qx5T7eS+BU= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 h1:+dggnR89/BIIlRlQ6d19dkhhdd/mQUiQbXhyHUFiB4w= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0/go.mod h1:tI9M2Q/ueFi287QRkdrhb9LHm6ZnXgkVYLRC3FhYkPw= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/internal/v2 v2.0.0 h1:PTFGRSlMKCQelWwxUyYVEUqseBJVemLyqWJjvMyt0do= @@ -42,13 +42,13 @@ github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v0.8.0 h1:T028g github.com/Azure/azure-sdk-for-go/sdk/security/keyvault/internal v0.8.0/go.mod h1:cw4zVQgBby0Z5f2v0itn6se2dDP17nTjbZFXW5uPyHA= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.2.1 h1:AMf7YbZOZIW5b66cXNHMWWT/zkjhz5+a+k/3x40EO7E= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.2.1/go.mod h1:uwfk06ZBcvL/g4VHNjurPfVln9NMbsk2XIZxJ+hu81k= -github.com/Azure/go-amqp v1.0.3 h1:wEXQyG2tromTkSdTTOhpYSmosOlJjtVX5UVk2pxyC0Y= -github.com/Azure/go-amqp v1.0.3/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= +github.com/Azure/go-amqp v1.0.4 h1:GX5OFOs706UjuFRD5PDKm3aOuLQ92F7DMbua+DKAYCc= +github.com/Azure/go-amqp v1.0.4/go.mod h1:vZAogwdrkbyK3Mla8m/CxSc/aKdnTZ4IbPxl51Y5WZE= 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/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/ClickHouse/ch-go v0.58.2 h1:jSm2szHbT9MCAB1rJ3WuCJqmGLi5UTjlNu+f530UTS0= -github.com/ClickHouse/ch-go v0.58.2/go.mod h1:Ap/0bEmiLa14gYjCiRkYGbXvbe8vwdrfTYWhsuQ99aw= +github.com/ClickHouse/ch-go v0.61.1 h1:j5rx3qnvcnYjhnP1IdXE/vdIRQiqgwAzyqOaasA6QCw= +github.com/ClickHouse/ch-go v0.61.1/go.mod h1:myxt/JZgy2BYHFGQqzmaIpbfr5CMbs3YHVULaWQj5YU= github.com/ClickHouse/clickhouse-go/v2 v2.17.1 h1:ZCmAYWpu75IyEi7+Yrs/uaAjiCGY5wfW5kXo64exkX4= github.com/ClickHouse/clickhouse-go/v2 v2.17.1/go.mod h1:rkGTvFDTLqLIm0ma+13xmcCfr/08Gvs7KmFt1tgiWHQ= github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= @@ -64,22 +64,22 @@ github.com/andybalholm/brotli v1.1.0/go.mod h1:sms7XGricyQI9K10gOSf56VKKWS4oLer5 github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/arrow/go/v12 v12.0.1 h1:JsR2+hzYYjgSUkBSaahpqCetqZMr76djX80fF/DiJbg= github.com/apache/arrow/go/v12 v12.0.1/go.mod h1:weuTY7JvTG/HDPtMQxEUp7pU73vkLWMLpY67QwZ/WWw= +github.com/apache/arrow/go/v14 v14.0.2 h1:N8OkaJEOfI3mEZt07BIkvo4sC6XDbL+48MBPWO5IONw= +github.com/apache/arrow/go/v14 v14.0.2/go.mod h1:u3fgh3EdgN/YQ8cVQRguVW3R+seMybFg8QBQ5LU+eBY= github.com/apache/thrift v0.19.0 h1:sOqkWPzMj7w6XaYbJQG7m4sGqVolaW/0D28Ln7yPzMk= github.com/apache/thrift v0.19.0/go.mod h1:SUALL216IiaOw2Oy+5Vs9lboJ/t9g40C+G07Dc0QC1I= -github.com/aws/aws-sdk-go v1.49.20 h1:VgEUq2/ZbUkLbqPyDcxrirfXB+PgiZUUF5XbsgWe2S0= -github.com/aws/aws-sdk-go v1.49.20/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/aws/aws-sdk-go-v2 v1.24.1 h1:xAojnj+ktS95YZlDf0zxWBkbFtymPeDP+rvUQIH3uAU= github.com/aws/aws-sdk-go-v2 v1.24.1/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 h1:OCs21ST2LrepDfD3lwlQiOqIGp6JiEUqG84GzTDoyJs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4/go.mod h1:usURWEKSNNAcAZuzRn/9ZYPT8aZQkR7xcCtunK/LkJo= -github.com/aws/aws-sdk-go-v2/config v1.26.3 h1:dKuc2jdp10y13dEEvPqWxqLoc0vF3Z9FC45MvuQSxOA= -github.com/aws/aws-sdk-go-v2/config v1.26.3/go.mod h1:Bxgi+DeeswYofcYO0XyGClwlrq3DZEXli0kLf4hkGA0= -github.com/aws/aws-sdk-go-v2/credentials v1.16.14 h1:mMDTwwYO9A0/JbOCOG7EOZHtYM+o7OfGWfu0toa23VE= -github.com/aws/aws-sdk-go-v2/credentials v1.16.14/go.mod h1:cniAUh3ErQPHtCQGPT5ouvSAQ0od8caTO9OOuufZOAE= +github.com/aws/aws-sdk-go-v2/config v1.26.5 h1:lodGSevz7d+kkFJodfauThRxK9mdJbyutUxGq1NNhvw= +github.com/aws/aws-sdk-go-v2/config v1.26.5/go.mod h1:DxHrz6diQJOc9EwDslVRh84VjjrE17g+pVZXUeSxaDU= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16 h1:8q6Rliyv0aUFAVtzaldUEcS+T5gbadPbWdV1WcAddK8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16/go.mod h1:UHVZrdUsv63hPXFo1H7c5fEneoVo9UXiz36QG1GEPi0= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11 h1:c5I5iH+DZcH3xOIMlz3/tCKJDaHFwYEmxvlh2fAcFo8= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11/go.mod h1:cRrYDYAMUohBJUtUnOhydaMHtiK/1NZ0Otc9lIb6O0Y= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.11 h1:I6lAa3wBWfCz/cKkOpAcumsETRkFAl70sWi8ItcMEsM= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.11/go.mod h1:be1NIO30kJA23ORBLqPo1LttEM6tPNSEcjkd1eKzNW0= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.13 h1:8Nt4LBUEKV0FxLBO2BmRzDKax3hp2LRMKySMBwL4vMc= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.13/go.mod h1:t5QEDu/FBJJM4kslbQlTSpYtnhoWDNmHSsgQojIxE0o= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 h1:vF+Zgd9s+H4vOXd5BMaPWykta2a6Ih0AKLq/X6NYKn4= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10/go.mod h1:6BkRjejp/GR4411UGqkX8+wFMbFbqsUIimfK4XjOKR4= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 h1:nYPe006ktcqUji8S2mqXf9c/7NdiKriOwMvWQHgYztw= @@ -98,10 +98,10 @@ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.10 h1:KOxnQeWy5sXyS github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.10/go.mod h1:jMx5INQFYFYB3lQD9W0D8Ohgq6Wnl7NYOJ2TQndbulI= github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0 h1:PJTdBMsyvra6FtED7JZtDpQrIAflYDHFoZAu/sKYkwU= github.com/aws/aws-sdk-go-v2/service/s3 v1.48.0/go.mod h1:4qXHrG1Ne3VGIMZPCB8OjH/pLFO94sKABIusjh0KWPU= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.6 h1:dGrs+Q/WzhsiUKh82SfTVN66QzyulXuMDTV/G8ZxOac= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.6/go.mod h1:+mJNDdF+qiUlNKNC3fxn74WWNN+sOiGOEImje+3ScPM= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.6 h1:Yf2MIo9x+0tyv76GljxzqA3WtC5mw7NmazD2chwjxE4= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.6/go.mod h1:ykf3COxYI0UJmxcfcxcVuz7b6uADi1FkiUz6Eb7AgM8= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7 h1:eajuO3nykDPdYicLlP3AGgOyVN3MOlFmZv7WGTuJPow= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7/go.mod h1:+mJNDdF+qiUlNKNC3fxn74WWNN+sOiGOEImje+3ScPM= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7 h1:QPMJf+Jw8E1l7zqhZmMlFw6w1NmfkfiSK8mS4zOx3BA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7/go.mod h1:ykf3COxYI0UJmxcfcxcVuz7b6uADi1FkiUz6Eb7AgM8= github.com/aws/aws-sdk-go-v2/service/sts v1.26.7 h1:NzO4Vrau795RkUdSHKEwiR01FaGzGOH1EETJ+5QHnm0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.7/go.mod h1:6h2YuIoxaMSCFf5fi1EgZAwdfkGMgDY+DVfa61uLe4U= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= @@ -165,8 +165,8 @@ github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxI github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-faster/city v1.0.1 h1:4WAxSZ3V2Ws4QRDrscLEDcibJY8uf41H6AhXDrNDcGw= github.com/go-faster/city v1.0.1/go.mod h1:jKcUJId49qdW3L1qKHH/3wPeUstCVpVSXTM6vO3VcTw= -github.com/go-faster/errors v0.6.1 h1:nNIPOBkprlKzkThvS/0YaX8Zs9KewLCOSFQS5BU06FI= -github.com/go-faster/errors v0.6.1/go.mod h1:5MGV2/2T9yvlrbhe9pD9LO5Z/2zCSq2T8j+Jpi2LAyY= +github.com/go-faster/errors v0.7.1 h1:MkJTnDoEdi9pDabt1dpWf7AA8/BaSYZqibYyhZ20AYg= +github.com/go-faster/errors v0.7.1/go.mod h1:5ySTjWFiphBs07IKuiL69nxdfd5+fzh1u7FPGZP2quo= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -274,8 +274,8 @@ github.com/jackc/pgservicefile v0.0.0-20200714003250-2b9c44734f2b/go.mod h1:vsD4 github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9 h1:L0QtFUgDarD7Fpv9jeVMgy/+Ec0mtnmYuImjTz6dtDA= github.com/jackc/pgservicefile v0.0.0-20231201235250-de7065d80cb9/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= github.com/jackc/pgx/v5 v5.0.3/go.mod h1:JBbvW3Hdw77jKl9uJrEDATUZIFM2VFPzRq4RWIhkF4o= -github.com/jackc/pgx/v5 v5.5.1 h1:5I9etrGkLrN+2XPCsi6XLlV5DITbSL/xBZdmAxFcXPI= -github.com/jackc/pgx/v5 v5.5.1/go.mod h1:Ig06C2Vu0t5qXC60W8sqIthScaEnFvojjj9dSljmHRA= +github.com/jackc/pgx/v5 v5.5.2 h1:iLlpgp4Cp/gC9Xuscl7lFL1PhhW+ZLtXZcrfCt4C3tA= +github.com/jackc/pgx/v5 v5.5.2/go.mod h1:ez9gk+OAat140fv9ErkZDYFWmXLfV+++K0uAOiwgm1A= github.com/jackc/puddle/v2 v2.0.0/go.mod h1:itE7ZJY8xnoo0JqJEpSMprN0f+NQkMCuEV/N9j8h0oc= github.com/jackc/puddle/v2 v2.2.1 h1:RhxXJtFG022u4ibrCSMSiu5aOq1i77R3OHKNJj77OAk= github.com/jackc/puddle/v2 v2.2.1/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= @@ -316,8 +316,6 @@ github.com/linkedin/goavro/v2 v2.12.0 h1:rIQQSj8jdAUlKQh6DttK8wCRv4t4QO09g1C4aBW github.com/linkedin/goavro/v2 v2.12.0/go.mod h1:KXx+erlq+RPlGSPmLF7xGo6SAbh8sCQ53x064+ioxhk= github.com/mattn/go-sqlite3 v1.14.6 h1:dNPt6NO46WmLVt2DLNpwczCmdV5boIZ6g/tlDrlRUbg= github.com/mattn/go-sqlite3 v1.14.6/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= github.com/microsoft/go-mssqldb v1.6.0 h1:mM3gYdVwEPFrlg/Dvr2DNVEgYFG7L42l+dGc67NNNpc= github.com/microsoft/go-mssqldb v1.6.0/go.mod h1:00mDtPbeQCRGC1HwOOR5K/gr30P1NcEG0vx6Kbv2aJU= github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs= @@ -331,8 +329,8 @@ github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLA github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/orcaman/concurrent-map/v2 v2.0.1 h1:jOJ5Pg2w1oeB6PeDurIYf6k9PQ+aTITr/6lP/L/zp6c= github.com/orcaman/concurrent-map/v2 v2.0.1/go.mod h1:9Eq3TG2oBe5FirmYWQfYO5iH1q0Jv47PLaNK++uCdOM= -github.com/paulmach/orb v0.10.0 h1:guVYVqzxHE/CQ1KpfGO077TR0ATHSNjp4s6XGLn3W9s= -github.com/paulmach/orb v0.10.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= +github.com/paulmach/orb v0.11.0 h1:JfVXJUBeH9ifc/OrhBY0lL16QsmPgpCHMlqSSYhcgAA= +github.com/paulmach/orb v0.11.0/go.mod h1:5mULz1xQfs3bmQm63QEJA6lNGujuRafwA5S/EnuLaLU= github.com/paulmach/protoscan v0.2.1/go.mod h1:SpcSwydNLrxUGSDvXvO0P7g7AuhJ7lcKfDlhJCDw2gY= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -355,8 +353,8 @@ github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlk github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= -github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= -github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= +github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= +github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= @@ -375,8 +373,8 @@ github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/slack-go/slack v0.12.3 h1:92/dfFU8Q5XP6Wp5rr5/T5JHLM5c5Smtn53fhToAP88= github.com/slack-go/slack v0.12.3/go.mod h1:hlGi5oXA+Gt+yWTPP0plCdRKmjsDxecdHxYQdlMQKOw= -github.com/snowflakedb/gosnowflake v1.7.1 h1:c9JjyjjDlvxex9ud71TwKL+Wu54Vfx+39h4DAwbIdqU= -github.com/snowflakedb/gosnowflake v1.7.1/go.mod h1:JI3eRZL8CpimPek6CJO0aTbDQjDGOt7Rxv9A/ti4f5c= +github.com/snowflakedb/gosnowflake v1.7.2 h1:HRSwva8YXC64WUppfmHcMNVVzSE1+EwXXaJxgS0EkTo= +github.com/snowflakedb/gosnowflake v1.7.2/go.mod h1:03tW856vc3ceM4rJuj7KO4dzqN7qoezTm+xw7aPIIFo= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -397,8 +395,8 @@ github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/twpayne/go-geos v0.15.0 h1:L8RCcbaEDfhRz/HhzOvw8fU2s7SzxBLh1sID125EneY= -github.com/twpayne/go-geos v0.15.0/go.mod h1:zmBwZNTaMTB1usptcCl4n7FjIDoBi2IGtm6h6nq9G8c= +github.com/twpayne/go-geos v0.16.1 h1:2tqZyAtDFxBqKn7HR2/g2tQkC/GuQQ2D8dxgK9BjdV8= +github.com/twpayne/go-geos v0.16.1/go.mod h1:zmBwZNTaMTB1usptcCl4n7FjIDoBi2IGtm6h6nq9G8c= github.com/urfave/cli/v3 v3.0.0-alpha8 h1:H+qxFPoCkGzdF8KUMs2fEOZl5io/1QySgUiGfar8occ= github.com/urfave/cli/v3 v3.0.0-alpha8/go.mod h1:0kK/RUFHyh+yIKSfWxwheGndfnrvYSmYFVeKCh03ZUc= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= @@ -420,18 +418,18 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD go.mongodb.org/mongo-driver v1.11.4/go.mod h1:PTSz5yu21bkT/wXpkS7WR5f0ddqw5quethTUn9WM+2g= 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.46.1 h1:SpGay3w+nEwMpfVnbqOLH5gY52/foP8RE8UzTZ1pdSE= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJMNjENlcleuuOkGAPH82y0yULBScfXcIEdS24= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= -go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= -go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= -go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= -go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0 h1:UNQQKPfTDe1J81ViolILjTKPr9WetKW6uei2hFgJmFs= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.47.0/go.mod h1:r9vWsPS/3AQItv3OSlEJ/E4mbrhUbbw18meOjArPtKQ= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.47.0 h1:sv9kVfal0MK0wBMCOGr+HeJm9v803BkJxGrk2au7j08= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.47.0/go.mod h1:SK2UL73Zy1quvRPonmOmRDiWk1KBV3LyIeeIxcEApWw= +go.opentelemetry.io/otel v1.22.0 h1:xS7Ku+7yTFvDfDraDIJVpw7XPyuHlB9MCiqqX5mcJ6Y= +go.opentelemetry.io/otel v1.22.0/go.mod h1:eoV4iAi3Ea8LkAEI9+GFT44O6T/D0GWAVFyZVCC6pMI= +go.opentelemetry.io/otel/metric v1.22.0 h1:lypMQnGyJYeuYPhOM/bgjbFM6WE44W1/T45er4d8Hhg= +go.opentelemetry.io/otel/metric v1.22.0/go.mod h1:evJGjVpZv0mQ5QBRJoBF64yMuOf4xCWdXjK8pzFvliY= go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= -go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= -go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= +go.opentelemetry.io/otel/trace v1.22.0 h1:Hg6pPujv0XG9QaVbGOBVHunyuLcCC3jN7WEhPx83XD0= +go.opentelemetry.io/otel/trace v1.22.0/go.mod h1:RbbHXVqKES9QhzZq/fE5UnOSILqRt40a21sPw2He1xo= go.temporal.io/api v1.26.0 h1:N4V0Daqa0qqK5+9LELSZV7clBYrwB4l33iaFfKgycPk= go.temporal.io/api v1.26.0/go.mod h1:uVAcpQJ6bM4mxZ3m7vSHU65fHjrwy9ktGQMtsNfMZQQ= go.temporal.io/sdk v1.25.1 h1:jC9l9vHHz5OJ7PR6OjrpYSN4+uEG0bLe5rdF9nlMSGk= @@ -454,8 +452,8 @@ golang.org/x/crypto v0.0.0-20220829220503-c86fa9a7ed90/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3 h1:hNQpMuAJe5CtcUqCXaWga3FHu+kQvCqcsoVaQgSV60o= -golang.org/x/exp v0.0.0-20240112132812-db7319d0e0e3/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= +golang.org/x/exp v0.0.0-20240119083558-1b970713d09a h1:Q8/wZp0KX97QFTc2ywcOE0YRjZPVIx+MXInMzdvQqcA= +golang.org/x/exp v0.0.0-20240119083558-1b970713d09a/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= 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= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -548,10 +546,10 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T 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= -gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E= -gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= -google.golang.org/api v0.156.0 h1:yloYcGbBtVYjLKQe4enCunxvwn3s2w/XPrrhVf6MsvQ= -google.golang.org/api v0.156.0/go.mod h1:bUSmn4KFO0Q+69zo9CNIDp4Psi6BqM0np0CbzKRSiSY= +gonum.org/v1/gonum v0.12.0 h1:xKuo6hzt+gMav00meVPUlXwSdoEJP46BR+wdxQEFK2o= +gonum.org/v1/gonum v0.12.0/go.mod h1:73TDxJfAAHeA8Mk9mf8NlIppyhQNo5GLTcYeqgo2lvY= +google.golang.org/api v0.157.0 h1:ORAeqmbrrozeyw5NjnMxh7peHO0UzV4wWYSwZeCUb20= +google.golang.org/api v0.157.0/go.mod h1:+z4v4ufbZ1WEpld6yMGHyggs+PmAHiaLNj5ytP3N01g= 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.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= @@ -562,12 +560,12 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98 google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20240108191215-35c7eff3a6b1 h1:/IWabOtPziuXTEtI1KYCpM6Ss7vaAkeMxk+uXV/xvZs= -google.golang.org/genproto v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1 h1:OPXtXn7fNMaXwO3JvOmF1QyTc00jsSFFz1vXXBOdCDo= -google.golang.org/genproto/googleapis/api v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:B5xPO//w8qmBDjGReYLpR6UJPnkldGkCSMoH/2vxJeg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240108191215-35c7eff3a6b1 h1:gphdwh0npgs8elJ4T6J+DQJHPVF7RsuJHCfwztUb4J4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240108191215-35c7eff3a6b1/go.mod h1:daQN87bsDqDoe316QbbvX60nMoJQa4r6Ds0ZuoAe5yA= +google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac h1:ZL/Teoy/ZGnzyrqK/Optxxp2pmVh+fmJ97slxSRyzUg= +google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= +google.golang.org/genproto/googleapis/api v0.0.0-20240116215550-a9fa1716bcac h1:OZkkudMUu9LVQMCoRUbI/1p5VCo9BOrlvkqMvWtqa6s= +google.golang.org/genproto/googleapis/api v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:B5xPO//w8qmBDjGReYLpR6UJPnkldGkCSMoH/2vxJeg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240116215550-a9fa1716bcac h1:nUQEQmH/csSvFECKYRv6HWEyypysidKl2I6Qpsglq/0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:daQN87bsDqDoe316QbbvX60nMoJQa4r6Ds0ZuoAe5yA= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= From ddca5488edde3e17566e060ac15192912098c49e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 22 Jan 2024 21:48:23 +0000 Subject: [PATCH 23/28] Support specifying host key for ssh config (#1125) Empty host key interpreted as accept-all Fixes #804 UX would be greatly improved by implementing a "load host key" button which has server request host key from remote server & fill in ui field Host keys can be acquired with ```sh ssh-keyscan $HOST ``` --- .editorconfig | 2 +- flow/connectors/postgres/ssh_wrapped_pool.go | 6 +--- flow/connectors/utils/ssh.go | 30 ++++++++++++++------ protos/peers.proto | 3 +- ui/app/peers/create/[peerType]/helpers/pg.ts | 8 ++++++ ui/components/PeerForms/ClickhouseConfig.tsx | 9 ++---- ui/components/PeerForms/PostgresForm.tsx | 9 ++---- 7 files changed, 39 insertions(+), 28 deletions(-) diff --git a/.editorconfig b/.editorconfig index 174af3e952..004d9e96f2 100644 --- a/.editorconfig +++ b/.editorconfig @@ -10,6 +10,6 @@ trim_trailing_whitespace = true indent_style = space indent_size = 4 -[{package.json,*.yml,*.yaml}] +[{package.json,*.yml,*.yaml,*.proto}] indent_style = space indent_size = 2 diff --git a/flow/connectors/postgres/ssh_wrapped_pool.go b/flow/connectors/postgres/ssh_wrapped_pool.go index 9922c41675..1c5511fc49 100644 --- a/flow/connectors/postgres/ssh_wrapped_pool.go +++ b/flow/connectors/postgres/ssh_wrapped_pool.go @@ -39,11 +39,7 @@ func NewSSHWrappedPostgresPool( if sshConfig != nil { sshServer = fmt.Sprintf("%s:%d", sshConfig.Host, sshConfig.Port) var err error - clientConfig, err = utils.GetSSHClientConfig( - sshConfig.User, - sshConfig.Password, - sshConfig.PrivateKey, - ) + clientConfig, err = utils.GetSSHClientConfig(sshConfig) if err != nil { slog.Error("Failed to get SSH client config", slog.Any("error", err)) cancel() diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go index 511eea672a..d960b20488 100644 --- a/flow/connectors/utils/ssh.go +++ b/flow/connectors/utils/ssh.go @@ -4,6 +4,7 @@ import ( "encoding/base64" "fmt" + "github.com/PeerDB-io/peer-flow/generated/protos" "golang.org/x/crypto/ssh" ) @@ -13,17 +14,17 @@ import ( // user: SSH username // password: SSH password (can be empty if using a private key) // privateKeyString: Private key as a string (can be empty if using a password) -func GetSSHClientConfig(user, password, privateKeyString string) (*ssh.ClientConfig, error) { +func GetSSHClientConfig(config *protos.SSHConfig) (*ssh.ClientConfig, error) { var authMethods []ssh.AuthMethod // Password-based authentication - if password != "" { - authMethods = append(authMethods, ssh.Password(password)) + if config.Password != "" { + authMethods = append(authMethods, ssh.Password(config.Password)) } // Private key-based authentication - if privateKeyString != "" { - pkey, err := base64.StdEncoding.DecodeString(privateKeyString) + if config.PrivateKey != "" { + pkey, err := base64.StdEncoding.DecodeString(config.PrivateKey) if err != nil { return nil, fmt.Errorf("failed to base64 decode private key: %w", err) } @@ -40,10 +41,21 @@ func GetSSHClientConfig(user, password, privateKeyString string) (*ssh.ClientCon return nil, fmt.Errorf("no authentication methods provided") } - return &ssh.ClientConfig{ - User: user, - Auth: authMethods, + var hostKeyCallback ssh.HostKeyCallback + if config.HostKey != "" { + pubKey, err := ssh.ParsePublicKey([]byte(config.HostKey)) + if err != nil { + return nil, fmt.Errorf("failed to parse host key: %w", err) + } + hostKeyCallback = ssh.FixedHostKey(pubKey) + } else { //nolint:gosec - HostKeyCallback: ssh.InsecureIgnoreHostKey(), + hostKeyCallback = ssh.InsecureIgnoreHostKey() + } + + return &ssh.ClientConfig{ + User: config.User, + Auth: authMethods, + HostKeyCallback: hostKeyCallback, }, nil } diff --git a/protos/peers.proto b/protos/peers.proto index b16c35fccd..372c02936b 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -8,6 +8,7 @@ message SSHConfig { string user = 3; string password = 4; string private_key = 5; + string host_key = 6; } message SnowflakeConfig { @@ -115,7 +116,7 @@ enum DBType { S3 = 5; SQLSERVER = 6; EVENTHUB_GROUP = 7; - CLICKHOUSE = 8; + CLICKHOUSE = 8; } message Peer { diff --git a/ui/app/peers/create/[peerType]/helpers/pg.ts b/ui/app/peers/create/[peerType]/helpers/pg.ts index 8e822fb5c4..122b92be2e 100644 --- a/ui/app/peers/create/[peerType]/helpers/pg.ts +++ b/ui/app/peers/create/[peerType]/helpers/pg.ts @@ -86,6 +86,13 @@ export const sshSetting = [ optional: true, tips: 'Private key as a BASE64 string for authentication in order to SSH into your machine.', }, + { + label: 'Host Key', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, hostKey: value })), + optional: true, + tips: 'Public key of host to mitigate MITM attacks when SSHing into your machine.', + }, ]; export const blankSSHConfig: SSHConfig = { @@ -94,6 +101,7 @@ export const blankSSHConfig: SSHConfig = { user: '', password: '', privateKey: '', + hostKey: '', }; export const blankPostgresSetting: PostgresConfig = { diff --git a/ui/components/PeerForms/ClickhouseConfig.tsx b/ui/components/PeerForms/ClickhouseConfig.tsx index 1819e626df..1d76286db0 100644 --- a/ui/components/PeerForms/ClickhouseConfig.tsx +++ b/ui/components/PeerForms/ClickhouseConfig.tsx @@ -137,12 +137,9 @@ export default function PostgresForm({ settings, setter }: ConfigProps) { (sshConfig as SSHConfig)[ sshParam.label === 'BASE64 Private Key' ? 'privateKey' - : (sshParam.label.toLowerCase() as - | 'host' - | 'port' - | 'user' - | 'password' - | 'privateKey') + : sshParam.label === 'Host Key' + ? 'hostKey' + : (sshParam.label.toLowerCase() as keyof SSHConfig) ] || '' } /> diff --git a/ui/components/PeerForms/PostgresForm.tsx b/ui/components/PeerForms/PostgresForm.tsx index 1819e626df..1d76286db0 100644 --- a/ui/components/PeerForms/PostgresForm.tsx +++ b/ui/components/PeerForms/PostgresForm.tsx @@ -137,12 +137,9 @@ export default function PostgresForm({ settings, setter }: ConfigProps) { (sshConfig as SSHConfig)[ sshParam.label === 'BASE64 Private Key' ? 'privateKey' - : (sshParam.label.toLowerCase() as - | 'host' - | 'port' - | 'user' - | 'password' - | 'privateKey') + : sshParam.label === 'Host Key' + ? 'hostKey' + : (sshParam.label.toLowerCase() as keyof SSHConfig) ] || '' } /> From 7042307672edc7240911ecec39bc8d5601cfa595 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 23 Jan 2024 14:28:05 +0000 Subject: [PATCH 24/28] CDCRecordStream: remove RelationMessageMapping (#1129) RelationMessageMapping of input is mutated, thus it can be added to sync response from input at end --- flow/activities/flowable.go | 3 ++- flow/connectors/bigquery/qrep_avro_sync.go | 1 - flow/connectors/eventhub/eventhub.go | 1 - flow/connectors/postgres/cdc.go | 1 - flow/connectors/postgres/postgres.go | 1 - flow/connectors/s3/s3.go | 1 - flow/connectors/snowflake/snowflake.go | 1 - flow/model/model.go | 12 ++++-------- 8 files changed, 6 insertions(+), 15 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 7fdde3ef43..f9830d479d 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -286,8 +286,8 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, } return &model.SyncResponse{ - RelationMessageMapping: <-recordBatch.RelationMessageMapping, TableSchemaDeltas: tableSchemaDeltas, + RelationMessageMapping: input.RelationMessageMapping, }, nil } @@ -298,6 +298,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, TableMappings: input.FlowConnectionConfigs.TableMappings, StagingPath: input.FlowConnectionConfigs.CdcStagingPath, }) + res.RelationMessageMapping = input.RelationMessageMapping if err != nil { slog.Warn("failed to push records", slog.Any("error", err)) a.Alerter.LogFlowError(ctx, flowName, err) diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index c9da0d6634..ce51ed1e2c 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -119,7 +119,6 @@ func (s *QRepAvroSyncMethod) SyncRecords( CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: tableSchemaDeltas, - RelationMessageMapping: <-req.Records.RelationMessageMapping, }, nil } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index a0d900db75..f60cb547c7 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -261,7 +261,6 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S NumRecordsSynced: rowsSynced, TableNameRowsMapping: make(map[string]uint32), TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), - RelationMessageMapping: <-req.Records.RelationMessageMapping, }, nil } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 8de328d0f6..73250c3a31 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -236,7 +236,6 @@ func (p *PostgresCDCSource) consumeStream( if cdcRecordsStorage.IsEmpty() { records.SignalAsEmpty() } - records.RelationMessageMapping <- p.relationMessageMapping p.logger.Info(fmt.Sprintf("[finished] PullRecords streamed %d records", cdcRecordsStorage.Len())) err := cdcRecordsStorage.Close() if err != nil { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 4af329d3b4..98b4382df6 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -417,7 +417,6 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: tableSchemaDeltas, - RelationMessageMapping: <-req.Records.RelationMessageMapping, }, nil } diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 0ddb8abf5c..d24175ed45 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -229,7 +229,6 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes NumRecordsSynced: int64(numRecords), TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), - RelationMessageMapping: <-req.Records.RelationMessageMapping, }, nil } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 29f41d5995..23b335f8f8 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -569,7 +569,6 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: tableSchemaDeltas, - RelationMessageMapping: <-req.Records.RelationMessageMapping, }, nil } diff --git a/flow/model/model.go b/flow/model/model.go index b3bd44b0ed..524a1ce83f 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -425,8 +425,6 @@ type CDCRecordStream struct { records chan Record // Schema changes from the slot SchemaDeltas chan *protos.TableSchemaDelta - // Relation message mapping - RelationMessageMapping chan RelationMessageMapping // Indicates if the last checkpoint has been set. lastCheckpointSet bool // lastCheckPointID is the last ID of the commit that corresponds to this batch. @@ -440,11 +438,10 @@ func NewCDCRecordStream() *CDCRecordStream { return &CDCRecordStream{ records: make(chan Record, channelBuffer), // TODO (kaushik): more than 1024 schema deltas can cause problems! - SchemaDeltas: make(chan *protos.TableSchemaDelta, 1<<10), - emptySignal: make(chan bool, 1), - RelationMessageMapping: make(chan RelationMessageMapping, 1), - lastCheckpointSet: false, - lastCheckPointID: atomic.Int64{}, + SchemaDeltas: make(chan *protos.TableSchemaDelta, 1<<10), + emptySignal: make(chan bool, 1), + lastCheckpointSet: false, + lastCheckPointID: atomic.Int64{}, } } @@ -515,7 +512,6 @@ func (r *CDCRecordStream) Close() { close(r.emptySignal) close(r.records) close(r.SchemaDeltas) - close(r.RelationMessageMapping) r.lastCheckpointSet = true } From b87b2516199896e08ad43b0902cee91cdd8a02dd Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 23 Jan 2024 20:47:42 +0530 Subject: [PATCH 25/28] Numeric: adjust precision handling (#1130) `200` was being synced as `199.99999999` in initial load for PostgreSQL -> Snowflake. It resided in a numeric column. Noticed that in ToJSON where we transform numerics and other types for raw table insertion, we set the scale to 9 to conform with the scale value we've chosen for Snowflake for Numeric values (Number(38,9)). And this issue isn't present in CDC. This PR does the same at `processNumeric` in the Avro transform step where we write to Avro files --- flow/model/model.go | 3 ++- flow/model/numeric/scale.go | 3 +++ flow/model/qvalue/avro_converter.go | 4 ++++ 3 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 flow/model/numeric/scale.go diff --git a/flow/model/model.go b/flow/model/model.go index 524a1ce83f..c2ecd7a3b7 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -12,6 +12,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" hstore_util "github.com/PeerDB-io/peer-flow/hstore" + "github.com/PeerDB-io/peer-flow/model/numeric" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/peerdbenv" ) @@ -215,7 +216,7 @@ func (r *RecordItems) toMap(hstoreAsJSON bool) (map[string]interface{}, error) { if !ok { return nil, errors.New("expected *big.Rat value") } - jsonStruct[col] = bigRat.FloatString(9) + jsonStruct[col] = bigRat.FloatString(numeric.PeerDBNumericScale) case qvalue.QValueKindFloat64: floatVal, ok := v.Value.(float64) if !ok { diff --git a/flow/model/numeric/scale.go b/flow/model/numeric/scale.go new file mode 100644 index 0000000000..5ce1ecd024 --- /dev/null +++ b/flow/model/numeric/scale.go @@ -0,0 +1,3 @@ +package numeric + +const PeerDBNumericScale = 9 diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index 3ab06047c8..f404df7e73 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -7,6 +7,7 @@ import ( "time" hstore_util "github.com/PeerDB-io/peer-flow/hstore" + "github.com/PeerDB-io/peer-flow/model/numeric" "github.com/google/uuid" "github.com/linkedin/goavro/v2" ) @@ -323,6 +324,9 @@ func (c *QValueAvroConverter) processNumeric() (interface{}, error) { return nil, fmt.Errorf("invalid Numeric value: expected *big.Rat, got %T", c.Value.Value) } + scale := numeric.PeerDBNumericScale + decimalValue := num.FloatString(scale) + num.SetString(decimalValue) if c.Nullable { return goavro.Union("bytes.decimal", num), nil } From 5b3fe7bf04d17482ebff7da859db245ff143a79a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 23 Jan 2024 15:37:53 +0000 Subject: [PATCH 26/28] flow lints: enable gci, musttag, perfsprint (#1128) Also rename CheckPoint to Checkpoint musttag revealed we were passing a `[]byte` to postgres for a `jsonb` column pgx handles serializing structs to json itself --- flow/.golangci.yml | 8 +++ flow/activities/flowable.go | 19 ++++--- flow/cmd/api.go | 17 +++--- flow/cmd/handler.go | 7 ++- flow/cmd/main.go | 3 +- flow/cmd/mirror_status.go | 5 +- flow/cmd/peer_data.go | 7 ++- flow/cmd/snapshot_worker.go | 6 +- flow/cmd/worker.go | 9 ++- flow/connectors/bigquery/bigquery.go | 10 ++-- .../bigquery/merge_stmt_generator.go | 1 + flow/connectors/bigquery/qrep.go | 6 +- flow/connectors/bigquery/qrep_avro_sync.go | 12 ++-- flow/connectors/bigquery/qvalue_convert.go | 1 + flow/connectors/clickhouse/clickhouse.go | 1 + flow/connectors/clickhouse/client.go | 3 +- flow/connectors/clickhouse/qrep.go | 7 ++- flow/connectors/clickhouse/qrep_avro_sync.go | 3 +- flow/connectors/core.go | 3 +- flow/connectors/eventhub/eventhub.go | 5 +- flow/connectors/eventhub/hub_batches.go | 1 + flow/connectors/eventhub/hubmanager.go | 3 +- flow/connectors/external_metadata/store.go | 5 +- flow/connectors/postgres/cdc.go | 32 +++++------ flow/connectors/postgres/client.go | 7 ++- .../postgres/normalize_stmt_generator.go | 3 +- flow/connectors/postgres/postgres.go | 15 ++--- .../postgres/postgres_schema_delta_test.go | 5 +- flow/connectors/postgres/qrep.go | 7 ++- .../postgres/qrep_partition_test.go | 5 +- .../postgres/qrep_query_executor.go | 12 ++-- flow/connectors/postgres/qrep_sql_sync.go | 8 +-- flow/connectors/postgres/qvalue_convert.go | 6 +- flow/connectors/postgres/ssh_wrapped_pool.go | 5 +- flow/connectors/s3/s3.go | 10 ++-- .../snowflake/avro_file_writer_test.go | 5 +- flow/connectors/snowflake/client.go | 2 +- flow/connectors/snowflake/qrep.go | 9 +-- flow/connectors/snowflake/qrep_avro_sync.go | 5 +- flow/connectors/snowflake/snowflake.go | 13 +++-- flow/connectors/sql/query_executor.go | 8 +-- flow/connectors/sqlserver/qrep.go | 7 ++- flow/connectors/sqlserver/sqlserver.go | 5 +- flow/connectors/utils/avro/avro_writer.go | 8 +-- flow/connectors/utils/aws.go | 10 ++-- flow/connectors/utils/catalog/env.go | 3 +- .../utils/cdc_records/cdc_records_storage.go | 3 +- .../cdc_records/cdc_records_storage_test.go | 5 +- .../connectors/utils/monitoring/monitoring.go | 11 ++-- flow/connectors/utils/partition/partition.go | 4 +- flow/connectors/utils/partition_hash.go | 4 +- flow/connectors/utils/postgres.go | 3 +- flow/connectors/utils/ssh.go | 3 +- flow/connectors/utils/stream.go | 3 +- flow/dynamicconf/dynamicconf.go | 3 +- flow/e2e/bigquery/bigquery_helper.go | 3 +- flow/e2e/bigquery/peer_flow_bq_test.go | 11 ++-- flow/e2e/bigquery/qrep_flow_bq_test.go | 3 +- flow/e2e/congen.go | 7 ++- flow/e2e/postgres/peer_flow_pg_test.go | 11 ++-- flow/e2e/postgres/qrep_flow_pg_test.go | 11 ++-- flow/e2e/s3/cdc_s3_test.go | 3 +- flow/e2e/s3/qrep_flow_s3_test.go | 7 ++- flow/e2e/s3/s3_helper.go | 5 +- flow/e2e/snowflake/peer_flow_sf_test.go | 11 ++-- flow/e2e/snowflake/qrep_flow_sf_test.go | 5 +- .../snowflake/snowflake_schema_delta_test.go | 3 +- .../e2e/sqlserver/qrep_flow_sqlserver_test.go | 14 +++-- flow/e2e/test_utils.go | 13 +++-- flow/e2eshared/e2eshared.go | 3 +- flow/model/model.go | 56 +++++++++---------- flow/model/qrecord_batch.go | 5 +- flow/model/qrecord_test.go | 5 +- flow/model/qvalue/avro_converter.go | 10 ++-- flow/model/qvalue/qvalue.go | 3 +- flow/shared/alerting/alerting.go | 3 +- flow/workflows/cdc_flow.go | 7 ++- flow/workflows/drop_flow.go | 5 +- flow/workflows/normalize_flow.go | 5 +- flow/workflows/qrep_flow.go | 5 +- flow/workflows/setup_flow.go | 8 +-- flow/workflows/snapshot_flow.go | 10 ++-- flow/workflows/sync_flow.go | 5 +- flow/workflows/xmin_flow.go | 5 +- 84 files changed, 342 insertions(+), 271 deletions(-) diff --git a/flow/.golangci.yml b/flow/.golangci.yml index eee69b73a1..638fab22f9 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -7,6 +7,7 @@ linters: - durationcheck - errcheck - forbidigo + - gci - gocritic - gofumpt - gosec @@ -14,9 +15,11 @@ linters: - ineffassign - lll - misspell + - musttag - nakedret - nolintlint - nonamedreturns + - perfsprint - prealloc - staticcheck - stylecheck @@ -30,6 +33,11 @@ linters: - wastedassign - whitespace linters-settings: + gci: + sections: + - standard + - 'prefix(github.com/PeerDB-io)' + - default gocritic: disabled-checks: - ifElseChain diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index f9830d479d..b4c8911ed3 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -9,6 +9,14 @@ import ( "sync" "time" + "github.com/jackc/pglogrepl" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + "go.temporal.io/sdk/activity" + "golang.org/x/sync/errgroup" + "google.golang.org/protobuf/proto" + "github.com/PeerDB-io/peer-flow/connectors" connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" @@ -20,13 +28,6 @@ import ( "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" - "github.com/jackc/pglogrepl" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" - "go.temporal.io/sdk/activity" - "golang.org/x/sync/errgroup" - "google.golang.org/protobuf/proto" ) // CheckConnectionResult is the result of a CheckConnection call. @@ -577,7 +578,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, numRecords) if err != nil { - slog.ErrorContext(ctx, fmt.Sprintf("%v", err)) + slog.ErrorContext(ctx, err.Error()) goroutineErr = err } } @@ -966,7 +967,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, err = monitoring.UpdatePullEndTimeAndRowsForPartition( errCtx, a.CatalogPool, runUUID, partition, int64(numRecords)) if err != nil { - slog.Error(fmt.Sprintf("%v", err)) + slog.Error(err.Error()) return err } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index f466a6b9f9..1e6d3c6d17 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -10,21 +10,20 @@ import ( "net/http" "time" - utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" - peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/google/uuid" "github.com/grpc-ecosystem/grpc-gateway/v2/runtime" - - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/grpc/reflection" - "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/client" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/health" "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/reflection" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" + peerflow "github.com/PeerDB-io/peer-flow/workflows" ) type APIServerParams struct { diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 1cee1fd5fc..93d5ff1ea6 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -8,15 +8,16 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" - peerflow "github.com/PeerDB-io/peer-flow/workflows" backoff "github.com/cenkalti/backoff/v4" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "go.temporal.io/sdk/client" "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" + peerflow "github.com/PeerDB-io/peer-flow/workflows" ) // grpc server implementation diff --git a/flow/cmd/main.go b/flow/cmd/main.go index 79741669f4..1d924e3ccc 100644 --- a/flow/cmd/main.go +++ b/flow/cmd/main.go @@ -8,9 +8,10 @@ import ( "os/signal" "syscall" - "github.com/PeerDB-io/peer-flow/logger" "github.com/urfave/cli/v3" _ "go.uber.org/automaxprocs" + + "github.com/PeerDB-io/peer-flow/logger" ) func main() { diff --git a/flow/cmd/mirror_status.go b/flow/cmd/mirror_status.go index 1e17495085..df7862d8fe 100644 --- a/flow/cmd/mirror_status.go +++ b/flow/cmd/mirror_status.go @@ -6,11 +6,12 @@ import ( "fmt" "log/slog" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5/pgtype" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/timestamppb" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) func (h *FlowRequestHandler) MirrorStatus( diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index f91f37bf8c..110b9b5a7f 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -6,13 +6,14 @@ import ( "fmt" "log/slog" - connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "google.golang.org/protobuf/proto" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" ) func (h *FlowRequestHandler) getPGPeerConfig(ctx context.Context, peerName string) (*protos.PostgresConfig, error) { diff --git a/flow/cmd/snapshot_worker.go b/flow/cmd/snapshot_worker.go index 9239cf7276..d76a095ace 100644 --- a/flow/cmd/snapshot_worker.go +++ b/flow/cmd/snapshot_worker.go @@ -4,14 +4,14 @@ import ( "crypto/tls" "fmt" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/worker" + "github.com/PeerDB-io/peer-flow/activities" utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" - - "go.temporal.io/sdk/client" - "go.temporal.io/sdk/worker" ) type SnapshotWorkerOptions struct { diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index f060230b63..aacc7d85bc 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -10,16 +10,15 @@ import ( "runtime" "syscall" + "github.com/grafana/pyroscope-go" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/worker" + "github.com/PeerDB-io/peer-flow/activities" utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" - - "github.com/grafana/pyroscope-go" - - "go.temporal.io/sdk/client" - "go.temporal.io/sdk/worker" ) type WorkerOptions struct { diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 3d7e27d863..3e6499d513 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -12,17 +12,17 @@ import ( "cloud.google.com/go/bigquery" "cloud.google.com/go/storage" + "github.com/jackc/pgx/v5/pgxpool" + "go.temporal.io/sdk/activity" + "google.golang.org/api/iterator" + "google.golang.org/api/option" + "github.com/PeerDB-io/peer-flow/connectors/utils" cc "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgxpool" - - "go.temporal.io/sdk/activity" - "google.golang.org/api/iterator" - "google.golang.org/api/option" ) const ( diff --git a/flow/connectors/bigquery/merge_stmt_generator.go b/flow/connectors/bigquery/merge_stmt_generator.go index e93a139a73..d87a83a290 100644 --- a/flow/connectors/bigquery/merge_stmt_generator.go +++ b/flow/connectors/bigquery/merge_stmt_generator.go @@ -5,6 +5,7 @@ import ( "strings" "cloud.google.com/go/bigquery" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index 305bab01eb..4720e60cee 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -8,12 +8,12 @@ import ( "time" "cloud.google.com/go/bigquery" + "google.golang.org/api/iterator" + "google.golang.org/protobuf/encoding/protojson" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - - "google.golang.org/api/iterator" - "google.golang.org/protobuf/encoding/protojson" ) func (c *BigQueryConnector) SyncQRepRecords( diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index ce51ed1e2c..ab20343626 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -5,17 +5,19 @@ import ( "fmt" "log/slog" "os" + "strconv" "strings" "time" "cloud.google.com/go/bigquery" + "go.temporal.io/sdk/activity" + "github.com/PeerDB-io/peer-flow/connectors/utils" avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "go.temporal.io/sdk/activity" ) type QRepAvroSyncMethod struct { @@ -53,8 +55,8 @@ func (s *QRepAvroSyncMethod) SyncRecords( return nil, fmt.Errorf("failed to define Avro schema: %w", err) } - stagingTable := fmt.Sprintf("%s_%s_staging", rawTableName, fmt.Sprint(syncBatchID)) - numRecords, err := s.writeToStage(fmt.Sprint(syncBatchID), rawTableName, avroSchema, + stagingTable := fmt.Sprintf("%s_%s_staging", rawTableName, strconv.FormatInt(syncBatchID, 10)) + numRecords, err := s.writeToStage(strconv.FormatInt(syncBatchID, 10), rawTableName, avroSchema, &datasetTable{ dataset: s.connector.datasetID, table: stagingTable, @@ -105,7 +107,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( // just log the error this isn't fatal. slog.Error("failed to delete staging table "+stagingTable, slog.Any("error", err), - slog.String("syncBatchID", fmt.Sprint(syncBatchID)), + slog.Int64("syncBatchID", syncBatchID), slog.String("destinationTable", rawTableName)) } @@ -114,7 +116,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( slog.String("dstTableName", rawTableName)) return &model.SyncResponse{ - LastSyncedCheckPointID: lastCP, + LastSyncedCheckpointID: lastCP, NumRecordsSynced: int64(numRecords), CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, diff --git a/flow/connectors/bigquery/qvalue_convert.go b/flow/connectors/bigquery/qvalue_convert.go index d4e5032182..75fa2971b5 100644 --- a/flow/connectors/bigquery/qvalue_convert.go +++ b/flow/connectors/bigquery/qvalue_convert.go @@ -4,6 +4,7 @@ import ( "fmt" "cloud.google.com/go/bigquery" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) diff --git a/flow/connectors/clickhouse/clickhouse.go b/flow/connectors/clickhouse/clickhouse.go index 59f58b1379..6b3740935a 100644 --- a/flow/connectors/clickhouse/clickhouse.go +++ b/flow/connectors/clickhouse/clickhouse.go @@ -8,6 +8,7 @@ import ( _ "github.com/ClickHouse/clickhouse-go/v2" _ "github.com/ClickHouse/clickhouse-go/v2/lib/driver" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" ) diff --git a/flow/connectors/clickhouse/client.go b/flow/connectors/clickhouse/client.go index 9aa14cd57c..8bd5a0221e 100644 --- a/flow/connectors/clickhouse/client.go +++ b/flow/connectors/clickhouse/client.go @@ -4,10 +4,11 @@ import ( "context" "fmt" + "github.com/jmoiron/sqlx" + peersql "github.com/PeerDB-io/peer-flow/connectors/sql" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/jmoiron/sqlx" ) type ClickhouseClient struct { diff --git a/flow/connectors/clickhouse/qrep.go b/flow/connectors/clickhouse/qrep.go index 57f2f94f26..0695d955bf 100644 --- a/flow/connectors/clickhouse/qrep.go +++ b/flow/connectors/clickhouse/qrep.go @@ -7,13 +7,14 @@ import ( "strings" "time" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" + "google.golang.org/protobuf/encoding/protojson" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go-v2/aws" - "github.com/aws/aws-sdk-go-v2/service/s3" - "google.golang.org/protobuf/encoding/protojson" ) const qRepMetadataTableName = "_peerdb_query_replication_metadata" diff --git a/flow/connectors/clickhouse/qrep_avro_sync.go b/flow/connectors/clickhouse/qrep_avro_sync.go index 2adb391457..84c5d2eb89 100644 --- a/flow/connectors/clickhouse/qrep_avro_sync.go +++ b/flow/connectors/clickhouse/qrep_avro_sync.go @@ -6,13 +6,14 @@ import ( "log/slog" "time" + "go.temporal.io/sdk/activity" + "github.com/PeerDB-io/peer-flow/connectors/utils" avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "go.temporal.io/sdk/activity" ) type ClickhouseAvroSyncMethod struct { diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 8abc6f3bf8..1e28822181 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -6,6 +6,8 @@ import ( "fmt" "log/slog" + "github.com/jackc/pgx/v5/pgxpool" + connbigquery "github.com/PeerDB-io/peer-flow/connectors/bigquery" connclickhouse "github.com/PeerDB-io/peer-flow/connectors/clickhouse" conneventhub "github.com/PeerDB-io/peer-flow/connectors/eventhub" @@ -15,7 +17,6 @@ import ( connsqlserver "github.com/PeerDB-io/peer-flow/connectors/sqlserver" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" - "github.com/jackc/pgx/v5/pgxpool" ) var ErrUnsupportedFunctionality = errors.New("requested connector does not support functionality") diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index f60cb547c7..1a2ecb4a58 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -9,6 +9,7 @@ import ( "time" "github.com/Azure/azure-sdk-for-go/sdk/azidentity" + metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -155,7 +156,7 @@ func (c *EventHubConnector) processBatch( numRecords.Add(1) - recordLSN := record.GetCheckPointID() + recordLSN := record.GetCheckpointID() if recordLSN > lastSeenLSN { lastSeenLSN = recordLSN } @@ -257,7 +258,7 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return &model.SyncResponse{ CurrentSyncBatchID: syncBatchID, - LastSyncedCheckPointID: lastCheckpoint, + LastSyncedCheckpointID: lastCheckpoint, NumRecordsSynced: rowsSynced, TableNameRowsMapping: make(map[string]uint32), TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), diff --git a/flow/connectors/eventhub/hub_batches.go b/flow/connectors/eventhub/hub_batches.go index 5634173faf..e96d0e6896 100644 --- a/flow/connectors/eventhub/hub_batches.go +++ b/flow/connectors/eventhub/hub_batches.go @@ -9,6 +9,7 @@ import ( "time" azeventhubs "github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs" + "github.com/PeerDB-io/peer-flow/shared" ) diff --git a/flow/connectors/eventhub/hubmanager.go b/flow/connectors/eventhub/hubmanager.go index 3635544d3c..7f6d206728 100644 --- a/flow/connectors/eventhub/hubmanager.go +++ b/flow/connectors/eventhub/hubmanager.go @@ -12,9 +12,10 @@ import ( "github.com/Azure/azure-sdk-for-go/sdk/azidentity" "github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs" "github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub" + cmap "github.com/orcaman/concurrent-map/v2" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" - cmap "github.com/orcaman/concurrent-map/v2" ) type EventHubManager struct { diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index b26aacf637..4bcfcaa583 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -5,12 +5,13 @@ import ( "fmt" "log/slog" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/PeerDB-io/peer-flow/connectors/utils" cc "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" ) const ( diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 73250c3a31..7959e5ff67 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -3,19 +3,11 @@ package connpostgres import ( "context" "crypto/sha256" - "encoding/json" "fmt" "log/slog" "regexp" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/connectors/utils/cdc_records" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/geo" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" @@ -24,6 +16,14 @@ import ( "github.com/jackc/pgx/v5/pgxpool" "github.com/lib/pq/oid" "go.temporal.io/sdk/activity" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/connectors/utils/cdc_records" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/geo" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" ) const maxRetriesForWalSegmentRemoved = 5 @@ -599,7 +599,7 @@ func (p *PostgresCDCSource) processInsertMessage( } return &model.InsertRecord{ - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), Items: items, DestinationTableName: p.TableNameMapping[tableName].Name, SourceTableName: tableName, @@ -640,7 +640,7 @@ func (p *PostgresCDCSource) processUpdateMessage( } return &model.UpdateRecord{ - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), OldItems: oldItems, NewItems: newItems, DestinationTableName: p.TableNameMapping[tableName].Name, @@ -677,7 +677,7 @@ func (p *PostgresCDCSource) processDeleteMessage( } return &model.DeleteRecord{ - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), Items: items, DestinationTableName: p.TableNameMapping[tableName].Name, SourceTableName: tableName, @@ -809,16 +809,12 @@ func (p *PostgresCDCSource) auditSchemaDelta(flowJobName string, rec *model.Rela activityInfo := activity.GetInfo(p.ctx) workflowID := activityInfo.WorkflowExecution.ID runID := activityInfo.WorkflowExecution.RunID - recJSON, err := json.Marshal(rec) - if err != nil { - return fmt.Errorf("failed to marshal schema delta to JSON: %w", err) - } - _, err = p.catalogPool.Exec(p.ctx, + _, err := p.catalogPool.Exec(p.ctx, `INSERT INTO peerdb_stats.schema_deltas_audit_log(flow_job_name,workflow_id,run_id,delta_info) VALUES($1,$2,$3,$4)`, - flowJobName, workflowID, runID, recJSON) + flowJobName, workflowID, runID, rec) if err != nil { return fmt.Errorf("failed to insert row into table: %w", err) } @@ -885,7 +881,7 @@ func (p *PostgresCDCSource) processRelationMessage( p.relationMessageMapping[currRel.RelationId] = currRel rec := &model.RelationRecord{ TableSchemaDelta: schemaDelta, - CheckPointID: int64(lsn), + CheckpointID: int64(lsn), } return rec, p.auditSchemaDelta(p.flowJobName, rec) } diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 2527a560c8..eb0b3e7619 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -7,13 +7,14 @@ import ( "strconv" "strings" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/lib/pq/oid" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) type PGVersion int diff --git a/flow/connectors/postgres/normalize_stmt_generator.go b/flow/connectors/postgres/normalize_stmt_generator.go index 083021926a..3792c188af 100644 --- a/flow/connectors/postgres/normalize_stmt_generator.go +++ b/flow/connectors/postgres/normalize_stmt_generator.go @@ -6,10 +6,11 @@ import ( "slices" "strings" + "golang.org/x/exp/maps" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "golang.org/x/exp/maps" ) type normalizeStmtGenerator struct { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 98b4382df6..c7c7dd7feb 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -8,17 +8,18 @@ import ( "strings" "time" + "github.com/google/uuid" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/google/uuid" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" ) // PostgresConnector is a Connector implementation for Postgres. @@ -362,7 +363,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S if len(records) == 0 { return &model.SyncResponse{ - LastSyncedCheckPointID: 0, + LastSyncedCheckpointID: 0, NumRecordsSynced: 0, }, nil } @@ -412,7 +413,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } return &model.SyncResponse{ - LastSyncedCheckPointID: lastCP, + LastSyncedCheckpointID: lastCP, NumRecordsSynced: int64(len(records)), CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index e912201533..4c3b012243 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -6,13 +6,14 @@ import ( "strings" "testing" + "github.com/jackc/pgx/v5" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5" - "github.com/stretchr/testify/require" ) type PostgresSchemaDeltaTestSuite struct { diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index f34e0a13bd..51cb6e30e9 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -8,14 +8,15 @@ import ( "text/template" "time" + "github.com/google/uuid" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" + "github.com/PeerDB-io/peer-flow/connectors/utils" partition_utils "github.com/PeerDB-io/peer-flow/connectors/utils/partition" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/google/uuid" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgtype" ) const qRepMetadataTableName = "_peerdb_query_replication_metadata" diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index b18f9c2574..f99b248ec4 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -7,10 +7,11 @@ import ( "testing" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5/pgxpool" "github.com/stretchr/testify/assert" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type testCase struct { diff --git a/flow/connectors/postgres/qrep_query_executor.go b/flow/connectors/postgres/qrep_query_executor.go index 506668c9be..52648249e3 100644 --- a/flow/connectors/postgres/qrep_query_executor.go +++ b/flow/connectors/postgres/qrep_query_executor.go @@ -5,17 +5,17 @@ import ( "fmt" "log/slog" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/geo" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" - "go.temporal.io/sdk/activity" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/geo" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" ) type QRepQueryExecutor struct { diff --git a/flow/connectors/postgres/qrep_sql_sync.go b/flow/connectors/postgres/qrep_sql_sync.go index b8bcf38687..e75e38a970 100644 --- a/flow/connectors/postgres/qrep_sql_sync.go +++ b/flow/connectors/postgres/qrep_sql_sync.go @@ -7,14 +7,14 @@ import ( "strings" "time" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgxpool" + "google.golang.org/protobuf/encoding/protojson" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5" - "github.com/jackc/pgx/v5/pgxpool" - - "google.golang.org/protobuf/encoding/protojson" ) type QRepSyncMethod interface { diff --git a/flow/connectors/postgres/qvalue_convert.go b/flow/connectors/postgres/qvalue_convert.go index d357495810..d80baf69b1 100644 --- a/flow/connectors/postgres/qvalue_convert.go +++ b/flow/connectors/postgres/qvalue_convert.go @@ -10,9 +10,10 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pgx/v5/pgtype" "github.com/lib/pq/oid" + + "github.com/PeerDB-io/peer-flow/model/qvalue" ) func postgresOIDToQValueKind(recvOID uint32) qvalue.QValueKind { @@ -262,8 +263,7 @@ func parseFieldFromQValueKind(qvalueKind qvalue.QValueKind, value interface{}) ( val = qvalue.QValue{Kind: qvalue.QValueKindFloat64, Value: floatVal} case qvalue.QValueKindString: // handling all unsupported types with strings as well for now. - textVal := value - val = qvalue.QValue{Kind: qvalue.QValueKindString, Value: fmt.Sprint(textVal)} + val = qvalue.QValue{Kind: qvalue.QValueKindString, Value: fmt.Sprint(value)} case qvalue.QValueKindUUID: switch value.(type) { case string: diff --git a/flow/connectors/postgres/ssh_wrapped_pool.go b/flow/connectors/postgres/ssh_wrapped_pool.go index 1c5511fc49..4f17116ea4 100644 --- a/flow/connectors/postgres/ssh_wrapped_pool.go +++ b/flow/connectors/postgres/ssh_wrapped_pool.go @@ -8,10 +8,11 @@ import ( "sync" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5/pgxpool" "golang.org/x/crypto/ssh" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" ) type SSHWrappedPostgresPool struct { diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index d24175ed45..2a3cf640f9 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -4,16 +4,18 @@ import ( "context" "fmt" "log/slog" + "strconv" "strings" "time" + "github.com/aws/aws-sdk-go-v2/aws" + "github.com/aws/aws-sdk-go-v2/service/s3" + metadataStore "github.com/PeerDB-io/peer-flow/connectors/external_metadata" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go-v2/aws" - "github.com/aws/aws-sdk-go-v2/service/s3" ) const ( @@ -200,7 +202,7 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes DestinationTableIdentifier: fmt.Sprintf("raw_table_%s", req.FlowJobName), } partition := &protos.QRepPartition{ - PartitionId: fmt.Sprint(syncBatchID), + PartitionId: strconv.FormatInt(syncBatchID, 10), } numRecords, err := c.SyncQRepRecords(qrepConfig, partition, recordStream) if err != nil { @@ -225,7 +227,7 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes } return &model.SyncResponse{ - LastSyncedCheckPointID: lastCheckpoint, + LastSyncedCheckpointID: lastCheckpoint, NumRecordsSynced: int64(numRecords), TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), diff --git a/flow/connectors/snowflake/avro_file_writer_test.go b/flow/connectors/snowflake/avro_file_writer_test.go index 8a5753680c..1e531ca3ac 100644 --- a/flow/connectors/snowflake/avro_file_writer_test.go +++ b/flow/connectors/snowflake/avro_file_writer_test.go @@ -8,11 +8,12 @@ import ( "testing" "time" + "github.com/google/uuid" + "github.com/stretchr/testify/require" + avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/google/uuid" - "github.com/stretchr/testify/require" ) // createQValue creates a QValue of the appropriate kind for a given placeholder. diff --git a/flow/connectors/snowflake/client.go b/flow/connectors/snowflake/client.go index 089965d573..82a09cb1ed 100644 --- a/flow/connectors/snowflake/client.go +++ b/flow/connectors/snowflake/client.go @@ -6,6 +6,7 @@ import ( "strings" "time" + "github.com/jackc/pgx/v5/pgtype" "github.com/jmoiron/sqlx" "github.com/snowflakedb/gosnowflake" @@ -14,7 +15,6 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" ) type SnowflakeClient struct { diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 215fdeab41..b4c2642f57 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -7,14 +7,15 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/shared" "github.com/aws/aws-sdk-go-v2/aws" "github.com/aws/aws-sdk-go-v2/service/s3" "github.com/jackc/pgx/v5/pgtype" "google.golang.org/protobuf/encoding/protojson" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/shared" ) const qRepMetadataTableName = "_peerdb_query_replication_metadata" diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 8355ab8bb2..3c330d636c 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -8,14 +8,15 @@ import ( "strings" "time" + _ "github.com/snowflakedb/gosnowflake" + "go.temporal.io/sdk/activity" + "github.com/PeerDB-io/peer-flow/connectors/utils" avro "github.com/PeerDB-io/peer-flow/connectors/utils/avro" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - _ "github.com/snowflakedb/gosnowflake" - "go.temporal.io/sdk/activity" ) type SnowflakeAvroSyncHandler struct { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 23b335f8f8..da6affa827 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -12,15 +12,16 @@ import ( "sync/atomic" "time" + "github.com/jackc/pgx/v5/pgtype" + "github.com/snowflakedb/gosnowflake" + "go.temporal.io/sdk/activity" + "golang.org/x/sync/errgroup" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" - "github.com/snowflakedb/gosnowflake" - "go.temporal.io/sdk/activity" - "golang.org/x/sync/errgroup" ) const ( @@ -510,7 +511,7 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. }() // updating metadata with new offset and syncBatchID - err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckPointID, syncBatchID, syncRecordsTx) + err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckpointID, syncBatchID, syncRecordsTx) if err != nil { return nil, err } @@ -564,7 +565,7 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( } return &model.SyncResponse{ - LastSyncedCheckPointID: lastCheckpoint, + LastSyncedCheckpointID: lastCheckpoint, NumRecordsSynced: int64(numRecords), CurrentSyncBatchID: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, diff --git a/flow/connectors/sql/query_executor.go b/flow/connectors/sql/query_executor.go index 751230a643..b5e699e067 100644 --- a/flow/connectors/sql/query_executor.go +++ b/flow/connectors/sql/query_executor.go @@ -9,14 +9,14 @@ import ( "math/big" "strings" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" "github.com/jmoiron/sqlx" - "go.temporal.io/sdk/activity" + + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared" ) type SQLQueryExecutor interface { diff --git a/flow/connectors/sqlserver/qrep.go b/flow/connectors/sqlserver/qrep.go index 6a75373597..33474ef490 100644 --- a/flow/connectors/sqlserver/qrep.go +++ b/flow/connectors/sqlserver/qrep.go @@ -6,12 +6,13 @@ import ( "log/slog" "text/template" - utils "github.com/PeerDB-io/peer-flow/connectors/utils/partition" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" "github.com/jmoiron/sqlx" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/partition" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) func (c *SQLServerConnector) GetQRepPartitions( diff --git a/flow/connectors/sqlserver/sqlserver.go b/flow/connectors/sqlserver/sqlserver.go index 6e9235431b..805dcfb849 100644 --- a/flow/connectors/sqlserver/sqlserver.go +++ b/flow/connectors/sqlserver/sqlserver.go @@ -5,11 +5,12 @@ import ( "fmt" "log/slog" + "github.com/jmoiron/sqlx" + _ "github.com/microsoft/go-mssqldb" + peersql "github.com/PeerDB-io/peer-flow/connectors/sql" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jmoiron/sqlx" - _ "github.com/microsoft/go-mssqldb" ) type SQLServerConnector struct { diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index e96a642dae..6148ae2a6a 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -9,10 +9,6 @@ import ( "os" "sync/atomic" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/aws/aws-sdk-go-v2/aws" "github.com/aws/aws-sdk-go-v2/feature/s3/manager" "github.com/aws/aws-sdk-go-v2/service/s3" @@ -20,6 +16,10 @@ import ( "github.com/klauspost/compress/snappy" "github.com/klauspost/compress/zstd" "github.com/linkedin/goavro/v2" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) type ( diff --git a/flow/connectors/utils/aws.go b/flow/connectors/utils/aws.go index 6e6cbed022..63adfa8330 100644 --- a/flow/connectors/utils/aws.go +++ b/flow/connectors/utils/aws.go @@ -22,11 +22,11 @@ type AWSSecrets struct { } type S3PeerCredentials struct { - AccessKeyID string - SecretAccessKey string - AwsRoleArn string - Region string - Endpoint string + AccessKeyID string `json:"accessKeyId"` + SecretAccessKey string `json:"secretAccessKey"` + AwsRoleArn string `json:"awsRoleArn"` + Region string `json:"region"` + Endpoint string `json:"endpoint"` } func GetAWSSecrets(creds S3PeerCredentials) (*AWSSecrets, error) { diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go index f5c8e0507d..5a12172022 100644 --- a/flow/connectors/utils/catalog/env.go +++ b/flow/connectors/utils/catalog/env.go @@ -5,10 +5,11 @@ import ( "fmt" "sync" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/peerdbenv" - "github.com/jackc/pgx/v5/pgxpool" ) var ( diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage.go b/flow/connectors/utils/cdc_records/cdc_records_storage.go index c6dece9eb9..a1bde8d614 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage.go @@ -10,10 +10,11 @@ import ( "os" "time" + "github.com/cockroachdb/pebble" + "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" - "github.com/cockroachdb/pebble" ) func encVal(val any) ([]byte, error) { diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go index 1d7afae08c..9f0b7a9f6e 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage_test.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage_test.go @@ -6,9 +6,10 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/stretchr/testify/require" ) func getTimeForTesting(t *testing.T) time.Time { @@ -47,7 +48,7 @@ func genKeyAndRec(t *testing.T) (model.TableWithPkey, model.Record) { rec := &model.InsertRecord{ SourceTableName: "test_src_tbl", DestinationTableName: "test_dst_tbl", - CheckPointID: 1, + CheckpointID: 1, CommitID: 2, Items: &model.RecordItems{ ColToValIdx: map[string]int{ diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 7e9263cf26..5412323bfc 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -4,16 +4,17 @@ import ( "context" "fmt" "log/slog" + "strconv" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" - "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type CDCBatchInfo struct { @@ -233,8 +234,8 @@ func addPartitionToQRepRun(ctx context.Context, pool *pgxpool.Pool, flowJobName var rangeStart, rangeEnd string switch x := partition.Range.Range.(type) { case *protos.PartitionRange_IntRange: - rangeStart = fmt.Sprint(x.IntRange.Start) - rangeEnd = fmt.Sprint(x.IntRange.End) + rangeStart = strconv.FormatInt(x.IntRange.Start, 10) + rangeEnd = strconv.FormatInt(x.IntRange.End, 10) case *protos.PartitionRange_TimestampRange: rangeStart = x.TimestampRange.Start.AsTime().String() rangeEnd = x.TimestampRange.End.AsTime().String() diff --git a/flow/connectors/utils/partition/partition.go b/flow/connectors/utils/partition/partition.go index 37ecd15f37..7d4d2c4298 100644 --- a/flow/connectors/utils/partition/partition.go +++ b/flow/connectors/utils/partition/partition.go @@ -5,11 +5,11 @@ import ( "log/slog" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" - "google.golang.org/protobuf/types/known/timestamppb" + + "github.com/PeerDB-io/peer-flow/generated/protos" ) // Function to compare two values diff --git a/flow/connectors/utils/partition_hash.go b/flow/connectors/utils/partition_hash.go index 14de3ae943..e22b1a27d3 100644 --- a/flow/connectors/utils/partition_hash.go +++ b/flow/connectors/utils/partition_hash.go @@ -1,8 +1,8 @@ package utils import ( - "fmt" "hash/fnv" + "strconv" ) func hashString(s string) uint32 { @@ -14,5 +14,5 @@ func hashString(s string) uint32 { func HashedPartitionKey(s string, numPartitions uint32) string { hashValue := hashString(s) partition := hashValue % numPartitions - return fmt.Sprintf("%d", partition) + return strconv.FormatUint(uint64(partition), 10) } diff --git a/flow/connectors/utils/postgres.go b/flow/connectors/utils/postgres.go index c3cc48659f..ed97364417 100644 --- a/flow/connectors/utils/postgres.go +++ b/flow/connectors/utils/postgres.go @@ -6,12 +6,13 @@ import ( "fmt" "net/url" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgerrcode" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + + "github.com/PeerDB-io/peer-flow/generated/protos" ) func IsUniqueError(err error) bool { diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go index d960b20488..c4580e870f 100644 --- a/flow/connectors/utils/ssh.go +++ b/flow/connectors/utils/ssh.go @@ -4,8 +4,9 @@ import ( "encoding/base64" "fmt" - "github.com/PeerDB-io/peer-flow/generated/protos" "golang.org/x/crypto/ssh" + + "github.com/PeerDB-io/peer-flow/generated/protos" ) // getSSHClientConfig returns an *ssh.ClientConfig based on provided credentials. diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index bbfe0a2fa6..59602b676a 100644 --- a/flow/connectors/utils/stream.go +++ b/flow/connectors/utils/stream.go @@ -4,9 +4,10 @@ import ( "fmt" "time" + "github.com/google/uuid" + "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/google/uuid" ) func RecordsToRawTableStream(req *model.RecordsToStreamRequest) (*model.RecordsToStreamResponse, error) { diff --git a/flow/dynamicconf/dynamicconf.go b/flow/dynamicconf/dynamicconf.go index 5a22ba4058..d08ece7078 100644 --- a/flow/dynamicconf/dynamicconf.go +++ b/flow/dynamicconf/dynamicconf.go @@ -6,9 +6,10 @@ import ( "strconv" "time" - utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + + utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" ) func dynamicConfKeyExists(ctx context.Context, conn *pgxpool.Pool, key string) bool { diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 7d2c5bce6b..0d024f9dc7 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -11,13 +11,14 @@ import ( "cloud.google.com/go/bigquery" "cloud.google.com/go/civil" + "google.golang.org/api/iterator" + peer_bq "github.com/PeerDB-io/peer-flow/connectors/bigquery" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "google.golang.org/api/iterator" ) type BigQueryTestHelper struct { diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index efa47366db..c195ab5e56 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -10,6 +10,12 @@ import ( "testing" "time" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/e2eshared" @@ -18,11 +24,6 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgerrcode" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - "github.com/stretchr/testify/require" ) type PeerFlowE2ETestSuiteBQ struct { diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index 6ac9f3be5d..4302b222e5 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -3,8 +3,9 @@ package e2e_bigquery import ( "fmt" - "github.com/PeerDB-io/peer-flow/e2e" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" ) func (s PeerFlowE2ETestSuiteBQ) setupSourceTable(tableName string, rowCount int) { diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index f324a8e8f5..cb3c35bc29 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -6,13 +6,14 @@ import ( "log/slog" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/generated/protos" ) const ( diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index b02d03df05..27be975acb 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -7,16 +7,17 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model/qvalue" - peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/jackc/pgerrcode" "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" "github.com/stretchr/testify/require" "go.temporal.io/sdk/testsuite" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model/qvalue" + peerflow "github.com/PeerDB-io/peer-flow/workflows" ) func (s PeerFlowE2ETestSuitePG) attachSchemaSuffix(tableName string) string { diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 2943e201b5..dcb32c7fd9 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -8,16 +8,17 @@ import ( "testing" "time" - connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" + + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type PeerFlowE2ETestSuitePG struct { diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index 26eb42d38e..69c26b6809 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -5,9 +5,10 @@ import ( "fmt" "time" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/e2e" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/stretchr/testify/require" ) func (s PeerFlowE2ETestSuiteS3) attachSchemaSuffix(tableName string) string { diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 5a2f266ab7..919994b570 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -8,12 +8,13 @@ import ( "testing" "time" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/e2eshared" - "github.com/PeerDB-io/peer-flow/shared" "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/e2eshared" + "github.com/PeerDB-io/peer-flow/shared" ) type PeerFlowE2ETestSuiteS3 struct { diff --git a/flow/e2e/s3/s3_helper.go b/flow/e2e/s3/s3_helper.go index ecca066ea5..1be1765927 100644 --- a/flow/e2e/s3/s3_helper.go +++ b/flow/e2e/s3/s3_helper.go @@ -8,12 +8,13 @@ import ( "os" "time" + "github.com/aws/aws-sdk-go-v2/service/s3" + s3types "github.com/aws/aws-sdk-go-v2/service/s3/types" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" - "github.com/aws/aws-sdk-go-v2/service/s3" - s3types "github.com/aws/aws-sdk-go-v2/service/s3/types" ) const ( diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 95340c74f3..8f3cf46e7a 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -10,6 +10,12 @@ import ( "testing" "time" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2e" @@ -19,11 +25,6 @@ import ( "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/jackc/pgerrcode" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - "github.com/stretchr/testify/require" ) type PeerFlowE2ETestSuiteSF struct { diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index b23ec8bb67..574869ac82 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -3,10 +3,11 @@ package e2e_snowflake import ( "fmt" - "github.com/PeerDB-io/peer-flow/e2e" - "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/google/uuid" "github.com/stretchr/testify/require" + + "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" ) //nolint:unparam diff --git a/flow/e2e/snowflake/snowflake_schema_delta_test.go b/flow/e2e/snowflake/snowflake_schema_delta_test.go index f9f0c1ceca..52f02b005e 100644 --- a/flow/e2e/snowflake/snowflake_schema_delta_test.go +++ b/flow/e2e/snowflake/snowflake_schema_delta_test.go @@ -6,12 +6,13 @@ import ( "log/slog" "testing" + "github.com/stretchr/testify/require" + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/stretchr/testify/require" ) const schemaDeltaTestSchemaName = "PUBLIC" diff --git a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index f7378ea2ae..3cb9cda650 100644 --- a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -5,20 +5,22 @@ import ( "fmt" "log/slog" "os" + "strconv" "strings" "testing" "time" + "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/joho/godotenv" + "github.com/stretchr/testify/require" + "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/jackc/pgx/v5/pgtype" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/joho/godotenv" - "github.com/stretchr/testify/require" ) type PeerFlowE2ETestSuiteSQLServer struct { @@ -95,8 +97,8 @@ func (s PeerFlowE2ETestSuiteSQLServer) insertRowsIntoSQLServerTable(tableName st schemaQualified := fmt.Sprintf("%s.%s", s.sqlsHelper.SchemaName, tableName) for i := 0; i < numRows; i++ { params := make(map[string]interface{}) - params["id"] = "test_id_" + fmt.Sprintf("%d", i) - params["card_id"] = "test_card_id_" + fmt.Sprintf("%d", i) + params["id"] = "test_id_" + strconv.Itoa(i) + params["card_id"] = "test_card_id_" + strconv.Itoa(i) params["v_from"] = time.Now() params["price"] = 100.00 params["status"] = 1 diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 0aa5aaebbb..94ee47a8e2 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -13,6 +13,13 @@ import ( "testing" "time" + "github.com/google/uuid" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/stretchr/testify/require" + "go.temporal.io/sdk/testsuite" + "github.com/PeerDB-io/peer-flow/activities" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" @@ -25,12 +32,6 @@ import ( "github.com/PeerDB-io/peer-flow/shared" "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" - "github.com/google/uuid" - "github.com/jackc/pgerrcode" - "github.com/jackc/pgx/v5/pgconn" - "github.com/jackc/pgx/v5/pgxpool" - "github.com/stretchr/testify/require" - "go.temporal.io/sdk/testsuite" ) func RegisterWorkflowsAndActivities(t *testing.T, env *testsuite.TestWorkflowEnvironment) { diff --git a/flow/e2eshared/e2eshared.go b/flow/e2eshared/e2eshared.go index 176564d342..9ac22762f8 100644 --- a/flow/e2eshared/e2eshared.go +++ b/flow/e2eshared/e2eshared.go @@ -8,8 +8,9 @@ import ( "strings" "testing" - "github.com/PeerDB-io/peer-flow/model" "github.com/jackc/pgx/v5/pgxpool" + + "github.com/PeerDB-io/peer-flow/model" ) type Suite interface { diff --git a/flow/model/model.go b/flow/model/model.go index c2ecd7a3b7..7e56a402e4 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -58,8 +58,8 @@ type PullRecordsRequest struct { } type Record interface { - // GetCheckPointID returns the ID of the record. - GetCheckPointID() int64 + // GetCheckpointID returns the ID of the record. + GetCheckpointID() int64 // get table name GetDestinationTableName() string // get columns and values for the record @@ -339,8 +339,8 @@ type InsertRecord struct { SourceTableName string // Name of the destination table DestinationTableName string - // CheckPointID is the ID of the record. - CheckPointID int64 + // CheckpointID is the ID of the record. + CheckpointID int64 // CommitID is the ID of the commit corresponding to this record. CommitID int64 // Items is a map of column name to value. @@ -348,8 +348,8 @@ type InsertRecord struct { } // Implement Record interface for InsertRecord. -func (r *InsertRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *InsertRecord) GetCheckpointID() int64 { + return r.CheckpointID } func (r *InsertRecord) GetDestinationTableName() string { @@ -363,8 +363,8 @@ func (r *InsertRecord) GetItems() *RecordItems { type UpdateRecord struct { // Name of the source table SourceTableName string - // CheckPointID is the ID of the record. - CheckPointID int64 + // CheckpointID is the ID of the record. + CheckpointID int64 // Name of the destination table DestinationTableName string // OldItems is a map of column name to value. @@ -376,8 +376,8 @@ type UpdateRecord struct { } // Implement Record interface for UpdateRecord. -func (r *UpdateRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *UpdateRecord) GetCheckpointID() int64 { + return r.CheckpointID } // Implement Record interface for UpdateRecord. @@ -394,8 +394,8 @@ type DeleteRecord struct { SourceTableName string // Name of the destination table DestinationTableName string - // CheckPointID is the ID of the record. - CheckPointID int64 + // CheckpointID is the ID of the record. + CheckpointID int64 // Items is a map of column name to value. Items *RecordItems // unchanged toast columns, filled from latest UpdateRecord @@ -403,8 +403,8 @@ type DeleteRecord struct { } // Implement Record interface for DeleteRecord. -func (r *DeleteRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *DeleteRecord) GetCheckpointID() int64 { + return r.CheckpointID } func (r *DeleteRecord) GetDestinationTableName() string { @@ -428,8 +428,8 @@ type CDCRecordStream struct { SchemaDeltas chan *protos.TableSchemaDelta // Indicates if the last checkpoint has been set. lastCheckpointSet bool - // lastCheckPointID is the last ID of the commit that corresponds to this batch. - lastCheckPointID atomic.Int64 + // lastCheckpointID is the last ID of the commit that corresponds to this batch. + lastCheckpointID atomic.Int64 // empty signal to indicate if the records are going to be empty or not. emptySignal chan bool } @@ -442,16 +442,16 @@ func NewCDCRecordStream() *CDCRecordStream { SchemaDeltas: make(chan *protos.TableSchemaDelta, 1<<10), emptySignal: make(chan bool, 1), lastCheckpointSet: false, - lastCheckPointID: atomic.Int64{}, + lastCheckpointID: atomic.Int64{}, } } func (r *CDCRecordStream) UpdateLatestCheckpoint(val int64) { // TODO update with https://github.com/golang/go/issues/63999 once implemented - // r.lastCheckPointID.Max(val) - oldLast := r.lastCheckPointID.Load() - for oldLast < val && !r.lastCheckPointID.CompareAndSwap(oldLast, val) { - oldLast = r.lastCheckPointID.Load() + // r.lastCheckpointID.Max(val) + oldLast := r.lastCheckpointID.Load() + for oldLast < val && !r.lastCheckpointID.CompareAndSwap(oldLast, val) { + oldLast = r.lastCheckpointID.Load() } } @@ -459,7 +459,7 @@ func (r *CDCRecordStream) GetLastCheckpoint() (int64, error) { if !r.lastCheckpointSet { return 0, errors.New("last checkpoint not set, stream is still active") } - return r.lastCheckPointID.Load(), nil + return r.lastCheckpointID.Load(), nil } func (r *CDCRecordStream) AddRecord(record Record) { @@ -546,8 +546,8 @@ type NormalizeRecordsRequest struct { } type SyncResponse struct { - // LastSyncedCheckPointID is the last ID that was synced. - LastSyncedCheckPointID int64 + // LastSyncedCheckpointID is the last ID that was synced. + LastSyncedCheckpointID int64 // NumRecordsSynced is the number of records that were synced. NumRecordsSynced int64 // CurrentSyncBatchID is the ID of the currently synced batch. @@ -569,13 +569,13 @@ type NormalizeResponse struct { // being clever and passing the delta back as a regular record instead of heavy CDC refactoring. type RelationRecord struct { - CheckPointID int64 - TableSchemaDelta *protos.TableSchemaDelta + CheckpointID int64 `json:"checkpointId"` + TableSchemaDelta *protos.TableSchemaDelta `json:"tableSchemaDelta"` } // Implement Record interface for RelationRecord. -func (r *RelationRecord) GetCheckPointID() int64 { - return r.CheckPointID +func (r *RelationRecord) GetCheckpointID() int64 { + return r.CheckpointID } func (r *RelationRecord) GetDestinationTableName() string { diff --git a/flow/model/qrecord_batch.go b/flow/model/qrecord_batch.go index 4729e04baa..9b18dfbbc0 100644 --- a/flow/model/qrecord_batch.go +++ b/flow/model/qrecord_batch.go @@ -6,10 +6,11 @@ import ( "math/big" "time" - "github.com/PeerDB-io/peer-flow/geo" - "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgtype" + + "github.com/PeerDB-io/peer-flow/geo" + "github.com/PeerDB-io/peer-flow/model/qvalue" ) // QRecordBatch holds a batch of QRecord objects. diff --git a/flow/model/qrecord_test.go b/flow/model/qrecord_test.go index 40aead66c9..775d0c81fd 100644 --- a/flow/model/qrecord_test.go +++ b/flow/model/qrecord_test.go @@ -4,11 +4,12 @@ import ( "math/big" "testing" + "github.com/google/uuid" + "github.com/stretchr/testify/assert" + "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/google/uuid" - "github.com/stretchr/testify/assert" ) func TestEquals(t *testing.T) { diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index f404df7e73..b95cde93b8 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -4,12 +4,14 @@ import ( "fmt" "log/slog" "math/big" + "strconv" "time" - hstore_util "github.com/PeerDB-io/peer-flow/hstore" - "github.com/PeerDB-io/peer-flow/model/numeric" "github.com/google/uuid" "github.com/linkedin/goavro/v2" + + hstore_util "github.com/PeerDB-io/peer-flow/hstore" + "github.com/PeerDB-io/peer-flow/model/numeric" ) // https://avro.apache.org/docs/1.11.0/spec.html @@ -275,7 +277,7 @@ func (c *QValueAvroConverter) processGoTime() (interface{}, error) { // Snowflake has issues with avro timestamp types, returning as string form of the int64 // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file if c.TargetDWH == QDWHTypeSnowflake { - return fmt.Sprint(ret), nil + return strconv.FormatInt(ret, 10), nil } return ret, nil } @@ -294,7 +296,7 @@ func (c *QValueAvroConverter) processGoDate() (interface{}, error) { // See: https://stackoverflow.com/questions/66104762/snowflake-date-column-have-incorrect-date-from-avro-file if c.TargetDWH == QDWHTypeSnowflake { ret := t.UnixMicro() - return fmt.Sprint(ret), nil + return strconv.FormatInt(ret, 10), nil } return t, nil } diff --git a/flow/model/qvalue/qvalue.go b/flow/model/qvalue/qvalue.go index 786065227b..7abe9b5ef4 100644 --- a/flow/model/qvalue/qvalue.go +++ b/flow/model/qvalue/qvalue.go @@ -10,9 +10,10 @@ import ( "time" "cloud.google.com/go/civil" + "github.com/google/uuid" + "github.com/PeerDB-io/peer-flow/geo" hstore_util "github.com/PeerDB-io/peer-flow/hstore" - "github.com/google/uuid" ) // if new types are added, register them in gob - cdc_records_storage.go diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go index 18c6cc2b9b..98922d22bc 100644 --- a/flow/shared/alerting/alerting.go +++ b/flow/shared/alerting/alerting.go @@ -7,9 +7,10 @@ import ( "log/slog" "time" - "github.com/PeerDB-io/peer-flow/dynamicconf" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgxpool" + + "github.com/PeerDB-io/peer-flow/dynamicconf" ) // alerting service, no cool name :( diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 1dd453931b..e4f4559b0a 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -6,15 +6,16 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "go.temporal.io/api/enums/v1" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" "google.golang.org/protobuf/proto" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "github.com/PeerDB-io/peer-flow/shared" ) const ( diff --git a/flow/workflows/drop_flow.go b/flow/workflows/drop_flow.go index b21d89f198..b50da66334 100644 --- a/flow/workflows/drop_flow.go +++ b/flow/workflows/drop_flow.go @@ -3,10 +3,11 @@ package peerflow import ( "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) // DropFlowWorkflowExecution represents the state for execution of a drop flow. diff --git a/flow/workflows/normalize_flow.go b/flow/workflows/normalize_flow.go index 720cbba06f..70f6463aef 100644 --- a/flow/workflows/normalize_flow.go +++ b/flow/workflows/normalize_flow.go @@ -4,10 +4,11 @@ import ( "fmt" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) type NormalizeFlowState struct { diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 47354a148b..1ae1518e21 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -6,13 +6,14 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "go.temporal.io/api/enums/v1" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type QRepFlowExecution struct { diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index 9b9b168a89..8959c07edf 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -6,13 +6,13 @@ import ( "sort" "time" + "go.temporal.io/sdk/log" + "go.temporal.io/sdk/workflow" + "golang.org/x/exp/maps" + "github.com/PeerDB-io/peer-flow/activities" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" - "golang.org/x/exp/maps" - - "go.temporal.io/sdk/log" - "go.temporal.io/sdk/workflow" ) // SetupFlow is the workflow that is responsible for ensuring all the diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 8830d0feec..d38801b599 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -7,15 +7,15 @@ import ( "strings" "time" - "github.com/PeerDB-io/peer-flow/concurrency" - "github.com/PeerDB-io/peer-flow/connectors/utils" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" - "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/concurrency" + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type SnapshotFlowExecution struct { diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index d2b0e72136..4c930ebf66 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -4,10 +4,11 @@ import ( "fmt" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/model" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" ) type SyncFlowState struct { diff --git a/flow/workflows/xmin_flow.go b/flow/workflows/xmin_flow.go index 32b276ee96..c6885253df 100644 --- a/flow/workflows/xmin_flow.go +++ b/flow/workflows/xmin_flow.go @@ -5,11 +5,12 @@ import ( "fmt" "time" - "github.com/PeerDB-io/peer-flow/generated/protos" - "github.com/PeerDB-io/peer-flow/shared" "github.com/google/uuid" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/shared" ) type XminFlowExecution struct { From 87bdc8c3261c04183cfecfc6c90aba8af76270f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 23 Jan 2024 16:04:07 +0000 Subject: [PATCH 27/28] Pass sync batch id to SyncRecords (#1131) No need to query it twice CatalogPool is always present, so remove nil check --- flow/activities/flowable.go | 41 +++++++++++++------------- flow/connectors/bigquery/bigquery.go | 10 +------ flow/connectors/eventhub/eventhub.go | 10 ++----- flow/connectors/postgres/postgres.go | 17 ++++------- flow/connectors/s3/s3.go | 10 ++----- flow/connectors/snowflake/snowflake.go | 12 ++------ flow/model/model.go | 3 +- 7 files changed, 37 insertions(+), 66 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index b4c8911ed3..2c1f3bb51a 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -252,24 +252,6 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, hasRecords := !recordBatch.WaitAndCheckEmpty() slog.InfoContext(ctx, fmt.Sprintf("the current sync flow has records: %v", hasRecords)) - if a.CatalogPool != nil && hasRecords { - syncBatchID, err := dstConn.GetLastSyncBatchID(flowName) - if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { - return nil, err - } - - err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, flowName, - monitoring.CDCBatchInfo{ - BatchID: syncBatchID + 1, - RowsInBatch: 0, - BatchEndlSN: 0, - StartTime: startTime, - }) - if err != nil { - a.Alerter.LogFlowError(ctx, flowName, err) - return nil, err - } - } if !hasRecords { // wait for the pull goroutine to finish @@ -292,8 +274,27 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, }, nil } + syncBatchID, err := dstConn.GetLastSyncBatchID(flowName) + if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { + return nil, err + } + syncBatchID += 1 + + err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, flowName, + monitoring.CDCBatchInfo{ + BatchID: syncBatchID, + RowsInBatch: 0, + BatchEndlSN: 0, + StartTime: startTime, + }) + if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) + return nil, err + } + syncStartTime := time.Now() res, err := dstConn.SyncRecords(&model.SyncRecordsRequest{ + SyncBatchID: syncBatchID, Records: recordBatch, FlowJobName: input.FlowConnectionConfigs.FlowJobName, TableMappings: input.FlowConnectionConfigs.TableMappings, @@ -377,13 +378,13 @@ func (a *FlowableActivity) StartNormalize( if errors.Is(err, connectors.ErrUnsupportedFunctionality) { dstConn, err := connectors.GetCDCSyncConnector(ctx, conn.Destination) if err != nil { - return nil, fmt.Errorf("failed to get connector: %v", err) + return nil, fmt.Errorf("failed to get connector: %w", err) } defer connectors.CloseConnector(dstConn) lastSyncBatchID, err := dstConn.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil { - return nil, fmt.Errorf("failed to get last sync batch ID: %v", err) + return nil, fmt.Errorf("failed to get last sync batch ID: %w", err) } err = monitoring.UpdateEndTimeForCDCBatch(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 3e6499d513..7c4534fe73 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -479,15 +479,7 @@ func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S c.logger.Info(fmt.Sprintf("pushing records to %s.%s...", c.datasetID, rawTableName)) - // generate a sequential number for last synced batch this sequence will be - // used to keep track of records that are normalized in NormalizeFlowWorkflow - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get batch for the current mirror: %v", err) - } - syncBatchID += 1 - - res, err := c.syncRecordsViaAvro(req, rawTableName, syncBatchID) + res, err := c.syncRecordsViaAvro(req, rawTableName, req.SyncBatchID) if err != nil { return nil, err } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 1a2ecb4a58..b14400cb34 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -250,16 +250,10 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return nil, err } - rowsSynced := int64(numRecords) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - c.logger.Error("failed to get last sync batch id", slog.Any("error", err)) - } - return &model.SyncResponse{ - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchID: req.SyncBatchID, LastSyncedCheckpointID: lastCheckpoint, - NumRecordsSynced: rowsSynced, + NumRecordsSynced: int64(numRecords), TableNameRowsMapping: make(map[string]uint32), TableSchemaDeltas: req.Records.WaitForSchemaDeltas(req.TableMappings), }, nil diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index c7c7dd7feb..7fdd6b032d 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -273,11 +273,6 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to Postgres table %s via COPY", rawTableIdentifier)) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get previous syncBatchID: %w", err) - } - syncBatchID += 1 records := make([][]interface{}, 0) tableNameRowsMapping := make(map[string]uint32) @@ -299,7 +294,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S itemsJSON, 0, "{}", - syncBatchID, + req.SyncBatchID, "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 @@ -326,7 +321,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S newItemsJSON, 1, oldItemsJSON, - syncBatchID, + req.SyncBatchID, utils.KeysToString(typedRecord.UnchangedToastColumns), }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 @@ -346,7 +341,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S itemsJSON, 2, itemsJSON, - syncBatchID, + req.SyncBatchID, "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 @@ -356,7 +351,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } tableSchemaDeltas := req.Records.WaitForSchemaDeltas(req.TableMappings) - err = c.ReplayTableSchemaDeltas(req.FlowJobName, tableSchemaDeltas) + err := c.ReplayTableSchemaDeltas(req.FlowJobName, tableSchemaDeltas) if err != nil { return nil, fmt.Errorf("failed to sync schema changes: %w", err) } @@ -402,7 +397,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } // updating metadata with new offset and syncBatchID - err = c.updateSyncMetadata(req.FlowJobName, lastCP, syncBatchID, syncRecordsTx) + err = c.updateSyncMetadata(req.FlowJobName, lastCP, req.SyncBatchID, syncRecordsTx) if err != nil { return nil, err } @@ -415,7 +410,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return &model.SyncResponse{ LastSyncedCheckpointID: lastCP, NumRecordsSynced: int64(len(records)), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchID: req.SyncBatchID, TableNameRowsMapping: tableNameRowsMapping, TableSchemaDeltas: tableSchemaDeltas, }, nil diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 2a3cf640f9..7a4e723e32 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -184,14 +184,8 @@ func (c *S3Connector) SetLastOffset(jobName string, offset int64) error { } func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get previous syncBatchID: %w", err) - } - syncBatchID += 1 - tableNameRowsMapping := make(map[string]uint32) - streamReq := model.NewRecordsToStreamRequest(req.Records.GetRecords(), tableNameRowsMapping, syncBatchID) + streamReq := model.NewRecordsToStreamRequest(req.Records.GetRecords(), tableNameRowsMapping, req.SyncBatchID) streamRes, err := utils.RecordsToRawTableStream(streamReq) if err != nil { return nil, fmt.Errorf("failed to convert records to raw table stream: %w", err) @@ -202,7 +196,7 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes DestinationTableIdentifier: fmt.Sprintf("raw_table_%s", req.FlowJobName), } partition := &protos.QRepPartition{ - PartitionId: strconv.FormatInt(syncBatchID, 10), + PartitionId: strconv.FormatInt(req.SyncBatchID, 10), } numRecords, err := c.SyncQRepRecords(qrepConfig, partition, recordStream) if err != nil { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index da6affa827..b96d6d9360 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -485,13 +485,7 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to Snowflake table %s", rawTableIdentifier)) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get previous syncBatchID: %w", err) - } - syncBatchID += 1 - - res, err := c.syncRecordsViaAvro(req, rawTableIdentifier, syncBatchID) + res, err := c.syncRecordsViaAvro(req, rawTableIdentifier, req.SyncBatchID) if err != nil { return nil, err } @@ -506,12 +500,12 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. deferErr := syncRecordsTx.Rollback() if deferErr != sql.ErrTxDone && deferErr != nil { c.logger.Error("error while rolling back transaction for SyncRecords: %v", - slog.Any("error", deferErr), slog.Int64("syncBatchID", syncBatchID)) + slog.Any("error", deferErr), slog.Int64("syncBatchID", req.SyncBatchID)) } }() // updating metadata with new offset and syncBatchID - err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckpointID, syncBatchID, syncRecordsTx) + err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckpointID, req.SyncBatchID, syncRecordsTx) if err != nil { return nil, err } diff --git a/flow/model/model.go b/flow/model/model.go index 7e56a402e4..09800251bf 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -526,7 +526,8 @@ type SyncAndNormalizeBatchID struct { } type SyncRecordsRequest struct { - Records *CDCRecordStream + SyncBatchID int64 + Records *CDCRecordStream // FlowJobName is the name of the flow job. FlowJobName string // SyncMode to use for pushing raw records From 346e34c89d8ab1cb55f77a40dc7d2384456d1c3f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 23 Jan 2024 16:23:38 +0000 Subject: [PATCH 28/28] external metadata: prevent sql injection (#1132) --- flow/connectors/eventhub/eventhub.go | 6 +----- flow/connectors/external_metadata/store.go | 19 ++++++++++++------- flow/connectors/s3/s3.go | 6 +----- 3 files changed, 14 insertions(+), 17 deletions(-) diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index b14400cb34..9eab9ae1b2 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -302,9 +302,5 @@ func (c *EventHubConnector) SetupNormalizedTables( } func (c *EventHubConnector) SyncFlowCleanup(jobName string) error { - err := c.pgMetadata.DropMetadata(jobName) - if err != nil { - return err - } - return nil + return c.pgMetadata.DropMetadata(jobName) } diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index 4bcfcaa583..e1858680cf 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -8,6 +8,7 @@ import ( "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/connectors/utils" cc "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" @@ -67,6 +68,10 @@ func (p *PostgresMetadataStore) Close() error { return nil } +func (p *PostgresMetadataStore) QualifyTable(table string) string { + return connpostgres.QuoteIdentifier(p.schemaName) + "." + connpostgres.QuoteIdentifier(table) +} + func (p *PostgresMetadataStore) Ping() error { if p.pool == nil { return fmt.Errorf("metadata db ping failed as pool does not exist") @@ -107,7 +112,7 @@ func (p *PostgresMetadataStore) SetupMetadata() error { // create the last sync state table _, err = p.pool.Exec(p.ctx, ` - CREATE TABLE IF NOT EXISTS `+p.schemaName+`.`+lastSyncStateTableName+` ( + CREATE TABLE IF NOT EXISTS `+p.QualifyTable(lastSyncStateTableName)+` ( job_name TEXT PRIMARY KEY NOT NULL, last_offset BIGINT NOT NULL, updated_at TIMESTAMP NOT NULL DEFAULT NOW(), @@ -126,7 +131,7 @@ func (p *PostgresMetadataStore) SetupMetadata() error { func (p *PostgresMetadataStore) FetchLastOffset(jobName string) (int64, error) { rows := p.pool.QueryRow(p.ctx, ` SELECT last_offset - FROM `+p.schemaName+`.`+lastSyncStateTableName+` + FROM `+p.QualifyTable(lastSyncStateTableName)+` WHERE job_name = $1 `, jobName) var offset pgtype.Int8 @@ -148,7 +153,7 @@ func (p *PostgresMetadataStore) FetchLastOffset(jobName string) (int64, error) { func (p *PostgresMetadataStore) GetLastBatchID(jobName string) (int64, error) { rows := p.pool.QueryRow(p.ctx, ` SELECT sync_batch_id - FROM `+p.schemaName+`.`+lastSyncStateTableName+` + FROM `+p.QualifyTable(lastSyncStateTableName)+` WHERE job_name = $1 `, jobName) @@ -180,10 +185,10 @@ func (p *PostgresMetadataStore) UpdateLastOffset(jobName string, offset int64) e // update the last offset p.logger.Info("updating last offset", slog.Int64("offset", offset)) _, err = tx.Exec(p.ctx, ` - INSERT INTO `+p.schemaName+`.`+lastSyncStateTableName+` (job_name, last_offset, sync_batch_id) + INSERT INTO `+p.QualifyTable(lastSyncStateTableName)+` (job_name, last_offset, sync_batch_id) VALUES ($1, $2, $3) ON CONFLICT (job_name) - DO UPDATE SET last_offset = GREATEST(`+lastSyncStateTableName+`.last_offset, excluded.last_offset), + DO UPDATE SET last_offset = GREATEST(`+connpostgres.QuoteIdentifier(lastSyncStateTableName)+`.last_offset, excluded.last_offset), updated_at = NOW() `, jobName, offset, 0) @@ -206,7 +211,7 @@ func (p *PostgresMetadataStore) UpdateLastOffset(jobName string, offset int64) e func (p *PostgresMetadataStore) IncrementID(jobName string) error { p.logger.Info("incrementing sync batch id for job") _, err := p.pool.Exec(p.ctx, ` - UPDATE `+p.schemaName+`.`+lastSyncStateTableName+` + UPDATE `+p.QualifyTable(lastSyncStateTableName)+` SET sync_batch_id=sync_batch_id+1 WHERE job_name=$1 `, jobName) if err != nil { @@ -219,7 +224,7 @@ func (p *PostgresMetadataStore) IncrementID(jobName string) error { func (p *PostgresMetadataStore) DropMetadata(jobName string) error { _, err := p.pool.Exec(p.ctx, ` - DELETE FROM `+p.schemaName+`.`+lastSyncStateTableName+` + DELETE FROM `+p.QualifyTable(lastSyncStateTableName)+` WHERE job_name = $1 `, jobName) return err diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 7a4e723e32..db96a927fb 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -242,9 +242,5 @@ func (c *S3Connector) SetupNormalizedTables(req *protos.SetupNormalizedTableBatc } func (c *S3Connector) SyncFlowCleanup(jobName string) error { - err := c.pgMetadata.DropMetadata(jobName) - if err != nil { - return err - } - return nil + return c.pgMetadata.DropMetadata(jobName) }