diff --git a/.github/workflows/dev-docker.yml b/.github/workflows/dev-docker.yml index c4e3e313ec..ad29acfb6e 100644 --- a/.github/workflows/dev-docker.yml +++ b/.github/workflows/dev-docker.yml @@ -10,7 +10,7 @@ jobs: docker-build: strategy: matrix: - runner: [ubuntu-latest] + runner: [ubicloud-standard-2-ubuntu-2204-arm] runs-on: ${{ matrix.runner }} permissions: contents: read diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 6d965122a3..2ddff33bd0 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -13,7 +13,7 @@ jobs: matrix: runner: [ubicloud-standard-8-ubuntu-2204-arm] runs-on: ${{ matrix.runner }} - timeout-minutes: 40 + timeout-minutes: 30 services: pg_cdc: image: imresamu/postgis:15-3.4-alpine @@ -80,13 +80,14 @@ jobs: name: "gcs_creds.json" json: ${{ secrets.GCS_CREDS }} - - name: create hstore extension and increase logical replication limits + - name: create hstore extension, increase logical replication limits, and setup catalog database run: > docker exec pg_cdc psql -h localhost -p 5432 -U postgres -c "CREATE EXTENSION hstore;" -c "ALTER SYSTEM SET wal_level=logical;" -c "ALTER SYSTEM SET max_replication_slots=192;" -c "ALTER SYSTEM SET max_wal_senders=256;" -c "ALTER SYSTEM SET max_connections=2048;" && + (cat ../nexus/catalog/migrations/V{?,??}__* | docker exec -i pg_cdc psql -h localhost -p 5432 -U postgres) && docker restart pg_cdc working-directory: ./flow env: diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 9d5dd70ab1..ace3e166fe 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -13,7 +13,6 @@ import ( connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" - catalog "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" @@ -21,6 +20,7 @@ import ( "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" + "github.com/jackc/pgx/v5/pgxpool" log "github.com/sirupsen/logrus" "go.temporal.io/sdk/activity" "golang.org/x/sync/errgroup" @@ -40,7 +40,7 @@ type SlotSnapshotSignal struct { } type FlowableActivity struct { - CatalogMirrorMonitor *monitoring.CatalogMirrorMonitor + CatalogPool *pgxpool.Pool } // CheckConnection implements CheckConnection. @@ -114,7 +114,7 @@ func (a *FlowableActivity) CreateRawTable( ctx context.Context, config *protos.CreateRawTableInput, ) (*protos.CreateRawTableOutput, error) { - ctx = context.WithValue(ctx, shared.CDCMirrorMonitorKey, a.CatalogMirrorMonitor) + ctx = context.WithValue(ctx, shared.CDCMirrorMonitorKey, a.CatalogPool) dstConn, err := connectors.GetCDCSyncConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) @@ -125,7 +125,7 @@ func (a *FlowableActivity) CreateRawTable( if err != nil { return nil, err } - err = a.CatalogMirrorMonitor.InitializeCDCFlow(ctx, config.FlowJobName) + err = monitoring.InitializeCDCFlow(ctx, a.CatalogPool, config.FlowJobName) if err != nil { return nil, err } @@ -174,7 +174,7 @@ func (a *FlowableActivity) handleSlotInfo( } if len(slotInfo) != 0 { - return a.CatalogMirrorMonitor.AppendSlotSizeInfo(ctx, peerName, slotInfo[0]) + return monitoring.AppendSlotSizeInfo(ctx, a.CatalogPool, peerName, slotInfo[0]) } return nil } @@ -208,7 +208,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlowInput) (*model.SyncResponse, error) { activity.RecordHeartbeat(ctx, "starting flow...") conn := input.FlowConnectionConfigs - ctx = context.WithValue(ctx, shared.CDCMirrorMonitorKey, a.CatalogMirrorMonitor) + ctx = context.WithValue(ctx, shared.CDCMirrorMonitorKey, a.CatalogPool) dstConn, err := connectors.GetCDCSyncConnector(ctx, conn.Destination) if err != nil { return nil, fmt.Errorf("failed to get destination connector: %w", err) @@ -275,13 +275,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, "flowName": input.FlowConnectionConfigs.FlowJobName, }).Infof("the current sync flow has records: %v", hasRecords) - if a.CatalogMirrorMonitor.IsActive() && hasRecords { + if a.CatalogPool != nil && hasRecords { syncBatchID, err := dstConn.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } - err = a.CatalogMirrorMonitor.AddCDCBatchForFlow(ctx, input.FlowConnectionConfigs.FlowJobName, + err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, monitoring.CDCBatchInfo{ BatchID: syncBatchID + 1, RowsInBatch: 0, @@ -346,8 +346,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, return nil, fmt.Errorf("failed to get last checkpoint: %w", err) } - err = a.CatalogMirrorMonitor.UpdateNumRowsAndEndLSNForCDCBatch( + err = monitoring.UpdateNumRowsAndEndLSNForCDCBatch( ctx, + a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, res.CurrentSyncBatchID, uint32(numRecords), @@ -357,13 +358,17 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, return nil, err } - err = a.CatalogMirrorMonitor. - UpdateLatestLSNAtTargetForCDCFlow(ctx, input.FlowConnectionConfigs.FlowJobName, pglogrepl.LSN(lastCheckpoint)) + err = monitoring.UpdateLatestLSNAtTargetForCDCFlow( + ctx, + a.CatalogPool, + input.FlowConnectionConfigs.FlowJobName, + pglogrepl.LSN(lastCheckpoint), + ) if err != nil { return nil, err } if res.TableNameRowsMapping != nil { - err = a.CatalogMirrorMonitor.AddCDCBatchTablesForFlow(ctx, input.FlowConnectionConfigs.FlowJobName, + err = monitoring.AddCDCBatchTablesForFlow(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, res.CurrentSyncBatchID, res.TableNameRowsMapping) if err != nil { return nil, err @@ -396,7 +401,7 @@ func (a *FlowableActivity) StartNormalize( } defer connectors.CloseConnector(dstConn) - err = a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, + err = monitoring.UpdateEndTimeForCDCBatch(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, syncBatchID) return nil, err } else if err != nil { @@ -430,8 +435,12 @@ func (a *FlowableActivity) StartNormalize( // normalize flow did not run due to no records, no need to update end time. if res.Done { - err = a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, - res.EndBatchID) + err = monitoring.UpdateEndTimeForCDCBatch( + ctx, + a.CatalogPool, + input.FlowConnectionConfigs.FlowJobName, + res.EndBatchID, + ) if err != nil { return nil, err } @@ -496,8 +505,9 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, return nil, fmt.Errorf("failed to get partitions from source: %w", err) } if len(partitions) > 0 { - err = a.CatalogMirrorMonitor.InitializeQRepRun( + err = monitoring.InitializeQRepRun( ctx, + a.CatalogPool, config, runUUID, partitions, @@ -518,7 +528,7 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, partitions *protos.QRepPartitionBatch, runUUID string, ) error { - err := a.CatalogMirrorMonitor.UpdateStartTimeForQRepRun(ctx, runUUID) + err := monitoring.UpdateStartTimeForQRepRun(ctx, a.CatalogPool, runUUID) if err != nil { return fmt.Errorf("failed to update start time for qrep run: %w", err) } @@ -545,12 +555,13 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, partition *protos.QRepPartition, runUUID string, ) error { - err := a.CatalogMirrorMonitor.UpdateStartTimeForPartition(ctx, runUUID, partition, time.Now()) + err := monitoring.UpdateStartTimeForPartition(ctx, a.CatalogPool, runUUID, partition, time.Now()) if err != nil { return fmt.Errorf("failed to update start time for partition: %w", err) } - srcConn, err := connectors.GetQRepPullConnector(ctx, config.SourcePeer) + pullCtx, pullCancel := context.WithCancel(ctx) + srcConn, err := connectors.GetQRepPullConnector(pullCtx, config.SourcePeer) if err != nil { return fmt.Errorf("failed to get qrep source connector: %w", err) } @@ -583,7 +594,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, }).Errorf("failed to pull records: %v", err) goroutineErr = err } else { - err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) + err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, numRecords) if err != nil { log.Errorf("%v", err) goroutineErr = err @@ -603,7 +614,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, "flowName": config.FlowJobName, }).Infof("pulled %d records\n", len(recordBatch.Records)) - err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(ctx, runUUID, partition, numRecords) + err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, numRecords) if err != nil { return err } @@ -628,6 +639,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, } if rowsSynced == 0 { + pullCancel() log.WithFields(log.Fields{ "flowName": config.FlowJobName, }).Infof("no records to push for partition %s\n", partition.PartitionId) @@ -637,7 +649,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, return goroutineErr } - err := a.CatalogMirrorMonitor.UpdateRowsSyncedForPartition(ctx, rowsSynced, runUUID, partition) + err := monitoring.UpdateRowsSyncedForPartition(ctx, a.CatalogPool, rowsSynced, runUUID, partition) if err != nil { return err } @@ -647,7 +659,7 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, }).Infof("pushed %d records\n", rowsSynced) } - err = a.CatalogMirrorMonitor.UpdateEndTimeForPartition(ctx, runUUID, partition) + err = monitoring.UpdateEndTimeForPartition(ctx, a.CatalogPool, runUUID, partition) if err != nil { return err } @@ -659,7 +671,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config runUUID string) error { dstConn, err := connectors.GetQRepConsolidateConnector(ctx, config.DestinationPeer) if errors.Is(err, connectors.ErrUnsupportedFunctionality) { - return a.CatalogMirrorMonitor.UpdateEndTimeForQRepRun(ctx, runUUID) + return monitoring.UpdateEndTimeForQRepRun(ctx, a.CatalogPool, runUUID) } else if err != nil { return err } @@ -677,7 +689,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config return err } - return a.CatalogMirrorMonitor.UpdateEndTimeForQRepRun(ctx, runUUID) + return monitoring.UpdateEndTimeForQRepRun(ctx, a.CatalogPool, runUUID) } func (a *FlowableActivity) CleanupQRepFlow(ctx context.Context, config *protos.QRepConfig) error { @@ -715,14 +727,8 @@ func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.Shutdown return nil } -func getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { - catalogPool, catalogErr := catalog.GetCatalogConnectionPoolFromEnv() - if catalogErr != nil { - return nil, fmt.Errorf("error getting catalog connection pool: %w", catalogErr) - } - defer catalogPool.Close() - - optionRows, err := catalogPool.Query(ctx, ` +func (a *FlowableActivity) getPostgresPeerConfigs(ctx context.Context) ([]*protos.Peer, error) { + optionRows, err := a.CatalogPool.Query(ctx, ` SELECT DISTINCT p.name, p.options FROM peers p JOIN flows f ON p.id = f.source_peer @@ -764,7 +770,7 @@ func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { log.Info("context is done, exiting wal heartbeat send loop") return nil case <-ticker.C: - pgPeers, err := getPostgresPeerConfigs(ctx) + pgPeers, err := a.getPostgresPeerConfigs(ctx) if err != nil { log.Warn("[sendwalheartbeat]: warning: unable to fetch peers." + "Skipping walheartbeat send. error encountered: " + err.Error()) @@ -946,17 +952,17 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, }}, } } - updateErr := a.CatalogMirrorMonitor.InitializeQRepRun(ctx, config, runUUID, []*protos.QRepPartition{partitionForMetrics}) + updateErr := monitoring.InitializeQRepRun(ctx, a.CatalogPool, config, runUUID, []*protos.QRepPartition{partitionForMetrics}) if updateErr != nil { return updateErr } - err := a.CatalogMirrorMonitor.UpdateStartTimeForPartition(ctx, runUUID, partition, startTime) + err := monitoring.UpdateStartTimeForPartition(ctx, a.CatalogPool, runUUID, partition, startTime) if err != nil { return fmt.Errorf("failed to update start time for partition: %w", err) } - err = a.CatalogMirrorMonitor.UpdatePullEndTimeAndRowsForPartition(errCtx, runUUID, partition, int64(numRecords)) + err = monitoring.UpdatePullEndTimeAndRowsForPartition(errCtx, a.CatalogPool, runUUID, partition, int64(numRecords)) if err != nil { log.Errorf("%v", err) return err @@ -988,7 +994,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, return 0, err } - err = a.CatalogMirrorMonitor.UpdateRowsSyncedForPartition(ctx, rowsSynced, runUUID, partition) + err = monitoring.UpdateRowsSyncedForPartition(ctx, a.CatalogPool, rowsSynced, runUUID, partition) if err != nil { return 0, err } @@ -998,7 +1004,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, }).Infof("pushed %d records\n", rowsSynced) } - err = a.CatalogMirrorMonitor.UpdateEndTimeForPartition(ctx, runUUID, partition) + err = monitoring.UpdateEndTimeForPartition(ctx, a.CatalogPool, runUUID, partition) if err != nil { return 0, err } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index 6715ff3c55..3da1286cfe 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -125,7 +125,6 @@ func APIMain(args *APIServerParams) error { } flowHandler := NewFlowRequestHandler(tc, catalogConn, taskQueue) - defer flowHandler.Close() err = killExistingHeartbeatFlows(ctx, tc, args.TemporalNamespace, taskQueue) if err != nil { diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 40a8e9147f..586c9811c3 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -3,6 +3,7 @@ package main import ( "context" "fmt" + "log/slog" "strconv" "strings" "time" @@ -116,13 +117,6 @@ func (h *FlowRequestHandler) createQrepJobEntry(ctx context.Context, return nil } -// Close closes the connection pool -func (h *FlowRequestHandler) Close() { - if h.pool != nil { - h.pool.Close() - } -} - func (h *FlowRequestHandler) CreateCDCFlow( ctx context.Context, req *protos.CreateCDCFlowRequest) (*protos.CreateCDCFlowResponse, error) { cfg := req.ConnectionConfigs @@ -244,6 +238,8 @@ func (h *FlowRequestHandler) CreateQRepFlow( if req.CreateCatalogEntry { err := h.createQrepJobEntry(ctx, req, workflowID) if err != nil { + slog.Error("unable to create flow job entry", + slog.Any("error", err), slog.String("flowName", cfg.FlowJobName)) return nil, fmt.Errorf("unable to create flow job entry: %w", err) } } @@ -258,6 +254,8 @@ func (h *FlowRequestHandler) CreateQRepFlow( // hack to facilitate migrating from existing xmin sync txid, err := strconv.ParseInt(postColon, 10, 64) if err != nil { + slog.Error("invalid xmin txid for xmin rep", + slog.Any("error", err), slog.String("flowName", cfg.FlowJobName)) return nil, fmt.Errorf("invalid xmin txid for xmin rep: %w", err) } state.LastPartition.Range = &protos.PartitionRange{Range: &protos.PartitionRange_IntRange{IntRange: &protos.IntPartitionRange{Start: txid}}} @@ -269,11 +267,15 @@ func (h *FlowRequestHandler) CreateQRepFlow( } _, err := h.temporalClient.ExecuteWorkflow(ctx, workflowOptions, workflowFn, cfg, state) if err != nil { + slog.Error("unable to start QRepFlow workflow", + slog.Any("error", err), slog.String("flowName", cfg.FlowJobName)) return nil, fmt.Errorf("unable to start QRepFlow workflow: %w", err) } err = h.updateQRepConfigInCatalog(cfg) if err != nil { + slog.Error("unable to update qrep config in catalog", + slog.Any("error", err), slog.String("flowName", cfg.FlowJobName)) return nil, fmt.Errorf("unable to update qrep config in catalog: %w", err) } @@ -308,6 +310,10 @@ func (h *FlowRequestHandler) ShutdownFlow( ctx context.Context, req *protos.ShutdownRequest, ) (*protos.ShutdownResponse, error) { + logs := slog.Group("shutdown-log", + slog.String("flowName", req.FlowJobName), + slog.String("workflowId", req.WorkflowId), + ) err := h.temporalClient.SignalWorkflow( ctx, req.WorkflowId, @@ -316,6 +322,10 @@ func (h *FlowRequestHandler) ShutdownFlow( shared.ShutdownSignal, ) if err != nil { + slog.Error("unable to signal PeerFlow workflow", + logs, + slog.Any("error", err), + ) return &protos.ShutdownResponse{ Ok: false, ErrorMessage: fmt.Sprintf("unable to signal PeerFlow workflow: %v", err), @@ -324,6 +334,10 @@ func (h *FlowRequestHandler) ShutdownFlow( err = h.waitForWorkflowClose(ctx, req.WorkflowId) if err != nil { + slog.Error("unable to wait for PeerFlow workflow to close", + logs, + slog.Any("error", err), + ) return &protos.ShutdownResponse{ Ok: false, ErrorMessage: fmt.Sprintf("unable to wait for PeerFlow workflow to close: %v", err), @@ -345,6 +359,9 @@ func (h *FlowRequestHandler) ShutdownFlow( req, // workflow input ) if err != nil { + slog.Error("unable to start DropFlow workflow", + logs, + slog.Any("error", err)) return &protos.ShutdownResponse{ Ok: false, ErrorMessage: fmt.Sprintf("unable to start DropFlow workflow: %v", err), @@ -362,6 +379,10 @@ func (h *FlowRequestHandler) ShutdownFlow( select { case err := <-errChan: if err != nil { + slog.Error("DropFlow workflow did not execute successfully", + logs, + slog.Any("error", err), + ) return &protos.ShutdownResponse{ Ok: false, ErrorMessage: fmt.Sprintf("DropFlow workflow did not execute successfully: %v", err), @@ -370,6 +391,10 @@ func (h *FlowRequestHandler) ShutdownFlow( case <-time.After(1 * time.Minute): err := h.handleWorkflowNotClosed(ctx, workflowID, "") if err != nil { + slog.Error("unable to wait for DropFlow workflow to close", + logs, + slog.Any("error", err), + ) return &protos.ShutdownResponse{ Ok: false, ErrorMessage: fmt.Sprintf("unable to wait for DropFlow workflow to close: %v", err), @@ -380,6 +405,10 @@ func (h *FlowRequestHandler) ShutdownFlow( if req.RemoveFlowEntry { delErr := h.removeFlowEntryInCatalog(req.FlowJobName) if delErr != nil { + slog.Error("unable to remove flow job entry", + slog.String("flowName", req.FlowJobName), + slog.Any("error", err), + slog.String("workflowId", req.WorkflowId)) return &protos.ShutdownResponse{ Ok: false, ErrorMessage: err.Error(), diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 31ef2a2989..1b9ced3a3c 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -10,6 +10,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" + "github.com/sirupsen/logrus" "google.golang.org/protobuf/proto" ) @@ -146,36 +147,36 @@ func (h *FlowRequestHandler) GetColumns( defer peerPool.Close() rows, err := peerPool.Query(ctx, ` - SELECT + SELECT cols.column_name, cols.data_type, - CASE + CASE WHEN constraint_type = 'PRIMARY KEY' THEN true ELSE false END AS is_primary_key - FROM + FROM information_schema.columns cols - LEFT JOIN + LEFT JOIN ( - SELECT + SELECT kcu.column_name, tc.constraint_type - FROM + FROM information_schema.key_column_usage kcu - JOIN + JOIN information_schema.table_constraints tc - ON + ON kcu.constraint_name = tc.constraint_name AND kcu.constraint_schema = tc.constraint_schema AND kcu.constraint_name = tc.constraint_name - WHERE + WHERE tc.constraint_type = 'PRIMARY KEY' AND kcu.table_schema = $1 AND kcu.table_name = $2 ) AS pk - ON + ON cols.column_name = pk.column_name - WHERE + WHERE cols.table_schema = $3 AND cols.table_name = $4; `, req.SchemaName, req.TableName, req.SchemaName, req.TableName) @@ -210,14 +211,17 @@ func (h *FlowRequestHandler) GetSlotInfo( pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) if err != nil { + logrus.Errorf("Failed to create postgres connector: %v", err) return &protos.PeerSlotResponse{SlotData: nil}, err } defer pgConnector.Close() slotInfo, err := pgConnector.GetSlotInfo("") if err != nil { + logrus.Errorf("Failed to get slot info: %v", err) return &protos.PeerSlotResponse{SlotData: nil}, err } + return &protos.PeerSlotResponse{ SlotData: slotInfo, }, nil @@ -227,16 +231,27 @@ func (h *FlowRequestHandler) GetStatInfo( ctx context.Context, req *protos.PostgresPeerActivityInfoRequest, ) (*protos.PeerStatResponse, error) { - peerPool, peerUser, err := h.getPoolForPGPeer(ctx, req.PeerName) + pgConfig, err := h.getPGPeerConfig(ctx, req.PeerName) if err != nil { return &protos.PeerStatResponse{StatData: nil}, err } - defer peerPool.Close() + + pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) + if err != nil { + logrus.Errorf("Failed to create postgres connector: %v", err) + return &protos.PeerStatResponse{StatData: nil}, err + } + defer pgConnector.Close() + + peerPool := pgConnector.GetPool() + peerUser := pgConfig.User + rows, err := peerPool.Query(ctx, "SELECT pid, wait_event, wait_event_type, query_start::text, query,"+ "EXTRACT(epoch FROM(now()-query_start)) AS dur"+ " FROM pg_stat_activity WHERE "+ "usename=$1 AND state != 'idle';", peerUser) if err != nil { + logrus.Errorf("Failed to get stat info: %v", err) return &protos.PeerStatResponse{StatData: nil}, err } defer rows.Close() @@ -251,6 +266,7 @@ func (h *FlowRequestHandler) GetStatInfo( err := rows.Scan(&pid, &waitEvent, &waitEventType, &queryStart, &query, &duration) if err != nil { + logrus.Errorf("Failed to scan row: %v", err) return &protos.PeerStatResponse{StatData: nil}, err } @@ -288,6 +304,7 @@ func (h *FlowRequestHandler) GetStatInfo( Duration: float32(d), }) } + return &protos.PeerStatResponse{ StatData: statInfoRows, }, nil diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index f7431e52cc..a9e1281619 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -10,7 +10,6 @@ import ( "github.com/PeerDB-io/peer-flow/activities" utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" - "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" @@ -108,8 +107,6 @@ func WorkerMain(opts *WorkerOptions) error { if err != nil { return fmt.Errorf("unable to create catalog connection pool: %w", err) } - catalogMirrorMonitor := monitoring.NewCatalogMirrorMonitor(conn) - defer catalogMirrorMonitor.Close() c, err := client.Dial(clientOptions) if err != nil { @@ -134,7 +131,7 @@ func WorkerMain(opts *WorkerOptions) error { w.RegisterWorkflow(peerflow.DropFlowWorkflow) w.RegisterWorkflow(peerflow.HeartbeatFlowWorkflow) w.RegisterActivity(&activities.FlowableActivity{ - CatalogMirrorMonitor: catalogMirrorMonitor, + CatalogPool: conn, }) err = w.Run(worker.InterruptCh()) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index fc63036185..fbf170fe05 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -199,7 +199,6 @@ func (c *BigQueryConnector) Close() error { if c == nil || c.client == nil { return nil } - c.catalogPool.Close() return c.client.Close() } diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 896b7fd9a7..259fe9b103 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -62,6 +62,12 @@ func (c *EventHubConnector) Close() error { allErrors = errors.Join(allErrors, err) } + err = c.hubManager.Close(context.Background()) + if err != nil { + log.Errorf("failed to close event hub manager: %v", err) + allErrors = errors.Join(allErrors, err) + } + return allErrors } diff --git a/flow/connectors/eventhub/hubmanager.go b/flow/connectors/eventhub/hubmanager.go index a30f94d162..f56c7217ab 100644 --- a/flow/connectors/eventhub/hubmanager.go +++ b/flow/connectors/eventhub/hubmanager.go @@ -2,6 +2,7 @@ package conneventhub import ( "context" + "errors" "fmt" "strings" "sync" @@ -62,6 +63,10 @@ func (m *EventHubManager) GetOrCreateHubClient(ctx context.Context, name ScopedE _, err := hubTmp.GetEventHubProperties(ctx, nil) if err != nil { log.Infof("eventhub %s not reachable. Will re-establish connection and re-create it. Err: %v", name, err) + closeError := m.closeProducerClient(ctx, hubTmp) + if closeError != nil { + log.Errorf("failed to close producer client: %v", closeError) + } m.hubs.Delete(name) hubConnectOK = false } @@ -86,6 +91,30 @@ func (m *EventHubManager) GetOrCreateHubClient(ctx context.Context, name ScopedE return hub.(*azeventhubs.ProducerClient), nil } +func (m *EventHubManager) closeProducerClient(ctx context.Context, pc *azeventhubs.ProducerClient) error { + if pc != nil { + return pc.Close(ctx) + } + return nil +} + +func (m *EventHubManager) Close(ctx context.Context) error { + var allErrors error + + m.hubs.Range(func(key any, value any) bool { + name := key.(ScopedEventhub) + hub := value.(*azeventhubs.ProducerClient) + err := m.closeProducerClient(ctx, hub) + if err != nil { + log.Errorf("failed to close eventhub client for %s: %v", name, err) + allErrors = errors.Join(allErrors, err) + } + return true + }) + + return allErrors +} + func (m *EventHubManager) CreateEventDataBatch(ctx context.Context, name ScopedEventhub) ( *azeventhubs.EventDataBatch, error) { hub, err := m.GetOrCreateHubClient(ctx, name) diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index 3d4f9baf99..637432b7fe 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -54,7 +54,7 @@ func NewPostgresMetadataStore(ctx context.Context, pgConfig *protos.PostgresConf } func (p *PostgresMetadataStore) Close() error { - if p.pool != nil { + if p.config != nil && p.pool != nil { p.pool.Close() } diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 76bafd7be6..268498b62c 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -25,8 +25,8 @@ type PostgresConnector struct { connStr string ctx context.Context config *protos.PostgresConfig - pool *pgxpool.Pool - replPool *pgxpool.Pool + pool *SSHWrappedPostgresPool + replPool *SSHWrappedPostgresPool tableSchemaMapping map[string]*protos.TableSchema customTypesMapping map[uint32]string metadataSchema string @@ -51,12 +51,12 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) // set pool size to 3 to avoid connection pool exhaustion connConfig.MaxConns = 3 - pool, err := pgxpool.NewWithConfig(ctx, connConfig) + pool, err := NewSSHWrappedPostgresPool(ctx, connConfig, pgConfig.SshConfig) if err != nil { return nil, fmt.Errorf("failed to create connection pool: %w", err) } - customTypeMap, err := utils.GetCustomDataTypes(ctx, pool) + customTypeMap, err := utils.GetCustomDataTypes(ctx, pool.Pool) if err != nil { return nil, fmt.Errorf("failed to get custom type map: %w", err) } @@ -73,7 +73,7 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) // TODO: replPool not initializing might be intentional, if we only want to use QRep mirrors // and the user doesn't have the REPLICATION permission - replPool, err := pgxpool.NewWithConfig(ctx, replConnConfig) + replPool, err := NewSSHWrappedPostgresPool(ctx, replConnConfig, pgConfig.SshConfig) if err != nil { return nil, fmt.Errorf("failed to create connection pool: %w", err) } @@ -94,6 +94,11 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) }, nil } +// GetPool returns the connection pool. +func (c *PostgresConnector) GetPool() *SSHWrappedPostgresPool { + return c.pool +} + // Close closes all connections. func (c *PostgresConnector) Close() error { if c.pool != nil { @@ -230,7 +235,7 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) error { cdc, err := NewPostgresCDCSource(&PostgresCDCConfig{ AppContext: c.ctx, - Connection: c.replPool, + Connection: c.replPool.Pool, SrcTableIDNameMapping: req.SrcTableIDNameMapping, Slot: slotName, Publication: publicationName, @@ -246,13 +251,13 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) error { return err } - cdcMirrorMonitor, ok := c.ctx.Value(shared.CDCMirrorMonitorKey).(*monitoring.CatalogMirrorMonitor) + catalogPool, ok := c.ctx.Value(shared.CDCMirrorMonitorKey).(*pgxpool.Pool) if ok { latestLSN, err := c.getCurrentLSN() if err != nil { return fmt.Errorf("failed to get current LSN: %w", err) } - err = cdcMirrorMonitor.UpdateLatestLSNAtSourceForCDCFlow(c.ctx, req.FlowJobName, latestLSN) + err = monitoring.UpdateLatestLSNAtSourceForCDCFlow(c.ctx, catalogPool, req.FlowJobName, latestLSN) if err != nil { return fmt.Errorf("failed to update latest LSN at source for CDC flow: %w", err) } diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index 6d514dccf4..2c7fa9b295 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -315,7 +315,8 @@ func (c *PostgresConnector) PullQRepRecords( log.WithFields(log.Fields{ "partitionId": partition.PartitionId, }).Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return nil, err @@ -361,7 +362,8 @@ func (c *PostgresConnector) PullQRepRecords( return nil, err } - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return nil, err @@ -386,7 +388,8 @@ func (c *PostgresConnector) PullQRepRecordStream( "flowName": config.FlowJobName, "partitionId": partition.PartitionId, }).Infof("pulling full table partition for flow job %s", config.FlowJobName) - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, err @@ -434,7 +437,8 @@ func (c *PostgresConnector) PullQRepRecordStream( return 0, err } - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, err @@ -558,7 +562,8 @@ func (c *PostgresConnector) PullXminRecordStream( query += " WHERE age(xmin) > 0 AND age(xmin) <= age($1::xid)" } - executor, err := NewQRepQueryExecutorSnapshot(c.pool, c.ctx, c.config.TransactionSnapshot, + executor, err := NewQRepQueryExecutorSnapshot( + c.pool.Pool, c.ctx, c.config.TransactionSnapshot, config.FlowJobName, partition.PartitionId) if err != nil { return 0, currentSnapshotXmin, err diff --git a/flow/connectors/postgres/qrep_partition_test.go b/flow/connectors/postgres/qrep_partition_test.go index 9eead5ec34..bc6956feec 100644 --- a/flow/connectors/postgres/qrep_partition_test.go +++ b/flow/connectors/postgres/qrep_partition_test.go @@ -70,9 +70,9 @@ func TestGetQRepPartitions(t *testing.T) { t.Fatalf("Failed to parse config: %v", err) } - pool, err := pgxpool.NewWithConfig(context.Background(), config) + pool, err := NewSSHWrappedPostgresPool(context.Background(), config, nil) if err != nil { - t.Fatalf("unable to connect to database: %v", err) + t.Fatalf("Failed to create pool: %v", err) } // Generate a random schema name @@ -101,7 +101,7 @@ func TestGetQRepPartitions(t *testing.T) { } // from 2010 Jan 1 10:00 AM UTC to 2010 Jan 30 10:00 AM UTC - numRows := prepareTestData(t, pool, schemaName) + numRows := prepareTestData(t, pool.Pool, schemaName) secondsInADay := uint32(24 * time.Hour / time.Second) fmt.Printf("secondsInADay: %d\n", secondsInADay) diff --git a/flow/connectors/postgres/ssh_wrapped_pool.go b/flow/connectors/postgres/ssh_wrapped_pool.go new file mode 100644 index 0000000000..ef19f78b32 --- /dev/null +++ b/flow/connectors/postgres/ssh_wrapped_pool.go @@ -0,0 +1,166 @@ +package connpostgres + +import ( + "context" + "fmt" + "net" + "sync" + "time" + + "github.com/PeerDB-io/peer-flow/connectors/utils" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgxpool" + "github.com/sirupsen/logrus" + "golang.org/x/crypto/ssh" +) + +type SSHWrappedPostgresPool struct { + *pgxpool.Pool + + poolConfig *pgxpool.Config + sshConfig *ssh.ClientConfig + sshServer string + once sync.Once + sshClient *ssh.Client + ctx context.Context + cancel context.CancelFunc +} + +func NewSSHWrappedPostgresPool( + ctx context.Context, + poolConfig *pgxpool.Config, + sshConfig *protos.SSHConfig, +) (*SSHWrappedPostgresPool, error) { + swCtx, cancel := context.WithCancel(ctx) + + var sshServer string + var clientConfig *ssh.ClientConfig + + 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, + ) + if err != nil { + logrus.Error("Failed to get SSH client config: ", err) + cancel() + return nil, err + } + } + + pool := &SSHWrappedPostgresPool{ + poolConfig: poolConfig, + sshConfig: clientConfig, + sshServer: sshServer, + ctx: swCtx, + cancel: cancel, + } + + err := pool.connect() + if err != nil { + return nil, err + } + + return pool, nil +} + +func (swpp *SSHWrappedPostgresPool) connect() error { + var err error + swpp.once.Do(func() { + err = swpp.setupSSH() + if err != nil { + return + } + + swpp.Pool, err = pgxpool.NewWithConfig(swpp.ctx, swpp.poolConfig) + if err != nil { + logrus.Errorf("Failed to create pool: %v", err) + return + } + + logrus.Infof("Established pool to %s:%d", + swpp.poolConfig.ConnConfig.Host, swpp.poolConfig.ConnConfig.Port) + + err = retryWithBackoff(func() error { + err = swpp.Ping(swpp.ctx) + if err != nil { + logrus.Errorf("Failed to ping pool: %v", err) + return err + } + return nil + }, 5, 5*time.Second) + + if err != nil { + logrus.Errorf("Failed to create pool: %v", err) + } + }) + + if err == nil { + logrus.Info("Successfully connected to Postgres") + } + + return err +} + +func (swpp *SSHWrappedPostgresPool) setupSSH() error { + if swpp.sshConfig == nil { + logrus.Info("SSH config is nil, skipping SSH setup") + return nil + } + + logrus.Info("Setting up SSH connection to ", swpp.sshServer) + + var err error + swpp.sshClient, err = ssh.Dial("tcp", swpp.sshServer, swpp.sshConfig) + if err != nil { + return err + } + + swpp.poolConfig.ConnConfig.DialFunc = func(ctx context.Context, network, addr string) (net.Conn, error) { + conn, err := swpp.sshClient.Dial(network, addr) + if err != nil { + return nil, err + } + return &noDeadlineConn{Conn: conn}, nil + } + + return nil +} + +func (swpp *SSHWrappedPostgresPool) Close() { + swpp.cancel() + + if swpp.Pool != nil { + swpp.Pool.Close() + } + + if swpp.sshClient != nil { + swpp.sshClient.Close() + } +} + +type retryFunc func() error + +func retryWithBackoff(fn retryFunc, maxRetries int, backoff time.Duration) (err error) { + for i := 0; i < maxRetries; i++ { + err = fn() + if err == nil { + return nil + } + if i < maxRetries-1 { + logrus.Infof("Attempt #%d failed, retrying in %s", i+1, backoff) + time.Sleep(backoff) + } + } + return err +} + +// see: https://github.com/jackc/pgx/issues/382#issuecomment-1496586216 +type noDeadlineConn struct{ net.Conn } + +func (c *noDeadlineConn) SetDeadline(t time.Time) error { return nil } +func (c *noDeadlineConn) SetReadDeadline(t time.Time) error { return nil } +func (c *noDeadlineConn) SetWriteDeadline(t time.Time) error { return nil } diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go index e1bd33d25a..0563f9201b 100644 --- a/flow/connectors/utils/catalog/env.go +++ b/flow/connectors/utils/catalog/env.go @@ -5,29 +5,37 @@ import ( "fmt" "os" "strconv" + "sync" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5/pgxpool" ) +var poolMutex = &sync.Mutex{} +var pool *pgxpool.Pool + func GetCatalogConnectionPoolFromEnv() (*pgxpool.Pool, error) { - catalogConnectionString, err := genCatalogConnectionString() - if err != nil { - return nil, fmt.Errorf("unable to generate catalog connection string: %w", err) - } + poolMutex.Lock() + defer poolMutex.Unlock() + if pool == nil { + catalogConnectionString, err := genCatalogConnectionString() + if err != nil { + return nil, fmt.Errorf("unable to generate catalog connection string: %w", err) + } - catalogConn, err := pgxpool.New(context.Background(), catalogConnectionString) - if err != nil { - return nil, fmt.Errorf("unable to establish connection with catalog: %w", err) + pool, err = pgxpool.New(context.Background(), catalogConnectionString) + if err != nil { + return nil, fmt.Errorf("unable to establish connection with catalog: %w", err) + } } - err = catalogConn.Ping(context.Background()) + err := pool.Ping(context.Background()) if err != nil { - return nil, fmt.Errorf("unable to establish connection with catalog: %w", err) + return pool, fmt.Errorf("unable to establish connection with catalog: %w", err) } - return catalogConn, nil + return pool, nil } func genCatalogConnectionString() (string, error) { diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 99415cd6da..59d8d5a82a 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -14,10 +14,6 @@ import ( "google.golang.org/protobuf/proto" ) -type CatalogMirrorMonitor struct { - catalogConn *pgxpool.Pool -} - type CDCBatchInfo struct { BatchID int64 RowsInBatch uint32 @@ -26,29 +22,8 @@ type CDCBatchInfo struct { StartTime time.Time } -func NewCatalogMirrorMonitor(catalogConn *pgxpool.Pool) *CatalogMirrorMonitor { - return &CatalogMirrorMonitor{ - catalogConn: catalogConn, - } -} - -func (c *CatalogMirrorMonitor) IsActive() bool { - return !(c == nil || c.catalogConn == nil) -} - -func (c *CatalogMirrorMonitor) Close() { - if c == nil || c.catalogConn == nil { - return - } - c.catalogConn.Close() -} - -func (c *CatalogMirrorMonitor) InitializeCDCFlow(ctx context.Context, flowJobName string) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, +func InitializeCDCFlow(ctx context.Context, pool *pgxpool.Pool, flowJobName string) error { + _, err := pool.Exec(ctx, `INSERT INTO peerdb_stats.cdc_flows(flow_name,latest_lsn_at_source,latest_lsn_at_target) VALUES($1,0,0) ON CONFLICT DO NOTHING`, flowJobName) if err != nil { @@ -57,13 +32,9 @@ func (c *CatalogMirrorMonitor) InitializeCDCFlow(ctx context.Context, flowJobNam return nil } -func (c *CatalogMirrorMonitor) UpdateLatestLSNAtSourceForCDCFlow(ctx context.Context, flowJobName string, +func UpdateLatestLSNAtSourceForCDCFlow(ctx context.Context, pool *pgxpool.Pool, flowJobName string, latestLSNAtSource pglogrepl.LSN) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, "UPDATE peerdb_stats.cdc_flows SET latest_lsn_at_source=$1 WHERE flow_name=$2", uint64(latestLSNAtSource), flowJobName) if err != nil { @@ -72,13 +43,9 @@ func (c *CatalogMirrorMonitor) UpdateLatestLSNAtSourceForCDCFlow(ctx context.Con return nil } -func (c *CatalogMirrorMonitor) UpdateLatestLSNAtTargetForCDCFlow(ctx context.Context, flowJobName string, +func UpdateLatestLSNAtTargetForCDCFlow(ctx context.Context, pool *pgxpool.Pool, flowJobName string, latestLSNAtTarget pglogrepl.LSN) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, "UPDATE peerdb_stats.cdc_flows SET latest_lsn_at_target=$1 WHERE flow_name=$2", uint64(latestLSNAtTarget), flowJobName) if err != nil { @@ -87,13 +54,9 @@ func (c *CatalogMirrorMonitor) UpdateLatestLSNAtTargetForCDCFlow(ctx context.Con return nil } -func (c *CatalogMirrorMonitor) AddCDCBatchForFlow(ctx context.Context, flowJobName string, +func AddCDCBatchForFlow(ctx context.Context, pool *pgxpool.Pool, flowJobName string, batchInfo CDCBatchInfo) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, `INSERT INTO peerdb_stats.cdc_batches(flow_name,batch_id,rows_in_batch,batch_start_lsn,batch_end_lsn, start_time) VALUES($1,$2,$3,$4,$5,$6) ON CONFLICT DO NOTHING`, flowJobName, batchInfo.BatchID, batchInfo.RowsInBatch, @@ -105,18 +68,15 @@ func (c *CatalogMirrorMonitor) AddCDCBatchForFlow(ctx context.Context, flowJobNa } // update num records and end-lsn for a cdc batch -func (c *CatalogMirrorMonitor) UpdateNumRowsAndEndLSNForCDCBatch( +func UpdateNumRowsAndEndLSNForCDCBatch( ctx context.Context, + pool *pgxpool.Pool, flowJobName string, batchID int64, numRows uint32, batchEndLSN pglogrepl.LSN, ) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, "UPDATE peerdb_stats.cdc_batches SET rows_in_batch=$1,batch_end_lsn=$2 WHERE flow_name=$3 AND batch_id=$4", numRows, uint64(batchEndLSN), flowJobName, batchID) if err != nil { @@ -125,16 +85,13 @@ func (c *CatalogMirrorMonitor) UpdateNumRowsAndEndLSNForCDCBatch( return nil } -func (c *CatalogMirrorMonitor) UpdateEndTimeForCDCBatch( +func UpdateEndTimeForCDCBatch( ctx context.Context, + pool *pgxpool.Pool, flowJobName string, batchID int64, ) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, "UPDATE peerdb_stats.cdc_batches SET end_time=$1 WHERE flow_name=$2 AND batch_id=$3", time.Now(), flowJobName, batchID) if err != nil { @@ -143,13 +100,9 @@ func (c *CatalogMirrorMonitor) UpdateEndTimeForCDCBatch( return nil } -func (c *CatalogMirrorMonitor) AddCDCBatchTablesForFlow(ctx context.Context, flowJobName string, +func AddCDCBatchTablesForFlow(ctx context.Context, pool *pgxpool.Pool, flowJobName string, batchID int64, tableNameRowsMapping map[string]uint32) error { - if c == nil || c.catalogConn == nil { - return nil - } - - insertBatchTablesTx, err := c.catalogConn.Begin(ctx) + insertBatchTablesTx, err := pool.Begin(ctx) if err != nil { return fmt.Errorf("error while beginning transaction for inserting statistics into cdc_batch_table: %w", err) } @@ -177,18 +130,15 @@ func (c *CatalogMirrorMonitor) AddCDCBatchTablesForFlow(ctx context.Context, flo return nil } -func (c *CatalogMirrorMonitor) InitializeQRepRun( +func InitializeQRepRun( ctx context.Context, + pool *pgxpool.Pool, config *protos.QRepConfig, runUUID string, partitions []*protos.QRepPartition, ) error { - if c == nil || c.catalogConn == nil { - return nil - } - flowJobName := config.GetFlowJobName() - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, "INSERT INTO peerdb_stats.qrep_runs(flow_name,run_uuid) VALUES($1,$2) ON CONFLICT DO NOTHING", flowJobName, runUUID) if err != nil { @@ -200,7 +150,7 @@ func (c *CatalogMirrorMonitor) InitializeQRepRun( return fmt.Errorf("unable to marshal flow config: %w", err) } - _, err = c.catalogConn.Exec(ctx, + _, err = pool.Exec(ctx, "UPDATE peerdb_stats.qrep_runs SET config_proto = $1 WHERE flow_name = $2", cfgBytes, flowJobName) if err != nil { @@ -208,7 +158,7 @@ func (c *CatalogMirrorMonitor) InitializeQRepRun( } for _, partition := range partitions { - if err := c.addPartitionToQRepRun(ctx, flowJobName, runUUID, partition); err != nil { + if err := addPartitionToQRepRun(ctx, pool, flowJobName, runUUID, partition); err != nil { return fmt.Errorf("unable to add partition to qrep run: %w", err) } } @@ -216,12 +166,8 @@ func (c *CatalogMirrorMonitor) InitializeQRepRun( return nil } -func (c *CatalogMirrorMonitor) UpdateStartTimeForQRepRun(ctx context.Context, runUUID string) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, +func UpdateStartTimeForQRepRun(ctx context.Context, pool *pgxpool.Pool, runUUID string) error { + _, err := pool.Exec(ctx, "UPDATE peerdb_stats.qrep_runs SET start_time=$1 WHERE run_uuid=$2", time.Now(), runUUID) if err != nil { @@ -231,12 +177,8 @@ func (c *CatalogMirrorMonitor) UpdateStartTimeForQRepRun(ctx context.Context, ru return nil } -func (c *CatalogMirrorMonitor) UpdateEndTimeForQRepRun(ctx context.Context, runUUID string) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, +func UpdateEndTimeForQRepRun(ctx context.Context, pool *pgxpool.Pool, runUUID string) error { + _, err := pool.Exec(ctx, "UPDATE peerdb_stats.qrep_runs SET end_time=$1 WHERE run_uuid=$2", time.Now(), runUUID) if err != nil { @@ -246,16 +188,13 @@ func (c *CatalogMirrorMonitor) UpdateEndTimeForQRepRun(ctx context.Context, runU return nil } -func (c *CatalogMirrorMonitor) AppendSlotSizeInfo( +func AppendSlotSizeInfo( ctx context.Context, + pool *pgxpool.Pool, peerName string, slotInfo *protos.SlotInfo, ) error { - if c == nil || c.catalogConn == nil || slotInfo == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, "INSERT INTO peerdb_stats.peer_slot_size"+ "(peer_name, slot_name, restart_lsn, redo_lsn, confirmed_flush_lsn, slot_size, wal_status) "+ "VALUES($1,$2,$3,$4,$5,$6) ON CONFLICT DO NOTHING;", @@ -274,12 +213,8 @@ func (c *CatalogMirrorMonitor) AppendSlotSizeInfo( return nil } -func (c *CatalogMirrorMonitor) addPartitionToQRepRun(ctx context.Context, flowJobName string, +func addPartitionToQRepRun(ctx context.Context, pool *pgxpool.Pool, flowJobName string, runUUID string, partition *protos.QRepPartition) error { - if c == nil || c.catalogConn == nil { - return nil - } - if partition.Range == nil && partition.FullTablePartition { log.Infof("partition %s is a full table partition. Metrics logging is skipped.", partition.PartitionId) return nil @@ -317,7 +252,7 @@ func (c *CatalogMirrorMonitor) addPartitionToQRepRun(ctx context.Context, flowJo return fmt.Errorf("unknown range type: %v", x) } - _, err := c.catalogConn.Exec(ctx, + _, err := pool.Exec(ctx, `INSERT INTO peerdb_stats.qrep_partitions (flow_name,run_uuid,partition_uuid,partition_start,partition_end,restart_count) VALUES($1,$2,$3,$4,$5,$6) ON CONFLICT(run_uuid,partition_uuid) DO UPDATE SET @@ -330,17 +265,14 @@ func (c *CatalogMirrorMonitor) addPartitionToQRepRun(ctx context.Context, flowJo return nil } -func (c *CatalogMirrorMonitor) UpdateStartTimeForPartition( +func UpdateStartTimeForPartition( ctx context.Context, + pool *pgxpool.Pool, runUUID string, partition *protos.QRepPartition, startTime time.Time, ) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET start_time=$1 + _, err := pool.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET start_time=$1 WHERE run_uuid=$2 AND partition_uuid=$3`, startTime, runUUID, partition.PartitionId) if err != nil { return fmt.Errorf("error while updating qrep partition in qrep_partitions: %w", err) @@ -348,13 +280,9 @@ func (c *CatalogMirrorMonitor) UpdateStartTimeForPartition( return nil } -func (c *CatalogMirrorMonitor) UpdatePullEndTimeAndRowsForPartition(ctx context.Context, runUUID string, +func UpdatePullEndTimeAndRowsForPartition(ctx context.Context, pool *pgxpool.Pool, runUUID string, partition *protos.QRepPartition, rowsInPartition int64) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET pull_end_time=$1,rows_in_partition=$2 + _, err := pool.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET pull_end_time=$1,rows_in_partition=$2 WHERE run_uuid=$3 AND partition_uuid=$4`, time.Now(), rowsInPartition, runUUID, partition.PartitionId) if err != nil { return fmt.Errorf("error while updating qrep partition in qrep_partitions: %w", err) @@ -362,13 +290,9 @@ func (c *CatalogMirrorMonitor) UpdatePullEndTimeAndRowsForPartition(ctx context. return nil } -func (c *CatalogMirrorMonitor) UpdateEndTimeForPartition(ctx context.Context, runUUID string, +func UpdateEndTimeForPartition(ctx context.Context, pool *pgxpool.Pool, runUUID string, partition *protos.QRepPartition) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET end_time=$1 + _, err := pool.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET end_time=$1 WHERE run_uuid=$2 AND partition_uuid=$3`, time.Now(), runUUID, partition.PartitionId) if err != nil { return fmt.Errorf("error while updating qrep partition in qrep_partitions: %w", err) @@ -376,13 +300,9 @@ func (c *CatalogMirrorMonitor) UpdateEndTimeForPartition(ctx context.Context, ru return nil } -func (c *CatalogMirrorMonitor) UpdateRowsSyncedForPartition(ctx context.Context, rowsSynced int, runUUID string, +func UpdateRowsSyncedForPartition(ctx context.Context, pool *pgxpool.Pool, rowsSynced int, runUUID string, partition *protos.QRepPartition) error { - if c == nil || c.catalogConn == nil { - return nil - } - - _, err := c.catalogConn.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET rows_synced=$1 + _, err := pool.Exec(ctx, `UPDATE peerdb_stats.qrep_partitions SET rows_synced=$1 WHERE run_uuid=$2 AND partition_uuid=$3`, rowsSynced, runUUID, partition.PartitionId) if err != nil { return fmt.Errorf("error while updating rows_synced in qrep_partitions: %w", err) diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go new file mode 100644 index 0000000000..7bd8ed141f --- /dev/null +++ b/flow/connectors/utils/ssh.go @@ -0,0 +1,48 @@ +package utils + +import ( + "encoding/base64" + "fmt" + + "golang.org/x/crypto/ssh" +) + +// getSSHClientConfig returns an *ssh.ClientConfig based on provided credentials. +// Parameters: +// +// 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) { + var authMethods []ssh.AuthMethod + + // Password-based authentication + if password != "" { + authMethods = append(authMethods, ssh.Password(password)) + } + + // Private key-based authentication + if privateKeyString != "" { + pkey, err := base64.StdEncoding.DecodeString(privateKeyString) + if err != nil { + return nil, fmt.Errorf("failed to base64 decode private key: %w", err) + } + + signer, err := ssh.ParsePrivateKey(pkey) + if err != nil { + return nil, fmt.Errorf("failed to parse private key: %w", err) + } + + authMethods = append(authMethods, ssh.PublicKeys(signer)) + } + + if len(authMethods) == 0 { + return nil, fmt.Errorf("no authentication methods provided") + } + + return &ssh.ClientConfig{ + User: user, + Auth: authMethods, + HostKeyCallback: ssh.InsecureIgnoreHostKey(), + }, nil +} diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 559f03ba62..d642d8eed1 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -117,7 +117,7 @@ func (s PeerFlowE2ETestSuiteBQ) tearDownSuite() { func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Connection_Config() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) // TODO (kaushikiska): ensure flow name can only be alpha numeric and underscores. limits := peerflow.CDCFlowLimits{ @@ -139,7 +139,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Invalid_Connection_Config() { func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_no_data") dstTableName := "test_no_data" @@ -183,7 +183,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Flow_No_Data() { func (s PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_char_coltype") dstTableName := "test_char_coltype" @@ -230,7 +230,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Char_ColType_Error() { // correctly synced to the destination table after sync flow completes. func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_simple_flow_bq") dstTableName := "test_simple_flow_bq" @@ -297,7 +297,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_1") dstTableName := "test_toast_bq_1" @@ -365,7 +365,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_2") dstTableName := "test_toast_bq_2" @@ -398,6 +398,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup // and execute a transaction touching toast columns + done := make(chan struct{}) go func() { e2e.SetupCDCFlowStatusQuery(env, connectionGen) /* transaction updating no rows */ @@ -409,6 +410,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { `, srcTableName, srcTableName)) require.NoError(s.t, err) fmt.Println("Executed a transaction touching toast columns") + done <- struct{}{} }() env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) @@ -422,11 +424,12 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Nochanges_BQ() { s.compareTableContentsBQ(dstTableName, "id,t1,t2,k") env.AssertExpectations(s.t) + <-done } func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_3") dstTableName := "test_toast_bq_3" @@ -500,7 +503,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_1_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_4") dstTableName := "test_toast_bq_4" @@ -567,7 +570,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_2_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_bq_5") dstTableName := "test_toast_bq_5" @@ -634,7 +637,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Toast_Advance_3_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_types_bq") dstTableName := "test_types_bq" @@ -712,7 +715,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Types_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTable1Name := s.attachSchemaSuffix("test1_bq") dstTable1Name := "test1_bq" @@ -774,7 +777,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { // TODO: not checking schema exactly, add later func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := "test_simple_schema_changes" @@ -874,7 +877,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := "test_simple_cpkey" @@ -947,7 +950,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := "test_cpkey_toast1" @@ -1024,7 +1027,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") dstTableName := "test_cpkey_toast2" diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index c12634cec8..f520014b04 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -48,7 +48,7 @@ func (s PeerFlowE2ETestSuiteBQ) compareTableContentsBQ(tableName string, colsStr func (s PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) numRows := 10 diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index 3f5718bb72..2720891fb6 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -20,7 +20,7 @@ func (s *PeerFlowE2ETestSuitePG) attachSuffix(input string) string { func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) srcTableName := s.attachSchemaSuffix("test_simple_flow") dstTableName := s.attachSchemaSuffix("test_simple_flow_dst") @@ -83,7 +83,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := s.attachSchemaSuffix("test_simple_schema_changes_dst") @@ -244,7 +244,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := s.attachSchemaSuffix("test_simple_cpkey_dst") @@ -319,7 +319,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := s.attachSchemaSuffix("test_cpkey_toast1_dst") @@ -400,7 +400,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") dstTableName := s.attachSchemaSuffix("test_cpkey_toast2_dst") diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index c9b82a4c6c..9ca4db2a79 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -138,7 +138,7 @@ func (s *PeerFlowE2ETestSuitePG) compareQuery(srcSchemaQualified, dstSchemaQuali func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) numRows := 10 diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index 85993e2fe8..478877ef4b 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -20,7 +20,7 @@ func (s *PeerFlowE2ETestSuiteS3) attachSuffix(input string) string { func (s *PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) srcTableName := s.attachSchemaSuffix("test_simple_flow_s3") dstTableName := fmt.Sprintf("%s.%s", "peerdb_test_s3", "test_simple_flow_s3") @@ -88,7 +88,7 @@ func (s *PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { func (s *PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_GCS_Interop() { env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) setupErr := s.setupS3("gcs") if setupErr != nil { s.Fail("failed to setup S3", setupErr) diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index d807cf6335..810b25aa42 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -93,7 +93,7 @@ func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { } env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) jobName := "test_complete_flow_s3" schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s3Suffix, jobName) @@ -140,7 +140,7 @@ func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { } env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) jobName := "test_complete_flow_s3_ctid" schemaQualifiedName := fmt.Sprintf("e2e_test_%s.%s", s3Suffix, jobName) diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 759ab34b05..1960c03ae5 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -128,7 +128,7 @@ func (s PeerFlowE2ETestSuiteSF) tearDownSuite() { func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_simple_flow_sf") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_flow_sf") @@ -203,7 +203,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_invalid_geo_sf_avro_cdc") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_invalid_geo_sf_avro_cdc") @@ -288,7 +288,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { func (s PeerFlowE2ETestSuiteSF) Test_Toast_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_1") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_1") @@ -355,7 +355,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_2") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_2") @@ -424,7 +424,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Nochanges_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_3") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_3") @@ -497,7 +497,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_1_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_4") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_4") @@ -563,7 +563,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_2_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_toast_sf_5") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_toast_sf_5") @@ -629,7 +629,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Toast_Advance_3_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_types_sf") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_types_sf") @@ -706,7 +706,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Types_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTable1Name := s.attachSchemaSuffix("test1_sf") srcTable2Name := s.attachSchemaSuffix("test2_sf") @@ -765,7 +765,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_schema_changes") @@ -925,7 +925,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_simple_cpkey") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_cpkey") @@ -998,7 +998,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_cpkey_toast1") @@ -1074,7 +1074,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_cpkey_toast2") @@ -1146,7 +1146,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_exclude_sf") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_exclude_sf") @@ -1228,7 +1228,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) cmpTableName := s.attachSchemaSuffix("test_softdel") srcTableName := fmt.Sprintf("%s_src", cmpTableName) @@ -1314,7 +1314,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) cmpTableName := s.attachSchemaSuffix("test_softdel_iud") srcTableName := fmt.Sprintf("%s_src", cmpTableName) @@ -1396,7 +1396,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) cmpTableName := s.attachSchemaSuffix("test_softdel_ud") srcTableName := fmt.Sprintf("%s_src", cmpTableName) @@ -1482,7 +1482,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) srcTableName := s.attachSchemaSuffix("test_softdel_iad") dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_softdel_iad") diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index 81884d2e3d..a541be9283 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -58,7 +58,7 @@ func (s PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, selecto func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) numRows := 10 @@ -97,7 +97,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) numRows := 10 @@ -140,7 +140,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) numRows := 10 @@ -180,7 +180,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_XMIN() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) numRows := 10 @@ -224,7 +224,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_XMIN() { func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() { env := e2e.NewTemporalTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.t) numRows := 10 diff --git a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go index 013685a364..8ea5370ee1 100644 --- a/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go +++ b/flow/e2e/sqlserver/qrep_flow_sqlserver_test.go @@ -137,7 +137,7 @@ func (s *PeerFlowE2ETestSuiteSQLServer) Test_Complete_QRep_Flow_SqlServer_Append } env := s.NewTestWorkflowEnvironment() - e2e.RegisterWorkflowsAndActivities(env) + e2e.RegisterWorkflowsAndActivities(env, s.T()) numRows := 10 tblName := "test_qrep_flow_avro_ss_append" diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index c16040062a..e4652b4854 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -7,9 +7,11 @@ import ( "io" "os" "strings" + "testing" "time" "github.com/PeerDB-io/peer-flow/activities" + utils "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" @@ -39,7 +41,12 @@ func ReadFileToBytes(path string) ([]byte, error) { return ret, nil } -func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { +func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment, t *testing.T) { + conn, err := utils.GetCatalogConnectionPoolFromEnv() + if err != nil { + t.Fatalf("unable to create catalog connection pool: %v", err) + } + // set a 300 second timeout for the workflow to execute a few runs. env.SetTestTimeout(300 * time.Second) @@ -51,7 +58,7 @@ func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment) { env.RegisterWorkflow(peerflow.QRepFlowWorkflow) env.RegisterWorkflow(peerflow.XminFlowWorkflow) env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) - env.RegisterActivity(&activities.FlowableActivity{}) + env.RegisterActivity(&activities.FlowableActivity{CatalogPool: conn}) env.RegisterActivity(&activities.SnapshotActivity{}) } diff --git a/flow/generated/protos/peers.pb.go b/flow/generated/protos/peers.pb.go index 6ac88a4c11..aa03dcfd82 100644 --- a/flow/generated/protos/peers.pb.go +++ b/flow/generated/protos/peers.pb.go @@ -84,6 +84,85 @@ func (DBType) EnumDescriptor() ([]byte, []int) { return file_peers_proto_rawDescGZIP(), []int{0} } +type SSHConfig struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Host string `protobuf:"bytes,1,opt,name=host,proto3" json:"host,omitempty"` + Port uint32 `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"` + User string `protobuf:"bytes,3,opt,name=user,proto3" json:"user,omitempty"` + Password string `protobuf:"bytes,4,opt,name=password,proto3" json:"password,omitempty"` + PrivateKey string `protobuf:"bytes,5,opt,name=private_key,json=privateKey,proto3" json:"private_key,omitempty"` +} + +func (x *SSHConfig) Reset() { + *x = SSHConfig{} + if protoimpl.UnsafeEnabled { + mi := &file_peers_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SSHConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SSHConfig) ProtoMessage() {} + +func (x *SSHConfig) ProtoReflect() protoreflect.Message { + mi := &file_peers_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SSHConfig.ProtoReflect.Descriptor instead. +func (*SSHConfig) Descriptor() ([]byte, []int) { + return file_peers_proto_rawDescGZIP(), []int{0} +} + +func (x *SSHConfig) GetHost() string { + if x != nil { + return x.Host + } + return "" +} + +func (x *SSHConfig) GetPort() uint32 { + if x != nil { + return x.Port + } + return 0 +} + +func (x *SSHConfig) GetUser() string { + if x != nil { + return x.User + } + return "" +} + +func (x *SSHConfig) GetPassword() string { + if x != nil { + return x.Password + } + return "" +} + +func (x *SSHConfig) GetPrivateKey() string { + if x != nil { + return x.PrivateKey + } + return "" +} + type SnowflakeConfig struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -105,7 +184,7 @@ type SnowflakeConfig struct { func (x *SnowflakeConfig) Reset() { *x = SnowflakeConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[0] + mi := &file_peers_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -118,7 +197,7 @@ func (x *SnowflakeConfig) String() string { func (*SnowflakeConfig) ProtoMessage() {} func (x *SnowflakeConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[0] + mi := &file_peers_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -131,7 +210,7 @@ func (x *SnowflakeConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use SnowflakeConfig.ProtoReflect.Descriptor instead. func (*SnowflakeConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{0} + return file_peers_proto_rawDescGZIP(), []int{1} } func (x *SnowflakeConfig) GetAccountId() string { @@ -225,7 +304,7 @@ type BigqueryConfig struct { func (x *BigqueryConfig) Reset() { *x = BigqueryConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[1] + mi := &file_peers_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -238,7 +317,7 @@ func (x *BigqueryConfig) String() string { func (*BigqueryConfig) ProtoMessage() {} func (x *BigqueryConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[1] + mi := &file_peers_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -251,7 +330,7 @@ func (x *BigqueryConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use BigqueryConfig.ProtoReflect.Descriptor instead. func (*BigqueryConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{1} + return file_peers_proto_rawDescGZIP(), []int{2} } func (x *BigqueryConfig) GetAuthType() string { @@ -346,7 +425,7 @@ type MongoConfig struct { func (x *MongoConfig) Reset() { *x = MongoConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[2] + mi := &file_peers_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -359,7 +438,7 @@ func (x *MongoConfig) String() string { func (*MongoConfig) ProtoMessage() {} func (x *MongoConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[2] + mi := &file_peers_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -372,7 +451,7 @@ func (x *MongoConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use MongoConfig.ProtoReflect.Descriptor instead. func (*MongoConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{2} + return file_peers_proto_rawDescGZIP(), []int{3} } func (x *MongoConfig) GetUsername() string { @@ -423,13 +502,14 @@ type PostgresConfig struct { // this is used only in query replication mode right now. TransactionSnapshot string `protobuf:"bytes,6,opt,name=transaction_snapshot,json=transactionSnapshot,proto3" json:"transaction_snapshot,omitempty"` // defaults to _peerdb_internal - MetadataSchema *string `protobuf:"bytes,7,opt,name=metadata_schema,json=metadataSchema,proto3,oneof" json:"metadata_schema,omitempty"` + MetadataSchema *string `protobuf:"bytes,7,opt,name=metadata_schema,json=metadataSchema,proto3,oneof" json:"metadata_schema,omitempty"` + SshConfig *SSHConfig `protobuf:"bytes,8,opt,name=ssh_config,json=sshConfig,proto3,oneof" json:"ssh_config,omitempty"` } func (x *PostgresConfig) Reset() { *x = PostgresConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[3] + mi := &file_peers_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -442,7 +522,7 @@ func (x *PostgresConfig) String() string { func (*PostgresConfig) ProtoMessage() {} func (x *PostgresConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[3] + mi := &file_peers_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -455,7 +535,7 @@ func (x *PostgresConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use PostgresConfig.ProtoReflect.Descriptor instead. func (*PostgresConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{3} + return file_peers_proto_rawDescGZIP(), []int{4} } func (x *PostgresConfig) GetHost() string { @@ -507,6 +587,13 @@ func (x *PostgresConfig) GetMetadataSchema() string { return "" } +func (x *PostgresConfig) GetSshConfig() *SSHConfig { + if x != nil { + return x.SshConfig + } + return nil +} + type EventHubConfig struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -527,7 +614,7 @@ type EventHubConfig struct { func (x *EventHubConfig) Reset() { *x = EventHubConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[4] + mi := &file_peers_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -540,7 +627,7 @@ func (x *EventHubConfig) String() string { func (*EventHubConfig) ProtoMessage() {} func (x *EventHubConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[4] + mi := &file_peers_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -553,7 +640,7 @@ func (x *EventHubConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use EventHubConfig.ProtoReflect.Descriptor instead. func (*EventHubConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{4} + return file_peers_proto_rawDescGZIP(), []int{5} } func (x *EventHubConfig) GetNamespace() string { @@ -619,7 +706,7 @@ type EventHubGroupConfig struct { func (x *EventHubGroupConfig) Reset() { *x = EventHubGroupConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[5] + mi := &file_peers_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -632,7 +719,7 @@ func (x *EventHubGroupConfig) String() string { func (*EventHubGroupConfig) ProtoMessage() {} func (x *EventHubGroupConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[5] + mi := &file_peers_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -645,7 +732,7 @@ func (x *EventHubGroupConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use EventHubGroupConfig.ProtoReflect.Descriptor instead. func (*EventHubGroupConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{5} + return file_peers_proto_rawDescGZIP(), []int{6} } func (x *EventHubGroupConfig) GetEventhubs() map[string]*EventHubConfig { @@ -686,7 +773,7 @@ type S3Config struct { func (x *S3Config) Reset() { *x = S3Config{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[6] + mi := &file_peers_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -699,7 +786,7 @@ func (x *S3Config) String() string { func (*S3Config) ProtoMessage() {} func (x *S3Config) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[6] + mi := &file_peers_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -712,7 +799,7 @@ func (x *S3Config) ProtoReflect() protoreflect.Message { // Deprecated: Use S3Config.ProtoReflect.Descriptor instead. func (*S3Config) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{6} + return file_peers_proto_rawDescGZIP(), []int{7} } func (x *S3Config) GetUrl() string { @@ -779,7 +866,7 @@ type SqlServerConfig struct { func (x *SqlServerConfig) Reset() { *x = SqlServerConfig{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[7] + mi := &file_peers_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -792,7 +879,7 @@ func (x *SqlServerConfig) String() string { func (*SqlServerConfig) ProtoMessage() {} func (x *SqlServerConfig) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[7] + mi := &file_peers_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -805,7 +892,7 @@ func (x *SqlServerConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use SqlServerConfig.ProtoReflect.Descriptor instead. func (*SqlServerConfig) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{7} + return file_peers_proto_rawDescGZIP(), []int{8} } func (x *SqlServerConfig) GetServer() string { @@ -866,7 +953,7 @@ type Peer struct { func (x *Peer) Reset() { *x = Peer{} if protoimpl.UnsafeEnabled { - mi := &file_peers_proto_msgTypes[8] + mi := &file_peers_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -879,7 +966,7 @@ func (x *Peer) String() string { func (*Peer) ProtoMessage() {} func (x *Peer) ProtoReflect() protoreflect.Message { - mi := &file_peers_proto_msgTypes[8] + mi := &file_peers_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -892,7 +979,7 @@ func (x *Peer) ProtoReflect() protoreflect.Message { // Deprecated: Use Peer.ProtoReflect.Descriptor instead. func (*Peer) Descriptor() ([]byte, []int) { - return file_peers_proto_rawDescGZIP(), []int{8} + return file_peers_proto_rawDescGZIP(), []int{9} } func (x *Peer) GetName() string { @@ -1028,209 +1115,223 @@ var File_peers_proto protoreflect.FileDescriptor var file_peers_proto_rawDesc = []byte{ 0x0a, 0x0b, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0c, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x22, 0xf7, 0x02, 0x0a, 0x0f, - 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, - 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x77, 0x61, 0x72, 0x65, 0x68, - 0x6f, 0x75, 0x73, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x77, 0x61, 0x72, 0x65, - 0x68, 0x6f, 0x75, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, - 0x0a, 0x0e, 0x73, 0x33, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x33, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, - 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, - 0x6f, 0x72, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x48, - 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x88, 0x01, 0x01, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, - 0x64, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x99, 0x03, 0x0a, 0x0e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1b, 0x0a, 0x09, 0x61, 0x75, 0x74, 0x68, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x75, 0x74, - 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, - 0x63, 0x74, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, - 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, - 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x6d, 0x61, 0x69, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x12, 0x1b, - 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x61, - 0x75, 0x74, 0x68, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, - 0x75, 0x74, 0x68, 0x55, 0x72, 0x69, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x5f, - 0x75, 0x72, 0x69, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x6f, 0x6b, 0x65, 0x6e, - 0x55, 0x72, 0x69, 0x12, 0x3c, 0x0a, 0x1b, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x70, 0x72, 0x6f, 0x76, - 0x69, 0x64, 0x65, 0x72, 0x5f, 0x78, 0x35, 0x30, 0x39, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, - 0x72, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x61, 0x75, 0x74, 0x68, 0x50, 0x72, - 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x58, 0x35, 0x30, 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, 0x72, - 0x6c, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x78, 0x35, 0x30, 0x39, - 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x11, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x58, 0x35, 0x30, 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, - 0x72, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x49, - 0x64, 0x22, 0xa3, 0x01, 0x0a, 0x0b, 0x4d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, - 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6c, 0x75, - 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, - 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0xf9, 0x01, 0x0a, 0x0e, 0x50, 0x6f, 0x73, 0x74, - 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, - 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, - 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, - 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, - 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, - 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x31, - 0x0a, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6e, - 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x72, - 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, - 0x74, 0x12, 0x2c, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0e, 0x6d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x42, - 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, - 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1a, 0x0a, 0x08, 0x6c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x09, + 0x53, 0x53, 0x48, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, 0x0a, + 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, 0x72, + 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, + 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, + 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x22, 0xf7, 0x02, 0x0a, 0x0f, 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, + 0x0a, 0x09, 0x77, 0x61, 0x72, 0x65, 0x68, 0x6f, 0x75, 0x73, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x77, 0x61, 0x72, 0x65, 0x68, 0x6f, 0x75, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, + 0x72, 0x6f, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6c, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x33, 0x5f, 0x69, 0x6e, 0x74, 0x65, + 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x73, + 0x33, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x08, + 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, + 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, + 0x0f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x42, 0x0b, 0x0a, 0x09, 0x5f, + 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x99, 0x03, 0x0a, + 0x0e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, + 0x1b, 0x0a, 0x09, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x61, 0x75, 0x74, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x0a, + 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x70, + 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x49, + 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, + 0x65, 0x79, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x6d, 0x61, + 0x69, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x45, 0x6d, 0x61, 0x69, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x75, 0x74, 0x68, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x75, 0x74, 0x68, 0x55, 0x72, 0x69, 0x12, 0x1b, 0x0a, + 0x09, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x5f, 0x75, 0x72, 0x69, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x55, 0x72, 0x69, 0x12, 0x3c, 0x0a, 0x1b, 0x61, 0x75, + 0x74, 0x68, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x5f, 0x78, 0x35, 0x30, 0x39, + 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x17, 0x61, 0x75, 0x74, 0x68, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x58, 0x35, 0x30, + 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x2f, 0x0a, 0x14, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x5f, 0x78, 0x35, 0x30, 0x39, 0x5f, 0x63, 0x65, 0x72, 0x74, 0x5f, 0x75, 0x72, 0x6c, + 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x58, 0x35, + 0x30, 0x39, 0x43, 0x65, 0x72, 0x74, 0x55, 0x72, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x61, 0x74, + 0x61, 0x73, 0x65, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, + 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x49, 0x64, 0x22, 0xa3, 0x01, 0x0a, 0x0b, 0x4d, 0x6f, 0x6e, + 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x72, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, + 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x75, 0x72, 0x6c, + 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, 0x72, 0x74, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x70, 0x6f, + 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0xc5, + 0x02, 0x0a, 0x0e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, + 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, + 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x31, 0x0a, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x2c, 0x0a, 0x0f, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x48, 0x00, 0x52, 0x0e, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x3b, 0x0a, 0x0a, 0x73, 0x73, 0x68, 0x5f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x53, 0x48, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x48, 0x01, 0x52, 0x09, 0x73, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x88, 0x01, 0x01, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x73, 0x73, 0x68, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1a, + 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, + 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0a, 0x6d, + 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x39, 0x0a, 0x19, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x79, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x44, 0x61, 0x79, 0x73, 0x22, 0xa7, 0x02, 0x0a, 0x13, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4e, + 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x30, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x12, 0x3d, + 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x25, 0x0a, + 0x0e, 0x75, 0x6e, 0x6e, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x75, 0x6e, 0x6e, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x5a, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xe0, 0x02, 0x0a, 0x08, 0x53, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, + 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, + 0x27, 0x0a, 0x0d, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, + 0x4b, 0x65, 0x79, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x11, 0x73, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, + 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, 0x88, 0x01, 0x01, 0x12, 0x1e, 0x0a, 0x08, 0x72, 0x6f, 0x6c, + 0x65, 0x5f, 0x61, 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x07, 0x72, + 0x6f, 0x6c, 0x65, 0x41, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x72, 0x65, 0x67, + 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x06, 0x72, 0x65, 0x67, + 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1f, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, + 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x04, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x61, - 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, - 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x75, - 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x39, 0x0a, 0x19, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, - 0x5f, 0x64, 0x61, 0x79, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x44, - 0x61, 0x79, 0x73, 0x22, 0xa7, 0x02, 0x0a, 0x13, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4e, 0x0a, 0x09, 0x65, - 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, - 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x64, 0x62, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0a, - 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x12, 0x25, 0x0a, 0x0e, 0x75, 0x6e, - 0x6e, 0x65, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0d, 0x75, 0x6e, 0x6e, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x1a, 0x5a, 0x0a, 0x0e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, + 0x64, 0x61, 0x74, 0x61, 0x44, 0x62, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x73, 0x65, 0x63, + 0x72, 0x65, 0x74, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x42, 0x0b, + 0x0a, 0x09, 0x5f, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x61, 0x72, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x5f, + 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x53, 0x71, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, + 0x12, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, + 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, + 0x6f, 0x72, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, + 0x6f, 0x72, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, + 0x91, 0x05, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x04, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, + 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, + 0x61, 0x6b, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, + 0x00, 0x52, 0x0f, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x42, 0x69, 0x67, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x62, 0x69, 0x67, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3e, 0x0a, 0x0c, 0x6d, + 0x6f, 0x6e, 0x67, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x2e, 0x4d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0b, + 0x6d, 0x6f, 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x70, + 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, - 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xe0, 0x02, - 0x0a, 0x08, 0x53, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, - 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12, 0x27, 0x0a, 0x0d, - 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x4b, 0x65, 0x79, - 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x11, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x5f, - 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x48, 0x01, 0x52, 0x0f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, 0x41, 0x63, 0x63, 0x65, 0x73, 0x73, - 0x4b, 0x65, 0x79, 0x88, 0x01, 0x01, 0x12, 0x1e, 0x0a, 0x08, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x61, - 0x72, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x07, 0x72, 0x6f, 0x6c, 0x65, - 0x41, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x03, 0x52, 0x06, 0x72, 0x65, 0x67, 0x69, 0x6f, 0x6e, - 0x88, 0x01, 0x01, 0x12, 0x1f, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x04, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, - 0x74, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x5f, 0x64, 0x62, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, - 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x44, 0x62, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, - 0x65, 0x79, 0x5f, 0x69, 0x64, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x73, 0x65, 0x63, 0x72, 0x65, 0x74, - 0x5f, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x42, 0x0b, 0x0a, 0x09, 0x5f, - 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x61, 0x72, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x72, 0x65, 0x67, - 0x69, 0x6f, 0x6e, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, - 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x53, 0x71, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, - 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x75, 0x73, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x75, 0x73, 0x65, 0x72, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64, - 0x12, 0x1a, 0x0a, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22, 0x91, 0x05, 0x0a, - 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, - 0x79, 0x70, 0x65, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x6e, 0x6f, - 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0f, - 0x73, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x47, 0x0a, 0x0f, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x42, 0x69, 0x67, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x62, 0x69, 0x67, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x3e, 0x0a, 0x0c, 0x6d, 0x6f, 0x6e, 0x67, - 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x4d, 0x6f, - 0x6e, 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0b, 0x6d, 0x6f, 0x6e, - 0x67, 0x6f, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x70, 0x6f, 0x73, 0x74, - 0x67, 0x72, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, - 0x00, 0x52, 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, - 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x68, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x35, 0x0a, 0x09, 0x73, 0x33, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x33, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x08, 0x73, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x71, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x71, 0x6c, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0f, 0x73, 0x71, - 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x57, 0x0a, - 0x15, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, - 0x00, 0x52, 0x13, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x2a, 0x77, 0x0a, 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x49, - 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x4e, 0x4f, 0x57, - 0x46, 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, 0x4e, 0x47, 0x4f, - 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, 0x53, 0x10, 0x03, - 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, 0x04, 0x12, 0x06, - 0x0a, 0x02, 0x53, 0x33, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, 0x53, 0x45, 0x52, - 0x56, 0x45, 0x52, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, - 0x42, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x07, 0x42, 0x7c, 0x0a, 0x10, 0x63, 0x6f, 0x6d, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x42, 0x0a, 0x50, - 0x65, 0x65, 0x72, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, - 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, - 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, - 0x73, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0xe2, - 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x5c, 0x47, 0x50, - 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x72, 0x73, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x47, 0x0a, 0x0f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x48, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x35, 0x0a, + 0x09, 0x73, 0x33, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, + 0x53, 0x33, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x08, 0x73, 0x33, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4a, 0x0a, 0x10, 0x73, 0x71, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x71, + 0x6c, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, + 0x0f, 0x73, 0x71, 0x6c, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x57, 0x0a, 0x15, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x48, 0x75, 0x62, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x48, 0x00, 0x52, 0x13, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x68, 0x75, 0x62, 0x47, 0x72, + 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x2a, 0x77, 0x0a, 0x06, 0x44, 0x42, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x0a, + 0x08, 0x42, 0x49, 0x47, 0x51, 0x55, 0x45, 0x52, 0x59, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x53, + 0x4e, 0x4f, 0x57, 0x46, 0x4c, 0x41, 0x4b, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x4f, + 0x4e, 0x47, 0x4f, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x50, 0x4f, 0x53, 0x54, 0x47, 0x52, 0x45, + 0x53, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x48, 0x55, 0x42, 0x10, + 0x04, 0x12, 0x06, 0x0a, 0x02, 0x53, 0x33, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x51, 0x4c, + 0x53, 0x45, 0x52, 0x56, 0x45, 0x52, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x45, 0x4e, + 0x54, 0x48, 0x55, 0x42, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x07, 0x42, 0x7c, 0x0a, 0x10, + 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x42, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, + 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, + 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, + 0x65, 0x65, 0x72, 0x73, 0xca, 0x02, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, + 0x72, 0x73, 0xe2, 0x02, 0x17, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, + 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0b, 0x50, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x50, 0x65, 0x65, 0x72, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, } var ( @@ -1246,40 +1347,42 @@ func file_peers_proto_rawDescGZIP() []byte { } var file_peers_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 10) +var file_peers_proto_msgTypes = make([]protoimpl.MessageInfo, 11) var file_peers_proto_goTypes = []interface{}{ (DBType)(0), // 0: peerdb_peers.DBType - (*SnowflakeConfig)(nil), // 1: peerdb_peers.SnowflakeConfig - (*BigqueryConfig)(nil), // 2: peerdb_peers.BigqueryConfig - (*MongoConfig)(nil), // 3: peerdb_peers.MongoConfig - (*PostgresConfig)(nil), // 4: peerdb_peers.PostgresConfig - (*EventHubConfig)(nil), // 5: peerdb_peers.EventHubConfig - (*EventHubGroupConfig)(nil), // 6: peerdb_peers.EventHubGroupConfig - (*S3Config)(nil), // 7: peerdb_peers.S3Config - (*SqlServerConfig)(nil), // 8: peerdb_peers.SqlServerConfig - (*Peer)(nil), // 9: peerdb_peers.Peer - nil, // 10: peerdb_peers.EventHubGroupConfig.EventhubsEntry + (*SSHConfig)(nil), // 1: peerdb_peers.SSHConfig + (*SnowflakeConfig)(nil), // 2: peerdb_peers.SnowflakeConfig + (*BigqueryConfig)(nil), // 3: peerdb_peers.BigqueryConfig + (*MongoConfig)(nil), // 4: peerdb_peers.MongoConfig + (*PostgresConfig)(nil), // 5: peerdb_peers.PostgresConfig + (*EventHubConfig)(nil), // 6: peerdb_peers.EventHubConfig + (*EventHubGroupConfig)(nil), // 7: peerdb_peers.EventHubGroupConfig + (*S3Config)(nil), // 8: peerdb_peers.S3Config + (*SqlServerConfig)(nil), // 9: peerdb_peers.SqlServerConfig + (*Peer)(nil), // 10: peerdb_peers.Peer + nil, // 11: peerdb_peers.EventHubGroupConfig.EventhubsEntry } var file_peers_proto_depIdxs = []int32{ - 4, // 0: peerdb_peers.EventHubConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig - 10, // 1: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry - 4, // 2: peerdb_peers.EventHubGroupConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig - 4, // 3: peerdb_peers.S3Config.metadata_db:type_name -> peerdb_peers.PostgresConfig - 0, // 4: peerdb_peers.Peer.type:type_name -> peerdb_peers.DBType - 1, // 5: peerdb_peers.Peer.snowflake_config:type_name -> peerdb_peers.SnowflakeConfig - 2, // 6: peerdb_peers.Peer.bigquery_config:type_name -> peerdb_peers.BigqueryConfig - 3, // 7: peerdb_peers.Peer.mongo_config:type_name -> peerdb_peers.MongoConfig - 4, // 8: peerdb_peers.Peer.postgres_config:type_name -> peerdb_peers.PostgresConfig - 5, // 9: peerdb_peers.Peer.eventhub_config:type_name -> peerdb_peers.EventHubConfig - 7, // 10: peerdb_peers.Peer.s3_config:type_name -> peerdb_peers.S3Config - 8, // 11: peerdb_peers.Peer.sqlserver_config:type_name -> peerdb_peers.SqlServerConfig - 6, // 12: peerdb_peers.Peer.eventhub_group_config:type_name -> peerdb_peers.EventHubGroupConfig - 5, // 13: peerdb_peers.EventHubGroupConfig.EventhubsEntry.value:type_name -> peerdb_peers.EventHubConfig - 14, // [14:14] is the sub-list for method output_type - 14, // [14:14] is the sub-list for method input_type - 14, // [14:14] is the sub-list for extension type_name - 14, // [14:14] is the sub-list for extension extendee - 0, // [0:14] is the sub-list for field type_name + 1, // 0: peerdb_peers.PostgresConfig.ssh_config:type_name -> peerdb_peers.SSHConfig + 5, // 1: peerdb_peers.EventHubConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig + 11, // 2: peerdb_peers.EventHubGroupConfig.eventhubs:type_name -> peerdb_peers.EventHubGroupConfig.EventhubsEntry + 5, // 3: peerdb_peers.EventHubGroupConfig.metadata_db:type_name -> peerdb_peers.PostgresConfig + 5, // 4: peerdb_peers.S3Config.metadata_db:type_name -> peerdb_peers.PostgresConfig + 0, // 5: peerdb_peers.Peer.type:type_name -> peerdb_peers.DBType + 2, // 6: peerdb_peers.Peer.snowflake_config:type_name -> peerdb_peers.SnowflakeConfig + 3, // 7: peerdb_peers.Peer.bigquery_config:type_name -> peerdb_peers.BigqueryConfig + 4, // 8: peerdb_peers.Peer.mongo_config:type_name -> peerdb_peers.MongoConfig + 5, // 9: peerdb_peers.Peer.postgres_config:type_name -> peerdb_peers.PostgresConfig + 6, // 10: peerdb_peers.Peer.eventhub_config:type_name -> peerdb_peers.EventHubConfig + 8, // 11: peerdb_peers.Peer.s3_config:type_name -> peerdb_peers.S3Config + 9, // 12: peerdb_peers.Peer.sqlserver_config:type_name -> peerdb_peers.SqlServerConfig + 7, // 13: peerdb_peers.Peer.eventhub_group_config:type_name -> peerdb_peers.EventHubGroupConfig + 6, // 14: peerdb_peers.EventHubGroupConfig.EventhubsEntry.value:type_name -> peerdb_peers.EventHubConfig + 15, // [15:15] is the sub-list for method output_type + 15, // [15:15] is the sub-list for method input_type + 15, // [15:15] is the sub-list for extension type_name + 15, // [15:15] is the sub-list for extension extendee + 0, // [0:15] is the sub-list for field type_name } func init() { file_peers_proto_init() } @@ -1289,7 +1392,7 @@ func file_peers_proto_init() { } if !protoimpl.UnsafeEnabled { file_peers_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SnowflakeConfig); i { + switch v := v.(*SSHConfig); i { case 0: return &v.state case 1: @@ -1301,7 +1404,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BigqueryConfig); i { + switch v := v.(*SnowflakeConfig); i { case 0: return &v.state case 1: @@ -1313,7 +1416,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MongoConfig); i { + switch v := v.(*BigqueryConfig); i { case 0: return &v.state case 1: @@ -1325,7 +1428,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PostgresConfig); i { + switch v := v.(*MongoConfig); i { case 0: return &v.state case 1: @@ -1337,7 +1440,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EventHubConfig); i { + switch v := v.(*PostgresConfig); i { case 0: return &v.state case 1: @@ -1349,7 +1452,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EventHubGroupConfig); i { + switch v := v.(*EventHubConfig); i { case 0: return &v.state case 1: @@ -1361,7 +1464,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*S3Config); i { + switch v := v.(*EventHubGroupConfig); i { case 0: return &v.state case 1: @@ -1373,7 +1476,7 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SqlServerConfig); i { + switch v := v.(*S3Config); i { case 0: return &v.state case 1: @@ -1385,6 +1488,18 @@ func file_peers_proto_init() { } } file_peers_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SqlServerConfig); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_peers_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*Peer); i { case 0: return &v.state @@ -1397,10 +1512,10 @@ func file_peers_proto_init() { } } } - file_peers_proto_msgTypes[0].OneofWrappers = []interface{}{} - file_peers_proto_msgTypes[3].OneofWrappers = []interface{}{} - file_peers_proto_msgTypes[6].OneofWrappers = []interface{}{} - file_peers_proto_msgTypes[8].OneofWrappers = []interface{}{ + file_peers_proto_msgTypes[1].OneofWrappers = []interface{}{} + file_peers_proto_msgTypes[4].OneofWrappers = []interface{}{} + file_peers_proto_msgTypes[7].OneofWrappers = []interface{}{} + file_peers_proto_msgTypes[9].OneofWrappers = []interface{}{ (*Peer_SnowflakeConfig)(nil), (*Peer_BigqueryConfig)(nil), (*Peer_MongoConfig)(nil), @@ -1416,7 +1531,7 @@ func file_peers_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_peers_proto_rawDesc, NumEnums: 1, - NumMessages: 10, + NumMessages: 11, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/shared/constants.go b/flow/shared/constants.go index a10d529189..fa53bb007e 100644 --- a/flow/shared/constants.go +++ b/flow/shared/constants.go @@ -7,9 +7,10 @@ import ( ) const ( - peerFlowTaskQueue = "peer-flow-task-queue" - snapshotFlowTaskQueue = "snapshot-flow-task-queue" - CDCFlowSignalName = "peer-flow-signal" + peerFlowTaskQueue = "peer-flow-task-queue" + snapshotFlowTaskQueue = "snapshot-flow-task-queue" + CDCFlowSignalName = "peer-flow-signal" + CDCBatchSizeSignalName = "cdc-batch-size-signal" ) const MirrorNameSearchAttribute = "MirrorName" diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 883df13046..4f4f5e3ccc 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -284,6 +284,20 @@ func CDCFlowWorkflowWithConfig( BatchSize: int32(limits.MaxBatchSize), } + // add a signal to change the batch size + batchSizeSignalChan := workflow.GetSignalChannel(ctx, shared.CDCBatchSizeSignalName) + batchSizeSelector := workflow.NewSelector(ctx) + batchSizeSelector.AddReceive(batchSizeSignalChan, func(c workflow.ReceiveChannel, more bool) { + var batchSize int32 + c.Receive(ctx, &batchSize) + w.logger.Info("received batch size signal: ", batchSize) + syncFlowOptions.BatchSize = batchSize + }) + batchSizeSelector.AddDefault(func() { + w.logger.Info("no batch size signal received, batch size remains: ", + syncFlowOptions.BatchSize) + }) + currentSyncFlowNum := 0 totalRecordsSynced := 0 diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 49784c2ff4..f65fe4b2d0 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -647,6 +647,7 @@ fn parse_db_options( .to_string(), metadata_schema: opts.get("metadata_schema").map(|s| s.to_string()), transaction_snapshot: "".to_string(), + ssh_config: None, }; let config = Config::PostgresConfig(postgres_config); Some(config) diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index 77467348e3..94119ff6c1 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -75,6 +75,7 @@ impl CatalogConfig { database: self.database.clone(), transaction_snapshot: "".to_string(), metadata_schema: Some("".to_string()), + ssh_config: None, } } diff --git a/nexus/pt/src/peerdb_peers.rs b/nexus/pt/src/peerdb_peers.rs index 017a36b8ab..8266eea158 100644 --- a/nexus/pt/src/peerdb_peers.rs +++ b/nexus/pt/src/peerdb_peers.rs @@ -1,6 +1,20 @@ // @generated #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SshConfig { + #[prost(string, tag="1")] + pub host: ::prost::alloc::string::String, + #[prost(uint32, tag="2")] + pub port: u32, + #[prost(string, tag="3")] + pub user: ::prost::alloc::string::String, + #[prost(string, tag="4")] + pub password: ::prost::alloc::string::String, + #[prost(string, tag="5")] + pub private_key: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct SnowflakeConfig { #[prost(string, tag="1")] pub account_id: ::prost::alloc::string::String, @@ -83,6 +97,8 @@ pub struct PostgresConfig { /// defaults to _peerdb_internal #[prost(string, optional, tag="7")] pub metadata_schema: ::core::option::Option<::prost::alloc::string::String>, + #[prost(message, optional, tag="8")] + pub ssh_config: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_peers.serde.rs b/nexus/pt/src/peerdb_peers.serde.rs index 2cdf144f4b..18c206865e 100644 --- a/nexus/pt/src/peerdb_peers.serde.rs +++ b/nexus/pt/src/peerdb_peers.serde.rs @@ -1134,6 +1134,9 @@ impl serde::Serialize for PostgresConfig { if self.metadata_schema.is_some() { len += 1; } + if self.ssh_config.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_peers.PostgresConfig", len)?; if !self.host.is_empty() { struct_ser.serialize_field("host", &self.host)?; @@ -1156,6 +1159,9 @@ impl serde::Serialize for PostgresConfig { if let Some(v) = self.metadata_schema.as_ref() { struct_ser.serialize_field("metadataSchema", v)?; } + if let Some(v) = self.ssh_config.as_ref() { + struct_ser.serialize_field("sshConfig", v)?; + } struct_ser.end() } } @@ -1175,6 +1181,8 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { "transactionSnapshot", "metadata_schema", "metadataSchema", + "ssh_config", + "sshConfig", ]; #[allow(clippy::enum_variant_names)] @@ -1186,6 +1194,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { Database, TransactionSnapshot, MetadataSchema, + SshConfig, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1215,6 +1224,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { "database" => Ok(GeneratedField::Database), "transactionSnapshot" | "transaction_snapshot" => Ok(GeneratedField::TransactionSnapshot), "metadataSchema" | "metadata_schema" => Ok(GeneratedField::MetadataSchema), + "sshConfig" | "ssh_config" => Ok(GeneratedField::SshConfig), _ => Ok(GeneratedField::__SkipField__), } } @@ -1241,6 +1251,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { let mut database__ = None; let mut transaction_snapshot__ = None; let mut metadata_schema__ = None; + let mut ssh_config__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Host => { @@ -1287,6 +1298,12 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { } metadata_schema__ = map.next_value()?; } + GeneratedField::SshConfig => { + if ssh_config__.is_some() { + return Err(serde::de::Error::duplicate_field("sshConfig")); + } + ssh_config__ = map.next_value()?; + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1300,6 +1317,7 @@ impl<'de> serde::Deserialize<'de> for PostgresConfig { database: database__.unwrap_or_default(), transaction_snapshot: transaction_snapshot__.unwrap_or_default(), metadata_schema: metadata_schema__, + ssh_config: ssh_config__, }) } } @@ -1507,6 +1525,172 @@ impl<'de> serde::Deserialize<'de> for S3Config { deserializer.deserialize_struct("peerdb_peers.S3Config", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SshConfig { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.host.is_empty() { + len += 1; + } + if self.port != 0 { + len += 1; + } + if !self.user.is_empty() { + len += 1; + } + if !self.password.is_empty() { + len += 1; + } + if !self.private_key.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_peers.SSHConfig", len)?; + if !self.host.is_empty() { + struct_ser.serialize_field("host", &self.host)?; + } + if self.port != 0 { + struct_ser.serialize_field("port", &self.port)?; + } + if !self.user.is_empty() { + struct_ser.serialize_field("user", &self.user)?; + } + if !self.password.is_empty() { + struct_ser.serialize_field("password", &self.password)?; + } + if !self.private_key.is_empty() { + struct_ser.serialize_field("privateKey", &self.private_key)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SshConfig { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "host", + "port", + "user", + "password", + "private_key", + "privateKey", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Host, + Port, + User, + Password, + PrivateKey, + __SkipField__, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "host" => Ok(GeneratedField::Host), + "port" => Ok(GeneratedField::Port), + "user" => Ok(GeneratedField::User), + "password" => Ok(GeneratedField::Password), + "privateKey" | "private_key" => Ok(GeneratedField::PrivateKey), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SshConfig; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_peers.SSHConfig") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut host__ = None; + let mut port__ = None; + let mut user__ = None; + let mut password__ = None; + let mut private_key__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Host => { + if host__.is_some() { + return Err(serde::de::Error::duplicate_field("host")); + } + host__ = Some(map.next_value()?); + } + GeneratedField::Port => { + if port__.is_some() { + return Err(serde::de::Error::duplicate_field("port")); + } + port__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::User => { + if user__.is_some() { + return Err(serde::de::Error::duplicate_field("user")); + } + user__ = Some(map.next_value()?); + } + GeneratedField::Password => { + if password__.is_some() { + return Err(serde::de::Error::duplicate_field("password")); + } + password__ = Some(map.next_value()?); + } + GeneratedField::PrivateKey => { + if private_key__.is_some() { + return Err(serde::de::Error::duplicate_field("privateKey")); + } + private_key__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(SshConfig { + host: host__.unwrap_or_default(), + port: port__.unwrap_or_default(), + user: user__.unwrap_or_default(), + password: password__.unwrap_or_default(), + private_key: private_key__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_peers.SSHConfig", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SnowflakeConfig { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/peers.proto b/protos/peers.proto index fe343d906e..18a4577aeb 100644 --- a/protos/peers.proto +++ b/protos/peers.proto @@ -2,6 +2,14 @@ syntax = "proto3"; package peerdb_peers; +message SSHConfig { + string host = 1; + uint32 port = 2; + string user = 3; + string password = 4; + string private_key = 5; +} + message SnowflakeConfig { string account_id = 1; string username = 2; @@ -48,6 +56,7 @@ message PostgresConfig { string transaction_snapshot = 6; // defaults to _peerdb_internal optional string metadata_schema = 7; + optional SSHConfig ssh_config = 8; } message EventHubConfig { diff --git a/ui/app/api/mirrors/cdc/route.ts b/ui/app/api/mirrors/cdc/route.ts index 025dfc149a..12efa4a826 100644 --- a/ui/app/api/mirrors/cdc/route.ts +++ b/ui/app/api/mirrors/cdc/route.ts @@ -14,19 +14,23 @@ export async function POST(request: Request) { connectionConfigs: config, createCatalogEntry: true, }; - const createStatus: CreateCDCFlowResponse = await fetch( - `${flowServiceAddr}/v1/flows/cdc/create`, - { - method: 'POST', - body: JSON.stringify(req), - } - ).then((res) => { - return res.json(); - }); + try { + const createStatus: CreateCDCFlowResponse = await fetch( + `${flowServiceAddr}/v1/flows/cdc/create`, + { + method: 'POST', + body: JSON.stringify(req), + } + ).then((res) => { + return res.json(); + }); - let response: UCreateMirrorResponse = { - created: !!createStatus.worflowId, - }; + let response: UCreateMirrorResponse = { + created: !!createStatus.worflowId, + }; - return new Response(JSON.stringify(response)); + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/api/mirrors/drop/route.ts b/ui/app/api/mirrors/drop/route.ts index 6652bf5897..e3be0f7c41 100644 --- a/ui/app/api/mirrors/drop/route.ts +++ b/ui/app/api/mirrors/drop/route.ts @@ -14,19 +14,23 @@ export async function POST(request: Request) { removeFlowEntry: true, }; console.log('/drop/mirror: req:', req); - const dropStatus: ShutdownResponse = await fetch( - `${flowServiceAddr}/v1/mirrors/drop`, - { - method: 'POST', - body: JSON.stringify(req), - } - ).then((res) => { - return res.json(); - }); - let response: UDropMirrorResponse = { - dropped: dropStatus.ok, - errorMessage: dropStatus.errorMessage, - }; + try { + const dropStatus: ShutdownResponse = await fetch( + `${flowServiceAddr}/v1/mirrors/drop`, + { + method: 'POST', + body: JSON.stringify(req), + } + ).then((res) => { + return res.json(); + }); + let response: UDropMirrorResponse = { + dropped: dropStatus.ok, + errorMessage: dropStatus.errorMessage, + }; - return new Response(JSON.stringify(response)); + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/api/mirrors/qrep/route.ts b/ui/app/api/mirrors/qrep/route.ts index ff160e703e..6f199dc53f 100644 --- a/ui/app/api/mirrors/qrep/route.ts +++ b/ui/app/api/mirrors/qrep/route.ts @@ -14,18 +14,22 @@ export async function POST(request: Request) { qrepConfig: config, createCatalogEntry: true, }; - const createStatus: CreateQRepFlowResponse = await fetch( - `${flowServiceAddr}/v1/flows/qrep/create`, - { - method: 'POST', - body: JSON.stringify(req), - } - ).then((res) => { - return res.json(); - }); - let response: UCreateMirrorResponse = { - created: !!createStatus.worflowId, - }; + try { + const createStatus: CreateQRepFlowResponse = await fetch( + `${flowServiceAddr}/v1/flows/qrep/create`, + { + method: 'POST', + body: JSON.stringify(req), + } + ).then((res) => { + return res.json(); + }); + let response: UCreateMirrorResponse = { + created: !!createStatus.worflowId, + }; - return new Response(JSON.stringify(response)); + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/api/peers/columns/route.ts b/ui/app/api/peers/columns/route.ts index ce944907fb..9f45db092e 100644 --- a/ui/app/api/peers/columns/route.ts +++ b/ui/app/api/peers/columns/route.ts @@ -6,13 +6,17 @@ export async function POST(request: Request) { const body = await request.json(); const { peerName, schemaName, tableName } = body; const flowServiceAddr = GetFlowHttpAddressFromEnv(); - const columnsList: TableColumnsResponse = await fetch( - `${flowServiceAddr}/v1/peers/columns?peer_name=${peerName}&schema_name=${schemaName}&table_name=${tableName}` - ).then((res) => { - return res.json(); - }); - let response: UColumnsResponse = { - columns: columnsList.columns, - }; - return new Response(JSON.stringify(response)); + try { + const columnsList: TableColumnsResponse = await fetch( + `${flowServiceAddr}/v1/peers/columns?peer_name=${peerName}&schema_name=${schemaName}&table_name=${tableName}` + ).then((res) => { + return res.json(); + }); + let response: UColumnsResponse = { + columns: columnsList.columns, + }; + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/api/peers/drop/route.ts b/ui/app/api/peers/drop/route.ts index 0a7eb25418..7449480033 100644 --- a/ui/app/api/peers/drop/route.ts +++ b/ui/app/api/peers/drop/route.ts @@ -10,19 +10,23 @@ export async function POST(request: Request) { peerName, }; console.log('/drop/peer: req:', req); - const dropStatus: DropPeerResponse = await fetch( - `${flowServiceAddr}/v1/peers/drop`, - { - method: 'POST', - body: JSON.stringify(req), - } - ).then((res) => { - return res.json(); - }); - let response: UDropPeerResponse = { - dropped: dropStatus.ok, - errorMessage: dropStatus.errorMessage, - }; + try { + const dropStatus: DropPeerResponse = await fetch( + `${flowServiceAddr}/v1/peers/drop`, + { + method: 'POST', + body: JSON.stringify(req), + } + ).then((res) => { + return res.json(); + }); + let response: UDropPeerResponse = { + dropped: dropStatus.ok, + errorMessage: dropStatus.errorMessage, + }; - return new Response(JSON.stringify(response)); + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index ca74ba0127..c865979efe 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -65,46 +65,55 @@ const constructPeer = ( export async function POST(request: Request) { const body = await request.json(); + console.log('POST Validate Peer:', body); const { name, type, config, mode } = body; const flowServiceAddr = GetFlowHttpAddressFromEnv(); const peer = constructPeer(name, type, config); if (mode === 'validate') { const validateReq: ValidatePeerRequest = { peer }; - const validateStatus: ValidatePeerResponse = await fetch( - `${flowServiceAddr}/v1/peers/validate`, - { - method: 'POST', - body: JSON.stringify(validateReq), - } - ).then((res) => { - return res.json(); - }); - let response: UValidatePeerResponse = { - valid: - validatePeerStatusFromJSON(validateStatus.status) === - ValidatePeerStatus.VALID, - message: validateStatus.message, - }; - return new Response(JSON.stringify(response)); + try { + const validateStatus: ValidatePeerResponse = await fetch( + `${flowServiceAddr}/v1/peers/validate`, + { + method: 'POST', + body: JSON.stringify(validateReq), + } + ).then((res) => { + return res.json(); + }); + let response: UValidatePeerResponse = { + valid: + validatePeerStatusFromJSON(validateStatus.status) === + ValidatePeerStatus.VALID, + message: validateStatus.message, + }; + return new Response(JSON.stringify(response)); + } catch (error) { + console.error('Error validating peer:', error); + } } else if (mode === 'create') { const req: CreatePeerRequest = { peer }; console.log('/peer/create req:', req); - const createStatus: CreatePeerResponse = await fetch( - `${flowServiceAddr}/v1/peers/create`, - { - method: 'POST', - body: JSON.stringify(req), - } - ).then((res) => { - return res.json(); - }); - let response: UCreatePeerResponse = { - created: - createPeerStatusFromJSON(createStatus.status) === - CreatePeerStatus.CREATED, - message: createStatus.message, - }; - return new Response(JSON.stringify(response)); + try { + const createStatus: CreatePeerResponse = await fetch( + `${flowServiceAddr}/v1/peers/create`, + { + method: 'POST', + body: JSON.stringify(req), + } + ).then((res) => { + return res.json(); + }); + let response: UCreatePeerResponse = { + created: + createPeerStatusFromJSON(createStatus.status) === + CreatePeerStatus.CREATED, + message: createStatus.message, + }; + return new Response(JSON.stringify(response)); + } catch (error) { + console.error('Error creating peer:', error); + } } } diff --git a/ui/app/api/peers/schemas/route.ts b/ui/app/api/peers/schemas/route.ts index 0848692702..0c701cb713 100644 --- a/ui/app/api/peers/schemas/route.ts +++ b/ui/app/api/peers/schemas/route.ts @@ -5,18 +5,22 @@ export async function POST(request: Request) { const body = await request.json(); const { peerName } = body; const flowServiceAddr = GetFlowHttpAddressFromEnv(); - const schemaList = await fetch( - `${flowServiceAddr}/v1/peers/schemas?peer_name=${peerName}` - ).then((res) => { - return res.json(); - }); - let response: USchemasResponse = { - schemas: schemaList.schemas, - }; - if (schemaList.message === 'no rows in result set') { - response = { - schemas: [], + try { + const schemaList = await fetch( + `${flowServiceAddr}/v1/peers/schemas?peer_name=${peerName}` + ).then((res) => { + return res.json(); + }); + let response: USchemasResponse = { + schemas: schemaList.schemas, }; + if (schemaList.message === 'no rows in result set') { + response = { + schemas: [], + }; + } + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); } - return new Response(JSON.stringify(response)); } diff --git a/ui/app/api/peers/tables/all/route.ts b/ui/app/api/peers/tables/all/route.ts index c372b41a32..92223f7bbf 100644 --- a/ui/app/api/peers/tables/all/route.ts +++ b/ui/app/api/peers/tables/all/route.ts @@ -6,13 +6,17 @@ export async function POST(request: Request) { const body = await request.json(); const { peerName } = body; const flowServiceAddr = GetFlowHttpAddressFromEnv(); - const tableList: SchemaTablesResponse = await fetch( - `${flowServiceAddr}/v1/peers/tables/all?peer_name=${peerName}` - ).then((res) => { - return res.json(); - }); - let response: UTablesResponse = { - tables: tableList.tables, - }; - return new Response(JSON.stringify(response)); + try { + const tableList: SchemaTablesResponse = await fetch( + `${flowServiceAddr}/v1/peers/tables/all?peer_name=${peerName}` + ).then((res) => { + return res.json(); + }); + let response: UTablesResponse = { + tables: tableList.tables, + }; + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/api/peers/tables/route.ts b/ui/app/api/peers/tables/route.ts index 53f606d48c..b4c73500dd 100644 --- a/ui/app/api/peers/tables/route.ts +++ b/ui/app/api/peers/tables/route.ts @@ -6,13 +6,17 @@ export async function POST(request: Request) { const body = await request.json(); const { peerName, schemaName } = body; const flowServiceAddr = GetFlowHttpAddressFromEnv(); - const tableList: SchemaTablesResponse = await fetch( - `${flowServiceAddr}/v1/peers/tables?peer_name=${peerName}&schema_name=${schemaName}` - ).then((res) => { - return res.json(); - }); - let response: UTablesResponse = { - tables: tableList.tables, - }; - return new Response(JSON.stringify(response)); + try { + const tableList: SchemaTablesResponse = await fetch( + `${flowServiceAddr}/v1/peers/tables?peer_name=${peerName}&schema_name=${schemaName}` + ).then((res) => { + return res.json(); + }); + let response: UTablesResponse = { + tables: tableList.tables, + }; + return new Response(JSON.stringify(response)); + } catch (e) { + console.log(e); + } } diff --git a/ui/app/dto/PeersDTO.ts b/ui/app/dto/PeersDTO.ts index 0e6dfef818..232e1b2c3c 100644 --- a/ui/app/dto/PeersDTO.ts +++ b/ui/app/dto/PeersDTO.ts @@ -43,3 +43,4 @@ export type CatalogPeer = { type: number; options: Buffer; }; +export type PeerSetter = React.Dispatch>; diff --git a/ui/app/peers/create/[peerType]/helpers/common.ts b/ui/app/peers/create/[peerType]/helpers/common.ts index 62b2bb26c2..b1c27e0edc 100644 --- a/ui/app/peers/create/[peerType]/helpers/common.ts +++ b/ui/app/peers/create/[peerType]/helpers/common.ts @@ -1,5 +1,4 @@ -import { PeerConfig } from '@/app/dto/PeersDTO'; -import { PeerSetter } from '@/components/ConfigForm'; +import { PeerConfig, PeerSetter } from '@/app/dto/PeersDTO'; import { blankBigquerySetting } from './bq'; import { blankPostgresSetting } from './pg'; import { blankS3Setting } from './s3'; diff --git a/ui/app/peers/create/[peerType]/helpers/pg.ts b/ui/app/peers/create/[peerType]/helpers/pg.ts index 84c464bf7a..8e822fb5c4 100644 --- a/ui/app/peers/create/[peerType]/helpers/pg.ts +++ b/ui/app/peers/create/[peerType]/helpers/pg.ts @@ -1,4 +1,5 @@ -import { PostgresConfig } from '@/grpc_generated/peers'; +import { PostgresConfig, SSHConfig } from '@/grpc_generated/peers'; +import { Dispatch, SetStateAction } from 'react'; import { PeerSetting } from './common'; export const postgresSetting: PeerSetting[] = [ @@ -48,6 +49,53 @@ export const postgresSetting: PeerSetting[] = [ }, ]; +type sshSetter = Dispatch>; +export const sshSetting = [ + { + label: 'Host', + stateHandler: (value: string, setter: sshSetter) => + setter((curr: SSHConfig) => ({ ...curr, host: value })), + tips: 'Specifies the IP host name or address of your instance.', + }, + { + label: 'Port', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, port: parseInt(value, 10) })), + type: 'number', + default: 5432, + tips: 'Specifies the TCP/IP port or local Unix domain socket file extension on which clients can connect.', + }, + { + label: 'User', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, user: value })), + tips: 'Specify the user that we should use to connect to this host.', + }, + { + label: 'Password', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, password: value })), + type: 'password', + optional: true, + tips: 'Password associated with the user you provided.', + }, + { + label: 'BASE64 Private Key', + stateHandler: (value: string, setter: sshSetter) => + setter((curr) => ({ ...curr, privateKey: value })), + optional: true, + tips: 'Private key as a BASE64 string for authentication in order to SSH into your machine.', + }, +]; + +export const blankSSHConfig: SSHConfig = { + host: '', + port: 22, + user: '', + password: '', + privateKey: '', +}; + export const blankPostgresSetting: PostgresConfig = { host: '', port: 5432, diff --git a/ui/app/peers/create/[peerType]/page.tsx b/ui/app/peers/create/[peerType]/page.tsx index cbefbfad8b..a611df87ab 100644 --- a/ui/app/peers/create/[peerType]/page.tsx +++ b/ui/app/peers/create/[peerType]/page.tsx @@ -1,7 +1,9 @@ 'use client'; import { PeerConfig } from '@/app/dto/PeersDTO'; -import BQConfig from '@/components/BigqueryConfig'; -import S3ConfigForm from '@/components/S3Form'; +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 { Button } from '@/lib/Button'; import { ButtonGroup } from '@/lib/ButtonGroup'; import { Label } from '@/lib/Label'; @@ -12,9 +14,8 @@ import { Tooltip } from '@/lib/Tooltip'; import Link from 'next/link'; import { useRouter } from 'next/navigation'; import { useState } from 'react'; -import ConfigForm from '../../../../components/ConfigForm'; import { handleCreate, handleValidate } from './handlers'; -import { PeerSetting, getBlankSetting } from './helpers/common'; +import { getBlankSetting } from './helpers/common'; import { postgresSetting } from './helpers/pg'; import { snowflakeSetting } from './helpers/sf'; @@ -36,18 +37,15 @@ export default function CreateConfig({ }); const [loading, setLoading] = useState(false); const configComponentMap = (dbType: string) => { - const configForm = (settingList: PeerSetting[]) => ( - - ); switch (dbType) { case 'POSTGRES': - return configForm(postgresSetting); + return ; case 'SNOWFLAKE': - return configForm(snowflakeSetting); + return ; case 'BIGQUERY': - return ; + return ; case 'S3': - return ; + return ; default: return <>; } @@ -103,7 +101,7 @@ export default function CreateConfig({ - {dbType && configComponentMap(dbType)} + {configComponentMap(dbType)} diff --git a/ui/app/peers/create/[peerType]/schema.ts b/ui/app/peers/create/[peerType]/schema.ts index 01a82386e2..5bedeaa26f 100644 --- a/ui/app/peers/create/[peerType]/schema.ts +++ b/ui/app/peers/create/[peerType]/schema.ts @@ -52,6 +52,42 @@ export const pgSchema = z.object({ .string() .max(100, 'Transaction snapshot too long (100 char limit)') .optional(), + sshConfig: z + .object({ + host: z + .string({ + required_error: 'SSH Host is required', + invalid_type_error: 'SSH Host must be a string', + }) + .min(1, { message: 'SSH Host cannot be empty' }) + .max(255, 'SSH Host must be less than 255 characters'), + port: z + .number({ + required_error: 'SSH Port is required', + invalid_type_error: 'SSH Port must be a number', + }) + .int() + .min(1, 'SSH Port must be a positive integer') + .max(65535, 'SSH Port must be below 65535'), + user: z + .string({ + required_error: 'SSH User is required', + invalid_type_error: 'SSH User must be a string', + }) + .min(1, 'SSH User must be non-empty') + .max(64, 'SSH User must be less than 64 characters'), + password: z + .string({ + required_error: 'SSH Password is required', + invalid_type_error: 'SSH Password must be a string', + }) + .max(100, 'SSH Password must be less than 100 characters'), + privateKey: z.string({ + required_error: 'SSH Private Key is required', + invalid_type_error: 'SSH Private Key must be a string', + }), + }) + .optional(), }); export const sfSchema = z.object({ diff --git a/ui/components/BigqueryConfig.tsx b/ui/components/PeerForms/BigqueryConfig.tsx similarity index 96% rename from ui/components/BigqueryConfig.tsx rename to ui/components/PeerForms/BigqueryConfig.tsx index 9da7c9601d..db435bb17a 100644 --- a/ui/components/BigqueryConfig.tsx +++ b/ui/components/PeerForms/BigqueryConfig.tsx @@ -1,4 +1,5 @@ 'use client'; +import { PeerSetter } from '@/app/dto/PeersDTO'; import { blankBigquerySetting } from '@/app/peers/create/[peerType]/helpers/bq'; import { BigqueryConfig } from '@/grpc_generated/peers'; import { Label } from '@/lib/Label'; @@ -7,13 +8,12 @@ import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; import Link from 'next/link'; import { useState } from 'react'; -import { PeerSetter } from './ConfigForm'; -import { InfoPopover } from './InfoPopover'; +import { InfoPopover } from '../InfoPopover'; interface BQProps { setter: PeerSetter; } -export default function BQConfig(props: BQProps) { +export default function BigqueryForm(props: BQProps) { const [datasetID, setDatasetID] = useState(''); const handleJSONFile = (file: File) => { if (file) { diff --git a/ui/components/PeerForms/PostgresForm.tsx b/ui/components/PeerForms/PostgresForm.tsx new file mode 100644 index 0000000000..1819e626df --- /dev/null +++ b/ui/components/PeerForms/PostgresForm.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 && } +
+ } + /> + ))} + + ); +} diff --git a/ui/components/S3Form.tsx b/ui/components/PeerForms/S3Form.tsx similarity index 93% rename from ui/components/S3Form.tsx rename to ui/components/PeerForms/S3Form.tsx index 2a3f005671..cafef330f9 100644 --- a/ui/components/S3Form.tsx +++ b/ui/components/PeerForms/S3Form.tsx @@ -1,5 +1,5 @@ 'use client'; -import { PeerConfig } from '@/app/dto/PeersDTO'; +import { PeerConfig, PeerSetter } from '@/app/dto/PeersDTO'; import { postgresSetting } from '@/app/peers/create/[peerType]/helpers/pg'; import { blankS3Setting, @@ -13,13 +13,12 @@ import { Switch } from '@/lib/Switch'; import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; import { useEffect, useState } from 'react'; -import { PeerSetter } from './ConfigForm'; -import { InfoPopover } from './InfoPopover'; +import { InfoPopover } from '../InfoPopover'; interface S3Props { setter: PeerSetter; } -const S3ConfigForm = ({ setter }: S3Props) => { +const S3Form = ({ setter }: S3Props) => { const [showMetadata, setShowMetadata] = useState(false); const [metadataDB, setMetadataDB] = useState( blankS3Setting.metadataDb! @@ -31,6 +30,7 @@ const S3ConfigForm = ({ setter }: S3Props) => { storageType === 'GCS' ); }; + useEffect(() => { const endpoint = storageType === 'S3' ? '' : 'storage.googleapis.com'; setter((prev) => { @@ -189,7 +189,12 @@ const S3ConfigForm = ({ setter }: S3Props) => { } defaultValue={ (metadataDB as PostgresConfig)[ - pgSetting.label.toLowerCase() as keyof PostgresConfig + pgSetting.label.toLowerCase() as + | 'host' + | 'port' + | 'user' + | 'password' + | 'database' ] || '' } /> @@ -208,4 +213,4 @@ const S3ConfigForm = ({ setter }: S3Props) => { ); }; -export default S3ConfigForm; +export default S3Form; diff --git a/ui/components/ConfigForm.tsx b/ui/components/PeerForms/SnowflakeForm.tsx similarity index 91% rename from ui/components/ConfigForm.tsx rename to ui/components/PeerForms/SnowflakeForm.tsx index 7cd816415c..e2db335723 100644 --- a/ui/components/ConfigForm.tsx +++ b/ui/components/PeerForms/SnowflakeForm.tsx @@ -1,19 +1,18 @@ 'use client'; -import { PeerConfig } from '@/app/dto/PeersDTO'; +import { PeerSetter } from '@/app/dto/PeersDTO'; import { PeerSetting } from '@/app/peers/create/[peerType]/helpers/common'; import { Label } from '@/lib/Label'; import { RowWithTextField } from '@/lib/Layout'; import { TextField } from '@/lib/TextField'; import { Tooltip } from '@/lib/Tooltip'; -import { InfoPopover } from './InfoPopover'; +import { InfoPopover } from '../InfoPopover'; -export type PeerSetter = React.Dispatch>; interface ConfigProps { settings: PeerSetting[]; setter: PeerSetter; } -export default function ConfigForm(props: ConfigProps) { +export default function SnowflakeForm(props: ConfigProps) { const handleFile = ( file: File, setFile: (value: string, setter: PeerSetter) => void diff --git a/ui/grpc_generated/peers.ts b/ui/grpc_generated/peers.ts index 8d4b9b4eef..7cdd2ca50b 100644 --- a/ui/grpc_generated/peers.ts +++ b/ui/grpc_generated/peers.ts @@ -73,6 +73,14 @@ export function dBTypeToJSON(object: DBType): string { } } +export interface SSHConfig { + host: string; + port: number; + user: string; + password: string; + privateKey: string; +} + export interface SnowflakeConfig { accountId: string; username: string; @@ -121,6 +129,7 @@ export interface PostgresConfig { transactionSnapshot: string; /** defaults to _peerdb_internal */ metadataSchema?: string | undefined; + sshConfig?: SSHConfig | undefined; } export interface EventHubConfig { @@ -181,6 +190,125 @@ export interface Peer { eventhubGroupConfig?: EventHubGroupConfig | undefined; } +function createBaseSSHConfig(): SSHConfig { + return { host: "", port: 0, user: "", password: "", privateKey: "" }; +} + +export const SSHConfig = { + encode(message: SSHConfig, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.host !== "") { + writer.uint32(10).string(message.host); + } + if (message.port !== 0) { + writer.uint32(16).uint32(message.port); + } + if (message.user !== "") { + writer.uint32(26).string(message.user); + } + if (message.password !== "") { + writer.uint32(34).string(message.password); + } + if (message.privateKey !== "") { + writer.uint32(42).string(message.privateKey); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SSHConfig { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSSHConfig(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.host = reader.string(); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.port = reader.uint32(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.user = reader.string(); + continue; + case 4: + if (tag !== 34) { + break; + } + + message.password = reader.string(); + continue; + case 5: + if (tag !== 42) { + break; + } + + message.privateKey = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SSHConfig { + return { + host: isSet(object.host) ? String(object.host) : "", + port: isSet(object.port) ? Number(object.port) : 0, + user: isSet(object.user) ? String(object.user) : "", + password: isSet(object.password) ? String(object.password) : "", + privateKey: isSet(object.privateKey) ? String(object.privateKey) : "", + }; + }, + + toJSON(message: SSHConfig): unknown { + const obj: any = {}; + if (message.host !== "") { + obj.host = message.host; + } + if (message.port !== 0) { + obj.port = Math.round(message.port); + } + if (message.user !== "") { + obj.user = message.user; + } + if (message.password !== "") { + obj.password = message.password; + } + if (message.privateKey !== "") { + obj.privateKey = message.privateKey; + } + return obj; + }, + + create, I>>(base?: I): SSHConfig { + return SSHConfig.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): SSHConfig { + const message = createBaseSSHConfig(); + message.host = object.host ?? ""; + message.port = object.port ?? 0; + message.user = object.user ?? ""; + message.password = object.password ?? ""; + message.privateKey = object.privateKey ?? ""; + return message; + }, +}; + function createBaseSnowflakeConfig(): SnowflakeConfig { return { accountId: "", @@ -735,6 +863,7 @@ function createBasePostgresConfig(): PostgresConfig { database: "", transactionSnapshot: "", metadataSchema: undefined, + sshConfig: undefined, }; } @@ -761,6 +890,9 @@ export const PostgresConfig = { if (message.metadataSchema !== undefined) { writer.uint32(58).string(message.metadataSchema); } + if (message.sshConfig !== undefined) { + SSHConfig.encode(message.sshConfig, writer.uint32(66).fork()).ldelim(); + } return writer; }, @@ -820,6 +952,13 @@ export const PostgresConfig = { message.metadataSchema = reader.string(); continue; + case 8: + if (tag !== 66) { + break; + } + + message.sshConfig = SSHConfig.decode(reader, reader.uint32()); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -838,6 +977,7 @@ export const PostgresConfig = { database: isSet(object.database) ? String(object.database) : "", transactionSnapshot: isSet(object.transactionSnapshot) ? String(object.transactionSnapshot) : "", metadataSchema: isSet(object.metadataSchema) ? String(object.metadataSchema) : undefined, + sshConfig: isSet(object.sshConfig) ? SSHConfig.fromJSON(object.sshConfig) : undefined, }; }, @@ -864,6 +1004,9 @@ export const PostgresConfig = { if (message.metadataSchema !== undefined) { obj.metadataSchema = message.metadataSchema; } + if (message.sshConfig !== undefined) { + obj.sshConfig = SSHConfig.toJSON(message.sshConfig); + } return obj; }, @@ -879,6 +1022,9 @@ export const PostgresConfig = { message.database = object.database ?? ""; message.transactionSnapshot = object.transactionSnapshot ?? ""; message.metadataSchema = object.metadataSchema ?? undefined; + message.sshConfig = (object.sshConfig !== undefined && object.sshConfig !== null) + ? SSHConfig.fromPartial(object.sshConfig) + : undefined; return message; }, };