diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index dad255119..5a5291f82 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -4,9 +4,7 @@ on: push: branches: [main, release/*] pull_request: - branches: - - "main" - - "release/*" + branches: [main, release/*] jobs: build: diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 2ddff33bd..dffa4e084 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -2,8 +2,7 @@ name: Flow build and test on: pull_request: - branches: - - "main" + branches: [main] push: branches: [main] @@ -96,7 +95,7 @@ jobs: - name: run tests run: | - gotestsum --format testname -- -p 8 ./... -timeout 2400s + gotestsum --format testname -- -p 8 ./... -timeout 1200s working-directory: ./flow env: AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }} diff --git a/.github/workflows/golang-lint.yml b/.github/workflows/golang-lint.yml index 23b3adee0..6bead0102 100644 --- a/.github/workflows/golang-lint.yml +++ b/.github/workflows/golang-lint.yml @@ -2,8 +2,8 @@ name: GolangCI-Lint on: pull_request: - branches: - - "main" + branches: [main] + paths: [flow/**] jobs: golangci-lint: diff --git a/.github/workflows/rust-lint.yml b/.github/workflows/rust-lint.yml index 8a56cac05..c4a43ad79 100644 --- a/.github/workflows/rust-lint.yml +++ b/.github/workflows/rust-lint.yml @@ -2,8 +2,8 @@ name: clippy-action on: pull_request: - branches: - - "main" + branches: [main] + paths: [nexus/**] jobs: clippy: diff --git a/.github/workflows/ui-build.yml b/.github/workflows/ui-build.yml index ec59e1396..752bb9bd7 100644 --- a/.github/workflows/ui-build.yml +++ b/.github/workflows/ui-build.yml @@ -2,11 +2,10 @@ name: Build & Test UI on: push: - branches: - - main + branches: [main] pull_request: - branches: - - main + branches: [main] + paths: [ui/**] jobs: build-test: diff --git a/.github/workflows/ui-lint.yml b/.github/workflows/ui-lint.yml index 84fe76b0b..48428e157 100644 --- a/.github/workflows/ui-lint.yml +++ b/.github/workflows/ui-lint.yml @@ -2,11 +2,10 @@ name: Lint UI on: push: - branches: - - main + branches: [main] pull_request: - branches: - - main + branches: [main] + paths: [ui/**] permissions: checks: write diff --git a/dev-peerdb.sh b/dev-peerdb.sh index 230588421..796425f3d 100755 --- a/dev-peerdb.sh +++ b/dev-peerdb.sh @@ -1,12 +1,27 @@ -#!/bin/bash -set -Eeuo pipefail +#!/bin/sh +if test -z "$USE_PODMAN" +then + if ! command -v docker &> /dev/null + then + if command -v podman-compose + then + echo "docker could not be found on PATH, using podman-compose" + USE_PODMAN=1 + else + echo "docker could not be found on PATH" + exit 1 + fi + fi +fi -if ! command -v docker &> /dev/null +if test -z "$USE_PODMAN" then - echo "docker could not be found on PATH" - exit 1 + DOCKER="docker compose" + EXTRA_ARGS="--no-attach temporal --no-attach pyroscope --no-attach temporal-ui" +else + DOCKER="podman-compose --podman-run-args=--replace" + EXTRA_ARGS="" fi export PEERDB_VERSION_SHA_SHORT=local-$(git rev-parse --short HEAD) -docker compose -f docker-compose-dev.yml up --build \ - --no-attach temporal --no-attach pyroscope --no-attach temporal-ui +exec $DOCKER -f docker-compose-dev.yml up --build $EXTRA_ARGS diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index f5d8942ba..4f5a8ca8d 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -44,7 +44,7 @@ services: volumes: - pgdata:/var/lib/postgresql/data healthcheck: - test: ["CMD-SHELL", "pg_isready", "-d", "postgres", "-U", "postgres"] + test: ["CMD", "pg_isready", "-d", "postgres", "-U", "postgres"] interval: 10s timeout: 30s retries: 5 @@ -133,7 +133,7 @@ services: dockerfile: stacks/flow.Dockerfile target: flow-snapshot-worker environment: - <<: [*flow-worker-env] + <<: [*catalog-config, *flow-worker-env] depends_on: temporal-admin-tools: condition: service_healthy @@ -209,6 +209,8 @@ services: DATABASE_URL: postgres://postgres:postgres@catalog:5432/postgres PEERDB_FLOW_SERVER_HTTP: http://flow_api:8113 PEERDB_PASSWORD: + depends_on: + - flow-api volumes: pgdata: diff --git a/docker-compose.yml b/docker-compose.yml index 5645f2b1e..d6a4fbb12 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -37,7 +37,7 @@ services: volumes: - pgdata:/var/lib/postgresql/data healthcheck: - test: ["CMD-SHELL", "pg_isready", "-d", "postgres", "-U", "postgres"] + test: ["CMD", "pg_isready", "-d", "postgres", "-U", "postgres"] interval: 10s timeout: 30s retries: 5 @@ -112,7 +112,7 @@ services: container_name: flow-snapshot-worker image: ghcr.io/peerdb-io/flow-snapshot-worker:latest-dev environment: - <<: [*flow-worker-env] + <<: [*catalog-config, *flow-worker-env] depends_on: temporal-admin-tools: condition: service_healthy @@ -174,6 +174,8 @@ services: <<: *catalog-config DATABASE_URL: postgres://postgres:postgres@catalog:5432/postgres PEERDB_FLOW_SERVER_HTTP: http://flow_api:8113 + depends_on: + - flow-api volumes: pgdata: diff --git a/flow/.golangci.yml b/flow/.golangci.yml index 177566f39..ab1e6264d 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -21,8 +21,12 @@ linters: linters-settings: forbidigo: tests: false + stylecheck: + checks: + - all + - '-ST1003' lll: - line-length: 120 + line-length: 144 tab-width: 4 revive: rules: diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 69628c28d..66a4a2033 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -19,6 +19,7 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/peerdbenv" "github.com/PeerDB-io/peer-flow/shared" + "github.com/PeerDB-io/peer-flow/shared/alerting" "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" @@ -42,6 +43,7 @@ type SlotSnapshotSignal struct { type FlowableActivity struct { CatalogPool *pgxpool.Pool + Alerter *alerting.Alerter } // CheckConnection implements CheckConnection. @@ -70,7 +72,9 @@ func (a *FlowableActivity) SetupMetadataTables(ctx context.Context, config *prot } defer connectors.CloseConnector(dstConn) + flowName, _ := ctx.Value(shared.FlowNameKey).(string) if err := dstConn.SetupMetadataTables(); err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return fmt.Errorf("failed to setup metadata tables: %w", err) } @@ -109,6 +113,7 @@ func (a *FlowableActivity) EnsurePullability( output, err := srcConn.EnsurePullability(config) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return nil, fmt.Errorf("failed to ensure pullability: %w", err) } @@ -163,53 +168,16 @@ func (a *FlowableActivity) CreateNormalizedTable( } defer connectors.CloseConnector(conn) - return conn.SetupNormalizedTables(config) -} - -func (a *FlowableActivity) handleSlotInfo( - ctx context.Context, - srcConn connectors.CDCPullConnector, - slotName string, - peerName string, -) error { - slotInfo, err := srcConn.GetSlotInfo(slotName) + setupNormalizedTablesOutput, err := conn.SetupNormalizedTables(config) if err != nil { - slog.Warn("warning: failed to get slot info", slog.Any("error", err)) - return err + flowName, _ := ctx.Value(shared.FlowNameKey).(string) + a.Alerter.LogFlowError(ctx, flowName, err) + return nil, fmt.Errorf("failed to setup normalized tables: %w", err) } - if len(slotInfo) != 0 { - return monitoring.AppendSlotSizeInfo(ctx, a.CatalogPool, peerName, slotInfo[0]) - } - return nil + return setupNormalizedTablesOutput, nil } -func (a *FlowableActivity) recordSlotSizePeriodically( - ctx context.Context, - srcConn connectors.CDCPullConnector, - slotName string, - peerName string, -) { - timeout := 10 * time.Minute - ticker := time.NewTicker(timeout) - - defer ticker.Stop() - for { - select { - case <-ticker.C: - err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) - if err != nil { - return - } - case <-ctx.Done(): - return - } - ticker.Stop() - ticker = time.NewTicker(timeout) - } -} - -// StartFlow implements StartFlow. func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlowInput, ) (*model.SyncResponse, error) { @@ -220,6 +188,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, return nil, fmt.Errorf("failed to get destination connector: %w", err) } defer connectors.CloseConnector(dstConn) + slog.InfoContext(ctx, "initializing table schema...") err = dstConn.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { @@ -232,10 +201,6 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, tblNameMapping[v.SourceTableIdentifier] = model.NewNameAndExclude(v.DestinationTableIdentifier, v.Exclude) } - recordBatch := model.NewCDCRecordStream() - - startTime := time.Now() - errGroup, errCtx := errgroup.WithContext(ctx) srcConn, err := connectors.GetCDCPullConnector(errCtx, conn.Source) if err != nil { @@ -251,31 +216,37 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, go a.recordSlotSizePeriodically(errCtx, srcConn, slotNameForMetrics, input.FlowConnectionConfigs.Source.Name) // start a goroutine to pull records from the source + recordBatch := model.NewCDCRecordStream() + startTime := time.Now() + flowName := input.FlowConnectionConfigs.FlowJobName errGroup.Go(func() error { return srcConn.PullRecords(a.CatalogPool, &model.PullRecordsRequest{ - FlowJobName: input.FlowConnectionConfigs.FlowJobName, + FlowJobName: flowName, SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, TableNameMapping: tblNameMapping, LastOffset: input.LastSyncState.Checkpoint, MaxBatchSize: uint32(input.SyncFlowOptions.BatchSize), - IdleTimeout: peerdbenv.GetPeerDBCDCIdleTimeoutSeconds(), + IdleTimeout: peerdbenv.PeerDBCDCIdleTimeoutSeconds(), TableNameSchemaMapping: input.FlowConnectionConfigs.TableNameSchemaMapping, OverridePublicationName: input.FlowConnectionConfigs.PublicationName, OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, RelationMessageMapping: input.RelationMessageMapping, RecordStream: recordBatch, + SetLastOffset: func(lastOffset int64) error { + return dstConn.SetLastOffset(flowName, lastOffset) + }, }) }) hasRecords := !recordBatch.WaitAndCheckEmpty() slog.InfoContext(ctx, fmt.Sprintf("the current sync flow has records: %v", hasRecords)) if a.CatalogPool != nil && hasRecords { - syncBatchID, err := dstConn.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) + syncBatchID, err := dstConn.GetLastSyncBatchID(flowName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } - err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, + err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, flowName, monitoring.CDCBatchInfo{ BatchID: syncBatchID + 1, RowsInBatch: 0, @@ -283,6 +254,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, StartTime: startTime, }) if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, err } } @@ -291,6 +263,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, // wait for the pull goroutine to finish err = errGroup.Wait() if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to pull records: %w", err) } slog.InfoContext(ctx, "no records to push") @@ -319,11 +292,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, }) if err != nil { slog.Warn("failed to push records", slog.Any("error", err)) + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to push records: %w", err) } err = errGroup.Wait() if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to pull records: %w", err) } @@ -426,6 +401,7 @@ func (a *FlowableActivity) StartNormalize( SyncedAtColName: input.FlowConnectionConfigs.SyncedAtColName, }) if err != nil { + a.Alerter.LogFlowError(ctx, input.FlowConnectionConfigs.FlowJobName, err) return nil, fmt.Errorf("failed to normalized records: %w", err) } @@ -463,7 +439,13 @@ func (a *FlowableActivity) ReplayTableSchemaDeltas( } defer connectors.CloseConnector(dest) - return dest.ReplayTableSchemaDeltas(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDeltas) + err = dest.ReplayTableSchemaDeltas(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDeltas) + if err != nil { + a.Alerter.LogFlowError(ctx, input.FlowConnectionConfigs.FlowJobName, err) + return fmt.Errorf("failed to replay table schema deltas: %w", err) + } + + return nil } // SetupQRepMetadataTables sets up the metadata tables for QReplication. @@ -474,7 +456,13 @@ func (a *FlowableActivity) SetupQRepMetadataTables(ctx context.Context, config * } defer connectors.CloseConnector(conn) - return conn.SetupQRepMetadataTables(config) + err = conn.SetupQRepMetadataTables(config) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to setup metadata tables: %w", err) + } + + return nil } // GetQRepPartitions returns the partitions for a given QRepConfig. @@ -499,6 +487,7 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, partitions, err := srcConn.GetQRepPartitions(config, last) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return nil, fmt.Errorf("failed to get partitions from source: %w", err) } if len(partitions) > 0 { @@ -539,6 +528,7 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, slog.InfoContext(ctx, fmt.Sprintf("batch-%d - replicating partition - %s\n", partitions.BatchId, p.PartitionId)) err := a.replicateQRepPartition(ctx, config, i+1, numPartitions, p, runUUID) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return err } } @@ -592,7 +582,8 @@ func (a *FlowableActivity) replicateQRepPartition(ctx context.Context, slog.Error("failed to pull records", slog.Any("error", err)) goroutineErr = err } else { - err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, a.CatalogPool, runUUID, partition, numRecords) + err = monitoring.UpdatePullEndTimeAndRowsForPartition(ctx, + a.CatalogPool, runUUID, partition, numRecords) if err != nil { slog.Error(fmt.Sprintf("%v", err)) goroutineErr = err @@ -678,6 +669,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config err = dstConn.ConsolidateQRepPartitions(config) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return err } @@ -752,6 +744,11 @@ func (a *FlowableActivity) getPostgresPeerConfigs(ctx context.Context) ([]*proto } func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { + if !peerdbenv.PeerDBEnableWALHeartbeat() { + slog.InfoContext(ctx, "wal heartbeat is disabled") + return nil + } + sendTimeout := 10 * time.Minute ticker := time.NewTicker(sendTimeout) defer ticker.Stop() @@ -941,7 +938,8 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, }, } } - updateErr := monitoring.InitializeQRepRun(ctx, a.CatalogPool, config, runUUID, []*protos.QRepPartition{partitionForMetrics}) + updateErr := monitoring.InitializeQRepRun( + ctx, a.CatalogPool, config, runUUID, []*protos.QRepPartition{partitionForMetrics}) if updateErr != nil { return updateErr } @@ -951,7 +949,8 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, return fmt.Errorf("failed to update start time for partition: %w", err) } - err = monitoring.UpdatePullEndTimeAndRowsForPartition(errCtx, a.CatalogPool, runUUID, partition, int64(numRecords)) + err = monitoring.UpdatePullEndTimeAndRowsForPartition( + errCtx, a.CatalogPool, runUUID, partition, int64(numRecords)) if err != nil { slog.Error(fmt.Sprintf("%v", err)) return err @@ -978,6 +977,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, } else { err := errGroup.Wait() if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return 0, err } diff --git a/flow/activities/slot.go b/flow/activities/slot.go new file mode 100644 index 000000000..baa0fbc0f --- /dev/null +++ b/flow/activities/slot.go @@ -0,0 +1,94 @@ +package activities + +import ( + "context" + "fmt" + "log/slog" + "time" + + "github.com/PeerDB-io/peer-flow/connectors" + "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" + "github.com/PeerDB-io/peer-flow/peerdbenv" +) + +func (a *FlowableActivity) handleSlotInfo( + ctx context.Context, + srcConn connectors.CDCPullConnector, + slotName string, + peerName string, +) error { + slotInfo, err := srcConn.GetSlotInfo(slotName) + if err != nil { + slog.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) + return err + } + + if len(slotInfo) == 0 { + slog.WarnContext(ctx, "warning: unable to get slot info", slog.Any("slotName", slotName)) + return nil + } + + deploymentUIDPrefix := "" + if peerdbenv.PeerDBDeploymentUID() != "" { + deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.PeerDBDeploymentUID()) + } + + slotLagInMBThreshold := peerdbenv.PeerDBSlotLagMBAlertThreshold() + if (slotLagInMBThreshold > 0) && (slotInfo[0].LagInMb >= float32(slotLagInMBThreshold)) { + a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-slot-lag-threshold-exceeded", peerName), + fmt.Sprintf(`%sSlot `+"`%s`"+` on peer `+"`%s`"+` has exceeded threshold size of %dMB, currently at %.2fMB! +cc: `, + deploymentUIDPrefix, slotName, peerName, slotLagInMBThreshold, slotInfo[0].LagInMb)) + } + + // Also handles alerts for PeerDB user connections exceeding a given limit here + maxOpenConnectionsThreshold := peerdbenv.PeerDBOpenConnectionsAlertThreshold() + res, err := srcConn.GetOpenConnectionsForUser() + if err != nil { + slog.WarnContext(ctx, "warning: failed to get current open connections", slog.Any("error", err)) + return err + } + if (maxOpenConnectionsThreshold > 0) && (res.CurrentOpenConnections >= int64(maxOpenConnectionsThreshold)) { + a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-max-open-connections-threshold-exceeded", peerName), + fmt.Sprintf(`%sOpen connections from PeerDB user `+"`%s`"+` on peer `+"`%s`"+ + ` has exceeded threshold size of %d connections, currently at %d connections! +cc: `, + deploymentUIDPrefix, res.UserName, peerName, maxOpenConnectionsThreshold, res.CurrentOpenConnections)) + } + + if len(slotInfo) != 0 { + return monitoring.AppendSlotSizeInfo(ctx, a.CatalogPool, peerName, slotInfo[0]) + } + return nil +} + +func (a *FlowableActivity) recordSlotSizePeriodically( + ctx context.Context, + srcConn connectors.CDCPullConnector, + slotName string, + peerName string, +) { + // ensures slot info is logged at least once per SyncFlow + err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) + if err != nil { + return + } + + timeout := 5 * time.Minute + ticker := time.NewTicker(timeout) + + defer ticker.Stop() + for { + select { + case <-ticker.C: + err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) + if err != nil { + return + } + case <-ctx.Done(): + return + } + ticker.Stop() + ticker = time.NewTicker(timeout) + } +} diff --git a/flow/cmd/api.go b/flow/cmd/api.go index 09185a0fc..b16034bf2 100644 --- a/flow/cmd/api.go +++ b/flow/cmd/api.go @@ -104,7 +104,10 @@ func APIMain(args *APIServerParams) error { } connOptions := client.ConnectionOptions{ - TLS: &tls.Config{Certificates: certs}, + TLS: &tls.Config{ + Certificates: certs, + MinVersion: tls.VersionTLS13, + }, } clientOptions.ConnectionOptions = connOptions } diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 9efdc59fb..04a9ccd70 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -138,10 +138,9 @@ func (h *FlowRequestHandler) CreateCDCFlow( } limits := &peerflow.CDCFlowLimits{ - TotalSyncFlows: 0, - ExitAfterRecords: -1, - TotalNormalizeFlows: 0, - MaxBatchSize: maxBatchSize, + TotalSyncFlows: 0, + ExitAfterRecords: -1, + MaxBatchSize: maxBatchSize, } if req.ConnectionConfigs.SoftDeleteColName == "" { @@ -261,13 +260,22 @@ func (h *FlowRequestHandler) CreateQRepFlow( 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}}} + state.LastPartition.Range = &protos.PartitionRange{ + Range: &protos.PartitionRange_IntRange{IntRange: &protos.IntPartitionRange{Start: txid}}, + } } workflowFn = peerflow.XminFlowWorkflow } else { workflowFn = peerflow.QRepFlowWorkflow } + + if req.QrepConfig.SyncedAtColName == "" { + cfg.SyncedAtColName = "_PEERDB_SYNCED_AT" + } else { + // make them all uppercase + cfg.SyncedAtColName = strings.ToUpper(req.QrepConfig.SyncedAtColName) + } _, err := h.temporalClient.ExecuteWorkflow(ctx, workflowOptions, workflowFn, cfg, state) if err != nil { slog.Error("unable to start QRepFlow workflow", diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 454dceb20..f9383d8c5 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -31,24 +31,24 @@ func (h *FlowRequestHandler) getPGPeerConfig(ctx context.Context, peerName strin return &pgPeerConfig, nil } -func (h *FlowRequestHandler) getPoolForPGPeer(ctx context.Context, peerName string) (*pgxpool.Pool, string, error) { +func (h *FlowRequestHandler) getPoolForPGPeer(ctx context.Context, peerName string) (*pgxpool.Pool, error) { pgPeerConfig, err := h.getPGPeerConfig(ctx, peerName) if err != nil { - return nil, "", err + return nil, err } connStr := utils.GetPGConnectionString(pgPeerConfig) peerPool, err := pgxpool.New(ctx, connStr) if err != nil { - return nil, "", err + return nil, err } - return peerPool, pgPeerConfig.User, nil + return peerPool, nil } func (h *FlowRequestHandler) GetSchemas( ctx context.Context, req *protos.PostgresPeerActivityInfoRequest, ) (*protos.PeerSchemasResponse, error) { - peerPool, _, err := h.getPoolForPGPeer(ctx, req.PeerName) + peerPool, err := h.getPoolForPGPeer(ctx, req.PeerName) if err != nil { return &protos.PeerSchemasResponse{Schemas: nil}, err } @@ -78,7 +78,7 @@ func (h *FlowRequestHandler) GetTablesInSchema( ctx context.Context, req *protos.SchemaTablesRequest, ) (*protos.SchemaTablesResponse, error) { - peerPool, _, err := h.getPoolForPGPeer(ctx, req.PeerName) + peerPool, err := h.getPoolForPGPeer(ctx, req.PeerName) if err != nil { return &protos.SchemaTablesResponse{Tables: nil}, err } @@ -110,7 +110,7 @@ func (h *FlowRequestHandler) GetAllTables( ctx context.Context, req *protos.PostgresPeerActivityInfoRequest, ) (*protos.AllTablesResponse, error) { - peerPool, _, err := h.getPoolForPGPeer(ctx, req.PeerName) + peerPool, err := h.getPoolForPGPeer(ctx, req.PeerName) if err != nil { return &protos.AllTablesResponse{Tables: nil}, err } @@ -140,7 +140,7 @@ func (h *FlowRequestHandler) GetColumns( ctx context.Context, req *protos.TableColumnsRequest, ) (*protos.TableColumnsResponse, error) { - peerPool, _, err := h.getPoolForPGPeer(ctx, req.PeerName) + peerPool, err := h.getPoolForPGPeer(ctx, req.PeerName) if err != nil { return &protos.TableColumnsResponse{Columns: nil}, err } @@ -209,7 +209,7 @@ func (h *FlowRequestHandler) GetSlotInfo( return &protos.PeerSlotResponse{SlotData: nil}, err } - pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) + pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig, false) if err != nil { slog.Error("Failed to create postgres connector", slog.Any("error", err)) return &protos.PeerSlotResponse{SlotData: nil}, err @@ -236,7 +236,7 @@ func (h *FlowRequestHandler) GetStatInfo( return &protos.PeerStatResponse{StatData: nil}, err } - pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) + pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig, false) if err != nil { slog.Error("Failed to create postgres connector", slog.Any("error", err)) return &protos.PeerStatResponse{StatData: nil}, err diff --git a/flow/cmd/snapshot_worker.go b/flow/cmd/snapshot_worker.go index 16008cc6a..c68d44d92 100644 --- a/flow/cmd/snapshot_worker.go +++ b/flow/cmd/snapshot_worker.go @@ -32,7 +32,10 @@ func SnapshotWorkerMain(opts *SnapshotWorkerOptions) error { } connOptions := client.ConnectionOptions{ - TLS: &tls.Config{Certificates: certs}, + TLS: &tls.Config{ + Certificates: certs, + MinVersion: tls.VersionTLS13, + }, } clientOptions.ConnectionOptions = connOptions } diff --git a/flow/cmd/version.go b/flow/cmd/version.go index 94210ea9c..3338a20e6 100644 --- a/flow/cmd/version.go +++ b/flow/cmd/version.go @@ -11,6 +11,6 @@ func (h *FlowRequestHandler) GetVersion( ctx context.Context, req *protos.PeerDBVersionRequest, ) (*protos.PeerDBVersionResponse, error) { - version := peerdbenv.GetPeerDBVersionShaShort() + version := peerdbenv.PeerDBVersionShaShort() return &protos.PeerDBVersionResponse{Version: version}, nil } diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index ba6e0d0e1..f060230b6 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -13,6 +13,7 @@ import ( "github.com/PeerDB-io/peer-flow/activities" utils "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/shared" + "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/grafana/pyroscope-go" @@ -99,7 +100,10 @@ func WorkerMain(opts *WorkerOptions) error { return fmt.Errorf("unable to process certificate and key: %w", err) } connOptions := client.ConnectionOptions{ - TLS: &tls.Config{Certificates: certs}, + TLS: &tls.Config{ + Certificates: certs, + MinVersion: tls.VersionTLS13, + }, } clientOptions.ConnectionOptions = connOptions } @@ -131,8 +135,15 @@ func WorkerMain(opts *WorkerOptions) error { w.RegisterWorkflow(peerflow.XminFlowWorkflow) w.RegisterWorkflow(peerflow.DropFlowWorkflow) w.RegisterWorkflow(peerflow.HeartbeatFlowWorkflow) + + alerter, err := alerting.NewAlerter(conn) + if err != nil { + return fmt.Errorf("unable to create alerter: %w", err) + } + w.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, + Alerter: alerter, }) err = w.Run(worker.InterruptCh()) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 1ed1982bf..7fd86ba52 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -18,7 +18,6 @@ import ( "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" - "github.com/google/uuid" "github.com/jackc/pgx/v5/pgxpool" "go.temporal.io/sdk/activity" @@ -29,9 +28,7 @@ import ( const ( /* Different batch Ids in code/BigQuery - 1. batchID - identifier in raw/staging tables on target to depict which batch a row was inserted. - 2. stagingBatchID - the random batch id we generate before ingesting into staging table. - helps filter rows in the current batch before inserting into raw table. + 1. batchID - identifier in raw table on target to depict which batch a row was inserted. 3. syncBatchID - batch id that was last synced or will be synced 4. normalizeBatchID - batch id that was last normalized or will be normalized. */ @@ -71,19 +68,6 @@ type BigQueryConnector struct { logger slog.Logger } -type StagingBQRecord struct { - uid string `bigquery:"_peerdb_uid"` - timestamp time.Time `bigquery:"_peerdb_timestamp"` - timestampNanos int64 `bigquery:"_peerdb_timestamp_nanos"` - destinationTableName string `bigquery:"_peerdb_destination_table_name"` - data string `bigquery:"_peerdb_data"` - recordType int `bigquery:"_peerdb_record_type"` - matchData string `bigquery:"_peerdb_match_data"` - batchID int64 `bigquery:"_peerdb_batch_id"` - stagingBatchID int64 `bigquery:"_peerdb_staging_batch_id"` - unchangedToastColumns string `bigquery:"_peerdb_unchanged_toast_columns"` -} - // Create BigQueryServiceAccount from BigqueryConfig func NewBigQueryServiceAccount(bqConfig *protos.BigqueryConfig) (*BigQueryServiceAccount, error) { var serviceAccount BigQueryServiceAccount @@ -233,16 +217,16 @@ func (c *BigQueryConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } -func (c *BigQueryConnector) WaitForTableReady(tblName string) error { - table := c.client.Dataset(c.datasetID).Table(tblName) +func (c *BigQueryConnector) waitForTableReady(datasetTable *datasetTable) error { + table := c.client.Dataset(datasetTable.dataset).Table(datasetTable.table) maxDuration := 5 * time.Minute deadline := time.Now().Add(maxDuration) - sleepInterval := 15 * time.Second + sleepInterval := 5 * time.Second attempt := 0 for { if time.Now().After(deadline) { - return fmt.Errorf("timeout reached while waiting for table %s to be ready", tblName) + return fmt.Errorf("timeout reached while waiting for table %s to be ready", datasetTable) } _, err := table.Metadata(c.ctx) @@ -250,7 +234,8 @@ func (c *BigQueryConnector) WaitForTableReady(tblName string) error { return nil } - slog.Info("waiting for table to be ready", slog.String("table", tblName), slog.Int("attempt", attempt)) + slog.Info("waiting for table to be ready", + slog.String("table", datasetTable.table), slog.Int("attempt", attempt)) attempt++ time.Sleep(sleepInterval) } @@ -267,9 +252,10 @@ func (c *BigQueryConnector) ReplayTableSchemaDeltas(flowJobName string, } for _, addedColumn := range schemaDelta.AddedColumns { + dstDatasetTable, _ := c.convertToDatasetTable(schemaDelta.DstTableName) _, err := c.client.Query(fmt.Sprintf( - "ALTER TABLE %s.%s ADD COLUMN IF NOT EXISTS `%s` %s", c.datasetID, - schemaDelta.DstTableName, addedColumn.ColumnName, + "ALTER TABLE %s.%s ADD COLUMN IF NOT EXISTS `%s` %s", dstDatasetTable.dataset, + dstDatasetTable.table, addedColumn.ColumnName, qValueKindToBigQueryType(addedColumn.ColumnType))).Read(c.ctx) if err != nil { return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, @@ -340,6 +326,23 @@ func (c *BigQueryConnector) GetLastOffset(jobName string) (int64, error) { } } +func (c *BigQueryConnector) SetLastOffset(jobName string, lastOffset int64) error { + query := fmt.Sprintf( + "UPDATE %s.%s SET offset = GREATEST(offset, %d) WHERE mirror_job_name = '%s'", + c.datasetID, + MirrorJobsTable, + lastOffset, + jobName, + ) + q := c.client.Query(query) + _, err := q.Read(c.ctx) + if err != nil { + return fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) + } + + return nil +} + func (c *BigQueryConnector) GetLastSyncBatchID(jobName string) (int64, error) { query := fmt.Sprintf("SELECT sync_batch_id FROM %s.%s WHERE mirror_job_name = '%s'", c.datasetID, MirrorJobsTable, jobName) @@ -365,29 +368,35 @@ func (c *BigQueryConnector) GetLastSyncBatchID(jobName string) (int64, error) { } } -func (c *BigQueryConnector) GetLastNormalizeBatchID(jobName string) (int64, error) { - query := fmt.Sprintf("SELECT normalize_batch_id FROM %s.%s WHERE mirror_job_name = '%s'", +func (c *BigQueryConnector) GetLastSyncAndNormalizeBatchID(jobName string) (model.SyncAndNormalizeBatchID, error) { + query := fmt.Sprintf("SELECT sync_batch_id, normalize_batch_id FROM %s.%s WHERE mirror_job_name = '%s'", c.datasetID, MirrorJobsTable, jobName) q := c.client.Query(query) it, err := q.Read(c.ctx) if err != nil { err = fmt.Errorf("failed to run query %s on BigQuery:\n %w", query, err) - return -1, err + return model.SyncAndNormalizeBatchID{}, err } var row []bigquery.Value err = it.Next(&row) if err != nil { c.logger.Info("no row found for job") - return 0, nil + return model.SyncAndNormalizeBatchID{}, nil } - if row[0] == nil { - c.logger.Info("no normalize_batch_id found returning 0") - return 0, nil - } else { - return row[0].(int64), nil + syncBatchID := int64(0) + normBatchID := int64(0) + if row[0] != nil { + syncBatchID = row[0].(int64) } + if row[1] != nil { + normBatchID = row[1].(int64) + } + return model.SyncAndNormalizeBatchID{ + SyncBatchID: syncBatchID, + NormalizeBatchID: normBatchID, + }, nil } func (c *BigQueryConnector) getDistinctTableNamesInBatch(flowJobName string, syncBatchID int64, @@ -433,9 +442,13 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync rawTableName := c.getRawTableName(flowJobName) // Prepare the query to retrieve distinct tables in that batch + // we want to only select the unchanged cols from UpdateRecords, as we have a workaround + // where a placeholder value for unchanged cols can be set in DeleteRecord if there is no backfill + // we don't want these particular DeleteRecords to be used in the update statement query := fmt.Sprintf(`SELECT _peerdb_destination_table_name, array_agg(DISTINCT _peerdb_unchanged_toast_columns) as unchanged_toast_columns FROM %s.%s - WHERE _peerdb_batch_id > %d and _peerdb_batch_id <= %d GROUP BY _peerdb_destination_table_name`, + WHERE _peerdb_batch_id > %d AND _peerdb_batch_id <= %d AND _peerdb_record_type != 2 + GROUP BY _peerdb_destination_table_name`, c.datasetID, rawTableName, normalizeBatchID, syncBatchID) // Run the query q := c.client.Query(query) @@ -466,16 +479,15 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync } // SyncRecords pushes records to the destination. -// currently only supports inserts,updates and deletes -// more record types will be added in the future. +// Currently only supports inserts, updates, and deletes. +// More record types will be added in the future. func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { rawTableName := c.getRawTableName(req.FlowJobName) c.logger.Info(fmt.Sprintf("pushing records to %s.%s...", c.datasetID, rawTableName)) - // generate a sequential number for the last synced batch - // this sequence will be used to keep track of records that are normalized - // in the NormalizeFlowWorkflow + // generate a sequential number for last synced batch this sequence will be + // used to keep track of records that are normalized in NormalizeFlowWorkflow syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) if err != nil { return nil, fmt.Errorf("failed to get batch for the current mirror: %v", err) @@ -496,211 +508,31 @@ func (c *BigQueryConnector) syncRecordsViaAvro( syncBatchID int64, ) (*model.SyncResponse, error) { tableNameRowsMapping := make(map[string]uint32) - recordStream := model.NewQRecordStream(1 << 20) - err := recordStream.SetSchema(&model.QRecordSchema{ - Fields: []model.QField{ - { - Name: "_peerdb_uid", - Type: qvalue.QValueKindString, - Nullable: false, - }, - { - Name: "_peerdb_timestamp", - Type: qvalue.QValueKindTimestamp, - Nullable: false, - }, - { - Name: "_peerdb_timestamp_nanos", - Type: qvalue.QValueKindInt64, - Nullable: false, - }, - { - Name: "_peerdb_destination_table_name", - Type: qvalue.QValueKindString, - Nullable: false, - }, - { - Name: "_peerdb_data", - Type: qvalue.QValueKindString, - Nullable: false, - }, - { - Name: "_peerdb_record_type", - Type: qvalue.QValueKindInt64, - Nullable: true, - }, - { - Name: "_peerdb_match_data", - Type: qvalue.QValueKindString, - Nullable: true, - }, - { - Name: "_peerdb_staging_batch_id", - Type: qvalue.QValueKindInt64, - Nullable: true, - }, - { - Name: "_peerdb_batch_id", - Type: qvalue.QValueKindInt64, - Nullable: true, - }, - { - Name: "_peerdb_unchanged_toast_columns", - Type: qvalue.QValueKindString, - Nullable: true, - }, - }, - }) + streamReq := model.NewRecordsToStreamRequest(req.Records.GetRecords(), tableNameRowsMapping, syncBatchID) + streamRes, err := utils.RecordsToRawTableStream(streamReq) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to convert records to raw table stream: %w", err) } - // loop over req.Records - for record := range req.Records.GetRecords() { - var entries [10]qvalue.QValue - switch r := record.(type) { - case *model.InsertRecord: - - itemsJSON, err := r.Items.ToJSON() - if err != nil { - return nil, fmt.Errorf("failed to create items to json: %v", err) - } - - entries[3] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: r.DestinationTableName, - } - entries[4] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: itemsJSON, - } - entries[5] = qvalue.QValue{ - Kind: qvalue.QValueKindInt64, - Value: 0, - } - entries[6] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: "", - } - entries[9] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: "", - } - - tableNameRowsMapping[r.DestinationTableName] += 1 - case *model.UpdateRecord: - newItemsJSON, err := r.NewItems.ToJSON() - if err != nil { - return nil, fmt.Errorf("failed to create new items to json: %v", err) - } - - oldItemsJSON, err := r.OldItems.ToJSON() - if err != nil { - return nil, fmt.Errorf("failed to create old items to json: %v", err) - } - - entries[3] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: r.DestinationTableName, - } - entries[4] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: newItemsJSON, - } - entries[5] = qvalue.QValue{ - Kind: qvalue.QValueKindInt64, - Value: 1, - } - entries[6] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: oldItemsJSON, - } - entries[9] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: utils.KeysToString(r.UnchangedToastColumns), - } - - tableNameRowsMapping[r.DestinationTableName] += 1 - case *model.DeleteRecord: - itemsJSON, err := r.Items.ToJSON() - if err != nil { - return nil, fmt.Errorf("failed to create items to json: %v", err) - } - - entries[3] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: r.DestinationTableName, - } - entries[4] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: itemsJSON, - } - entries[5] = qvalue.QValue{ - Kind: qvalue.QValueKindInt64, - Value: 2, - } - entries[6] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: itemsJSON, - } - entries[9] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: "", - } - - tableNameRowsMapping[r.DestinationTableName] += 1 - default: - return nil, fmt.Errorf("record type %T not supported", r) - } - - entries[0] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: uuid.New().String(), - } - entries[1] = qvalue.QValue{ - Kind: qvalue.QValueKindTimestamp, - Value: time.Now(), - } - entries[2] = qvalue.QValue{ - Kind: qvalue.QValueKindInt64, - Value: time.Now().UnixNano(), - } - entries[7] = qvalue.QValue{ - Kind: qvalue.QValueKindInt64, - Value: syncBatchID, - } - entries[8] = qvalue.QValue{ - Kind: qvalue.QValueKindInt64, - Value: syncBatchID, - } - recordStream.Records <- model.QRecordOrError{ - Record: model.QRecord{ - NumEntries: 10, - Entries: entries[:], - }, - } - } - - close(recordStream.Records) avroSync := NewQRepAvroSyncMethod(c, req.StagingPath, req.FlowJobName) rawTableMetadata, err := c.client.Dataset(c.datasetID).Table(rawTableName).Metadata(c.ctx) if err != nil { return nil, fmt.Errorf("failed to get metadata of destination table: %v", err) } - lastCP, err := req.Records.GetLastCheckpoint() - if err != nil { - return nil, fmt.Errorf("failed to get last checkpoint: %v", err) - } - numRecords, err := avroSync.SyncRecords(rawTableName, req.FlowJobName, - lastCP, rawTableMetadata, syncBatchID, recordStream) + req.Records, rawTableMetadata, syncBatchID, streamRes.Stream) if err != nil { return nil, fmt.Errorf("failed to sync records via avro : %v", err) } c.logger.Info(fmt.Sprintf("pushed %d records to %s.%s", numRecords, c.datasetID, rawTableName)) + lastCP, err := req.Records.GetLastCheckpoint() + if err != nil { + return nil, fmt.Errorf("failed to get last checkpoint: %v", err) + } + return &model.SyncResponse{ LastSyncedCheckPointID: lastCP, NumRecordsSynced: int64(numRecords), @@ -713,13 +545,7 @@ func (c *BigQueryConnector) syncRecordsViaAvro( func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { rawTableName := c.getRawTableName(req.FlowJobName) - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, fmt.Errorf("failed to get batch for the current mirror: %v", err) - } - - // get last batchid that has been normalize - normalizeBatchID, err := c.GetLastNormalizeBatchID(req.FlowJobName) + batchIDs, err := c.GetLastSyncAndNormalizeBatchID(req.FlowJobName) if err != nil { return nil, fmt.Errorf("failed to get batch for the current mirror: %v", err) } @@ -730,77 +556,76 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) } // if job is not yet found in the peerdb_mirror_jobs_table // OR sync is lagging end normalize - if !hasJob || normalizeBatchID == syncBatchID { + if !hasJob || batchIDs.NormalizeBatchID >= batchIDs.SyncBatchID { c.logger.Info("waiting for sync to catch up, so finishing") return &model.NormalizeResponse{ Done: false, - StartBatchID: normalizeBatchID, - EndBatchID: syncBatchID, + StartBatchID: batchIDs.NormalizeBatchID, + EndBatchID: batchIDs.SyncBatchID, }, nil } - distinctTableNames, err := c.getDistinctTableNamesInBatch(req.FlowJobName, syncBatchID, normalizeBatchID) + distinctTableNames, err := c.getDistinctTableNamesInBatch( + req.FlowJobName, + batchIDs.SyncBatchID, + batchIDs.NormalizeBatchID, + ) if err != nil { return nil, fmt.Errorf("couldn't get distinct table names to normalize: %w", err) } - tableNametoUnchangedToastCols, err := c.getTableNametoUnchangedCols(req.FlowJobName, syncBatchID, normalizeBatchID) + tableNametoUnchangedToastCols, err := c.getTableNametoUnchangedCols( + req.FlowJobName, + batchIDs.SyncBatchID, + batchIDs.NormalizeBatchID, + ) if err != nil { return nil, fmt.Errorf("couldn't get tablename to unchanged cols mapping: %w", err) } - stmts := []string{} + stmts := make([]string, 0, len(distinctTableNames)+1) // append all the statements to one list c.logger.Info(fmt.Sprintf("merge raw records to corresponding tables: %s %s %v", c.datasetID, rawTableName, distinctTableNames)) - release, err := c.grabJobsUpdateLock() - if err != nil { - return nil, fmt.Errorf("failed to grab lock: %v", err) - } - - defer func() { - err := release() - if err != nil { - c.logger.Error("failed to release lock", slog.Any("error", err)) - } - }() - - stmts = append(stmts, "BEGIN TRANSACTION;") - for _, tableName := range distinctTableNames { + dstDatasetTable, _ := c.convertToDatasetTable(tableName) mergeGen := &mergeStmtGenerator{ - Dataset: c.datasetID, - NormalizedTable: tableName, - RawTable: rawTableName, - NormalizedTableSchema: c.tableNameSchemaMapping[tableName], - SyncBatchID: syncBatchID, - NormalizeBatchID: normalizeBatchID, - UnchangedToastColumns: tableNametoUnchangedToastCols[tableName], + rawDatasetTable: &datasetTable{ + dataset: c.datasetID, + table: rawTableName, + }, + dstTableName: tableName, + dstDatasetTable: dstDatasetTable, + normalizedTableSchema: c.tableNameSchemaMapping[tableName], + syncBatchID: batchIDs.SyncBatchID, + normalizeBatchID: batchIDs.NormalizeBatchID, + unchangedToastColumns: tableNametoUnchangedToastCols[tableName], + peerdbCols: &protos.PeerDBColumns{ + SoftDeleteColName: req.SoftDeleteColName, + SyncedAtColName: req.SyncedAtColName, + SoftDelete: req.SoftDelete, + }, } // normalize anything between last normalized batch id to last sync batchid - mergeStmts := mergeGen.generateMergeStmts() - stmts = append(stmts, mergeStmts...) + mergeStmt := mergeGen.generateMergeStmt() + stmts = append(stmts, mergeStmt) } // update metadata to make the last normalized batch id to the recent last sync batch id. updateMetadataStmt := fmt.Sprintf( - "UPDATE %s.%s SET normalize_batch_id=%d WHERE mirror_job_name = '%s';", - c.datasetID, MirrorJobsTable, syncBatchID, req.FlowJobName) + "UPDATE %s.%s SET normalize_batch_id=%d WHERE mirror_job_name='%s';", + c.datasetID, MirrorJobsTable, batchIDs.SyncBatchID, req.FlowJobName) stmts = append(stmts, updateMetadataStmt) - stmts = append(stmts, "COMMIT TRANSACTION;") - - // put this within a transaction - // TODO - not truncating rows in staging table as of now. - // err = c.truncateTable(staging...) - _, err = c.client.Query(strings.Join(stmts, "\n")).Read(c.ctx) + query := strings.Join(stmts, "\n") + _, err = c.client.Query(query).Read(c.ctx) if err != nil { - return nil, fmt.Errorf("failed to execute statements %s in a transaction: %v", strings.Join(stmts, "\n"), err) + return nil, fmt.Errorf("failed to execute statements %s: %v", query, err) } return &model.NormalizeResponse{ Done: true, - StartBatchID: normalizeBatchID + 1, - EndBatchID: syncBatchID, + StartBatchID: batchIDs.NormalizeBatchID + 1, + EndBatchID: batchIDs.SyncBatchID, }, nil } @@ -816,8 +641,7 @@ func (c *BigQueryConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr schema := bigquery.Schema{ {Name: "_peerdb_uid", Type: bigquery.StringFieldType}, - {Name: "_peerdb_timestamp", Type: bigquery.TimestampFieldType}, - {Name: "_peerdb_timestamp_nanos", Type: bigquery.IntegerFieldType}, + {Name: "_peerdb_timestamp", Type: bigquery.IntegerFieldType}, {Name: "_peerdb_destination_table_name", Type: bigquery.StringFieldType}, {Name: "_peerdb_data", Type: bigquery.StringFieldType}, {Name: "_peerdb_record_type", Type: bigquery.IntegerFieldType}, @@ -826,19 +650,6 @@ func (c *BigQueryConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr {Name: "_peerdb_unchanged_toast_columns", Type: bigquery.StringFieldType}, } - stagingSchema := bigquery.Schema{ - {Name: "_peerdb_uid", Type: bigquery.StringFieldType}, - {Name: "_peerdb_timestamp", Type: bigquery.TimestampFieldType}, - {Name: "_peerdb_timestamp_nanos", Type: bigquery.IntegerFieldType}, - {Name: "_peerdb_destination_table_name", Type: bigquery.StringFieldType}, - {Name: "_peerdb_data", Type: bigquery.StringFieldType}, - {Name: "_peerdb_record_type", Type: bigquery.IntegerFieldType}, - {Name: "_peerdb_match_data", Type: bigquery.StringFieldType}, - {Name: "_peerdb_batch_id", Type: bigquery.IntegerFieldType}, - {Name: "_peerdb_staging_batch_id", Type: bigquery.IntegerFieldType}, - {Name: "_peerdb_unchanged_toast_columns", Type: bigquery.StringFieldType}, - } - // create the table table := c.client.Dataset(c.datasetID).Table(rawTableName) @@ -863,16 +674,6 @@ func (c *BigQueryConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr return nil, fmt.Errorf("failed to create table %s.%s: %w", c.datasetID, rawTableName, err) } - // also create a staging table for this raw table - stagingTableName := c.getStagingTableName(req.FlowJobName) - stagingTable := c.client.Dataset(c.datasetID).Table(stagingTableName) - err = stagingTable.Create(c.ctx, &bigquery.TableMetadata{ - Schema: stagingSchema, - }) - if err != nil { - return nil, fmt.Errorf("failed to create table %s.%s: %w", c.datasetID, stagingTableName, err) - } - return &protos.CreateRawTableOutput{ TableIdentifier: rawTableName, }, nil @@ -893,7 +694,7 @@ func (c *BigQueryConnector) getUpdateMetadataStmt(jobName string, lastSyncedChec c.datasetID, MirrorJobsTable, jobName, lastSyncedCheckpointID, batchID) if hasJob { jobStatement = fmt.Sprintf( - "UPDATE %s.%s SET offset = %d,sync_batch_id=%d WHERE mirror_job_name = '%s';", + "UPDATE %s.%s SET offset=GREATEST(offset,%d),sync_batch_id=%d WHERE mirror_job_name = '%s';", c.datasetID, MirrorJobsTable, lastSyncedCheckpointID, batchID, jobName) } @@ -932,19 +733,46 @@ func (c *BigQueryConnector) SetupNormalizedTables( req *protos.SetupNormalizedTableBatchInput, ) (*protos.SetupNormalizedTableBatchOutput, error) { tableExistsMapping := make(map[string]bool) + datasetTablesSet := make(map[datasetTable]struct{}) for tableIdentifier, tableSchema := range req.TableNameSchemaMapping { - table := c.client.Dataset(c.datasetID).Table(tableIdentifier) + // only place where we check for parsing errors + datasetTable, err := c.convertToDatasetTable(tableIdentifier) + if err != nil { + return nil, err + } + _, ok := datasetTablesSet[*datasetTable] + if ok { + return nil, fmt.Errorf("invalid mirror: two tables mirror to the same BigQuery table %s", + datasetTable.string()) + } + dataset := c.client.Dataset(datasetTable.dataset) + _, err = dataset.Metadata(c.ctx) + // just assume this means dataset don't exist, and create it + if err != nil { + // if err message does not contain `notFound`, then other error happened. + if !strings.Contains(err.Error(), "notFound") { + return nil, fmt.Errorf("error while checking metadata for BigQuery dataset %s: %w", + datasetTable.dataset, err) + } + c.logger.InfoContext(c.ctx, fmt.Sprintf("creating dataset %s...", dataset.DatasetID)) + err = dataset.Create(c.ctx, nil) + if err != nil { + return nil, fmt.Errorf("failed to create BigQuery dataset %s: %w", dataset.DatasetID, err) + } + } + table := dataset.Table(datasetTable.table) // check if the table exists - _, err := table.Metadata(c.ctx) + _, err = table.Metadata(c.ctx) if err == nil { // table exists, go to next table tableExistsMapping[tableIdentifier] = true + datasetTablesSet[*datasetTable] = struct{}{} continue } // convert the column names and types to bigquery types - columns := make([]*bigquery.FieldSchema, len(tableSchema.Columns)) + columns := make([]*bigquery.FieldSchema, len(tableSchema.Columns), len(tableSchema.Columns)+2) idx := 0 for colName, genericColType := range tableSchema.Columns { columns[idx] = &bigquery.FieldSchema{ @@ -955,6 +783,22 @@ func (c *BigQueryConnector) SetupNormalizedTables( idx++ } + if req.SoftDeleteColName != "" { + columns = append(columns, &bigquery.FieldSchema{ + Name: req.SoftDeleteColName, + Type: bigquery.BooleanFieldType, + Repeated: false, + }) + } + + if req.SyncedAtColName != "" { + columns = append(columns, &bigquery.FieldSchema{ + Name: req.SyncedAtColName, + Type: bigquery.TimestampFieldType, + Repeated: false, + }) + } + // create the table using the columns schema := bigquery.Schema(columns) err = table.Create(c.ctx, &bigquery.TableMetadata{Schema: schema}) @@ -963,6 +807,7 @@ func (c *BigQueryConnector) SetupNormalizedTables( } tableExistsMapping[tableIdentifier] = false + datasetTablesSet[*datasetTable] = struct{}{} // log that table was created c.logger.Info(fmt.Sprintf("created table %s", tableIdentifier)) } @@ -973,28 +818,12 @@ func (c *BigQueryConnector) SetupNormalizedTables( } func (c *BigQueryConnector) SyncFlowCleanup(jobName string) error { - release, err := c.grabJobsUpdateLock() - if err != nil { - return fmt.Errorf("failed to grab lock: %w", err) - } - - defer func() { - err := release() - if err != nil { - c.logger.Error("failed to release lock", slog.Any("error", err)) - } - }() - dataset := c.client.Dataset(c.datasetID) // deleting PeerDB specific tables - err = dataset.Table(c.getRawTableName(jobName)).Delete(c.ctx) + err := dataset.Table(c.getRawTableName(jobName)).Delete(c.ctx) if err != nil { return fmt.Errorf("failed to delete raw table: %w", err) } - err = dataset.Table(c.getStagingTableName(jobName)).Delete(c.ctx) - if err != nil { - return fmt.Errorf("failed to delete staging table: %w", err) - } // deleting job from metadata table query := fmt.Sprintf("DELETE FROM %s.%s WHERE mirror_job_name = '%s'", c.datasetID, MirrorJobsTable, jobName) @@ -1012,62 +841,33 @@ func (c *BigQueryConnector) getRawTableName(flowJobName string) string { return fmt.Sprintf("_peerdb_raw_%s", flowJobName) } -// getStagingTableName returns the staging table name for the given table identifier. -func (c *BigQueryConnector) getStagingTableName(flowJobName string) string { - // replace all non-alphanumeric characters with _ - flowJobName = regexp.MustCompile("[^a-zA-Z0-9]+").ReplaceAllString(flowJobName, "_") - return fmt.Sprintf("_peerdb_staging_%s", flowJobName) -} - -// Bigquery doesn't allow concurrent updates to the same table. -// we grab a lock on catalog to ensure that only one job is updating -// bigquery tables at a time. -// returns a function to release the lock. -func (c *BigQueryConnector) grabJobsUpdateLock() (func() error, error) { - tx, err := c.catalogPool.Begin(c.ctx) - if err != nil { - return nil, fmt.Errorf("failed to begin transaction: %w", err) - } - - // grab an advisory lock based on the mirror jobs table hash - mjTbl := fmt.Sprintf("%s.%s", c.datasetID, MirrorJobsTable) - _, err = tx.Exec(c.ctx, "SELECT pg_advisory_xact_lock(hashtext($1))", mjTbl) - if err != nil { - err = tx.Rollback(c.ctx) - return nil, fmt.Errorf("failed to grab lock on %s: %w", mjTbl, err) - } - - return func() error { - err = tx.Commit(c.ctx) - if err != nil { - return fmt.Errorf("failed to commit transaction: %w", err) - } - return nil - }, nil -} - func (c *BigQueryConnector) RenameTables(req *protos.RenameTablesInput) (*protos.RenameTablesOutput, error) { for _, renameRequest := range req.RenameTableOptions { - src := renameRequest.CurrentName - dst := renameRequest.NewName - c.logger.Info(fmt.Sprintf("renaming table '%s' to '%s'...", src, dst)) + srcDatasetTable, _ := c.convertToDatasetTable(renameRequest.CurrentName) + dstDatasetTable, _ := c.convertToDatasetTable(renameRequest.NewName) + c.logger.Info(fmt.Sprintf("renaming table '%s' to '%s'...", srcDatasetTable.string(), + dstDatasetTable.string())) - activity.RecordHeartbeat(c.ctx, fmt.Sprintf("renaming table '%s' to '%s'...", src, dst)) + activity.RecordHeartbeat(c.ctx, fmt.Sprintf("renaming table '%s' to '%s'...", srcDatasetTable.string(), + dstDatasetTable.string())) // drop the dst table if exists - _, err := c.client.Query(fmt.Sprintf("DROP TABLE IF EXISTS %s.%s", c.datasetID, dst)).Run(c.ctx) + _, err := c.client.Query(fmt.Sprintf("DROP TABLE IF EXISTS %s.%s", + dstDatasetTable.dataset, dstDatasetTable.table)).Run(c.ctx) if err != nil { - return nil, fmt.Errorf("unable to drop table %s: %w", dst, err) + return nil, fmt.Errorf("unable to drop table %s: %w", dstDatasetTable.string(), err) } // rename the src table to dst _, err = c.client.Query(fmt.Sprintf("ALTER TABLE %s.%s RENAME TO %s", - c.datasetID, src, dst)).Run(c.ctx) + srcDatasetTable.dataset, srcDatasetTable.table, dstDatasetTable.table)).Run(c.ctx) if err != nil { - return nil, fmt.Errorf("unable to rename table %s to %s: %w", src, dst, err) + return nil, fmt.Errorf("unable to rename table %s to %s: %w", srcDatasetTable.string(), + dstDatasetTable.string(), err) } - c.logger.Info(fmt.Sprintf("successfully renamed table '%s' to '%s'", src, dst)) + c.logger.Info(fmt.Sprintf("successfully renamed table '%s' to '%s'", srcDatasetTable.string(), + dstDatasetTable.string())) } return &protos.RenameTablesOutput{ @@ -1079,13 +879,15 @@ func (c *BigQueryConnector) CreateTablesFromExisting(req *protos.CreateTablesFro *protos.CreateTablesFromExistingOutput, error, ) { for newTable, existingTable := range req.NewToExistingTableMapping { + newDatasetTable, _ := c.convertToDatasetTable(newTable) + existingDatasetTable, _ := c.convertToDatasetTable(existingTable) c.logger.Info(fmt.Sprintf("creating table '%s' similar to '%s'", newTable, existingTable)) activity.RecordHeartbeat(c.ctx, fmt.Sprintf("creating table '%s' similar to '%s'", newTable, existingTable)) // rename the src table to dst - _, err := c.client.Query(fmt.Sprintf("CREATE TABLE IF NOT EXISTS %s.%s LIKE %s.%s", - c.datasetID, newTable, c.datasetID, existingTable)).Run(c.ctx) + _, err := c.client.Query(fmt.Sprintf("CREATE TABLE IF NOT EXISTS `%s` LIKE `%s`", + newDatasetTable.string(), existingDatasetTable.string())).Run(c.ctx) if err != nil { return nil, fmt.Errorf("unable to create table %s: %w", newTable, err) } @@ -1097,3 +899,29 @@ func (c *BigQueryConnector) CreateTablesFromExisting(req *protos.CreateTablesFro FlowJobName: req.FlowJobName, }, nil } + +type datasetTable struct { + dataset string + table string +} + +func (d *datasetTable) string() string { + return fmt.Sprintf("%s.%s", d.dataset, d.table) +} + +func (c *BigQueryConnector) convertToDatasetTable(tableName string) (*datasetTable, error) { + parts := strings.Split(tableName, ".") + if len(parts) == 1 { + return &datasetTable{ + dataset: c.datasetID, + table: parts[0], + }, nil + } else if len(parts) == 2 { + return &datasetTable{ + dataset: parts[0], + table: parts[1], + }, nil + } else { + return nil, fmt.Errorf("invalid BigQuery table name: %s", tableName) + } +} diff --git a/flow/connectors/bigquery/merge_statement_generator.go b/flow/connectors/bigquery/merge_statement_generator.go index 2a37ef5ec..569ff611e 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -8,42 +8,25 @@ import ( "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" - "github.com/PeerDB-io/peer-flow/shared" ) type mergeStmtGenerator struct { - // dataset of all the tables - Dataset string - // the table to merge into - NormalizedTable string - // the table where the data is currently staged. - RawTable string + // dataset + raw table + rawDatasetTable *datasetTable + // destination table name, used to retrieve records from raw table + dstTableName string + // dataset + destination table + dstDatasetTable *datasetTable // last synced batchID. - SyncBatchID int64 + syncBatchID int64 // last normalized batchID. - NormalizeBatchID int64 + normalizeBatchID int64 // the schema of the table to merge into - NormalizedTableSchema *protos.TableSchema + normalizedTableSchema *protos.TableSchema // array of toast column combinations that are unchanged - UnchangedToastColumns []string -} - -// GenerateMergeStmt generates a merge statements. -func (m *mergeStmtGenerator) generateMergeStmts() []string { - // return an empty array for now - flattenedCTE := m.generateFlattenedCTE() - deDupedCTE := m.generateDeDupedCTE() - tempTable := fmt.Sprintf("_peerdb_de_duplicated_data_%s", shared.RandomString(5)) - // create temp table stmt - createTempTableStmt := fmt.Sprintf( - "CREATE TEMP TABLE %s AS (%s, %s);", - tempTable, flattenedCTE, deDupedCTE) - - mergeStmt := m.generateMergeStmt(tempTable) - - dropTempTableStmt := fmt.Sprintf("DROP TABLE %s;", tempTable) - - return []string{createTempTableStmt, mergeStmt, dropTempTableStmt} + unchangedToastColumns []string + // _PEERDB_IS_DELETED and _SYNCED_AT columns + peerdbCols *protos.PeerDBColumns } // generateFlattenedCTE generates a flattened CTE. @@ -51,7 +34,7 @@ func (m *mergeStmtGenerator) generateFlattenedCTE() string { // for each column in the normalized table, generate CAST + JSON_EXTRACT_SCALAR // statement. flattenedProjs := make([]string, 0) - for colName, colType := range m.NormalizedTableSchema.Columns { + for colName, colType := range m.normalizedTableSchema.Columns { bqType := qValueKindToBigQueryType(colType) // CAST doesn't work for FLOAT, so rewrite it to FLOAT64. if bqType == bigquery.FloatFieldType { @@ -97,17 +80,16 @@ func (m *mergeStmtGenerator) generateFlattenedCTE() string { flattenedProjs = append( flattenedProjs, "_peerdb_timestamp", - "_peerdb_timestamp_nanos", "_peerdb_record_type", "_peerdb_unchanged_toast_columns", ) // normalize anything between last normalized batch id to last sync batchid return fmt.Sprintf(`WITH _peerdb_flattened AS - (SELECT %s FROM %s.%s WHERE _peerdb_batch_id > %d and _peerdb_batch_id <= %d and + (SELECT %s FROM %s WHERE _peerdb_batch_id > %d and _peerdb_batch_id <= %d and _peerdb_destination_table_name='%s')`, - strings.Join(flattenedProjs, ", "), m.Dataset, m.RawTable, m.NormalizeBatchID, - m.SyncBatchID, m.NormalizedTable) + strings.Join(flattenedProjs, ", "), m.rawDatasetTable.string(), m.normalizeBatchID, + m.syncBatchID, m.dstTableName) } // generateDeDupedCTE generates a de-duped CTE. @@ -116,47 +98,69 @@ func (m *mergeStmtGenerator) generateDeDupedCTE() string { SELECT _peerdb_ranked.* FROM ( SELECT RANK() OVER ( - PARTITION BY %s ORDER BY _peerdb_timestamp_nanos DESC + PARTITION BY %s ORDER BY _peerdb_timestamp DESC ) as _peerdb_rank, * FROM _peerdb_flattened ) _peerdb_ranked WHERE _peerdb_rank = 1 ) SELECT * FROM _peerdb_de_duplicated_data_res` - pkeyColsStr := fmt.Sprintf("(CONCAT(%s))", strings.Join(m.NormalizedTableSchema.PrimaryKeyColumns, + pkeyColsStr := fmt.Sprintf("(CONCAT(%s))", strings.Join(m.normalizedTableSchema.PrimaryKeyColumns, ", '_peerdb_concat_', ")) return fmt.Sprintf(cte, pkeyColsStr) } // generateMergeStmt generates a merge statement. -func (m *mergeStmtGenerator) generateMergeStmt(tempTable string) string { +func (m *mergeStmtGenerator) generateMergeStmt() string { // comma separated list of column names - backtickColNames := make([]string, 0, len(m.NormalizedTableSchema.Columns)) - pureColNames := make([]string, 0, len(m.NormalizedTableSchema.Columns)) - for colName := range m.NormalizedTableSchema.Columns { + backtickColNames := make([]string, 0, len(m.normalizedTableSchema.Columns)) + pureColNames := make([]string, 0, len(m.normalizedTableSchema.Columns)) + for colName := range m.normalizedTableSchema.Columns { backtickColNames = append(backtickColNames, fmt.Sprintf("`%s`", colName)) pureColNames = append(pureColNames, colName) } csep := strings.Join(backtickColNames, ", ") - - updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, m.UnchangedToastColumns) + insertColumnsSQL := csep + fmt.Sprintf(", `%s`", m.peerdbCols.SyncedAtColName) + insertValuesSQL := csep + ",CURRENT_TIMESTAMP" + + updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, + m.unchangedToastColumns, m.peerdbCols) + if m.peerdbCols.SoftDelete { + softDeleteInsertColumnsSQL := insertColumnsSQL + fmt.Sprintf(", `%s`", m.peerdbCols.SoftDeleteColName) + softDeleteInsertValuesSQL := insertValuesSQL + ", TRUE" + + updateStatementsforToastCols = append(updateStatementsforToastCols, + fmt.Sprintf("WHEN NOT MATCHED AND (_peerdb_deduped._PEERDB_RECORD_TYPE = 2) THEN INSERT (%s) VALUES(%s)", + softDeleteInsertColumnsSQL, softDeleteInsertValuesSQL)) + } updateStringToastCols := strings.Join(updateStatementsforToastCols, " ") - pkeySelectSQLArray := make([]string, 0, len(m.NormalizedTableSchema.PrimaryKeyColumns)) - for _, pkeyColName := range m.NormalizedTableSchema.PrimaryKeyColumns { + pkeySelectSQLArray := make([]string, 0, len(m.normalizedTableSchema.PrimaryKeyColumns)) + for _, pkeyColName := range m.normalizedTableSchema.PrimaryKeyColumns { pkeySelectSQLArray = append(pkeySelectSQLArray, fmt.Sprintf("_peerdb_target.%s = _peerdb_deduped.%s", pkeyColName, pkeyColName)) } // _peerdb_target. = _peerdb_deduped. AND _peerdb_target. = _peerdb_deduped. ... pkeySelectSQL := strings.Join(pkeySelectSQLArray, " AND ") + deletePart := "DELETE" + if m.peerdbCols.SoftDelete { + colName := m.peerdbCols.SoftDeleteColName + deletePart = fmt.Sprintf("UPDATE SET %s = TRUE", colName) + if m.peerdbCols.SyncedAtColName != "" { + deletePart = fmt.Sprintf("%s, %s = CURRENT_TIMESTAMP", + deletePart, m.peerdbCols.SyncedAtColName) + } + } + return fmt.Sprintf(` - MERGE %s.%s _peerdb_target USING %s _peerdb_deduped + MERGE %s _peerdb_target USING (%s,%s) _peerdb_deduped ON %s WHEN NOT MATCHED and (_peerdb_deduped._peerdb_record_type != 2) THEN INSERT (%s) VALUES (%s) %s WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type = 2) THEN - DELETE; - `, m.Dataset, m.NormalizedTable, tempTable, pkeySelectSQL, csep, csep, updateStringToastCols) + %s; + `, m.dstDatasetTable.string(), m.generateFlattenedCTE(), m.generateDeDupedCTE(), + pkeySelectSQL, insertColumnsSQL, insertValuesSQL, updateStringToastCols, deletePart) } /* @@ -174,7 +178,11 @@ and updating the other columns (not the unchanged toast columns) 6. Repeat steps 1-5 for each unique unchanged toast column group. 7. Return the list of generated update statements. */ -func (m *mergeStmtGenerator) generateUpdateStatements(allCols []string, unchangedToastCols []string) []string { +func (m *mergeStmtGenerator) generateUpdateStatements( + allCols []string, + unchangedToastCols []string, + peerdbCols *protos.PeerDBColumns, +) []string { updateStmts := make([]string, 0, len(unchangedToastCols)) for _, cols := range unchangedToastCols { @@ -184,11 +192,36 @@ func (m *mergeStmtGenerator) generateUpdateStatements(allCols []string, unchange for _, colName := range otherCols { tmpArray = append(tmpArray, fmt.Sprintf("`%s` = _peerdb_deduped.%s", colName, colName)) } + + // set the synced at column to the current timestamp + if peerdbCols.SyncedAtColName != "" { + tmpArray = append(tmpArray, fmt.Sprintf("`%s` = CURRENT_TIMESTAMP", + peerdbCols.SyncedAtColName)) + } + // set soft-deleted to false, tackles insert after soft-delete + if peerdbCols.SoftDeleteColName != "" { + tmpArray = append(tmpArray, fmt.Sprintf("`%s` = FALSE", + peerdbCols.SoftDeleteColName)) + } + ssep := strings.Join(tmpArray, ", ") updateStmt := fmt.Sprintf(`WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2) AND _peerdb_unchanged_toast_columns='%s' THEN UPDATE SET %s `, cols, ssep) updateStmts = append(updateStmts, updateStmt) + + // generates update statements for the case where updates and deletes happen in the same branch + // the backfill has happened from the pull side already, so treat the DeleteRecord as an update + // and then set soft-delete to true. + if peerdbCols.SoftDelete && (peerdbCols.SoftDeleteColName != "") { + tmpArray = append(tmpArray[:len(tmpArray)-1], + fmt.Sprintf("`%s` = TRUE", peerdbCols.SoftDeleteColName)) + ssep := strings.Join(tmpArray, ", ") + updateStmt := fmt.Sprintf(`WHEN MATCHED AND + (_peerdb_deduped._peerdb_record_type = 2) AND _peerdb_unchanged_toast_columns='%s' + THEN UPDATE SET %s `, cols, ssep) + updateStmts = append(updateStmts, updateStmt) + } } return updateStmts } diff --git a/flow/connectors/bigquery/merge_stmt_generator_test.go b/flow/connectors/bigquery/merge_stmt_generator_test.go index 41e54114e..37dd3e07e 100644 --- a/flow/connectors/bigquery/merge_stmt_generator_test.go +++ b/flow/connectors/bigquery/merge_stmt_generator_test.go @@ -4,6 +4,8 @@ import ( "reflect" "strings" "testing" + + "github.com/PeerDB-io/peer-flow/generated/protos" ) func TestGenerateUpdateStatement_WithUnchangedToastCols(t *testing.T) { @@ -12,25 +14,38 @@ func TestGenerateUpdateStatement_WithUnchangedToastCols(t *testing.T) { unchangedToastCols := []string{"", "col2, col3", "col2", "col3"} expected := []string{ - "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + - " AND _peerdb_unchanged_toast_columns='' " + - "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + - " `col2` = _peerdb_deduped.col2," + - " `col3` = _peerdb_deduped.col3", - "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + - " AND _peerdb_unchanged_toast_columns='col2, col3' " + - "THEN UPDATE SET `col1` = _peerdb_deduped.col1", - "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + - " AND _peerdb_unchanged_toast_columns='col2'" + - "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + - " `col3` = _peerdb_deduped.col3", - "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type != 2)" + - " AND _peerdb_unchanged_toast_columns='col3'" + - "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + - " `col2` = _peerdb_deduped.col2", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type!=2) AND _peerdb_unchanged_toast_columns=''" + + " THEN UPDATE SET `col1`=_peerdb_deduped.col1,`col2`=_peerdb_deduped.col2,`col3`=_peerdb_deduped.col3," + + "`synced_at`=CURRENT_TIMESTAMP,`deleted`=FALSE", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type=2) " + + "AND _peerdb_unchanged_toast_columns='' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1,`col2`=_peerdb_deduped.col2," + + "`col3`=_peerdb_deduped.col3,`synced_at`=CURRENT_TIMESTAMP,`deleted`=TRUE", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type!=2) AND _peerdb_unchanged_toast_columns='col2,col3' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1,`synced_at`=CURRENT_TIMESTAMP,`deleted`=FALSE ", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type=2) AND _peerdb_unchanged_toast_columns='col2,col3' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1,`synced_at`=CURRENT_TIMESTAMP,`deleted`=TRUE", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type!=2) " + + "AND _peerdb_unchanged_toast_columns='col2' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1,`col3`=_peerdb_deduped.col3," + + "`synced_at`=CURRENT_TIMESTAMP,`deleted`=FALSE", + "WHEN MATCHED AND(_peerdb_deduped._peerdb_record_type=2) " + + "AND _peerdb_unchanged_toast_columns='col2' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1,`col3`=_peerdb_deduped.col3," + + "`synced_at`=CURRENT_TIMESTAMP,`deleted`=TRUE ", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type!=2) AND _peerdb_unchanged_toast_columns='col3' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1," + + "`col2`=_peerdb_deduped.col2,`synced_at`=CURRENT_TIMESTAMP,`deleted`=FALSE ", + "WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type=2) AND _peerdb_unchanged_toast_columns='col3' " + + "THEN UPDATE SET `col1`=_peerdb_deduped.col1," + + "`col2`=_peerdb_deduped.col2,`synced_at`=CURRENT_TIMESTAMP,`deleted`=TRUE", } - result := m.generateUpdateStatements(allCols, unchangedToastCols) + result := m.generateUpdateStatements(allCols, unchangedToastCols, &protos.PeerDBColumns{ + SoftDelete: true, + SoftDeleteColName: "deleted", + SyncedAtColName: "synced_at", + }) for i := range expected { expected[i] = removeSpacesTabsNewlines(expected[i]) @@ -38,7 +53,7 @@ func TestGenerateUpdateStatement_WithUnchangedToastCols(t *testing.T) { } if !reflect.DeepEqual(result, expected) { - t.Errorf("Unexpected result. Expected: %v, but got: %v", expected, result) + t.Errorf("Unexpected result. Expected: %v,\nbut got: %v", expected, result) } } @@ -53,10 +68,21 @@ func TestGenerateUpdateStatement_NoUnchangedToastCols(t *testing.T) { "THEN UPDATE SET " + "`col1` = _peerdb_deduped.col1," + " `col2` = _peerdb_deduped.col2," + - " `col3` = _peerdb_deduped.col3", + " `col3` = _peerdb_deduped.col3," + + " `synced_at`=CURRENT_TIMESTAMP," + + "`deleted`=FALSE", + "WHEN MATCHED AND" + + "(_peerdb_deduped._peerdb_record_type = 2) AND _peerdb_unchanged_toast_columns=''" + + "THEN UPDATE SET `col1` = _peerdb_deduped.col1, `col2` = _peerdb_deduped.col2, " + + "`col3` = _peerdb_deduped.col3, `synced_at` = CURRENT_TIMESTAMP, `deleted` = TRUE", } - result := m.generateUpdateStatements(allCols, unchangedToastCols) + result := m.generateUpdateStatements(allCols, unchangedToastCols, + &protos.PeerDBColumns{ + SoftDelete: true, + SoftDeleteColName: "deleted", + SyncedAtColName: "synced_at", + }) for i := range expected { expected[i] = removeSpacesTabsNewlines(expected[i]) diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index a353d432e..305bab01e 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -45,18 +45,19 @@ func (c *BigQueryConnector) SyncQRepRecords( " partition %s of destination table %s", partition.PartitionId, destTable)) - avroSync := &QRepAvroSyncMethod{connector: c, gcsBucket: config.StagingPath} - return avroSync.SyncQRepRecords(config.FlowJobName, destTable, partition, tblMetadata, stream) + avroSync := NewQRepAvroSyncMethod(c, config.StagingPath, config.FlowJobName) + return avroSync.SyncQRepRecords(config.FlowJobName, destTable, partition, + tblMetadata, stream, config.SyncedAtColName, config.SoftDeleteColName) } func (c *BigQueryConnector) replayTableSchemaDeltasQRep(config *protos.QRepConfig, partition *protos.QRepPartition, srcSchema *model.QRecordSchema, ) (*bigquery.TableMetadata, error) { - destTable := config.DestinationTableIdentifier - bqTable := c.client.Dataset(c.datasetID).Table(destTable) + destDatasetTable, _ := c.convertToDatasetTable(config.DestinationTableIdentifier) + bqTable := c.client.Dataset(destDatasetTable.dataset).Table(destDatasetTable.table) dstTableMetadata, err := bqTable.Metadata(c.ctx) if err != nil { - return nil, fmt.Errorf("failed to get metadata of table %s: %w", destTable, err) + return nil, fmt.Errorf("failed to get metadata of table %s: %w", destDatasetTable, err) } tableSchemaDelta := &protos.TableSchemaDelta{ @@ -91,7 +92,7 @@ func (c *BigQueryConnector) replayTableSchemaDeltasQRep(config *protos.QRepConfi } dstTableMetadata, err = bqTable.Metadata(c.ctx) if err != nil { - return nil, fmt.Errorf("failed to get metadata of table %s: %w", destTable, err) + return nil, fmt.Errorf("failed to get metadata of table %s: %w", destDatasetTable, err) } return dstTableMetadata, nil } diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 9bb01157f..9f36ceb3a 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -35,9 +35,9 @@ func NewQRepAvroSyncMethod(connector *BigQueryConnector, gcsBucket string, } func (s *QRepAvroSyncMethod) SyncRecords( - dstTableName string, + rawTableName string, flowJobName string, - lastCP int64, + records *model.CDCRecordStream, dstTableMetadata *bigquery.TableMetadata, syncBatchID int64, stream *model.QRecordStream, @@ -45,16 +45,20 @@ func (s *QRepAvroSyncMethod) SyncRecords( activity.RecordHeartbeat(s.connector.ctx, time.Minute, fmt.Sprintf("Flow job %s: Obtaining Avro schema"+ " for destination table %s and sync batch ID %d", - flowJobName, dstTableName, syncBatchID), + flowJobName, rawTableName, syncBatchID), ) // You will need to define your Avro schema as a string - avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata) + avroSchema, err := DefineAvroSchema(rawTableName, dstTableMetadata, "", "") if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } - stagingTable := fmt.Sprintf("%s_%s_staging", dstTableName, fmt.Sprint(syncBatchID)) - numRecords, err := s.writeToStage(fmt.Sprint(syncBatchID), dstTableName, avroSchema, stagingTable, stream) + stagingTable := fmt.Sprintf("%s_%s_staging", rawTableName, fmt.Sprint(syncBatchID)) + numRecords, err := s.writeToStage(fmt.Sprint(syncBatchID), rawTableName, avroSchema, + &datasetTable{ + dataset: s.connector.datasetID, + table: stagingTable, + }, stream) if err != nil { return -1, fmt.Errorf("failed to push to avro stage: %v", err) } @@ -62,7 +66,12 @@ func (s *QRepAvroSyncMethod) SyncRecords( bqClient := s.connector.client datasetID := s.connector.datasetID insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT * FROM `%s.%s`;", - datasetID, dstTableName, datasetID, stagingTable) + datasetID, rawTableName, datasetID, stagingTable) + + lastCP, err := records.GetLastCheckpoint() + if err != nil { + return -1, fmt.Errorf("failed to get last checkpoint: %v", err) + } updateMetadataStmt, err := s.connector.getUpdateMetadataStmt(flowJobName, lastCP, syncBatchID) if err != nil { return -1, fmt.Errorf("failed to update metadata: %v", err) @@ -71,15 +80,15 @@ func (s *QRepAvroSyncMethod) SyncRecords( activity.RecordHeartbeat(s.connector.ctx, time.Minute, fmt.Sprintf("Flow job %s: performing insert and update transaction"+ " for destination table %s and sync batch ID %d", - flowJobName, dstTableName, syncBatchID), + flowJobName, rawTableName, syncBatchID), ) - // execute the statements in a transaction - stmts := []string{} - stmts = append(stmts, "BEGIN TRANSACTION;") - stmts = append(stmts, insertStmt) - stmts = append(stmts, updateMetadataStmt) - stmts = append(stmts, "COMMIT TRANSACTION;") + stmts := []string{ + "BEGIN TRANSACTION;", + insertStmt, + updateMetadataStmt, + "COMMIT TRANSACTION;", + } _, err = bqClient.Query(strings.Join(stmts, "\n")).Read(s.connector.ctx) if err != nil { return -1, fmt.Errorf("failed to execute statements in a transaction: %v", err) @@ -91,12 +100,12 @@ func (s *QRepAvroSyncMethod) SyncRecords( slog.Error("failed to delete staging table "+stagingTable, slog.Any("error", err), slog.String("syncBatchID", fmt.Sprint(syncBatchID)), - slog.String("destinationTable", dstTableName)) + slog.String("destinationTable", rawTableName)) } - slog.Info(fmt.Sprintf("loaded stage into %s.%s", datasetID, dstTableName), + slog.Info(fmt.Sprintf("loaded stage into %s.%s", datasetID, rawTableName), slog.String(string(shared.FlowNameKey), flowJobName), - slog.String("dstTableName", dstTableName)) + slog.String("dstTableName", rawTableName)) return numRecords, nil } @@ -107,6 +116,8 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( partition *protos.QRepPartition, dstTableMetadata *bigquery.TableMetadata, stream *model.QRecordStream, + syncedAtCol string, + softDeleteCol string, ) (int, error) { startTime := time.Now() flowLog := slog.Group("sync_metadata", @@ -115,15 +126,21 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( slog.String("destinationTable", dstTableName), ) // You will need to define your Avro schema as a string - avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata) + avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata, syncedAtCol, softDeleteCol) if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } slog.Info("Obtained Avro schema for destination table", flowLog) slog.Info(fmt.Sprintf("Avro schema: %v\n", avroSchema), flowLog) // create a staging table name with partitionID replace hyphens with underscores - stagingTable := fmt.Sprintf("%s_%s_staging", dstTableName, strings.ReplaceAll(partition.PartitionId, "-", "_")) - numRecords, err := s.writeToStage(partition.PartitionId, flowJobName, avroSchema, stagingTable, stream) + dstDatasetTable, _ := s.connector.convertToDatasetTable(dstTableName) + stagingDatasetTable := &datasetTable{ + dataset: dstDatasetTable.dataset, + table: fmt.Sprintf("%s_%s_staging", dstDatasetTable.table, + strings.ReplaceAll(partition.PartitionId, "-", "_")), + } + numRecords, err := s.writeToStage(partition.PartitionId, flowJobName, avroSchema, + stagingDatasetTable, stream) if err != nil { return -1, fmt.Errorf("failed to push to avro stage: %v", err) } @@ -133,38 +150,45 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( flowJobName, dstTableName, partition.PartitionId), ) bqClient := s.connector.client - datasetID := s.connector.datasetID - // Start a transaction - stmts := []string{"BEGIN TRANSACTION;"} + selector := "*" + if softDeleteCol != "" { // PeerDB column + selector += ", FALSE" + } + if syncedAtCol != "" { // PeerDB column + selector += ", CURRENT_TIMESTAMP" + } // Insert the records from the staging table into the destination table - insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT * FROM `%s.%s`;", - datasetID, dstTableName, datasetID, stagingTable) - - stmts = append(stmts, insertStmt) + insertStmt := fmt.Sprintf("INSERT INTO `%s` SELECT %s FROM `%s`;", + dstDatasetTable.string(), selector, stagingDatasetTable.string()) insertMetadataStmt, err := s.connector.createMetadataInsertStatement(partition, flowJobName, startTime) if err != nil { return -1, fmt.Errorf("failed to create metadata insert statement: %v", err) } slog.Info("Performing transaction inside QRep sync function", flowLog) - stmts = append(stmts, insertMetadataStmt) - stmts = append(stmts, "COMMIT TRANSACTION;") - // Execute the statements in a transaction + + stmts := []string{ + "BEGIN TRANSACTION;", + insertStmt, + insertMetadataStmt, + "COMMIT TRANSACTION;", + } _, err = bqClient.Query(strings.Join(stmts, "\n")).Read(s.connector.ctx) if err != nil { return -1, fmt.Errorf("failed to execute statements in a transaction: %v", err) } // drop the staging table - if err := bqClient.Dataset(datasetID).Table(stagingTable).Delete(s.connector.ctx); err != nil { + if err := bqClient.Dataset(stagingDatasetTable.dataset). + Table(stagingDatasetTable.table).Delete(s.connector.ctx); err != nil { // just log the error this isn't fatal. - slog.Error("failed to delete staging table "+stagingTable, + slog.Error("failed to delete staging table "+stagingDatasetTable.string(), slog.Any("error", err), flowLog) } - slog.Info(fmt.Sprintf("loaded stage into %s.%s", datasetID, dstTableName), flowLog) + slog.Info(fmt.Sprintf("loaded stage into %s", dstDatasetTable.string()), flowLog) return numRecords, nil } @@ -181,11 +205,16 @@ type AvroSchema struct { func DefineAvroSchema(dstTableName string, dstTableMetadata *bigquery.TableMetadata, + syncedAtCol string, + softDeleteCol string, ) (*model.QRecordAvroSchemaDefinition, error) { avroFields := []AvroField{} nullableFields := make(map[string]struct{}) for _, bqField := range dstTableMetadata.Schema { + if bqField.Name == syncedAtCol || bqField.Name == softDeleteCol { + continue + } avroType, err := GetAvroType(bqField) if err != nil { return nil, err @@ -309,7 +338,7 @@ func (s *QRepAvroSyncMethod) writeToStage( syncID string, objectFolder string, avroSchema *model.QRecordAvroSchemaDefinition, - stagingTable string, + stagingTable *datasetTable, stream *model.QRecordStream, ) (int, error) { shutdown := utils.HeartbeatRoutine(s.connector.ctx, time.Minute, @@ -329,7 +358,6 @@ func (s *QRepAvroSyncMethod) writeToStage( slog.String("batchOrPartitionID", syncID), ) if s.gcsBucket != "" { - bucket := s.connector.storageClient.Bucket(s.gcsBucket) avroFilePath := fmt.Sprintf("%s/%s.avro", objectFolder, syncID) obj := bucket.Object(avroFilePath) @@ -366,7 +394,6 @@ func (s *QRepAvroSyncMethod) writeToStage( slog.Info(fmt.Sprintf("wrote %d records", avroFile.NumRecords), idLog) bqClient := s.connector.client - datasetID := s.connector.datasetID var avroRef bigquery.LoadSource if s.gcsBucket != "" { gcsRef := bigquery.NewGCSReference(fmt.Sprintf("gs://%s/%s", s.gcsBucket, avroFile.FilePath)) @@ -383,7 +410,7 @@ func (s *QRepAvroSyncMethod) writeToStage( avroRef = localRef } - loader := bqClient.Dataset(datasetID).Table(stagingTable).LoaderFrom(avroRef) + loader := bqClient.Dataset(stagingTable.dataset).Table(stagingTable.table).LoaderFrom(avroRef) loader.UseAvroLogicalTypes = true loader.WriteDisposition = bigquery.WriteTruncate job, err := loader.Run(s.connector.ctx) @@ -399,9 +426,9 @@ func (s *QRepAvroSyncMethod) writeToStage( if err := status.Err(); err != nil { return 0, fmt.Errorf("failed to load Avro file into BigQuery table: %w", err) } - slog.Info(fmt.Sprintf("Pushed into %s/%s", avroFile.FilePath, syncID)) + slog.Info(fmt.Sprintf("Pushed from %s to BigQuery", avroFile.FilePath)) - err = s.connector.WaitForTableReady(stagingTable) + err = s.connector.waitForTableReady(stagingTable) if err != nil { return 0, fmt.Errorf("failed to wait for table to be ready: %w", err) } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 707a7f0b1..477b7cf46 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -43,11 +43,11 @@ type CDCPullConnector interface { // PullFlowCleanup drops both the Postgres publication and replication slot, as a part of DROP MIRROR PullFlowCleanup(jobName string) error - // SendWALHeartbeat allows for activity to progress restart_lsn on postgres. - SendWALHeartbeat() error - // GetSlotInfo returns the WAL (or equivalent) info of a slot for the connector. GetSlotInfo(slotName string) ([]*protos.SlotInfo, error) + + // GetOpenConnectionsForUser returns the number of open connections for the user configured in the peer. + GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) } type CDCSyncConnector interface { @@ -62,6 +62,9 @@ type CDCSyncConnector interface { // GetLastOffset gets the last offset from the metadata table on the destination GetLastOffset(jobName string) (int64, error) + // SetLastOffset updates the last offset on the metadata table on the destination + SetLastOffset(jobName string, lastOffset int64) error + // GetLastSyncBatchID gets the last batch synced to the destination from the metadata table GetLastSyncBatchID(jobName string) (int64, error) @@ -134,7 +137,7 @@ func GetCDCPullConnector(ctx context.Context, config *protos.Peer) (CDCPullConne inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), true) default: return nil, ErrUnsupportedFunctionality } @@ -144,7 +147,7 @@ func GetCDCSyncConnector(ctx context.Context, config *protos.Peer) (CDCSyncConne inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_BigqueryConfig: return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: @@ -166,7 +169,7 @@ func GetCDCNormalizeConnector(ctx context.Context, inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_BigqueryConfig: return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: @@ -180,7 +183,7 @@ func GetQRepPullConnector(ctx context.Context, config *protos.Peer) (QRepPullCon inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_SqlserverConfig: return connsqlserver.NewSQLServerConnector(ctx, config.GetSqlserverConfig()) default: @@ -192,7 +195,7 @@ func GetQRepSyncConnector(ctx context.Context, config *protos.Peer) (QRepSyncCon inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_BigqueryConfig: return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: @@ -213,7 +216,10 @@ func GetConnector(ctx context.Context, peer *protos.Peer) (Connector, error) { if pgConfig == nil { return nil, fmt.Errorf("missing postgres config for %s peer %s", peer.Type.String(), peer.Name) } - return connpostgres.NewPostgresConnector(ctx, pgConfig) + // we can't decide if a PG peer should have replication permissions on it because we don't know + // what the user wants to do with it, so defaulting to being permissive. + // can be revisited in the future or we can use some UI wizardry. + return connpostgres.NewPostgresConnector(ctx, pgConfig, false) case protos.DBType_BIGQUERY: bqConfig := peer.GetBigqueryConfig() if bqConfig == nil { diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 4be57309f..c8ba3dad4 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -109,7 +109,7 @@ func (c *EventHubConnector) GetLastOffset(jobName string) (int64, error) { return c.pgMetadata.FetchLastOffset(jobName) } -func (c *EventHubConnector) updateLastOffset(jobName string, offset int64) error { +func (c *EventHubConnector) SetLastOffset(jobName string, offset int64) error { err := c.pgMetadata.UpdateLastOffset(jobName, offset) if err != nil { c.logger.Error(fmt.Sprintf("failed to update last offset: %v", err)) @@ -129,7 +129,7 @@ func (c *EventHubConnector) processBatch( batchPerTopic := NewHubBatches(c.hubManager) toJSONOpts := model.NewToJSONOptions(c.config.UnnestColumns) - eventHubFlushTimeout := peerdbenv.GetPeerDBEventhubFlushTimeoutSeconds() + eventHubFlushTimeout := peerdbenv.PeerDBEventhubFlushTimeoutSeconds() ticker := time.NewTicker(eventHubFlushTimeout) defer ticker.Stop() @@ -164,7 +164,7 @@ func (c *EventHubConnector) processBatch( return 0, err } - topicName, err := NewScopedEventhub(record.GetTableName()) + topicName, err := NewScopedEventhub(record.GetDestinationTableName()) if err != nil { c.logger.Error("failed to get topic name", slog.Any("error", err)) return 0, err @@ -187,7 +187,7 @@ func (c *EventHubConnector) processBatch( } if lastSeenLSN > lastUpdatedOffset { - err = c.updateLastOffset(flowJobName, lastSeenLSN) + err = c.SetLastOffset(flowJobName, lastSeenLSN) lastUpdatedOffset = lastSeenLSN c.logger.Info("processBatch", slog.Int64("updated last offset", lastSeenLSN)) if err != nil { @@ -233,7 +233,7 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return nil, err } - err = c.updateLastOffset(req.FlowJobName, lastCheckpoint) + err = c.SetLastOffset(req.FlowJobName, lastCheckpoint) if err != nil { c.logger.Error("failed to update last offset", slog.Any("error", err)) return nil, err diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index ef2cf5e45..9fe72828a 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -2,6 +2,7 @@ package connmetadata import ( "context" + "errors" "fmt" "log/slog" @@ -9,6 +10,8 @@ import ( cc "github.com/PeerDB-io/peer-flow/connectors/utils/catalog" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/shared" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" ) @@ -17,6 +20,11 @@ const ( lastSyncStateTableName = "last_sync_state" ) +func isUniqueError(err error) bool { + var pgerr *pgconn.PgError + return errors.As(err, &pgerr) && pgerr.Code == pgerrcode.UniqueViolation +} + type PostgresMetadataStore struct { ctx context.Context config *protos.PostgresConfig @@ -106,7 +114,7 @@ func (p *PostgresMetadataStore) SetupMetadata() error { // create the schema _, err = tx.Exec(p.ctx, "CREATE SCHEMA IF NOT EXISTS "+p.schemaName) - if err != nil { + if err != nil && !isUniqueError(err) { p.logger.Error("failed to create schema", slog.Any("error", err)) return err } @@ -120,7 +128,7 @@ func (p *PostgresMetadataStore) SetupMetadata() error { sync_batch_id BIGINT NOT NULL ) `) - if err != nil { + if err != nil && !isUniqueError(err) { p.logger.Error("failed to create last sync state table", slog.Any("error", err)) return err } @@ -146,7 +154,6 @@ func (p *PostgresMetadataStore) FetchLastOffset(jobName string) (int64, error) { var offset pgtype.Int8 err := rows.Scan(&offset) if err != nil { - // if the job doesn't exist, return 0 if err.Error() == "no rows in result set" { return 0, nil } @@ -198,7 +205,8 @@ func (p *PostgresMetadataStore) UpdateLastOffset(jobName string, offset int64) e INSERT INTO `+p.schemaName+`.`+lastSyncStateTableName+` (job_name, last_offset, sync_batch_id) VALUES ($1, $2, $3) ON CONFLICT (job_name) - DO UPDATE SET last_offset = $2, updated_at = NOW() + DO UPDATE SET last_offset = GREATEST(`+lastSyncStateTableName+`.last_offset, excluded.last_offset), + updated_at = NOW() `, jobName, offset, 0) if err != nil { diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 031ae5a8e..4c5693f29 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -3,6 +3,7 @@ package connpostgres import ( "context" "crypto/sha256" + "encoding/json" "fmt" "log/slog" "time" @@ -20,6 +21,7 @@ import ( "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "github.com/lib/pq/oid" + "go.temporal.io/sdk/activity" ) type PostgresCDCSource struct { @@ -28,16 +30,20 @@ type PostgresCDCSource struct { SrcTableIDNameMapping map[uint32]string TableNameMapping map[string]model.NameAndExclude slot string + SetLastOffset func(int64) error publication string relationMessageMapping model.RelationMessageMapping typeMap *pgtype.Map - startLSN pglogrepl.LSN commitLock bool customTypeMapping map[uint32]string // for partitioned tables, maps child relid to parent relid childToParentRelIDMapping map[uint32]uint32 logger slog.Logger + + // for storing chema delta audit logs to catalog + catalogPool *pgxpool.Pool + flowJobName string } type PostgresCDCConfig struct { @@ -48,6 +54,9 @@ type PostgresCDCConfig struct { SrcTableIDNameMapping map[uint32]string TableNameMapping map[string]model.NameAndExclude RelationMessageMapping model.RelationMessageMapping + CatalogPool *pgxpool.Pool + FlowJobName string + SetLastOffset func(int64) error } // Create a new PostgresCDCSource @@ -64,6 +73,7 @@ func NewPostgresCDCSource(cdcConfig *PostgresCDCConfig, customTypeMap map[uint32 SrcTableIDNameMapping: cdcConfig.SrcTableIDNameMapping, TableNameMapping: cdcConfig.TableNameMapping, slot: cdcConfig.Slot, + SetLastOffset: cdcConfig.SetLastOffset, publication: cdcConfig.Publication, relationMessageMapping: cdcConfig.RelationMessageMapping, typeMap: pgtype.NewMap(), @@ -71,6 +81,8 @@ func NewPostgresCDCSource(cdcConfig *PostgresCDCConfig, customTypeMap map[uint32 commitLock: false, customTypeMapping: customTypeMap, logger: *slog.With(slog.String(string(shared.FlowNameKey), flowName)), + catalogPool: cdcConfig.CatalogPool, + flowJobName: cdcConfig.FlowJobName, }, nil } @@ -142,19 +154,20 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) error { sysident.SystemID, sysident.Timeline, sysident.XLogPos, sysident.DBName)) // start replication - p.startLSN = 0 + var clientXLogPos, startLSN pglogrepl.LSN if req.LastOffset > 0 { p.logger.Info("starting replication from last sync state", slog.Int64("last checkpoint", req.LastOffset)) - p.startLSN = pglogrepl.LSN(req.LastOffset + 1) + clientXLogPos = pglogrepl.LSN(req.LastOffset) + startLSN = clientXLogPos + 1 } - err = pglogrepl.StartReplication(p.ctx, pgConn, replicationSlot, p.startLSN, replicationOpts) + err = pglogrepl.StartReplication(p.ctx, pgConn, replicationSlot, startLSN, replicationOpts) if err != nil { - return fmt.Errorf("error starting replication at startLsn - %d: %w", p.startLSN, err) + return fmt.Errorf("error starting replication at startLsn - %d: %w", startLSN, err) } - p.logger.Info(fmt.Sprintf("started replication on slot %s at startLSN: %d", p.slot, p.startLSN)) + p.logger.Info(fmt.Sprintf("started replication on slot %s at startLSN: %d", p.slot, startLSN)) - return p.consumeStream(pgConn, req, p.startLSN, req.RecordStream) + return p.consumeStream(pgConn, req, clientXLogPos, req.RecordStream) } // start consuming the cdc stream @@ -171,12 +184,12 @@ func (p *PostgresCDCSource) consumeStream( } }() - // clientXLogPos is the last checkpoint id + 1, we need to ack that we have processed - // until clientXLogPos - 1 each time we send a standby status update. + // clientXLogPos is the last checkpoint id, we need to ack that we have processed + // until clientXLogPos each time we send a standby status update. // consumedXLogPos is the lsn that has been committed on the destination. consumedXLogPos := pglogrepl.LSN(0) if clientXLogPos > 0 { - consumedXLogPos = clientXLogPos - 1 + consumedXLogPos = clientXLogPos err := pglogrepl.SendStandbyStatusUpdate(p.ctx, conn, pglogrepl.StandbyStatusUpdate{WALWritePosition: consumedXLogPos}) @@ -186,7 +199,7 @@ func (p *PostgresCDCSource) consumeStream( } var standByLastLogged time.Time - cdcRecordsStorage := cdc_records.NewCDCRecordsStore(req.FlowJobName) + cdcRecordsStorage := cdc_records.NewCDCRecordsStore(p.flowJobName) defer func() { if cdcRecordsStorage.IsEmpty() { records.SignalAsEmpty() @@ -200,7 +213,7 @@ func (p *PostgresCDCSource) consumeStream( }() shutdown := utils.HeartbeatRoutine(p.ctx, 10*time.Second, func() string { - jobName := req.FlowJobName + jobName := p.flowJobName currRecords := cdcRecordsStorage.Len() return fmt.Sprintf("pulling records for job - %s, currently have %d records", jobName, currRecords) }) @@ -240,21 +253,18 @@ func (p *PostgresCDCSource) consumeStream( for { if pkmRequiresResponse { - // Update XLogPos to the last processed position, we can only confirm - // that this is the last row committed on the destination. err := pglogrepl.SendStandbyStatusUpdate(p.ctx, conn, pglogrepl.StandbyStatusUpdate{WALWritePosition: consumedXLogPos}) if err != nil { return fmt.Errorf("SendStandbyStatusUpdate failed: %w", err) } + pkmRequiresResponse = false if time.Since(standByLastLogged) > 10*time.Second { numRowsProcessedMessage := fmt.Sprintf("processed %d rows", cdcRecordsStorage.Len()) p.logger.Info(fmt.Sprintf("Sent Standby status message. %s", numRowsProcessedMessage)) standByLastLogged = time.Now() } - - pkmRequiresResponse = false } if (cdcRecordsStorage.Len() >= int(req.MaxBatchSize)) && !p.commitLock { @@ -264,7 +274,7 @@ func (p *PostgresCDCSource) consumeStream( if waitingForCommit && !p.commitLock { p.logger.Info(fmt.Sprintf( "[%s] commit received, returning currently accumulated records - %d", - req.FlowJobName, + p.flowJobName, cdcRecordsStorage.Len()), ) return nil @@ -274,7 +284,7 @@ func (p *PostgresCDCSource) consumeStream( if time.Now().After(nextStandbyMessageDeadline) { if !cdcRecordsStorage.IsEmpty() { p.logger.Info(fmt.Sprintf("[%s] standby deadline reached, have %d records, will return at next commit", - req.FlowJobName, + p.flowJobName, cdcRecordsStorage.Len()), ) @@ -286,7 +296,7 @@ func (p *PostgresCDCSource) consumeStream( waitingForCommit = true } else { p.logger.Info(fmt.Sprintf("[%s] standby deadline reached, no records accumulated, continuing to wait", - req.FlowJobName), + p.flowJobName), ) } nextStandbyMessageDeadline = time.Now().Add(standbyMessageTimeout) @@ -329,8 +339,9 @@ func (p *PostgresCDCSource) consumeStream( return fmt.Errorf("ParsePrimaryKeepaliveMessage failed: %w", err) } - p.logger.Debug(fmt.Sprintf("Primary Keepalive Message => ServerWALEnd: %s ServerTime: %s ReplyRequested: %t", - pkm.ServerWALEnd, pkm.ServerTime, pkm.ReplyRequested)) + p.logger.Debug( + fmt.Sprintf("Primary Keepalive Message => ServerWALEnd: %s ServerTime: %s ReplyRequested: %t", + pkm.ServerWALEnd, pkm.ServerTime, pkm.ReplyRequested)) if pkm.ServerWALEnd > clientXLogPos { clientXLogPos = pkm.ServerWALEnd @@ -348,13 +359,13 @@ func (p *PostgresCDCSource) consumeStream( p.logger.Debug(fmt.Sprintf("XLogData => WALStart %s ServerWALEnd %s ServerTime %s\n", xld.WALStart, xld.ServerWALEnd, xld.ServerTime)) - rec, err := p.processMessage(records, xld) + rec, err := p.processMessage(records, xld, clientXLogPos) if err != nil { return fmt.Errorf("error processing message: %w", err) } if rec != nil { - tableName := rec.GetTableName() + tableName := rec.GetDestinationTableName() switch r := rec.(type) { case *model.UpdateRecord: // tableName here is destination tableName. @@ -453,18 +464,13 @@ func (p *PostgresCDCSource) consumeStream( if xld.WALStart > clientXLogPos { clientXLogPos = xld.WALStart } - - if cdcRecordsStorage.IsEmpty() { - // given that we have no records it is safe to update the flush wal position - // to the clientXLogPos. clientXLogPos can be moved forward due to PKM messages. - consumedXLogPos = clientXLogPos - records.UpdateLatestCheckpoint(int64(clientXLogPos)) - } } } } -func (p *PostgresCDCSource) processMessage(batch *model.CDCRecordStream, xld pglogrepl.XLogData) (model.Record, error) { +func (p *PostgresCDCSource) processMessage(batch *model.CDCRecordStream, xld pglogrepl.XLogData, + currentClientXlogPos pglogrepl.LSN, +) (model.Record, error) { logicalMsg, err := pglogrepl.Parse(xld.WALData) if err != nil { return nil, fmt.Errorf("error parsing logical message: %w", err) @@ -503,7 +509,10 @@ func (p *PostgresCDCSource) processMessage(batch *model.CDCRecordStream, xld pgl if p.relationMessageMapping[msg.RelationID] == nil { p.relationMessageMapping[msg.RelationID] = convertRelationMessageToProto(msg) } else { - return p.processRelationMessage(xld.WALStart, convertRelationMessageToProto(msg)) + // RelationMessages don't contain an LSN, so we use current clientXlogPos instead. + //nolint:lll + // https://github.com/postgres/postgres/blob/8b965c549dc8753be8a38c4a1b9fabdb535a4338/src/backend/replication/logical/proto.c#L670 + return p.processRelationMessage(currentClientXlogPos, convertRelationMessageToProto(msg)) } case *pglogrepl.TruncateMessage: @@ -746,7 +755,27 @@ func convertRelationMessageToProto(msg *pglogrepl.RelationMessage) *protos.Relat } } -// processRelationMessage processes a delete message and returns a TableSchemaDelta +func (p *PostgresCDCSource) auditSchemaDelta(flowJobName string, rec *model.RelationRecord) error { + activityInfo := activity.GetInfo(p.ctx) + workflowID := activityInfo.WorkflowExecution.ID + runID := activityInfo.WorkflowExecution.RunID + recJSON, err := json.Marshal(rec) + if err != nil { + return fmt.Errorf("failed to marshal schema delta to JSON: %w", err) + } + + _, err = p.catalogPool.Exec(p.ctx, + `INSERT INTO + peerdb_stats.schema_deltas_audit_log(flow_job_name,workflow_id,run_id,delta_info) + VALUES($1,$2,$3,$4)`, + flowJobName, workflowID, runID, recJSON) + if err != nil { + return fmt.Errorf("failed to insert row into table: %w", err) + } + return nil +} + +// processRelationMessage processes a RelationMessage and returns a TableSchemaDelta func (p *PostgresCDCSource) processRelationMessage( lsn pglogrepl.LSN, currRel *protos.RelationMessage, @@ -804,16 +833,17 @@ func (p *PostgresCDCSource) processRelationMessage( } p.relationMessageMapping[currRel.RelationId] = currRel - return &model.RelationRecord{ + rec := &model.RelationRecord{ TableSchemaDelta: schemaDelta, CheckPointID: int64(lsn), - }, nil + } + return rec, p.auditSchemaDelta(p.flowJobName, rec) } func (p *PostgresCDCSource) recToTablePKey(req *model.PullRecordsRequest, rec model.Record, ) (*model.TableWithPkey, error) { - tableName := rec.GetTableName() + tableName := rec.GetDestinationTableName() pkeyColsMerged := make([]byte, 0) for _, pkeyCol := range req.TableNameSchemaMapping[tableName].PrimaryKeyColumns { diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index dbae60b9e..e48c71b29 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -15,6 +15,7 @@ import ( "github.com/jackc/pglogrepl" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgtype" + "github.com/lib/pq/oid" "golang.org/x/exp/maps" ) @@ -33,13 +34,14 @@ const ( createRawTableDstTableIndexSQL = "CREATE INDEX IF NOT EXISTS %s_dst_table_idx ON %s.%s(_peerdb_destination_table_name)" getLastOffsetSQL = "SELECT lsn_offset FROM %s.%s WHERE mirror_job_name=$1" + setLastOffsetSQL = "UPDATE %s.%s SET lsn_offset=GREATEST(lsn_offset, $1) WHERE mirror_job_name=$2" getLastSyncBatchID_SQL = "SELECT sync_batch_id FROM %s.%s WHERE mirror_job_name=$1" getLastNormalizeBatchID_SQL = "SELECT normalize_batch_id FROM %s.%s WHERE mirror_job_name=$1" createNormalizedTableSQL = "CREATE TABLE IF NOT EXISTS %s(%s)" insertJobMetadataSQL = "INSERT INTO %s.%s VALUES ($1,$2,$3,$4)" checkIfJobMetadataExistsSQL = "SELECT COUNT(1)::TEXT::BOOL FROM %s.%s WHERE mirror_job_name=$1" - updateMetadataForSyncRecordsSQL = "UPDATE %s.%s SET lsn_offset=$1, sync_batch_id=$2 WHERE mirror_job_name=$3" + updateMetadataForSyncRecordsSQL = "UPDATE %s.%s SET lsn_offset=GREATEST(lsn_offset, $1), sync_batch_id=$2 WHERE mirror_job_name=$3" updateMetadataForNormalizeRecordsSQL = "UPDATE %s.%s SET normalize_batch_id=$1 WHERE mirror_job_name=$2" getTableNameToUnchangedToastColsSQL = `SELECT _peerdb_destination_table_name, @@ -58,7 +60,7 @@ const ( INSERT (%s) VALUES (%s) %s WHEN MATCHED AND src._peerdb_record_type=2 THEN - DELETE` + %s` fallbackUpsertStatementSQL = `WITH src_rank AS ( SELECT _peerdb_data,_peerdb_record_type,_peerdb_unchanged_toast_columns, RANK() OVER (PARTITION BY %s ORDER BY _peerdb_timestamp DESC) AS _peerdb_rank @@ -71,10 +73,20 @@ const ( RANK() OVER (PARTITION BY %s ORDER BY _peerdb_timestamp DESC) AS _peerdb_rank FROM %s.%s WHERE _peerdb_batch_id>$1 AND _peerdb_batch_id<=$2 AND _peerdb_destination_table_name=$3 ) - DELETE FROM %s USING src_rank WHERE %s AND src_rank._peerdb_rank=1 AND src_rank._peerdb_record_type=2` + %s src_rank WHERE %s AND src_rank._peerdb_rank=1 AND src_rank._peerdb_record_type=2` - dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" - deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=$1" + dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" + deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE mirror_job_name=$1" + getNumConnectionsForUser = "SELECT COUNT(*) FROM pg_stat_activity WHERE usename=$1 AND client_addr IS NOT NULL" +) + +type ReplicaIdentityType rune + +const ( + ReplicaIdentityDefault ReplicaIdentityType = 'd' + ReplicaIdentityFull ReplicaIdentityType = 'f' + ReplicaIdentityIndex ReplicaIdentityType = 'i' + ReplicaIdentityNothing ReplicaIdentityType = 'n' ) // getRelIDForTable returns the relation ID for a table. @@ -92,10 +104,10 @@ func (c *PostgresConnector) getRelIDForTable(schemaTable *utils.SchemaTable) (ui } // getReplicaIdentity returns the replica identity for a table. -func (c *PostgresConnector) isTableFullReplica(schemaTable *utils.SchemaTable) (bool, error) { +func (c *PostgresConnector) getReplicaIdentityType(schemaTable *utils.SchemaTable) (ReplicaIdentityType, error) { relID, relIDErr := c.getRelIDForTable(schemaTable) if relIDErr != nil { - return false, fmt.Errorf("failed to get relation id for table %s: %w", schemaTable, relIDErr) + return ReplicaIdentityDefault, fmt.Errorf("failed to get relation id for table %s: %w", schemaTable, relIDErr) } var replicaIdentity rune @@ -103,43 +115,76 @@ func (c *PostgresConnector) isTableFullReplica(schemaTable *utils.SchemaTable) ( `SELECT relreplident FROM pg_class WHERE oid = $1;`, relID).Scan(&replicaIdentity) if err != nil { - return false, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, err) + return ReplicaIdentityDefault, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, err) } - return string(replicaIdentity) == "f", nil + + return ReplicaIdentityType(replicaIdentity), nil } -// getPrimaryKeyColumns for table returns the primary key column for a given table -// errors if there is no primary key column or if there is more than one primary key column. -func (c *PostgresConnector) getPrimaryKeyColumns(schemaTable *utils.SchemaTable) ([]string, error) { +// getPrimaryKeyColumns returns the primary key columns for a given table. +// Errors if there is no primary key column or if there is more than one primary key column. +func (c *PostgresConnector) getPrimaryKeyColumns( + replicaIdentity ReplicaIdentityType, + schemaTable *utils.SchemaTable, +) ([]string, error) { relID, err := c.getRelIDForTable(schemaTable) if err != nil { return nil, fmt.Errorf("failed to get relation id for table %s: %w", schemaTable, err) } - // Get the primary key column name - var pkCol pgtype.Text - pkCols := make([]string, 0) + if replicaIdentity == ReplicaIdentityIndex { + return c.getReplicaIdentityIndexColumns(relID, schemaTable) + } + + // Find the primary key index OID + var pkIndexOID oid.Oid + err = c.pool.QueryRow(c.ctx, + `SELECT indexrelid FROM pg_index WHERE indrelid = $1 AND indisprimary`, + relID).Scan(&pkIndexOID) + if err != nil { + return nil, fmt.Errorf("error finding primary key index for table %s: %w", schemaTable, err) + } + + return c.getColumnNamesForIndex(pkIndexOID) +} + +// getReplicaIdentityIndexColumns returns the columns used in the replica identity index. +func (c *PostgresConnector) getReplicaIdentityIndexColumns(relID uint32, schemaTable *utils.SchemaTable) ([]string, error) { + var indexRelID oid.Oid + // Fetch the OID of the index used as the replica identity + err := c.pool.QueryRow(c.ctx, + `SELECT indexrelid FROM pg_index + WHERE indrelid = $1 AND indisreplident = true`, + relID).Scan(&indexRelID) + if err != nil { + return nil, fmt.Errorf("error finding replica identity index for table %s: %w", schemaTable, err) + } + + return c.getColumnNamesForIndex(indexRelID) +} + +// getColumnNamesForIndex returns the column names for a given index. +func (c *PostgresConnector) getColumnNamesForIndex(indexOID oid.Oid) ([]string, error) { + var col pgtype.Text + cols := make([]string, 0) rows, err := c.pool.Query(c.ctx, `SELECT a.attname FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) - WHERE i.indrelid = $1 AND i.indisprimary ORDER BY a.attname ASC`, - relID) + WHERE i.indexrelid = $1 ORDER BY a.attname ASC`, + indexOID) if err != nil { - return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) + return nil, fmt.Errorf("error getting columns for index %v: %w", indexOID, err) } defer rows.Close() - for { - if !rows.Next() { - break - } - err = rows.Scan(&pkCol) + + for rows.Next() { + err = rows.Scan(&col) if err != nil { - return nil, fmt.Errorf("error scanning primary key column for table %s: %w", schemaTable, err) + return nil, fmt.Errorf("error scanning column for index %v: %w", indexOID, err) } - pkCols = append(pkCols, pkCol.String) + cols = append(cols, col.String) } - - return pkCols, nil + return cols, nil } func (c *PostgresConnector) tableExists(schemaTable *utils.SchemaTable) (bool, error) { @@ -209,7 +254,8 @@ func (c *PostgresConnector) GetSlotInfo(slotName string) ([]*protos.SlotInfo, er } rows, err := c.pool.Query(c.ctx, "SELECT slot_name, redo_lsn::Text,restart_lsn::text,wal_status,"+ "confirmed_flush_lsn::text,active,"+ - "round((pg_current_wal_lsn() - confirmed_flush_lsn) / 1024 / 1024) AS MB_Behind"+ + "round((CASE WHEN pg_is_in_recovery() THEN pg_last_wal_receive_lsn() ELSE pg_current_wal_lsn() END"+ + " - confirmed_flush_lsn) / 1024 / 1024) AS MB_Behind"+ " FROM pg_control_checkpoint(), pg_replication_slots"+specificSlotClause+";") if err != nil { return nil, err @@ -346,15 +392,28 @@ func getRawTableIdentifier(jobName string) string { return fmt.Sprintf("%s_%s", rawTablePrefix, strings.ToLower(jobName)) } -func generateCreateTableSQLForNormalizedTable(sourceTableIdentifier string, +func generateCreateTableSQLForNormalizedTable( + sourceTableIdentifier string, sourceTableSchema *protos.TableSchema, + softDeleteColName string, + syncedAtColName string, ) string { - createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)) + createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)+2) for columnName, genericColumnType := range sourceTableSchema.Columns { createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("\"%s\" %s,", columnName, qValueKindToPostgresType(genericColumnType))) } + if softDeleteColName != "" { + createTableSQLArray = append(createTableSQLArray, + fmt.Sprintf(`"%s" BOOL DEFAULT FALSE,`, softDeleteColName)) + } + + if syncedAtColName != "" { + createTableSQLArray = append(createTableSQLArray, + fmt.Sprintf(`"%s" TIMESTAMP DEFAULT CURRENT_TIMESTAMP,`, syncedAtColName)) + } + // add composite primary key to the table if len(sourceTableSchema.PrimaryKeyColumns) > 0 { primaryKeyColsQuoted := make([]string, 0, len(sourceTableSchema.PrimaryKeyColumns)) @@ -383,7 +442,7 @@ func (c *PostgresConnector) GetLastSyncBatchID(jobName string) (int64, error) { var result pgtype.Int8 if !rows.Next() { - c.logger.Info("No row found ,returning 0") + c.logger.Info("No row found, returning 0") return 0, nil } err = rows.Scan(&result) @@ -430,6 +489,7 @@ func (c *PostgresConnector) jobMetadataExistsTx(tx pgx.Tx, jobName string) (bool if err != nil { return false, fmt.Errorf("error reading result row: %w", err) } + return result.Bool, nil } @@ -523,17 +583,19 @@ func (c *PostgresConnector) getTableNametoUnchangedCols(flowJobName string, sync func (c *PostgresConnector) generateNormalizeStatements(destinationTableIdentifier string, unchangedToastColumns []string, rawTableIdentifier string, supportsMerge bool, + peerdbCols *protos.PeerDBColumns, ) []string { if supportsMerge { - return []string{c.generateMergeStatement(destinationTableIdentifier, unchangedToastColumns, rawTableIdentifier)} + return []string{c.generateMergeStatement(destinationTableIdentifier, unchangedToastColumns, + rawTableIdentifier, peerdbCols)} } c.logger.Warn("Postgres version is not high enough to support MERGE, falling back to UPSERT + DELETE") c.logger.Warn("TOAST columns will not be updated properly, use REPLICA IDENTITY FULL or upgrade Postgres") - return c.generateFallbackStatements(destinationTableIdentifier, rawTableIdentifier) + return c.generateFallbackStatements(destinationTableIdentifier, rawTableIdentifier, peerdbCols) } func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifier string, - rawTableIdentifier string, + rawTableIdentifier string, peerdbCols *protos.PeerDBColumns, ) []string { normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] columnNames := make([]string, 0, len(normalizedTableSchema.Columns)) @@ -569,20 +631,35 @@ func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifie parsedDstTable.String(), columnName, columnCast)) } deleteWhereClauseSQL := strings.TrimSuffix(strings.Join(deleteWhereClauseArray, ""), "AND ") - + deletePart := fmt.Sprintf( + "DELETE FROM %s USING", + parsedDstTable.String()) + + if peerdbCols.SoftDelete { + deletePart = fmt.Sprintf(`UPDATE %s SET "%s" = TRUE`, + parsedDstTable.String(), peerdbCols.SoftDeleteColName) + if peerdbCols.SyncedAtColName != "" { + deletePart = fmt.Sprintf(`%s, "%s" = CURRENT_TIMESTAMP`, + deletePart, peerdbCols.SyncedAtColName) + } + deletePart += " FROM" + } fallbackUpsertStatement := fmt.Sprintf(fallbackUpsertStatementSQL, strings.TrimSuffix(strings.Join(maps.Values(primaryKeyColumnCasts), ","), ","), c.metadataSchema, rawTableIdentifier, parsedDstTable.String(), insertColumnsSQL, flattenedCastsSQL, strings.Join(normalizedTableSchema.PrimaryKeyColumns, ","), updateColumnsSQL) fallbackDeleteStatement := fmt.Sprintf(fallbackDeleteStatementSQL, strings.Join(maps.Values(primaryKeyColumnCasts), ","), c.metadataSchema, - rawTableIdentifier, parsedDstTable.String(), deleteWhereClauseSQL) + rawTableIdentifier, deletePart, deleteWhereClauseSQL) return []string{fallbackUpsertStatement, fallbackDeleteStatement} } -func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier string, unchangedToastColumns []string, +func (c *PostgresConnector) generateMergeStatement( + destinationTableIdentifier string, + unchangedToastColumns []string, rawTableIdentifier string, + peerdbCols *protos.PeerDBColumns, ) string { normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] columnNames := maps.Keys(normalizedTableSchema.Columns) @@ -612,21 +689,60 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st } } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ","), ",") - - insertColumnsSQL := strings.TrimSuffix(strings.Join(columnNames, ","), ",") insertValuesSQLArray := make([]string, 0, len(columnNames)) for _, columnName := range columnNames { insertValuesSQLArray = append(insertValuesSQLArray, fmt.Sprintf("src.%s", columnName)) } + + updateStatementsforToastCols := c.generateUpdateStatement(columnNames, unchangedToastColumns, peerdbCols) + // append synced_at column + columnNames = append(columnNames, fmt.Sprintf(`"%s"`, peerdbCols.SyncedAtColName)) + insertColumnsSQL := strings.Join(columnNames, ",") + // fill in synced_at column + insertValuesSQLArray = append(insertValuesSQLArray, "CURRENT_TIMESTAMP") insertValuesSQL := strings.TrimSuffix(strings.Join(insertValuesSQLArray, ","), ",") - updateStatements := c.generateUpdateStatement(columnNames, unchangedToastColumns) - return fmt.Sprintf(mergeStatementSQL, strings.Join(maps.Values(primaryKeyColumnCasts), ","), - c.metadataSchema, rawTableIdentifier, parsedDstTable.String(), flattenedCastsSQL, - strings.Join(primaryKeySelectSQLArray, " AND "), insertColumnsSQL, insertValuesSQL, updateStatements) + if peerdbCols.SoftDelete { + softDeleteInsertColumnsSQL := strings.TrimSuffix(strings.Join(append(columnNames, + fmt.Sprintf(`"%s"`, peerdbCols.SoftDeleteColName)), ","), ",") + softDeleteInsertValuesSQL := strings.Join(append(insertValuesSQLArray, "TRUE"), ",") + + updateStatementsforToastCols = append(updateStatementsforToastCols, + fmt.Sprintf("WHEN NOT MATCHED AND (src._peerdb_record_type = 2) THEN INSERT (%s) VALUES(%s)", + softDeleteInsertColumnsSQL, softDeleteInsertValuesSQL)) + } + updateStringToastCols := strings.Join(updateStatementsforToastCols, "\n") + + deletePart := "DELETE" + if peerdbCols.SoftDelete { + colName := peerdbCols.SoftDeleteColName + deletePart = fmt.Sprintf(`UPDATE SET "%s" = TRUE`, colName) + if peerdbCols.SyncedAtColName != "" { + deletePart = fmt.Sprintf(`%s, "%s" = CURRENT_TIMESTAMP`, + deletePart, peerdbCols.SyncedAtColName) + } + } + + mergeStmt := fmt.Sprintf( + mergeStatementSQL, + strings.Join(maps.Values(primaryKeyColumnCasts), ","), + c.metadataSchema, + rawTableIdentifier, + parsedDstTable.String(), + flattenedCastsSQL, + strings.Join(primaryKeySelectSQLArray, " AND "), + insertColumnsSQL, + insertValuesSQL, + updateStringToastCols, + deletePart, + ) + + return mergeStmt } -func (c *PostgresConnector) generateUpdateStatement(allCols []string, unchangedToastColsLists []string) string { +func (c *PostgresConnector) generateUpdateStatement(allCols []string, + unchangedToastColsLists []string, peerdbCols *protos.PeerDBColumns, +) []string { updateStmts := make([]string, 0, len(unchangedToastColsLists)) for _, cols := range unchangedToastColsLists { @@ -640,17 +756,42 @@ func (c *PostgresConnector) generateUpdateStatement(allCols []string, unchangedT for _, colName := range otherCols { tmpArray = append(tmpArray, fmt.Sprintf("%s=src.%s", colName, colName)) } + // set the synced at column to the current timestamp + if peerdbCols.SyncedAtColName != "" { + tmpArray = append(tmpArray, fmt.Sprintf(`"%s" = CURRENT_TIMESTAMP`, + peerdbCols.SyncedAtColName)) + } + // set soft-deleted to false, tackles insert after soft-delete + if peerdbCols.SoftDelete && (peerdbCols.SoftDeleteColName != "") { + tmpArray = append(tmpArray, fmt.Sprintf(`"%s" = FALSE`, + peerdbCols.SoftDeleteColName)) + } + ssep := strings.Join(tmpArray, ",") updateStmt := fmt.Sprintf(`WHEN MATCHED AND src._peerdb_record_type=1 AND _peerdb_unchanged_toast_columns='%s' THEN UPDATE SET %s `, cols, ssep) updateStmts = append(updateStmts, updateStmt) + + // generates update statements for the case where updates and deletes happen in the same branch + // the backfill has happened from the pull side already, so treat the DeleteRecord as an update + // and then set soft-delete to true. + if peerdbCols.SoftDelete && (peerdbCols.SoftDeleteColName != "") { + tmpArray = append(tmpArray[:len(tmpArray)-1], + fmt.Sprintf(`"%s" = TRUE`, peerdbCols.SoftDeleteColName)) + ssep := strings.Join(tmpArray, ", ") + updateStmt := fmt.Sprintf(`WHEN MATCHED AND + src._peerdb_record_type = 2 AND _peerdb_unchanged_toast_columns='%s' + THEN UPDATE SET %s `, cols, ssep) + updateStmts = append(updateStmts, updateStmt) + } } - return strings.Join(updateStmts, "\n") + return updateStmts } func (c *PostgresConnector) getCurrentLSN() (pglogrepl.LSN, error) { - row := c.pool.QueryRow(c.ctx, "SELECT pg_current_wal_lsn();") + row := c.pool.QueryRow(c.ctx, + "SELECT CASE WHEN pg_is_in_recovery() THEN pg_last_wal_receive_lsn() ELSE pg_current_wal_lsn() END") var result pgtype.Text err := row.Scan(&result) if err != nil { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index da65b09c0..59a1b835b 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -34,7 +34,7 @@ type PostgresConnector struct { } // NewPostgresConnector creates a new instance of PostgresConnector. -func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) (*PostgresConnector, error) { +func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig, initializeReplPool bool) (*PostgresConnector, error) { connectionString := utils.GetPGConnectionString(pgConfig) // create a separate connection pool for non-replication queries as replication connections cannot @@ -62,21 +62,23 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) return nil, fmt.Errorf("failed to get custom type map: %w", err) } - // ensure that replication is set to database - replConnConfig, err := pgxpool.ParseConfig(connectionString) - if err != nil { - return nil, fmt.Errorf("failed to parse connection string: %w", err) - } + // only initialize for CDCPullConnector to reduce number of idle connections + var replPool *SSHWrappedPostgresPool + if initializeReplPool { + // ensure that replication is set to database + replConnConfig, err := pgxpool.ParseConfig(connectionString) + if err != nil { + return nil, fmt.Errorf("failed to parse connection string: %w", err) + } - replConnConfig.ConnConfig.RuntimeParams["replication"] = "database" - replConnConfig.ConnConfig.RuntimeParams["bytea_output"] = "hex" - replConnConfig.MaxConns = 1 + replConnConfig.ConnConfig.RuntimeParams["replication"] = "database" + replConnConfig.ConnConfig.RuntimeParams["bytea_output"] = "hex" + replConnConfig.MaxConns = 1 - // 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 := NewSSHWrappedPostgresPool(ctx, replConnConfig, pgConfig.SshConfig) - if err != nil { - return nil, fmt.Errorf("failed to create connection pool: %w", err) + replPool, err = NewSSHWrappedPostgresPool(ctx, replConnConfig, pgConfig.SshConfig) + if err != nil { + return nil, fmt.Errorf("failed to create replication connection pool: %w", err) + } } metadataSchema := "_peerdb_internal" @@ -185,13 +187,24 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (int64, error) { if err != nil { return 0, fmt.Errorf("error while reading result row: %w", err) } + if result.Int64 == 0 { c.logger.Warn("Assuming zero offset means no sync has happened") } - return result.Int64, nil } +// SetLastOffset updates the last synced offset for a job. +func (c *PostgresConnector) SetLastOffset(jobName string, lastOffset int64) error { + _, err := c.pool. + Exec(c.ctx, fmt.Sprintf(setLastOffsetSQL, c.metadataSchema, mirrorJobsTableIdentifier), lastOffset, jobName) + if err != nil { + return fmt.Errorf("error setting last offset for job %s: %w", jobName, err) + } + + return nil +} + // PullRecords pulls records from the source. func (c *PostgresConnector) PullRecords(catalogPool *pgxpool.Pool, req *model.PullRecordsRequest) error { defer func() { @@ -236,6 +249,9 @@ func (c *PostgresConnector) PullRecords(catalogPool *pgxpool.Pool, req *model.Pu Publication: publicationName, TableNameMapping: req.TableNameMapping, RelationMessageMapping: req.RelationMessageMapping, + CatalogPool: catalogPool, + FlowJobName: req.FlowJobName, + SetLastOffset: req.SetLastOffset, }, c.customTypesMapping) if err != nil { return fmt.Errorf("failed to create cdc source: %w", err) @@ -365,7 +381,8 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S len(records), syncedRecordsCount) } - c.logger.Info(fmt.Sprintf("synced %d records to Postgres table %s via COPY", syncedRecordsCount, rawTableIdentifier)) + c.logger.Info(fmt.Sprintf("synced %d records to Postgres table %s via COPY", + syncedRecordsCount, rawTableIdentifier)) lastCP, err := req.Records.GetLastCheckpoint() if err != nil { @@ -406,7 +423,7 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) return nil, err } // normalize has caught up with sync or no SyncFlow has run, chill until more records are loaded. - if syncBatchID == normalizeBatchID || !jobMetadataExists { + if normalizeBatchID >= syncBatchID || !jobMetadataExists { c.logger.Info(fmt.Sprintf("no records to normalize: syncBatchID %d, normalizeBatchID %d", syncBatchID, normalizeBatchID)) return &model.NormalizeResponse{ @@ -439,8 +456,13 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) mergeStatementsBatch := &pgx.Batch{} totalRowsAffected := 0 for destinationTableName, unchangedToastCols := range unchangedToastColsMap { + peerdbCols := protos.PeerDBColumns{ + SoftDeleteColName: req.SoftDeleteColName, + SyncedAtColName: req.SyncedAtColName, + SoftDelete: req.SoftDelete, + } normalizeStatements := c.generateNormalizeStatements(destinationTableName, unchangedToastCols, - rawTableIdentifier, supportsMerge) + rawTableIdentifier, supportsMerge, &peerdbCols) for _, normalizeStatement := range normalizeStatements { mergeStatementsBatch.Queue(normalizeStatement, normalizeBatchID, syncBatchID, destinationTableName).Exec( func(ct pgconn.CommandTag) error { @@ -550,12 +572,12 @@ func (c *PostgresConnector) getTableSchemaForTable( return nil, err } - isFullReplica, replErr := c.isTableFullReplica(schemaTable) + replicaIdentityType, replErr := c.getReplicaIdentityType(schemaTable) if replErr != nil { return nil, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, replErr) } - pKeyCols, err := c.getPrimaryKeyColumns(schemaTable) + pKeyCols, err := c.getPrimaryKeyColumns(replicaIdentityType, schemaTable) if err != nil { return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) } @@ -573,7 +595,7 @@ func (c *PostgresConnector) getTableSchemaForTable( TableIdentifier: tableName, Columns: make(map[string]string), PrimaryKeyColumns: pKeyCols, - IsReplicaIdentityFull: isFullReplica, + IsReplicaIdentityFull: replicaIdentityType == ReplicaIdentityFull, } for _, fieldDescription := range rows.FieldDescriptions() { @@ -631,7 +653,7 @@ func (c *PostgresConnector) SetupNormalizedTables(req *protos.SetupNormalizedTab // convert the column names and types to Postgres types normalizedTableCreateSQL := generateCreateTableSQLForNormalizedTable( - parsedNormalizedTable.String(), tableSchema) + parsedNormalizedTable.String(), tableSchema, req.SoftDeleteColName, req.SyncedAtColName) _, err = createNormalizedTablesTx.Exec(c.ctx, normalizedTableCreateSQL) if err != nil { return nil, fmt.Errorf("error while creating normalized table: %w", err) @@ -723,18 +745,18 @@ func (c *PostgresConnector) EnsurePullability(req *protos.EnsurePullabilityBatch return nil, err } - isFullReplica, replErr := c.isTableFullReplica(schemaTable) + replicaIdentity, replErr := c.getReplicaIdentityType(schemaTable) if replErr != nil { return nil, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, replErr) } - pKeyCols, err := c.getPrimaryKeyColumns(schemaTable) + pKeyCols, err := c.getPrimaryKeyColumns(replicaIdentity, schemaTable) if err != nil { return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) } // we only allow no primary key if the table has REPLICA IDENTITY FULL - if len(pKeyCols) == 0 && !isFullReplica { + if len(pKeyCols) == 0 && !(replicaIdentity == ReplicaIdentityFull) { return nil, fmt.Errorf("table %s has no primary keys and does not have REPLICA IDENTITY FULL", schemaTable) } @@ -860,18 +882,20 @@ func (c *PostgresConnector) SyncFlowCleanup(jobName string) error { return nil } -func (c *PostgresConnector) SendWALHeartbeat() error { - command := ` - BEGIN; - DROP aggregate IF EXISTS PEERDB_EPHEMERAL_HEARTBEAT(float4); - CREATE AGGREGATE PEERDB_EPHEMERAL_HEARTBEAT(float4) (SFUNC = float4pl, STYPE = float4); - DROP aggregate PEERDB_EPHEMERAL_HEARTBEAT(float4); - END; - ` - _, err := c.pool.Exec(c.ctx, command) +// GetLastOffset returns the last synced offset for a job. +func (c *PostgresConnector) GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) { + row := c.pool. + QueryRow(c.ctx, getNumConnectionsForUser, c.config.User) + + // COUNT() returns BIGINT + var result pgtype.Int8 + err := row.Scan(&result) if err != nil { - return fmt.Errorf("error bumping wal position: %w", err) + return nil, fmt.Errorf("error while reading result row: %w", err) } - return nil + return &protos.GetOpenConnectionsForUserResult{ + UserName: c.config.User, + CurrentOpenConnections: result.Int64, + }, nil } diff --git a/flow/connectors/postgres/postgres_repl_test.go b/flow/connectors/postgres/postgres_repl_test.go index b50a1f89f..df3a7de13 100644 --- a/flow/connectors/postgres/postgres_repl_test.go +++ b/flow/connectors/postgres/postgres_repl_test.go @@ -28,7 +28,7 @@ func (suite *PostgresReplicationSnapshotTestSuite) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, true) require.NoError(suite.T(), err) setupTx, err := suite.connector.pool.Begin(context.Background()) diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 98a6a47b9..8a919eb21 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -32,7 +32,7 @@ func (suite *PostgresSchemaDeltaTestSuite) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, false) suite.failTestError(err) setupTx, err := suite.connector.pool.Begin(context.Background()) diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index be8daa903..ce114b702 100644 --- a/flow/connectors/postgres/qrep.go +++ b/flow/connectors/postgres/qrep.go @@ -471,7 +471,8 @@ func (c *PostgresConnector) SyncQRepRecords( stagingTableSync := &QRepStagingTableSync{connector: c} return stagingTableSync.SyncQRepRecords( - config.FlowJobName, dstTable, partition, stream, config.WriteMode) + config.FlowJobName, dstTable, partition, stream, + config.WriteMode, config.SyncedAtColName) } // SetupQRepMetadataTables function for postgres connector diff --git a/flow/connectors/postgres/qrep_sync_method.go b/flow/connectors/postgres/qrep_sync_method.go index a54769e3d..672503241 100644 --- a/flow/connectors/postgres/qrep_sync_method.go +++ b/flow/connectors/postgres/qrep_sync_method.go @@ -35,6 +35,7 @@ func (s *QRepStagingTableSync) SyncQRepRecords( partition *protos.QRepPartition, stream *model.QRecordStream, writeMode *protos.QRepWriteMode, + syncedAtCol string, ) (int, error) { syncLog := slog.Group("sync-qrep-log", slog.String(string(shared.FlowNameKey), flowJobName), @@ -81,6 +82,19 @@ func (s *QRepStagingTableSync) SyncQRepRecords( if err != nil { return -1, fmt.Errorf("failed to copy records into destination table: %v", err) } + + if syncedAtCol != "" { + updateSyncedAtStmt := fmt.Sprintf( + `UPDATE %s SET "%s" = CURRENT_TIMESTAMP WHERE "%s" IS NULL;`, + pgx.Identifier{dstTableName.Schema, dstTableName.Table}.Sanitize(), + syncedAtCol, + syncedAtCol, + ) + _, err = tx.Exec(context.Background(), updateSyncedAtStmt) + if err != nil { + return -1, fmt.Errorf("failed to update synced_at column: %v", err) + } + } } else { // Step 2.1: Create a temp staging table stagingTableName := fmt.Sprintf("_peerdb_staging_%s", shared.RandomString(8)) @@ -128,16 +142,18 @@ func (s *QRepStagingTableSync) SyncQRepRecords( } selectStrArray = append(selectStrArray, fmt.Sprintf(`"%s"`, col)) } - + setClauseArray = append(setClauseArray, + fmt.Sprintf(`"%s" = CURRENT_TIMESTAMP`, syncedAtCol)) setClause := strings.Join(setClauseArray, ",") - selectStr := strings.Join(selectStrArray, ",") + selectSQL := strings.Join(selectStrArray, ",") // Step 2.3: Perform the upsert operation, ON CONFLICT UPDATE upsertStmt := fmt.Sprintf( - "INSERT INTO %s (%s) SELECT %s FROM %s ON CONFLICT (%s) DO UPDATE SET %s;", + `INSERT INTO %s (%s, "%s") SELECT %s, CURRENT_TIMESTAMP FROM %s ON CONFLICT (%s) DO UPDATE SET %s;`, dstTableIdentifier.Sanitize(), - selectStr, - selectStr, + selectSQL, + syncedAtCol, + selectSQL, stagingTableIdentifier.Sanitize(), strings.Join(writeMode.UpsertKeyColumns, ", "), setClause, diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 96d16930c..c40ef05bd 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -176,7 +176,7 @@ func (c *S3Connector) GetLastOffset(jobName string) (int64, error) { } // update offset for a job -func (c *S3Connector) updateLastOffset(jobName string, offset int64) error { +func (c *S3Connector) SetLastOffset(jobName string, offset int64) error { err := c.pgMetadata.UpdateLastOffset(jobName, offset) if err != nil { c.logger.Error("failed to update last offset: ", slog.Any("error", err)) @@ -218,7 +218,7 @@ func (c *S3Connector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncRes return nil, fmt.Errorf("failed to get last checkpoint: %w", err) } - err = c.updateLastOffset(req.FlowJobName, lastCheckpoint) + err = c.SetLastOffset(req.FlowJobName, lastCheckpoint) if err != nil { c.logger.Error("failed to update last offset for s3 cdc", slog.Any("error", err)) return nil, err diff --git a/flow/connectors/snowflake/avro_file_writer_test.go b/flow/connectors/snowflake/avro_file_writer_test.go index 46f18aaa3..f08b66a6c 100644 --- a/flow/connectors/snowflake/avro_file_writer_test.go +++ b/flow/connectors/snowflake/avro_file_writer_test.go @@ -55,6 +55,7 @@ func createQValue(t *testing.T, kind qvalue.QValueKind, placeHolder int) qvalue. } } +// nolint:unparam func generateRecords( t *testing.T, nullable bool, diff --git a/flow/connectors/snowflake/client.go b/flow/connectors/snowflake/client.go index ae2da1a89..beb38a4a0 100644 --- a/flow/connectors/snowflake/client.go +++ b/flow/connectors/snowflake/client.go @@ -9,6 +9,7 @@ import ( "github.com/snowflakedb/gosnowflake" peersql "github.com/PeerDB-io/peer-flow/connectors/sql" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" @@ -69,7 +70,7 @@ func NewSnowflakeClient(ctx context.Context, config *protos.SnowflakeConfig) (*S func (c *SnowflakeConnector) getTableCounts(tables []string) (int64, error) { var totalRecords int64 for _, table := range tables { - _, err := parseTableName(table) + _, err := utils.ParseSchemaTable(table) if err != nil { return 0, fmt.Errorf("failed to parse table name %s: %w", table, err) } diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index b099a54ff..1d81392d7 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -84,7 +84,7 @@ func (c *SnowflakeConnector) getTableSchema(tableName string) ([]*sql.ColumnType LIMIT 0 `, tableName) - rows, err := c.database.Query(queryString) + rows, err := c.database.QueryContext(c.ctx, queryString) if err != nil { return nil, fmt.Errorf("failed to execute query: %w", err) } @@ -249,7 +249,7 @@ func (c *SnowflakeConnector) createExternalStage(stageName string, config *proto } func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig) error { - c.logger.Error("Consolidating partitions") + c.logger.Info("Consolidating partitions") destTable := config.DestinationTableIdentifier stageName := c.getStageNameForJob(config.FlowJobName) @@ -272,29 +272,25 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig // CleanupQRepFlow function for snowflake connector func (c *SnowflakeConnector) CleanupQRepFlow(config *protos.QRepConfig) error { - c.logger.Error("Cleaning up flow job") + c.logger.Info("Cleaning up flow job") return c.dropStage(config.StagingPath, config.FlowJobName) } func (c *SnowflakeConnector) getColsFromTable(tableName string) (*model.ColumnInformation, error) { // parse the table name to get the schema and table name - components, err := parseTableName(tableName) + schemaTable, err := utils.ParseSchemaTable(tableName) if err != nil { return nil, fmt.Errorf("failed to parse table name: %w", err) } - // convert tableIdentifier and schemaIdentifier to upper case - components.tableIdentifier = strings.ToUpper(components.tableIdentifier) - components.schemaIdentifier = strings.ToUpper(components.schemaIdentifier) - //nolint:gosec queryString := fmt.Sprintf(` SELECT column_name, data_type FROM information_schema.columns WHERE UPPER(table_name) = '%s' AND UPPER(table_schema) = '%s' - `, components.tableIdentifier, components.schemaIdentifier) + `, strings.ToUpper(schemaTable.Table), strings.ToUpper(schemaTable.Schema)) - rows, err := c.database.Query(queryString) + rows, err := c.database.QueryContext(c.ctx, queryString) if err != nil { return nil, fmt.Errorf("failed to execute query: %w", err) } diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index d5902adac..30834c255 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -195,7 +195,7 @@ func (s *SnowflakeAvroSyncMethod) addMissingColumns( s.connector.logger.Info(fmt.Sprintf("altering destination table %s with command `%s`", dstTableName, alterTableCmd), partitionLog) - if _, err := tx.Exec(alterTableCmd); err != nil { + if _, err := tx.ExecContext(s.connector.ctx, alterTableCmd); err != nil { return fmt.Errorf("failed to alter destination table: %w", err) } } @@ -288,7 +288,7 @@ func (s *SnowflakeAvroSyncMethod) putFileToStage(avroFile *avro.AvroFile, stage shutdown <- struct{}{} }() - if _, err := s.connector.database.Exec(putCmd); err != nil { + if _, err := s.connector.database.ExecContext(s.connector.ctx, putCmd); err != nil { return fmt.Errorf("failed to put file to stage: %w", err) } @@ -298,16 +298,21 @@ func (s *SnowflakeAvroSyncMethod) putFileToStage(avroFile *avro.AvroFile, stage func (c *SnowflakeConnector) GetCopyTransformation( dstTableName string, + syncedAtCol string, ) (*CopyInfo, error) { colInfo, colsErr := c.getColsFromTable(dstTableName) if colsErr != nil { return nil, fmt.Errorf("failed to get columns from destination table: %w", colsErr) } - var transformations []string - var columnOrder []string + transformations := make([]string, 0, len(colInfo.ColumnMap)) + columnOrder := make([]string, 0, len(colInfo.ColumnMap)) for colName, colType := range colInfo.ColumnMap { columnOrder = append(columnOrder, fmt.Sprintf("\"%s\"", colName)) + if colName == syncedAtCol { + transformations = append(transformations, fmt.Sprintf("CURRENT_TIMESTAMP AS \"%s\"", colName)) + continue + } switch colType { case "GEOGRAPHY": transformations = append(transformations, @@ -352,7 +357,7 @@ func CopyStageToDestination( } } - copyTransformation, err := connector.GetCopyTransformation(dstTableName) + copyTransformation, err := connector.GetCopyTransformation(dstTableName, config.SyncedAtColName) if err != nil { return fmt.Errorf("failed to get copy transformation: %w", err) } @@ -388,7 +393,7 @@ func (s *SnowflakeAvroSyncMethod) insertMetadata( return fmt.Errorf("failed to create metadata insert statement: %v", err) } - if _, err := s.connector.database.Exec(insertMetadataStmt); err != nil { + if _, err := s.connector.database.ExecContext(s.connector.ctx, insertMetadataStmt); err != nil { s.connector.logger.Error("failed to execute metadata insert statement "+insertMetadataStmt, slog.Any("error", err), partitionLog) return fmt.Errorf("failed to execute metadata insert statement: %v", err) @@ -427,7 +432,7 @@ func (s *SnowflakeAvroWriteHandler) HandleAppendMode( copyCmd := fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", s.dstTableName, copyInfo.columnsSQL, copyInfo.transformationSQL, s.stage, strings.Join(s.copyOpts, ",")) s.connector.logger.Info("running copy command: " + copyCmd) - _, err := s.connector.database.Exec(copyCmd) + _, err := s.connector.database.ExecContext(s.connector.ctx, copyCmd) if err != nil { return fmt.Errorf("failed to run COPY INTO command: %w", err) } @@ -511,7 +516,7 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( //nolint:gosec createTempTableCmd := fmt.Sprintf("CREATE TEMPORARY TABLE %s AS SELECT * FROM %s LIMIT 0", tempTableName, s.dstTableName) - if _, err := s.connector.database.Exec(createTempTableCmd); err != nil { + if _, err := s.connector.database.ExecContext(s.connector.ctx, createTempTableCmd); err != nil { return fmt.Errorf("failed to create temp table: %w", err) } s.connector.logger.Info("created temp table " + tempTableName) @@ -519,7 +524,7 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( //nolint:gosec copyCmd := fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", tempTableName, copyInfo.columnsSQL, copyInfo.transformationSQL, s.stage, strings.Join(s.copyOpts, ",")) - _, err = s.connector.database.Exec(copyCmd) + _, err = s.connector.database.ExecContext(s.connector.ctx, copyCmd) if err != nil { return fmt.Errorf("failed to run COPY INTO command: %w", err) } @@ -531,7 +536,7 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( } startTime := time.Now() - rows, err := s.connector.database.Exec(mergeCmd) + rows, err := s.connector.database.ExecContext(s.connector.ctx, mergeCmd) if err != nil { return fmt.Errorf("failed to merge data into destination table '%s': %w", mergeCmd, err) } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index bc59765ab..55dd4444d 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -36,13 +36,13 @@ const ( _PEERDB_TIMESTAMP INT NOT NULL,_PEERDB_DESTINATION_TABLE_NAME STRING NOT NULL,_PEERDB_DATA STRING NOT NULL, _PEERDB_RECORD_TYPE INTEGER NOT NULL, _PEERDB_MATCH_DATA STRING,_PEERDB_BATCH_ID INT, _PEERDB_UNCHANGED_TOAST_COLUMNS STRING)` - createNormalizedTableSQL = "CREATE TABLE IF NOT EXISTS %s(%s)" - toVariantColumnName = "VAR_COLS" - mergeStatementSQL = `MERGE INTO %s TARGET USING (WITH VARIANT_CONVERTED AS (SELECT _PEERDB_UID, - _PEERDB_TIMESTAMP, - TO_VARIANT(PARSE_JSON(_PEERDB_DATA)) %s,_PEERDB_RECORD_TYPE,_PEERDB_MATCH_DATA,_PEERDB_BATCH_ID, - _PEERDB_UNCHANGED_TOAST_COLUMNS FROM - _PEERDB_INTERNAL.%s WHERE _PEERDB_BATCH_ID > %d AND _PEERDB_BATCH_ID <= %d AND + rawTableMultiValueInsertSQL = "INSERT INTO %s.%s VALUES%s" + createNormalizedTableSQL = "CREATE TABLE IF NOT EXISTS %s(%s)" + toVariantColumnName = "VAR_COLS" + mergeStatementSQL = `MERGE INTO %s TARGET USING (WITH VARIANT_CONVERTED AS ( + SELECT _PEERDB_UID,_PEERDB_TIMESTAMP,TO_VARIANT(PARSE_JSON(_PEERDB_DATA)) %s,_PEERDB_RECORD_TYPE, + _PEERDB_MATCH_DATA,_PEERDB_BATCH_ID,_PEERDB_UNCHANGED_TOAST_COLUMNS + FROM _PEERDB_INTERNAL.%s WHERE _PEERDB_BATCH_ID > %d AND _PEERDB_BATCH_ID <= %d AND _PEERDB_DESTINATION_TABLE_NAME = ? ), FLATTENED AS (SELECT _PEERDB_UID,_PEERDB_TIMESTAMP,_PEERDB_RECORD_TYPE,_PEERDB_MATCH_DATA,_PEERDB_BATCH_ID, _PEERDB_UNCHANGED_TOAST_COLUMNS,%s @@ -65,25 +65,22 @@ const ( insertJobMetadataSQL = "INSERT INTO %s.%s VALUES (?,?,?,?)" - updateMetadataForSyncRecordsSQL = "UPDATE %s.%s SET OFFSET=?, SYNC_BATCH_ID=? WHERE MIRROR_JOB_NAME=?" + updateMetadataForSyncRecordsSQL = `UPDATE %s.%s SET OFFSET=GREATEST(OFFSET, ?), SYNC_BATCH_ID=? + WHERE MIRROR_JOB_NAME=?` updateMetadataForNormalizeRecordsSQL = "UPDATE %s.%s SET NORMALIZE_BATCH_ID=? WHERE MIRROR_JOB_NAME=?" checkIfTableExistsSQL = `SELECT TO_BOOLEAN(COUNT(1)) FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA=? and TABLE_NAME=?` - checkIfJobMetadataExistsSQL = "SELECT TO_BOOLEAN(COUNT(1)) FROM %s.%s WHERE MIRROR_JOB_NAME=?" - getLastOffsetSQL = "SELECT OFFSET FROM %s.%s WHERE MIRROR_JOB_NAME=?" - getLastSyncBatchID_SQL = "SELECT SYNC_BATCH_ID FROM %s.%s WHERE MIRROR_JOB_NAME=?" - getLastNormalizeBatchID_SQL = "SELECT NORMALIZE_BATCH_ID FROM %s.%s WHERE MIRROR_JOB_NAME=?" - dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" - deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=?" - checkSchemaExistsSQL = "SELECT TO_BOOLEAN(COUNT(1)) FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME=?" + checkIfJobMetadataExistsSQL = "SELECT TO_BOOLEAN(COUNT(1)) FROM %s.%s WHERE MIRROR_JOB_NAME=?" + getLastOffsetSQL = "SELECT OFFSET FROM %s.%s WHERE MIRROR_JOB_NAME=?" + setLastOffsetSQL = "UPDATE %s.%s SET OFFSET=GREATEST(OFFSET, ?) WHERE MIRROR_JOB_NAME=?" + getLastSyncBatchID_SQL = "SELECT SYNC_BATCH_ID FROM %s.%s WHERE MIRROR_JOB_NAME=?" + getLastSyncNormalizeBatchID_SQL = "SELECT SYNC_BATCH_ID, NORMALIZE_BATCH_ID FROM %s.%s WHERE MIRROR_JOB_NAME=?" + dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" + deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=?" + checkSchemaExistsSQL = "SELECT TO_BOOLEAN(COUNT(1)) FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME=?" ) -type tableNameComponents struct { - schemaIdentifier string - tableIdentifier string -} - type SnowflakeConnector struct { ctx context.Context database *sql.DB @@ -243,12 +240,11 @@ func (c *SnowflakeConnector) GetTableSchema( } func (c *SnowflakeConnector) getTableSchemaForTable(tableName string) (*protos.TableSchema, error) { - tableNameComponents, err := parseTableName(tableName) + schemaTable, err := utils.ParseSchemaTable(tableName) if err != nil { return nil, fmt.Errorf("error while parsing table schema and name: %w", err) } - rows, err := c.database.QueryContext(c.ctx, getTableSchemaSQL, tableNameComponents.schemaIdentifier, - tableNameComponents.tableIdentifier) + rows, err := c.database.QueryContext(c.ctx, getTableSchemaSQL, schemaTable.Schema, schemaTable.Table) if err != nil { return nil, fmt.Errorf("error querying Snowflake peer for schema of table %s: %w", tableName, err) } @@ -300,7 +296,7 @@ func (c *SnowflakeConnector) GetLastOffset(jobName string) (int64, error) { }() if !rows.Next() { - c.logger.Warn("No row found ,returning nil") + c.logger.Warn("No row found, returning 0") return 0, nil } var result pgtype.Int8 @@ -310,10 +306,20 @@ func (c *SnowflakeConnector) GetLastOffset(jobName string) (int64, error) { } if result.Int64 == 0 { c.logger.Warn("Assuming zero offset means no sync has happened") + return 0, nil } return result.Int64, nil } +func (c *SnowflakeConnector) SetLastOffset(jobName string, lastOffset int64) error { + _, err := c.database.ExecContext(c.ctx, fmt.Sprintf(setLastOffsetSQL, + c.metadataSchema, mirrorJobsTableIdentifier), lastOffset, jobName) + if err != nil { + return fmt.Errorf("error querying Snowflake peer for last syncedID: %w", err) + } + return nil +} + func (c *SnowflakeConnector) GetLastSyncBatchID(jobName string) (int64, error) { rows, err := c.database.QueryContext(c.ctx, fmt.Sprintf(getLastSyncBatchID_SQL, c.metadataSchema, mirrorJobsTableIdentifier), jobName) @@ -333,23 +339,27 @@ func (c *SnowflakeConnector) GetLastSyncBatchID(jobName string) (int64, error) { return result.Int64, nil } -func (c *SnowflakeConnector) GetLastNormalizeBatchID(jobName string) (int64, error) { - rows, err := c.database.QueryContext(c.ctx, fmt.Sprintf(getLastNormalizeBatchID_SQL, c.metadataSchema, +func (c *SnowflakeConnector) GetLastSyncAndNormalizeBatchID(jobName string) (model.SyncAndNormalizeBatchID, error) { + rows, err := c.database.QueryContext(c.ctx, fmt.Sprintf(getLastSyncNormalizeBatchID_SQL, c.metadataSchema, mirrorJobsTableIdentifier), jobName) if err != nil { - return 0, fmt.Errorf("error querying Snowflake peer for last normalizeBatchId: %w", err) + return model.SyncAndNormalizeBatchID{}, + fmt.Errorf("error querying Snowflake peer for last normalizeBatchId: %w", err) } - var result pgtype.Int8 + var syncResult, normResult pgtype.Int8 if !rows.Next() { c.logger.Warn("No row found, returning 0") - return 0, nil + return model.SyncAndNormalizeBatchID{}, nil } - err = rows.Scan(&result) + err = rows.Scan(&syncResult, &normResult) if err != nil { - return 0, fmt.Errorf("error while reading result row: %w", err) + return model.SyncAndNormalizeBatchID{}, fmt.Errorf("error while reading result row: %w", err) } - return result.Int64, nil + return model.SyncAndNormalizeBatchID{ + SyncBatchID: syncResult.Int64, + NormalizeBatchID: normResult.Int64, + }, nil } func (c *SnowflakeConnector) getDistinctTableNamesInBatch(flowJobName string, syncBatchID int64, @@ -407,12 +417,11 @@ func (c *SnowflakeConnector) SetupNormalizedTables( ) (*protos.SetupNormalizedTableBatchOutput, error) { tableExistsMapping := make(map[string]bool) for tableIdentifier, tableSchema := range req.TableNameSchemaMapping { - normalizedTableNameComponents, err := parseTableName(tableIdentifier) + normalizedSchemaTable, err := utils.ParseSchemaTable(tableIdentifier) if err != nil { return nil, fmt.Errorf("error while parsing table schema and name: %w", err) } - tableAlreadyExists, err := c.checkIfTableExists(normalizedTableNameComponents.schemaIdentifier, - normalizedTableNameComponents.tableIdentifier) + tableAlreadyExists, err := c.checkIfTableExists(normalizedSchemaTable.Schema, normalizedSchemaTable.Table) if err != nil { return nil, fmt.Errorf("error occurred while checking if normalized table exists: %w", err) } @@ -468,8 +477,9 @@ func (c *SnowflakeConnector) ReplayTableSchemaDeltas(flowJobName string, return fmt.Errorf("failed to convert column type %s to snowflake type: %w", addedColumn.ColumnType, err) } - _, err = tableSchemaModifyTx.Exec(fmt.Sprintf("ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", - schemaDelta.DstTableName, strings.ToUpper(addedColumn.ColumnName), sfColtype)) + _, err = tableSchemaModifyTx.ExecContext(c.ctx, + fmt.Sprintf("ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", + schemaDelta.DstTableName, strings.ToUpper(addedColumn.ColumnName), sfColtype)) if err != nil { return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, schemaDelta.DstTableName, err) @@ -577,20 +587,16 @@ func (c *SnowflakeConnector) syncRecordsViaAvro( // NormalizeRecords normalizes raw table to destination table. func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) (*model.NormalizeResponse, error) { - syncBatchID, err := c.GetLastSyncBatchID(req.FlowJobName) - if err != nil { - return nil, err - } - normalizeBatchID, err := c.GetLastNormalizeBatchID(req.FlowJobName) + batchIDs, err := c.GetLastSyncAndNormalizeBatchID(req.FlowJobName) if err != nil { return nil, err } // normalize has caught up with sync, chill until more records are loaded. - if syncBatchID == normalizeBatchID { + if batchIDs.NormalizeBatchID >= batchIDs.SyncBatchID { return &model.NormalizeResponse{ Done: false, - StartBatchID: normalizeBatchID, - EndBatchID: syncBatchID, + StartBatchID: batchIDs.NormalizeBatchID, + EndBatchID: batchIDs.SyncBatchID, }, nil } @@ -604,12 +610,16 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest Done: false, }, nil } - destinationTableNames, err := c.getDistinctTableNamesInBatch(req.FlowJobName, syncBatchID, normalizeBatchID) + destinationTableNames, err := c.getDistinctTableNamesInBatch( + req.FlowJobName, + batchIDs.SyncBatchID, + batchIDs.NormalizeBatchID, + ) if err != nil { return nil, err } - tableNametoUnchangedToastCols, err := c.getTableNametoUnchangedCols(req.FlowJobName, syncBatchID, normalizeBatchID) + tableNametoUnchangedToastCols, err := c.getTableNametoUnchangedCols(req.FlowJobName, batchIDs.SyncBatchID, batchIDs.NormalizeBatchID) if err != nil { return nil, fmt.Errorf("couldn't tablename to unchanged cols mapping: %w", err) } @@ -627,7 +637,7 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest tableName, tableNametoUnchangedToastCols[tableName], getRawTableIdentifier(req.FlowJobName), - syncBatchID, normalizeBatchID, + batchIDs.SyncBatchID, batchIDs.NormalizeBatchID, req) if err != nil { c.logger.Error("[merge] error while normalizing records", slog.Any("error", err)) @@ -644,15 +654,15 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest } // updating metadata with new normalizeBatchID - err = c.updateNormalizeMetadata(req.FlowJobName, syncBatchID) + err = c.updateNormalizeMetadata(req.FlowJobName, batchIDs.SyncBatchID) if err != nil { return nil, err } return &model.NormalizeResponse{ Done: true, - StartBatchID: normalizeBatchID + 1, - EndBatchID: syncBatchID, + StartBatchID: batchIDs.NormalizeBatchID + 1, + EndBatchID: batchIDs.SyncBatchID, }, nil } @@ -750,7 +760,7 @@ func generateCreateTableSQLForNormalizedTable( softDeleteColName string, syncedAtColName string, ) string { - createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)) + createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)+2) for columnName, genericColumnType := range sourceTableSchema.Columns { columnNameUpper := strings.ToUpper(columnName) sfColType, err := qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)) @@ -811,7 +821,8 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) for columnName, genericColumnType := range normalizedTableSchema.Columns { - sfType, err := qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)) + qvKind := qvalue.QValueKind(genericColumnType) + sfType, err := qValueKindToSnowflakeType(qvKind) if err != nil { return 0, fmt.Errorf("failed to convert column type %s to snowflake type: %w", genericColumnType, err) @@ -836,8 +847,14 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( // "Microseconds*1000) "+ // "AS %s,", toVariantColumnName, columnName, columnName)) default: - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("CAST(%s:\"%s\" AS %s) AS %s,", - toVariantColumnName, columnName, sfType, targetColumnName)) + if qvKind == qvalue.QValueKindNumeric { + flattenedCastsSQLArray = append(flattenedCastsSQLArray, + fmt.Sprintf("TRY_CAST((%s:\"%s\")::text AS %s) AS %s,", + toVariantColumnName, columnName, sfType, targetColumnName)) + } else { + flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("CAST(%s:\"%s\" AS %s) AS %s,", + toVariantColumnName, columnName, sfType, targetColumnName)) + } } } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ""), ",") @@ -846,17 +863,21 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( for _, columnName := range columnNames { quotedUpperColNames = append(quotedUpperColNames, fmt.Sprintf(`"%s"`, strings.ToUpper(columnName))) } + // append synced_at column + quotedUpperColNames = append(quotedUpperColNames, + fmt.Sprintf(`"%s"`, strings.ToUpper(normalizeReq.SyncedAtColName)), + ) insertColumnsSQL := strings.TrimSuffix(strings.Join(quotedUpperColNames, ","), ",") insertValuesSQLArray := make([]string, 0, len(columnNames)) for _, columnName := range columnNames { quotedUpperColumnName := fmt.Sprintf(`"%s"`, strings.ToUpper(columnName)) - insertValuesSQLArray = append(insertValuesSQLArray, fmt.Sprintf("SOURCE.%s,", quotedUpperColumnName)) + insertValuesSQLArray = append(insertValuesSQLArray, fmt.Sprintf("SOURCE.%s", quotedUpperColumnName)) } - - insertValuesSQL := strings.TrimSuffix(strings.Join(insertValuesSQLArray, ""), ",") - + // fill in synced_at column + insertValuesSQLArray = append(insertValuesSQLArray, "CURRENT_TIMESTAMP") + insertValuesSQL := strings.Join(insertValuesSQLArray, ",") updateStatementsforToastCols := c.generateUpdateStatements(normalizeReq.SyncedAtColName, normalizeReq.SoftDeleteColName, normalizeReq.SoftDelete, columnNames, unchangedToastColumns) @@ -865,10 +886,9 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( // with soft-delete, we want the row to be in the destination with SOFT_DELETE true // the current merge statement doesn't do that, so we add another case to insert the DeleteRecord if normalizeReq.SoftDelete { - softDeleteInsertColumnsSQL := strings.TrimSuffix(strings.Join(append(quotedUpperColNames, - normalizeReq.SoftDeleteColName), ","), ",") - softDeleteInsertValuesSQL := strings.Join(append(insertValuesSQLArray, "TRUE"), "") - + softDeleteInsertColumnsSQL := strings.Join(append(quotedUpperColNames, + normalizeReq.SoftDeleteColName), ",") + softDeleteInsertValuesSQL := insertValuesSQL + ",TRUE" updateStatementsforToastCols = append(updateStatementsforToastCols, fmt.Sprintf("WHEN NOT MATCHED AND (SOURCE._PEERDB_RECORD_TYPE = 2) THEN INSERT (%s) VALUES(%s)", softDeleteInsertColumnsSQL, softDeleteInsertValuesSQL)) @@ -913,16 +933,6 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( return result.RowsAffected() } -// parseTableName parses a table name into schema and table name. -func parseTableName(tableName string) (*tableNameComponents, error) { - schemaIdentifier, tableIdentifier, hasDot := strings.Cut(tableName, ".") - if !hasDot || strings.ContainsRune(tableIdentifier, '.') { - return nil, fmt.Errorf("invalid table name: %s", tableName) - } - - return &tableNameComponents{schemaIdentifier, tableIdentifier}, nil -} - func (c *SnowflakeConnector) jobMetadataExists(jobName string) (bool, error) { var result pgtype.Bool err := c.database.QueryRowContext(c.ctx, @@ -1049,6 +1059,10 @@ func (c *SnowflakeConnector) generateUpdateStatements( (SOURCE._PEERDB_RECORD_TYPE != 2) AND _PEERDB_UNCHANGED_TOAST_COLUMNS='%s' THEN UPDATE SET %s `, cols, ssep) updateStmts = append(updateStmts, updateStmt) + + // generates update statements for the case where updates and deletes happen in the same branch + // the backfill has happened from the pull side already, so treat the DeleteRecord as an update + // and then set soft-delete to true. if softDelete && (softDeleteCol != "") { tmpArray = append(tmpArray[:len(tmpArray)-1], fmt.Sprintf(`"%s" = TRUE`, softDeleteCol)) ssep := strings.Join(tmpArray, ", ") diff --git a/flow/connectors/sql/query_executor.go b/flow/connectors/sql/query_executor.go index 3b3bb4664..acc434e3c 100644 --- a/flow/connectors/sql/query_executor.go +++ b/flow/connectors/sql/query_executor.go @@ -104,7 +104,7 @@ func (g *GenericSQLQueryExecutor) RecreateSchema(schemaName string) error { } func (g *GenericSQLQueryExecutor) CreateTable(schema *model.QRecordSchema, schemaName string, tableName string) error { - var fields []string + fields := make([]string, 0, len(schema.Fields)) for _, field := range schema.Fields { dbType, ok := g.qvalueKindToDBType[field.Type] if !ok { diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index 7e194f2d3..90c016b40 100644 --- a/flow/connectors/utils/avro/avro_writer.go +++ b/flow/connectors/utils/avro/avro_writer.go @@ -216,8 +216,9 @@ func (p *peerDBOCFWriter) WriteRecordsToS3(bucketName, key string, s3Creds utils Body: r, }) if err != nil { - slog.Error("failed to upload file: ", slog.Any("error", err)) - return nil, fmt.Errorf("failed to upload file: %w", err) + s3Path := "s3://" + bucketName + "/" + key + slog.Error("failed to upload file: ", slog.Any("error", err), slog.Any("s3_path", s3Path)) + return nil, fmt.Errorf("failed to upload file to path %s: %w", s3Path, err) } slog.Info("file uploaded to" + result.Location) diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go index cdd85535b..f5c8e0507 100644 --- a/flow/connectors/utils/catalog/env.go +++ b/flow/connectors/utils/catalog/env.go @@ -39,10 +39,10 @@ func GetCatalogConnectionPoolFromEnv() (*pgxpool.Pool, error) { func genCatalogConnectionString() string { return utils.GetPGConnectionString(&protos.PostgresConfig{ - Host: peerdbenv.GetPeerDBCatalogHost(), - Port: peerdbenv.GetPeerDBCatalogPort(), - User: peerdbenv.GetPeerDBCatalogUser(), - Password: peerdbenv.GetPeerDBCatalogPassword(), - Database: peerdbenv.GetPeerDBCatalogDatabase(), + Host: peerdbenv.PeerDBCatalogHost(), + Port: peerdbenv.PeerDBCatalogPort(), + User: peerdbenv.PeerDBCatalogUser(), + Password: peerdbenv.PeerDBCatalogPassword(), + Database: peerdbenv.PeerDBCatalogDatabase(), }) } diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage.go b/flow/connectors/utils/cdc_records/cdc_records_storage.go index 3147045a7..66722b69e 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage.go @@ -43,7 +43,7 @@ func NewCDCRecordsStore(flowJobName string) *cdcRecordsStore { numRecords: 0, flowJobName: flowJobName, dbFolderName: fmt.Sprintf("%s/%s_%s", os.TempDir(), flowJobName, shared.RandomString(8)), - numRecordsSwitchThreshold: peerdbenv.GetPeerDBCDCDiskSpillThreshold(), + numRecordsSwitchThreshold: peerdbenv.PeerDBCDCDiskSpillThreshold(), } } diff --git a/flow/connectors/utils/identifiers.go b/flow/connectors/utils/identifiers.go index 0b91b9e4f..2ae919488 100644 --- a/flow/connectors/utils/identifiers.go +++ b/flow/connectors/utils/identifiers.go @@ -1,7 +1,30 @@ package utils -import "fmt" +import ( + "fmt" + "strings" +) func QuoteIdentifier(identifier string) string { return fmt.Sprintf(`"%s"`, identifier) } + +// SchemaTable is a table in a schema. +type SchemaTable struct { + Schema string + Table string +} + +func (t *SchemaTable) String() string { + return fmt.Sprintf(`"%s"."%s"`, t.Schema, t.Table) +} + +// ParseSchemaTable parses a table name into schema and table name. +func ParseSchemaTable(tableName string) (*SchemaTable, error) { + schema, table, hasDot := strings.Cut(tableName, ".") + if !hasDot || strings.ContainsRune(table, '.') { + return nil, fmt.Errorf("invalid table name: %s", tableName) + } + + return &SchemaTable{schema, table}, nil +} diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 781537227..7e9263cf2 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -204,7 +204,7 @@ func AppendSlotSizeInfo( _, 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;", + "VALUES($1,$2,$3,$4,$5,$6,$7) ON CONFLICT DO NOTHING;", peerName, slotInfo.SlotName, slotInfo.RestartLSN, diff --git a/flow/connectors/utils/postgres.go b/flow/connectors/utils/postgres.go index 72aaf86f5..58cd02f20 100644 --- a/flow/connectors/utils/postgres.go +++ b/flow/connectors/utils/postgres.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "net/url" - "strings" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/jackc/pgx/v5/pgtype" @@ -49,26 +48,3 @@ func GetCustomDataTypes(ctx context.Context, pool *pgxpool.Pool) (map[uint32]str } return customTypeMap, nil } - -// SchemaTable is a table in a schema. -type SchemaTable struct { - Schema string - Table string -} - -func (t *SchemaTable) String() string { - return fmt.Sprintf(`"%s"."%s"`, t.Schema, t.Table) -} - -// ParseSchemaTable parses a table name into schema and table name. -func ParseSchemaTable(tableName string) (*SchemaTable, error) { - parts := strings.Split(tableName, ".") - if len(parts) != 2 { - return nil, fmt.Errorf("invalid table name: %s", tableName) - } - - return &SchemaTable{ - Schema: parts[0], - Table: parts[1], - }, nil -} diff --git a/flow/connectors/utils/ssh.go b/flow/connectors/utils/ssh.go index 7bd8ed141..511eea672 100644 --- a/flow/connectors/utils/ssh.go +++ b/flow/connectors/utils/ssh.go @@ -41,8 +41,9 @@ func GetSSHClientConfig(user, password, privateKeyString string) (*ssh.ClientCon } return &ssh.ClientConfig{ - User: user, - Auth: authMethods, + User: user, + Auth: authMethods, + //nolint:gosec HostKeyCallback: ssh.InsecureIgnoreHostKey(), }, nil } diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index 7cd8ae839..bbfe0a2fa 100644 --- a/flow/connectors/utils/stream.go +++ b/flow/connectors/utils/stream.go @@ -10,7 +10,7 @@ import ( ) func RecordsToRawTableStream(req *model.RecordsToStreamRequest) (*model.RecordsToStreamResponse, error) { - recordStream := model.NewQRecordStream(1 << 16) + recordStream := model.NewQRecordStream(1 << 17) err := recordStream.SetSchema(&model.QRecordSchema{ Fields: []model.QField{ { @@ -85,11 +85,6 @@ func recordToQRecordOrError(tableMapping map[string]uint32, batchID int64, recor } } - // add insert record to the raw table - entries[2] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: typedRecord.DestinationTableName, - } entries[3] = qvalue.QValue{ Kind: qvalue.QValueKindString, Value: itemsJSON, @@ -121,10 +116,6 @@ func recordToQRecordOrError(tableMapping map[string]uint32, batchID int64, recor } } - entries[2] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: typedRecord.DestinationTableName, - } entries[3] = qvalue.QValue{ Kind: qvalue.QValueKindString, Value: newItemsJSON, @@ -150,11 +141,6 @@ func recordToQRecordOrError(tableMapping map[string]uint32, batchID int64, recor } } - // append delete record to the raw table - entries[2] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: typedRecord.DestinationTableName, - } entries[3] = qvalue.QValue{ Kind: qvalue.QValueKindString, Value: itemsJSON, @@ -186,6 +172,10 @@ func recordToQRecordOrError(tableMapping map[string]uint32, batchID int64, recor Kind: qvalue.QValueKindInt64, Value: time.Now().UnixNano(), } + entries[2] = qvalue.QValue{ + Kind: qvalue.QValueKindString, + Value: record.GetDestinationTableName(), + } entries[6] = qvalue.QValue{ Kind: qvalue.QValueKindInt64, Value: batchID, diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 54b3a180c..40a42ea64 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -94,12 +94,11 @@ func generateBQPeer(bigQueryConfig *protos.BigqueryConfig) *protos.Peer { } // datasetExists checks if the dataset exists. -func (b *BigQueryTestHelper) datasetExists() (bool, error) { - dataset := b.client.Dataset(b.Config.DatasetId) +func (b *BigQueryTestHelper) datasetExists(datasetName string) (bool, error) { + dataset := b.client.Dataset(datasetName) meta, err := dataset.Metadata(context.Background()) if err != nil { // if err message contains `notFound` then dataset does not exist. - // first we cast the error to a bigquery.Error if strings.Contains(err.Error(), "notFound") { fmt.Printf("dataset %s does not exist\n", b.Config.DatasetId) return false, nil @@ -117,12 +116,12 @@ func (b *BigQueryTestHelper) datasetExists() (bool, error) { // RecreateDataset recreates the dataset, i.e, deletes it if exists and creates it again. func (b *BigQueryTestHelper) RecreateDataset() error { - exists, err := b.datasetExists() + exists, err := b.datasetExists(b.datasetName) if err != nil { return fmt.Errorf("failed to check if dataset %s exists: %w", b.Config.DatasetId, err) } - dataset := b.client.Dataset(b.Config.DatasetId) + dataset := b.client.Dataset(b.datasetName) if exists { err := dataset.DeleteWithContents(context.Background()) if err != nil { @@ -135,13 +134,13 @@ func (b *BigQueryTestHelper) RecreateDataset() error { return fmt.Errorf("failed to create dataset: %w", err) } - fmt.Printf("created dataset %s successfully\n", b.Config.DatasetId) + fmt.Printf("created dataset %s successfully\n", b.datasetName) return nil } // DropDataset drops the dataset. -func (b *BigQueryTestHelper) DropDataset() error { - exists, err := b.datasetExists() +func (b *BigQueryTestHelper) DropDataset(datasetName string) error { + exists, err := b.datasetExists(datasetName) if err != nil { return fmt.Errorf("failed to check if dataset %s exists: %w", b.Config.DatasetId, err) } @@ -150,7 +149,7 @@ func (b *BigQueryTestHelper) DropDataset() error { return nil } - dataset := b.client.Dataset(b.Config.DatasetId) + dataset := b.client.Dataset(datasetName) err = dataset.DeleteWithContents(context.Background()) if err != nil { return fmt.Errorf("failed to delete dataset: %w", err) @@ -171,7 +170,11 @@ func (b *BigQueryTestHelper) RunCommand(command string) error { // countRows(tableName) returns the number of rows in the given table. func (b *BigQueryTestHelper) countRows(tableName string) (int, error) { - command := fmt.Sprintf("SELECT COUNT(*) FROM `%s.%s`", b.Config.DatasetId, tableName) + return b.countRowsWithDataset(b.datasetName, tableName) +} + +func (b *BigQueryTestHelper) countRowsWithDataset(dataset, tableName string) (int, error) { + command := fmt.Sprintf("SELECT COUNT(*) FROM `%s.%s`", dataset, tableName) it, err := b.client.Query(command).Read(context.Background()) if err != nil { return 0, fmt.Errorf("failed to run command: %w", err) @@ -285,7 +288,7 @@ func bqFieldSchemaToQField(fieldSchema *bigquery.FieldSchema) (model.QField, err // bqSchemaToQRecordSchema converts a bigquery schema to a QRecordSchema. func bqSchemaToQRecordSchema(schema bigquery.Schema) (*model.QRecordSchema, error) { - var fields []model.QField + fields := make([]model.QField, 0, len(schema)) for _, fieldSchema := range schema { qField, err := bqFieldSchemaToQField(fieldSchema) if err != nil { @@ -433,7 +436,7 @@ func qValueKindToBqColTypeString(val qvalue.QValueKind) (string, error) { } func (b *BigQueryTestHelper) CreateTable(tableName string, schema *model.QRecordSchema) error { - var fields []string + fields := make([]string, 0, len(schema.Fields)) for _, field := range schema.Fields { bqType, err := qValueKindToBqColTypeString(field.Type) if err != nil { @@ -452,3 +455,15 @@ func (b *BigQueryTestHelper) CreateTable(tableName string, schema *model.QRecord return nil } + +func (b *BigQueryTestHelper) RunInt64Query(query string) (int64, error) { + recordBatch, err := b.ExecuteAndProcessQuery(query) + if err != nil { + return 0, fmt.Errorf("could not execute query: %w", err) + } + if recordBatch.NumRecords != 1 { + return 0, fmt.Errorf("expected only 1 record, got %d", recordBatch.NumRecords) + } + + return recordBatch.Records[0].Entries[0].Value.(int64), nil +} diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index de3ddae7e..6cbac4c91 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -5,10 +5,13 @@ import ( "fmt" "log/slog" "strings" + "sync" "testing" "time" "github.com/PeerDB-io/peer-flow/e2e" + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/PeerDB-io/peer-flow/shared" peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/jackc/pgx/v5/pgxpool" @@ -51,6 +54,50 @@ func (s PeerFlowE2ETestSuiteBQ) attachSuffix(input string) string { return fmt.Sprintf("%s_%s", input, s.bqSuffix) } +func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, softDelete bool) error { + qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, dstQualified) + selector := "`_PEERDB_SYNCED_AT`" + if softDelete { + selector += ", `_PEERDB_IS_DELETED`" + } + query := fmt.Sprintf("SELECT %s FROM %s", + selector, qualifiedTableName) + + recordBatch, err := s.bqHelper.ExecuteAndProcessQuery(query) + if err != nil { + return err + } + + recordCount := 0 + + for _, record := range recordBatch.Records { + for _, entry := range record.Entries { + if entry.Kind == qvalue.QValueKindBoolean { + isDeleteVal, ok := entry.Value.(bool) + if !(ok && isDeleteVal) { + return fmt.Errorf("peerdb column failed: _PEERDB_IS_DELETED is not true") + } + recordCount += 1 + } + + if entry.Kind == qvalue.QValueKindTimestamp { + _, ok := entry.Value.(time.Time) + if !ok { + return fmt.Errorf("peerdb column failed: _PEERDB_SYNCED_AT is not valid") + } + + recordCount += 1 + } + } + } + + if recordCount == 0 { + return fmt.Errorf("peerdb column check failed: no records found") + } + + return nil +} + // setupBigQuery sets up the bigquery connection. func setupBigQuery(t *testing.T) *BigQueryTestHelper { bqHelper, err := NewBigQueryTestHelper() @@ -105,7 +152,7 @@ func (s PeerFlowE2ETestSuiteBQ) tearDownSuite() { s.FailNow() } - err = s.bqHelper.DropDataset() + err = s.bqHelper.DropDataset(s.bqHelper.datasetName) if err != nil { slog.Error("failed to tear down bigquery", slog.Any("error", err)) s.FailNow() @@ -1095,3 +1142,470 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { env.AssertExpectations(s.t) } + +func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + srcTableName := s.attachSchemaSuffix("test_peerdb_cols") + dstTableName := "test_peerdb_cols_dst" + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_peerdb_cols_mirror"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + SoftDelete: true, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + require.NoError(s.t, err) + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 2, + MaxBatchSize: 100, + } + + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + // insert 1 row into the source table + testKey := fmt.Sprintf("test_key_%d", 1) + testValue := fmt.Sprintf("test_value_%d", 1) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + require.NoError(s.t, err) + + // delete that row + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id=1 + `, srcTableName)) + require.NoError(s.t, err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + require.Contains(s.t, err.Error(), "continue as new") + + err = s.checkPeerdbColumns(dstTableName, true) + require.NoError(s.t, err) + + env.AssertExpectations(s.t) +} + +func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_Multi_Dataset_BQ() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + srcTable1Name := s.attachSchemaSuffix("test1_bq") + dstTable1Name := "test1_bq" + secondDataset := fmt.Sprintf("%s_2", s.bqHelper.datasetName) + srcTable2Name := s.attachSchemaSuffix("test2_bq") + dstTable2Name := "test2_bq" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s(id serial primary key, c1 int, c2 text); + CREATE TABLE %s(id serial primary key, c1 int, c2 text); + `, srcTable1Name, srcTable2Name)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_multi_table_multi_dataset_bq"), + TableNameMapping: map[string]string{ + srcTable1Name: dstTable1Name, + srcTable2Name: fmt.Sprintf("%s.%s", secondDataset, dstTable2Name), + }, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + CdcStagingPath: "", + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + require.NoError(s.t, err) + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and execute a transaction touching toast columns + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + /* inserting across multiple tables*/ + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (c1,c2) VALUES (1,'dummy_1'); + INSERT INTO %s (c1,c2) VALUES (-1,'dummy_-1'); + `, srcTable1Name, srcTable2Name)) + require.NoError(s.t, err) + fmt.Println("Executed an insert on two tables") + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + require.True(s.t, env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + count1, err := s.bqHelper.countRows(dstTable1Name) + require.NoError(s.t, err) + count2, err := s.bqHelper.countRowsWithDataset(secondDataset, dstTable2Name) + require.NoError(s.t, err) + + s.Equal(1, count1) + s.Equal(1, count2) + + err = s.bqHelper.DropDataset(secondDataset) + require.NoError(s.t, err) + + env.AssertExpectations(s.t) +} + +func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Basic() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + cmpTableName := s.attachSchemaSuffix("test_softdel") + srcTableName := fmt.Sprintf("%s_src", cmpTableName) + dstTableName := "test_softdel" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 INT, + c2 INT, + t TEXT + ); + `, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_softdel"), + } + + config := &protos.FlowConnectionConfigs{ + FlowJobName: connectionGen.FlowJobName, + Destination: s.bqHelper.Peer, + TableMappings: []*protos.TableMapping{ + { + SourceTableIdentifier: srcTableName, + DestinationTableIdentifier: dstTableName, + }, + }, + Source: e2e.GeneratePostgresPeer(e2e.PostgresPort), + CdcStagingPath: connectionGen.CdcStagingPath, + SoftDelete: true, + SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", + } + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 3, + MaxBatchSize: 100, + } + + wg := sync.WaitGroup{} + wg.Add(1) + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) + require.NoError(s.t, err) + e2e.NormalizeFlowCountQuery(env, connectionGen, 1) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + // since we delete stuff, create another table to compare with + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) + require.NoError(s.t, err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + + wg.Done() + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, config, &limits, nil) + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + require.Contains(s.t, err.Error(), "continue as new") + + wg.Wait() + + // verify our updates and delete happened + s.compareTableContentsBQ("test_softdel", "id,c1,c2,t") + + newerSyncedAtQuery := fmt.Sprintf(` + SELECT COUNT(*) FROM`+"`%s.%s`"+`WHERE _PEERDB_IS_DELETED = TRUE`, + s.bqHelper.datasetName, dstTableName) + numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) + require.NoError(s.t, err) + s.Eq(1, numNewRows) +} + +func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_IUD_Same_Batch() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + cmpTableName := s.attachSchemaSuffix("test_softdel_iud") + srcTableName := fmt.Sprintf("%s_src", cmpTableName) + dstTableName := "test_softdel_iud" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 INT, + c2 INT, + t TEXT + ); + `, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_softdel_iud"), + } + + config := &protos.FlowConnectionConfigs{ + FlowJobName: connectionGen.FlowJobName, + Destination: s.bqHelper.Peer, + TableMappings: []*protos.TableMapping{ + { + SourceTableIdentifier: srcTableName, + DestinationTableIdentifier: dstTableName, + }, + }, + Source: e2e.GeneratePostgresPeer(e2e.PostgresPort), + CdcStagingPath: connectionGen.CdcStagingPath, + SoftDelete: true, + SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", + } + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 3, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + insertTx, err := s.pool.Begin(context.Background()) + require.NoError(s.t, err) + + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) + require.NoError(s.t, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + // since we delete stuff, create another table to compare with + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) + require.NoError(s.t, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + + require.NoError(s.t, insertTx.Commit(context.Background())) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, config, &limits, nil) + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + require.Contains(s.t, err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsBQ("test_softdel_iud", "id,c1,c2,t") + + newerSyncedAtQuery := fmt.Sprintf(` + SELECT COUNT(*) FROM`+"`%s.%s`"+`WHERE _PEERDB_IS_DELETED = TRUE`, + s.bqHelper.datasetName, dstTableName) + numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) + require.NoError(s.t, err) + s.Eq(1, numNewRows) +} + +func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_UD_Same_Batch() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + cmpTableName := s.attachSchemaSuffix("test_softdel_ud") + srcTableName := fmt.Sprintf("%s_src", cmpTableName) + dstTableName := "test_softdel_ud" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 INT, + c2 INT, + t TEXT + ); + `, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_softdel_ud"), + } + + config := &protos.FlowConnectionConfigs{ + FlowJobName: connectionGen.FlowJobName, + Destination: s.bqHelper.Peer, + TableMappings: []*protos.TableMapping{ + { + SourceTableIdentifier: srcTableName, + DestinationTableIdentifier: dstTableName, + }, + }, + Source: e2e.GeneratePostgresPeer(e2e.PostgresPort), + CdcStagingPath: connectionGen.CdcStagingPath, + SoftDelete: true, + SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", + } + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 4, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) + require.NoError(s.t, err) + e2e.NormalizeFlowCountQuery(env, connectionGen, 1) + + insertTx, err := s.pool.Begin(context.Background()) + require.NoError(s.t, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET t=random_string(10000) WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + UPDATE %s SET c1=c1+4 WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + // since we delete stuff, create another table to compare with + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE %s AS SELECT * FROM %s`, cmpTableName, srcTableName)) + require.NoError(s.t, err) + _, err = insertTx.Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + + require.NoError(s.t, insertTx.Commit(context.Background())) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, config, &limits, nil) + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + require.Contains(s.t, err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsBQ("test_softdel_ud", "id,c1,c2,t") + + newerSyncedAtQuery := fmt.Sprintf(` + SELECT COUNT(*) FROM`+"`%s.%s`"+`WHERE _PEERDB_IS_DELETED = TRUE`, + s.bqHelper.datasetName, dstTableName) + numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) + require.NoError(s.t, err) + s.Eq(1, numNewRows) +} + +func (s PeerFlowE2ETestSuiteBQ) Test_Soft_Delete_Insert_After_Delete() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + srcTableName := s.attachSchemaSuffix("test_softdel_iad") + dstTableName := "test_softdel_iad" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, + c1 INT, + c2 INT, + t TEXT + ); + `, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_softdel_iad"), + } + + config := &protos.FlowConnectionConfigs{ + FlowJobName: connectionGen.FlowJobName, + Destination: s.bqHelper.Peer, + TableMappings: []*protos.TableMapping{ + { + SourceTableIdentifier: srcTableName, + DestinationTableIdentifier: dstTableName, + }, + }, + Source: e2e.GeneratePostgresPeer(e2e.PostgresPort), + CdcStagingPath: connectionGen.CdcStagingPath, + SoftDelete: true, + SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", + } + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 3, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2,t) VALUES (1,2,random_string(9000))`, srcTableName)) + require.NoError(s.t, err) + e2e.NormalizeFlowCountQuery(env, connectionGen, 1) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id=1`, srcTableName)) + require.NoError(s.t, err) + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(id,c1,c2,t) VALUES (1,3,4,random_string(10000))`, srcTableName)) + require.NoError(s.t, err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, config, &limits, nil) + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + require.Contains(s.t, err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsBQ("test_softdel_iad", "id,c1,c2,t") + + newerSyncedAtQuery := fmt.Sprintf(` + SELECT COUNT(*) FROM`+"`%s.%s`"+`WHERE _PEERDB_IS_DELETED = TRUE`, + s.bqHelper.datasetName, dstTableName) + numNewRows, err := s.bqHelper.RunInt64Query(newerSyncedAtQuery) + require.NoError(s.t, err) + s.Eq(0, numNewRows) +} diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index f520014b0..ca74a412c 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -10,7 +10,7 @@ import ( ) func (s PeerFlowE2ETestSuiteBQ) setupSourceTable(tableName string, rowCount int) { - err := e2e.CreateSourceTableQRep(s.pool, s.bqSuffix, tableName) + err := e2e.CreateTableForQRep(s.pool, s.bqSuffix, tableName) require.NoError(s.t, err) err = e2e.PopulateSourceTable(s.pool, s.bqSuffix, tableName, rowCount) require.NoError(s.t, err) @@ -64,6 +64,8 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { tblName, query, s.bqHelper.Peer, + "", + false, "") require.NoError(s.t, err) e2e.RunQrepFlowWorkflow(env, qrepConfig) @@ -78,3 +80,38 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Complete_QRep_Flow_Avro() { env.AssertExpectations(s.t) } + +func (s PeerFlowE2ETestSuiteBQ) Test_PeerDB_Columns_QRep_BQ() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + numRows := 10 + + tblName := "test_columns_bq_qrep" + s.setupSourceTable(tblName, numRows) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + s.bqSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig("test_qrep_flow_avro", + fmt.Sprintf("e2e_test_%s.%s", s.bqSuffix, tblName), + tblName, + query, + s.bqHelper.Peer, + "", + true, + "_PEERDB_SYNCED_AT") + require.NoError(s.t, err) + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + err = env.GetWorkflowError() + require.NoError(s.t, err) + + err = s.checkPeerdbColumns(tblName, false) + require.NoError(s.t, err) + + env.AssertExpectations(s.t) +} diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index ac28879f4..df1ff17c1 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -115,7 +115,7 @@ func SetupPostgres(suffix string) (*pgxpool.Pool, error) { $$ language sql; CREATE OR REPLACE FUNCTION random_bytea(bytea_length integer) RETURNS bytea AS $body$ - SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0') ,''), 'hex') + SELECT decode(string_agg(lpad(to_hex(width_bucket(random(), 0, 1, 256)-1),2,'0'), ''), 'hex') FROM generate_series(1, $1); $body$ LANGUAGE 'sql' @@ -171,6 +171,7 @@ type FlowConnectionGenerationConfig struct { PostgresPort int Destination *protos.Peer CdcStagingPath string + SoftDelete bool } // GenerateSnowflakePeer generates a snowflake peer config for testing. @@ -201,7 +202,10 @@ func (c *FlowConnectionGenerationConfig) GenerateFlowConnectionConfigs() (*proto ret.Source = GeneratePostgresPeer(c.PostgresPort) ret.Destination = c.Destination ret.CdcStagingPath = c.CdcStagingPath - ret.SoftDeleteColName = "_PEERDB_IS_DELETED" + ret.SoftDelete = c.SoftDelete + if ret.SoftDelete { + ret.SoftDeleteColName = "_PEERDB_IS_DELETED" + } ret.SyncedAtColName = "_PEERDB_SYNCED_AT" return ret, nil } diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index 2720891fb..da050ccf6 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -8,6 +8,7 @@ import ( "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" peerflow "github.com/PeerDB-io/peer-flow/workflows" + "github.com/jackc/pgx/v5/pgtype" ) func (s *PeerFlowE2ETestSuitePG) attachSchemaSuffix(tableName string) string { @@ -18,6 +19,27 @@ func (s *PeerFlowE2ETestSuitePG) attachSuffix(input string) string { return fmt.Sprintf("%s_%s", input, postgresSuffix) } +func (s *PeerFlowE2ETestSuitePG) checkPeerdbColumns(dstSchemaQualified string, rowID int8) error { + query := fmt.Sprintf(`SELECT "_PEERDB_IS_DELETED","_PEERDB_SYNCED_AT" FROM %s WHERE id = %d`, + dstSchemaQualified, rowID) + var isDeleted pgtype.Bool + var syncedAt pgtype.Timestamp + err := s.pool.QueryRow(context.Background(), query).Scan(&isDeleted, &syncedAt) + if err != nil { + return fmt.Errorf("failed to query row: %w", err) + } + + if !isDeleted.Bool { + return fmt.Errorf("isDeleted is not true") + } + + if !syncedAt.Valid { + return fmt.Errorf("syncedAt is not valid") + } + + return nil +} + func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { env := s.NewTestWorkflowEnvironment() e2e.RegisterWorkflowsAndActivities(env, s.T()) @@ -474,3 +496,67 @@ func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuitePG) Test_PeerDB_Columns() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.T()) + + srcTableName := s.attachSchemaSuffix("test_peerdb_cols") + dstTableName := s.attachSchemaSuffix("test_peerdb_cols_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_peerdb_cols_mirror"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + SoftDelete: true, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 2, + MaxBatchSize: 100, + } + + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + // insert 1 row into the source table + testKey := fmt.Sprintf("test_key_%d", 1) + testValue := fmt.Sprintf("test_value_%d", 1) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + + // delete that row + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + DELETE FROM %s WHERE id=1 + `, srcTableName)) + s.NoError(err) + fmt.Println("Inserted and deleted a row for peerdb column check") + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + err = env.GetWorkflowError() + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + checkErr := s.checkPeerdbColumns(dstTableName, 1) + s.NoError(checkErr) + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 192863e39..0bb886f9a 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -10,6 +10,7 @@ import ( connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/e2e" "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "github.com/joho/godotenv" "github.com/stretchr/testify/suite" @@ -54,7 +55,7 @@ func (s *PeerFlowE2ETestSuitePG) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, false) s.NoError(err) } @@ -67,7 +68,7 @@ func (s *PeerFlowE2ETestSuitePG) TearDownSuite() { } func (s *PeerFlowE2ETestSuitePG) setupSourceTable(tableName string, rowCount int) { - err := e2e.CreateSourceTableQRep(s.pool, postgresSuffix, tableName) + err := e2e.CreateTableForQRep(s.pool, postgresSuffix, tableName) s.NoError(err) err = e2e.PopulateSourceTable(s.pool, postgresSuffix, tableName, rowCount) s.NoError(err) @@ -134,6 +135,27 @@ func (s *PeerFlowE2ETestSuitePG) compareQuery(srcSchemaQualified, dstSchemaQuali return nil } +func (s *PeerFlowE2ETestSuitePG) checkSyncedAt(dstSchemaQualified string) error { + query := fmt.Sprintf(`SELECT "_PEERDB_SYNCED_AT" FROM %s`, dstSchemaQualified) + + rows, _ := s.pool.Query(context.Background(), query) + + defer rows.Close() + for rows.Next() { + var syncedAt pgtype.Timestamp + err := rows.Scan(&syncedAt) + if err != nil { + return err + } + + if !syncedAt.Valid { + return fmt.Errorf("synced_at is not valid") + } + } + + return rows.Err() +} + func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { env := s.NewTestWorkflowEnvironment() e2e.RegisterWorkflowsAndActivities(env, s.T()) @@ -146,8 +168,8 @@ func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { //nolint:gosec dstTable := "test_qrep_flow_avro_pg_2" - // the name is misleading, but this is the destination table - err := e2e.CreateSourceTableQRep(s.pool, postgresSuffix, dstTable) + + err := e2e.CreateTableForQRep(s.pool, postgresSuffix, dstTable) s.NoError(err) srcSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, srcTable) @@ -165,6 +187,8 @@ func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { query, postgresPeer, "", + true, + "", ) s.NoError(err) @@ -183,3 +207,52 @@ func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuitePG) Test_Setup_Destination_And_PeerDB_Columns_QRep_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.T()) + + numRows := 10 + + //nolint:gosec + srcTable := "test_qrep_columns_pg_1" + s.setupSourceTable(srcTable, numRows) + + //nolint:gosec + dstTable := "test_qrep_columns_pg_2" + + srcSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, srcTable) + dstSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, dstTable) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + postgresSuffix, srcTable) + + postgresPeer := e2e.GeneratePostgresPeer(e2e.PostgresPort) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_qrep_columns_pg", + srcSchemaQualified, + dstSchemaQualified, + query, + postgresPeer, + "", + true, + "_PEERDB_SYNCED_AT", + ) + s.NoError(err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + err = env.GetWorkflowError() + s.NoError(err) + + err = s.checkSyncedAt(dstSchemaQualified) + if err != nil { + s.FailNow(err.Error()) + } + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index bfa19f866..a938f673b 100644 --- a/flow/e2e/s3/cdc_s3_test.go +++ b/flow/e2e/s3/cdc_s3_test.go @@ -22,6 +22,11 @@ func (s *PeerFlowE2ETestSuiteS3) Test_Complete_Simple_Flow_S3() { env := s.NewTestWorkflowEnvironment() e2e.RegisterWorkflowsAndActivities(env, s.T()) + setupErr := s.setupS3("s3") + if setupErr != nil { + s.Fail("failed to setup S3", setupErr) + } + srcTableName := s.attachSchemaSuffix("test_simple_flow_s3") dstTableName := fmt.Sprintf("%s.%s", "peerdb_test_s3", "test_simple_flow_s3") flowJobName := s.attachSuffix("test_simple_flow_s3") diff --git a/flow/e2e/s3/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 62523e1ad..fda57ced0 100644 --- a/flow/e2e/s3/qrep_flow_s3_test.go +++ b/flow/e2e/s3/qrep_flow_s3_test.go @@ -30,7 +30,7 @@ func TestPeerFlowE2ETestSuiteS3(t *testing.T) { } func (s *PeerFlowE2ETestSuiteS3) setupSourceTable(tableName string, rowCount int) { - err := e2e.CreateSourceTableQRep(s.pool, s3Suffix, tableName) + err := e2e.CreateTableForQRep(s.pool, s3Suffix, tableName) s.NoError(err) err = e2e.PopulateSourceTable(s.pool, s3Suffix, tableName, rowCount) s.NoError(err) @@ -106,6 +106,8 @@ func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3() { query, s.s3Helper.GetPeer(), "stage", + false, + "", ) s.NoError(err) qrepConfig.StagingPath = s.s3Helper.s3Config.Url @@ -152,6 +154,8 @@ func (s *PeerFlowE2ETestSuiteS3) Test_Complete_QRep_Flow_S3_CTID() { query, s.s3Helper.GetPeer(), "stage", + false, + "", ) s.NoError(err) qrepConfig.StagingPath = s.s3Helper.s3Config.Url diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 3e6f0c2bc..8d521dbb7 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -198,6 +198,72 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_Simple_Flow_SF() { env.AssertExpectations(s.t) } +func (s PeerFlowE2ETestSuiteSF) Test_Flow_ReplicaIdentity_Index_No_Pkey() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + srcTableName := s.attachSchemaSuffix("test_replica_identity_no_pkey") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_replica_identity_no_pkey") + + // Create a table without a primary key and create a named unique index + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + CREATE UNIQUE INDEX unique_idx_on_id_key ON %s (id, key); + ALTER TABLE %s REPLICA IDENTITY USING INDEX unique_idx_on_id_key; + `, srcTableName, srcTableName, srcTableName)) + require.NoError(s.t, err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + require.NoError(s.t, err) + + limits := peerflow.CDCFlowLimits{ + ExitAfterRecords: 20, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 20 rows into the source table + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + // insert 20 rows into the source table + for i := 0; i < 20; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s (id, key, value) VALUES ($1, $2, $3) + `, srcTableName), i, testKey, testValue) + require.NoError(s.t, err) + } + fmt.Println("Inserted 20 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + require.Contains(s.t, err.Error(), "continue as new") + + count, err := s.sfHelper.CountRows("test_replica_identity_no_pkey") + require.NoError(s.t, err) + s.Equal(20, count) + + env.AssertExpectations(s.t) +} + func (s PeerFlowE2ETestSuiteSF) Test_Invalid_Geo_SF_Avro_CDC() { env := e2e.NewTemporalTestWorkflowEnvironment() e2e.RegisterWorkflowsAndActivities(env, s.t) @@ -1176,8 +1242,9 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { Exclude: []string{"c2"}, }, }, - Source: e2e.GeneratePostgresPeer(e2e.PostgresPort), - CdcStagingPath: connectionGen.CdcStagingPath, + Source: e2e.GeneratePostgresPeer(e2e.PostgresPort), + CdcStagingPath: connectionGen.CdcStagingPath, + SyncedAtColName: "_PEERDB_SYNCED_AT", } limits := peerflow.CDCFlowLimits{ @@ -1221,7 +1288,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Column_Exclusion() { for _, field := range sfRows.Schema.Fields { require.NotEqual(s.t, field.Name, "c2") } - s.Equal(4, len(sfRows.Schema.Fields)) + s.Equal(5, len(sfRows.Schema.Fields)) s.Equal(10, len(sfRows.Records)) } @@ -1260,6 +1327,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Basic() { CdcStagingPath: connectionGen.CdcStagingPath, SoftDelete: true, SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", } limits := peerflow.CDCFlowLimits{ @@ -1346,6 +1414,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_IUD_Same_Batch() { CdcStagingPath: connectionGen.CdcStagingPath, SoftDelete: true, SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", } limits := peerflow.CDCFlowLimits{ @@ -1428,6 +1497,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_UD_Same_Batch() { CdcStagingPath: connectionGen.CdcStagingPath, SoftDelete: true, SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", } limits := peerflow.CDCFlowLimits{ @@ -1513,6 +1583,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Soft_Delete_Insert_After_Delete() { CdcStagingPath: connectionGen.CdcStagingPath, SoftDelete: true, SoftDeleteColName: "_PEERDB_IS_DELETED", + SyncedAtColName: "_PEERDB_SYNCED_AT", } limits := peerflow.CDCFlowLimits{ diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index 49ed3614b..b3cd9b9c2 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -12,8 +12,9 @@ import ( "github.com/stretchr/testify/require" ) +// nolint:unparam func (s PeerFlowE2ETestSuiteSF) setupSourceTable(tableName string, numRows int) { - err := e2e.CreateSourceTableQRep(s.pool, s.pgSuffix, tableName) + err := e2e.CreateTableForQRep(s.pool, s.pgSuffix, tableName) require.NoError(s.t, err) err = e2e.PopulateSourceTable(s.pool, s.pgSuffix, tableName, numRows) require.NoError(s.t, err) @@ -77,6 +78,8 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { query, s.sfHelper.Peer, "", + false, + "", ) require.NoError(s.t, err) @@ -116,6 +119,8 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_Simple() query, s.sfHelper.Peer, "", + false, + "", ) qrepConfig.WriteMode = &protos.QRepWriteMode{ WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, @@ -159,6 +164,8 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3() { query, s.sfHelper.Peer, "", + false, + "", ) require.NoError(s.t, err) qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) @@ -199,6 +206,8 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_Upsert_XMIN() { query, s.sfHelper.Peer, "", + false, + "", ) qrepConfig.WriteMode = &protos.QRepWriteMode{ WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, @@ -247,6 +256,8 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() sfPeer, "", + false, + "", ) require.NoError(s.t, err) qrepConfig.StagingPath = fmt.Sprintf("s3://peerdb-test-bucket/avro/%s", uuid.New()) @@ -264,3 +275,48 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF_S3_Integration() env.AssertExpectations(s.t) } + +func (s PeerFlowE2ETestSuiteSF) Test_PeerDB_Columns_QRep_SF() { + env := e2e.NewTemporalTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env, s.t) + + numRows := 10 + + tblName := "test_qrep_columns_sf" + s.setupSourceTable(tblName, numRows) + + dstSchemaQualified := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tblName) + + query := fmt.Sprintf("SELECT * FROM e2e_test_%s.%s WHERE updated_at BETWEEN {{.start}} AND {{.end}}", + s.pgSuffix, tblName) + + qrepConfig, err := e2e.CreateQRepWorkflowConfig( + "test_columns_qrep_sf", + fmt.Sprintf("e2e_test_%s.%s", s.pgSuffix, tblName), + dstSchemaQualified, + query, + s.sfHelper.Peer, + "", + true, + "_PEERDB_SYNCED_AT", + ) + qrepConfig.WriteMode = &protos.QRepWriteMode{ + WriteType: protos.QRepWriteType_QREP_WRITE_MODE_UPSERT, + UpsertKeyColumns: []string{"id"}, + } + require.NoError(s.t, err) + + e2e.RunQrepFlowWorkflow(env, qrepConfig) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + + err = env.GetWorkflowError() + require.NoError(s.t, err) + + err = s.sfHelper.checkSyncedAt(fmt.Sprintf(`SELECT "_PEERDB_SYNCED_AT" FROM %s.%s`, + s.sfHelper.testSchemaName, tblName)) + require.NoError(s.t, err) + + env.AssertExpectations(s.t) +} diff --git a/flow/e2e/snowflake/snowflake_helper.go b/flow/e2e/snowflake/snowflake_helper.go index 38fefeddc..0401d34f5 100644 --- a/flow/e2e/snowflake/snowflake_helper.go +++ b/flow/e2e/snowflake/snowflake_helper.go @@ -6,6 +6,7 @@ import ( "fmt" "math/big" "os" + "time" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/e2e" @@ -175,3 +176,25 @@ func (s *SnowflakeTestHelper) RunIntQuery(query string) (int, error) { return 0, fmt.Errorf("failed to execute query: %s, returned value of type %s", query, rec.Entries[0].Kind) } } + +// runs a query that returns an int result +func (s *SnowflakeTestHelper) checkSyncedAt(query string) error { + recordBatch, err := s.testClient.ExecuteAndProcessQuery(query) + if err != nil { + return err + } + + for _, record := range recordBatch.Records { + for _, entry := range record.Entries { + if entry.Kind != qvalue.QValueKindTimestamp { + return fmt.Errorf("synced_at column check failed: _PEERDB_SYNCED_AT is not timestamp") + } + _, ok := entry.Value.(time.Time) + if !ok { + return fmt.Errorf("synced_at column failed: _PEERDB_SYNCED_AT is not valid") + } + } + } + + return nil +} diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index abd0076d5..8bea8cf98 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -17,6 +17,7 @@ import ( "github.com/PeerDB-io/peer-flow/logger" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" + "github.com/PeerDB-io/peer-flow/shared/alerting" peerflow "github.com/PeerDB-io/peer-flow/workflows" "github.com/google/uuid" "github.com/jackc/pgx/v5/pgxpool" @@ -59,7 +60,16 @@ func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment, t *t env.RegisterWorkflow(peerflow.QRepFlowWorkflow) env.RegisterWorkflow(peerflow.XminFlowWorkflow) env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) - env.RegisterActivity(&activities.FlowableActivity{CatalogPool: conn}) + + alerter, err := alerting.NewAlerter(conn) + if err != nil { + t.Fatalf("unable to create alerter: %v", err) + } + + env.RegisterActivity(&activities.FlowableActivity{ + CatalogPool: conn, + Alerter: alerter, + }) env.RegisterActivity(&activities.SnapshotActivity{}) } @@ -123,7 +133,7 @@ func NormalizeFlowCountQuery(env *testsuite.TestWorkflowEnvironment, } } -func CreateSourceTableQRep(pool *pgxpool.Pool, suffix string, tableName string) error { +func CreateTableForQRep(pool *pgxpool.Pool, suffix string, tableName string) error { tblFields := []string{ "id UUID NOT NULL PRIMARY KEY", "card_id UUID", @@ -287,6 +297,8 @@ func CreateQRepWorkflowConfig( query string, dest *protos.Peer, stagingPath string, + setupDst bool, + syncedAtCol string, ) (*protos.QRepConfig, error) { connectionGen := QRepFlowConnectionGenerationConfig{ FlowJobName: flowJobName, @@ -304,6 +316,8 @@ func CreateQRepWorkflowConfig( return nil, err } qrepConfig.InitialCopyOnly = true + qrepConfig.SyncedAtColName = syncedAtCol + qrepConfig.SetupWatermarkTableOnDestination = setupDst return qrepConfig, nil } @@ -366,7 +380,7 @@ func GetOwnersSchema() *model.QRecordSchema { func GetOwnersSelectorString() string { schema := GetOwnersSchema() - var fields []string + fields := make([]string, 0, len(schema.Fields)) for _, field := range schema.Fields { // append quoted field name fields = append(fields, fmt.Sprintf(`"%s"`, field.Name)) diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index a4a8ff581..22b8ad06c 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -408,6 +408,7 @@ type FlowConnectionConfigs struct { Resync bool `protobuf:"varint,23,opt,name=resync,proto3" json:"resync,omitempty"` SoftDeleteColName string `protobuf:"bytes,24,opt,name=soft_delete_col_name,json=softDeleteColName,proto3" json:"soft_delete_col_name,omitempty"` SyncedAtColName string `protobuf:"bytes,25,opt,name=synced_at_col_name,json=syncedAtColName,proto3" json:"synced_at_col_name,omitempty"` + InitialCopyOnly bool `protobuf:"varint,26,opt,name=initial_copy_only,json=initialCopyOnly,proto3" json:"initial_copy_only,omitempty"` } func (x *FlowConnectionConfigs) Reset() { @@ -617,6 +618,13 @@ func (x *FlowConnectionConfigs) GetSyncedAtColName() string { return "" } +func (x *FlowConnectionConfigs) GetInitialCopyOnly() bool { + if x != nil { + return x.InitialCopyOnly + } + return false +} + type RenameTableOption struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2669,7 +2677,9 @@ type QRepConfig struct { SetupWatermarkTableOnDestination bool `protobuf:"varint,17,opt,name=setup_watermark_table_on_destination,json=setupWatermarkTableOnDestination,proto3" json:"setup_watermark_table_on_destination,omitempty"` // create new tables with "_peerdb_resync" suffix, perform initial load and then swap the new table with the old ones // to be used after the old mirror is dropped - DstTableFullResync bool `protobuf:"varint,18,opt,name=dst_table_full_resync,json=dstTableFullResync,proto3" json:"dst_table_full_resync,omitempty"` + DstTableFullResync bool `protobuf:"varint,18,opt,name=dst_table_full_resync,json=dstTableFullResync,proto3" json:"dst_table_full_resync,omitempty"` + SyncedAtColName string `protobuf:"bytes,19,opt,name=synced_at_col_name,json=syncedAtColName,proto3" json:"synced_at_col_name,omitempty"` + SoftDeleteColName string `protobuf:"bytes,20,opt,name=soft_delete_col_name,json=softDeleteColName,proto3" json:"soft_delete_col_name,omitempty"` } func (x *QRepConfig) Reset() { @@ -2830,6 +2840,20 @@ func (x *QRepConfig) GetDstTableFullResync() bool { return false } +func (x *QRepConfig) GetSyncedAtColName() string { + if x != nil { + return x.SyncedAtColName + } + return "" +} + +func (x *QRepConfig) GetSoftDeleteColName() string { + if x != nil { + return x.SoftDeleteColName + } + return "" +} + type QRepPartition struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3286,6 +3310,124 @@ func (x *QRepFlowState) GetDisableWaitForNewRows() bool { return false } +type PeerDBColumns struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SoftDeleteColName string `protobuf:"bytes,1,opt,name=soft_delete_col_name,json=softDeleteColName,proto3" json:"soft_delete_col_name,omitempty"` + SyncedAtColName string `protobuf:"bytes,2,opt,name=synced_at_col_name,json=syncedAtColName,proto3" json:"synced_at_col_name,omitempty"` + SoftDelete bool `protobuf:"varint,3,opt,name=soft_delete,json=softDelete,proto3" json:"soft_delete,omitempty"` +} + +func (x *PeerDBColumns) Reset() { + *x = PeerDBColumns{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PeerDBColumns) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PeerDBColumns) ProtoMessage() {} + +func (x *PeerDBColumns) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[48] + 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 PeerDBColumns.ProtoReflect.Descriptor instead. +func (*PeerDBColumns) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{48} +} + +func (x *PeerDBColumns) GetSoftDeleteColName() string { + if x != nil { + return x.SoftDeleteColName + } + return "" +} + +func (x *PeerDBColumns) GetSyncedAtColName() string { + if x != nil { + return x.SyncedAtColName + } + return "" +} + +func (x *PeerDBColumns) GetSoftDelete() bool { + if x != nil { + return x.SoftDelete + } + return false +} + +type GetOpenConnectionsForUserResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + UserName string `protobuf:"bytes,1,opt,name=user_name,json=userName,proto3" json:"user_name,omitempty"` + CurrentOpenConnections int64 `protobuf:"varint,2,opt,name=current_open_connections,json=currentOpenConnections,proto3" json:"current_open_connections,omitempty"` +} + +func (x *GetOpenConnectionsForUserResult) Reset() { + *x = GetOpenConnectionsForUserResult{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetOpenConnectionsForUserResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetOpenConnectionsForUserResult) ProtoMessage() {} + +func (x *GetOpenConnectionsForUserResult) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[49] + 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 GetOpenConnectionsForUserResult.ProtoReflect.Descriptor instead. +func (*GetOpenConnectionsForUserResult) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{49} +} + +func (x *GetOpenConnectionsForUserResult) GetUserName() string { + if x != nil { + return x.UserName + } + return "" +} + +func (x *GetOpenConnectionsForUserResult) GetCurrentOpenConnections() int64 { + if x != nil { + return x.CurrentOpenConnections + } + return 0 +} + var File_flow_proto protoreflect.FileDescriptor var file_flow_proto_rawDesc = []byte{ @@ -3329,7 +3471,7 @@ var file_flow_proto_rawDesc = []byte{ 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x22, - 0xc4, 0x0c, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0xf0, 0x0c, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x06, 0x73, @@ -3418,249 +3560,153 @@ var file_flow_proto_rawDesc = []byte{ 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x19, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, - 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8e, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, 0x6d, - 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, - 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0b, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6e, 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc9, 0x02, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, - 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, - 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x26, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, - 0x65, 0x65, 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x50, 0x0a, 0x14, 0x72, 0x65, 0x6e, - 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, 0x0a, 0x14, 0x73, - 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, 0x73, 0x6f, 0x66, - 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, - 0x01, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, - 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, - 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, - 0x88, 0x01, 0x01, 0x42, 0x17, 0x0a, 0x15, 0x5f, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x15, 0x0a, 0x13, - 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x22, 0x38, 0x0a, 0x12, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, + 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x1a, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, + 0x6e, 0x6c, 0x79, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, + 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x8e, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x72, 0x72, + 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, + 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, + 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x22, 0xc9, 0x02, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc7, 0x02, - 0x0a, 0x1d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, - 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, - 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x8b, 0x01, 0x0a, 0x1d, - 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, - 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, - 0x4e, 0x65, 0x77, 0x54, 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x19, - 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x4c, 0x0a, 0x1e, 0x4e, 0x65, 0x77, - 0x54, 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x44, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x8d, 0x02, - 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, - 0x12, 0x72, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, - 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, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x35, 0x0a, - 0x14, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x53, 0x69, 0x7a, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, - 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, - 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, - 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, - 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x22, 0xd6, 0x03, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, - 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, - 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, - 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, - 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, - 0x6e, 0x63, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x71, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x16, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 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, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, - 0x7a, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, - 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, - 0x79, 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, - 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, - 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, - 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, - 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, - 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, - 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, - 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, - 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, + 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, + 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, - 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, + 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x50, 0x0a, 0x14, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, + 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x30, 0x0a, + 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0f, 0x73, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, + 0x17, 0x0a, 0x15, 0x5f, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x73, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, + 0x38, 0x0a, 0x12, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, + 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, + 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc7, 0x02, 0x0a, 0x1d, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x26, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x8b, 0x01, 0x0a, 0x1d, 0x6e, 0x65, 0x77, 0x5f, + 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x49, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x4e, 0x65, 0x77, 0x54, + 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x19, 0x6e, 0x65, 0x77, 0x54, + 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x4c, 0x0a, 0x1e, 0x4e, 0x65, 0x77, 0x54, 0x6f, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x44, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, - 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, - 0x0a, 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, - 0x72, 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, - 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, - 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, - 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, - 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, - 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, - 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 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, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, - 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, - 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, - 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, - 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, - 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, - 0x70, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, - 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, 0x69, 0x73, - 0x74, 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, - 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, 0x78, 0x69, - 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, - 0x16, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, - 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, - 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x8d, 0x02, 0x0a, 0x0f, 0x53, 0x79, + 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x72, 0x0a, 0x18, + 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, + 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x52, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 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, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x35, 0x0a, 0x14, 0x4e, 0x6f, 0x72, + 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, + 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, + 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x41, 0x74, 0x22, 0xd6, 0x03, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, + 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x61, + 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, + 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, + 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x71, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, + 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 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, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x71, 0x0a, 0x13, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x22, + 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, + 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, + 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, + 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, 0x6e, 0x73, 0x75, 0x72, + 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, @@ -3668,321 +3714,442 @@ var file_flow_proto_rawDesc = []byte{ 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x64, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, - 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, - 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, - 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, 0x45, 0x6e, 0x73, 0x75, + 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, + 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, + 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, 0x0a, 0x06, 0x72, 0x65, + 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x72, 0x65, 0x6c, 0x49, + 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, + 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x48, 0x00, + 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, 0x10, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x62, 0x0a, + 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, + 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 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, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9a, 0x04, 0x0a, + 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, + 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x10, + 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, + 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, + 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, + 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, + 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, + 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x9e, 0x02, 0x0a, - 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x69, 0x6d, - 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, - 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x72, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, - 0x66, 0x75, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x73, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x46, 0x75, 0x6c, - 0x6c, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 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, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, - 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, - 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, - 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, - 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, - 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, 0x53, 0x65, 0x74, + 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, + 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, + 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, 0x0a, 0x12, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, + 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, + 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, + 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, + 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, + 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x9e, 0x02, 0x0a, 0x0b, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, + 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x11, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, 0x66, 0x75, 0x6c, 0x6c, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x73, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x46, 0x75, 0x6c, 0x6c, 0x1a, 0x3a, 0x0a, + 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 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, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, + 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, + 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0xda, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, + 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x12, 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xb2, 0x03, 0x0a, + 0x1e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, + 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, + 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, + 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, + 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, - 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, - 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x22, 0xb2, 0x03, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, - 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, - 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, - 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, - 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, - 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, - 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, - 0x0a, 0x0e, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, - 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, - 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, - 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2f, + 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, + 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x63, 0x0a, 0x1b, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, + 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, + 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, + 0x73, 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, - 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, - 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, - 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, - 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, - 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, - 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, - 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, - 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, - 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, - 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, - 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, - 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x99, 0x07, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, - 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, - 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, - 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, - 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, - 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, - 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, - 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, - 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, - 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, - 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, - 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, - 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, - 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, - 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, - 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, - 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, - 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, - 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, - 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, - 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, - 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, - 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, - 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, - 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, - 0x0a, 0x24, 0x73, 0x65, 0x74, 0x75, 0x70, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x73, 0x65, - 0x74, 0x75, 0x70, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4f, 0x6e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x31, - 0x0a, 0x15, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6c, 0x6c, - 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, - 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x79, 0x6e, - 0x63, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, - 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, - 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, - 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, - 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, - 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, + 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, + 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, + 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, + 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, + 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, + 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, + 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, + 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, + 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, + 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, + 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, + 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, + 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, + 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, + 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x22, 0xf7, 0x07, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, + 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, + 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, + 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, + 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, + 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, + 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, + 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, + 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, + 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, + 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, + 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, + 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, + 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, + 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, + 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, + 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, + 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, 0x24, 0x73, 0x65, + 0x74, 0x75, 0x70, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x73, 0x65, 0x74, 0x75, 0x70, 0x57, + 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x6e, 0x44, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x31, 0x0a, 0x15, 0x64, 0x73, + 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x72, 0x65, 0x73, + 0x79, 0x6e, 0x63, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x73, 0x74, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x2b, 0x0a, + 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, + 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x0d, + 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, + 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, + 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x24, 0x0a, + 0x0e, 0x73, 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x73, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x61, 0x64, 0x64, + 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, + 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, + 0x0c, 0x61, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xc8, 0x01, + 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, + 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, + 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x4d, 0x0a, 0x13, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x44, 0x65, 0x6c, 0x74, 0x61, 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x22, 0xe9, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, + 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x41, 0x0a, 0x0e, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, - 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, - 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, - 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, - 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0xa2, - 0x01, 0x0a, 0x10, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, - 0x6c, 0x74, 0x61, 0x12, 0x24, 0x0a, 0x0e, 0x73, 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x72, 0x63, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x73, 0x74, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x64, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x42, 0x0a, 0x0d, 0x61, 0x64, 0x64, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x0c, 0x61, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x22, 0xc8, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x79, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, - 0x4d, 0x0a, 0x13, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, - 0x64, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x52, 0x11, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x22, 0xe9, - 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x12, 0x41, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x38, 0x0a, 0x18, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x16, 0x6e, 0x75, 0x6d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x21, 0x0a, - 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x73, 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0b, 0x6e, 0x65, 0x65, 0x64, 0x73, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, - 0x12, 0x38, 0x0a, 0x19, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x77, 0x61, 0x69, 0x74, - 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x15, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x57, 0x61, 0x69, 0x74, - 0x46, 0x6f, 0x72, 0x4e, 0x65, 0x77, 0x52, 0x6f, 0x77, 0x73, 0x2a, 0x50, 0x0a, 0x0c, 0x51, 0x52, - 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x51, 0x52, - 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x55, 0x4c, - 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x51, - 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x53, 0x54, - 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, 0x66, 0x0a, 0x0d, - 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, - 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, - 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x50, 0x53, - 0x45, 0x52, 0x54, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, - 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x57, 0x52, 0x49, - 0x54, 0x45, 0x10, 0x02, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, 0x6f, 0x77, 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, 0x0a, 0x50, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, - 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, - 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, + 0x6c, 0x61, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x38, 0x0a, + 0x18, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, + 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x16, 0x6e, 0x75, 0x6d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x50, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x73, + 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x6e, + 0x65, 0x65, 0x64, 0x73, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x38, 0x0a, 0x19, 0x64, 0x69, + 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x6e, + 0x65, 0x77, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x64, + 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x4e, 0x65, 0x77, + 0x52, 0x6f, 0x77, 0x73, 0x22, 0x8e, 0x01, 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, 0x44, 0x42, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x22, 0x78, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x6e, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x46, 0x6f, 0x72, 0x55, 0x73, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x73, 0x65, 0x72, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, + 0x5f, 0x6f, 0x70, 0x65, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, + 0x4f, 0x70, 0x65, 0x6e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2a, + 0x50, 0x0a, 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, + 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, + 0x12, 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, + 0x44, 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, + 0x01, 0x2a, 0x66, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, + 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, + 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x51, 0x52, + 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x56, + 0x45, 0x52, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x02, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, + 0x6f, 0x77, 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, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, + 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, + 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -3998,7 +4165,7 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 60) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 62) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType @@ -4050,74 +4217,76 @@ var file_flow_proto_goTypes = []interface{}{ (*TableSchemaDelta)(nil), // 47: peerdb_flow.TableSchemaDelta (*ReplayTableSchemaDeltaInput)(nil), // 48: peerdb_flow.ReplayTableSchemaDeltaInput (*QRepFlowState)(nil), // 49: peerdb_flow.QRepFlowState - nil, // 50: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 51: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 52: peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry - nil, // 53: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry - nil, // 54: peerdb_flow.StartFlowInput.RelationMessageMappingEntry - nil, // 55: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 56: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 57: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 58: peerdb_flow.TableSchema.ColumnsEntry - nil, // 59: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 60: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 61: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 62: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 63: google.protobuf.Timestamp + (*PeerDBColumns)(nil), // 50: peerdb_flow.PeerDBColumns + (*GetOpenConnectionsForUserResult)(nil), // 51: peerdb_flow.GetOpenConnectionsForUserResult + nil, // 52: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 53: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 54: peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry + nil, // 55: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + nil, // 56: peerdb_flow.StartFlowInput.RelationMessageMappingEntry + nil, // 57: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + nil, // 58: peerdb_flow.SetupReplicationInput.TableNameMappingEntry + nil, // 59: peerdb_flow.CreateRawTableInput.TableNameMappingEntry + nil, // 60: peerdb_flow.TableSchema.ColumnsEntry + nil, // 61: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + nil, // 62: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + nil, // 63: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + (*Peer)(nil), // 64: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 65: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ 3, // 0: peerdb_flow.RelationMessage.columns:type_name -> peerdb_flow.RelationMessageColumn - 62, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 62, // 2: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer + 64, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 64, // 2: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer 28, // 3: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema 5, // 4: peerdb_flow.FlowConnectionConfigs.table_mappings:type_name -> peerdb_flow.TableMapping - 50, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 51, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 62, // 7: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer + 52, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + 53, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + 64, // 7: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer 0, // 8: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode 0, // 9: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode 28, // 10: peerdb_flow.RenameTableOption.table_schema:type_name -> peerdb_flow.TableSchema - 62, // 11: peerdb_flow.RenameTablesInput.peer:type_name -> peerdb_peers.Peer + 64, // 11: peerdb_flow.RenameTablesInput.peer:type_name -> peerdb_peers.Peer 7, // 12: peerdb_flow.RenameTablesInput.rename_table_options:type_name -> peerdb_flow.RenameTableOption - 62, // 13: peerdb_flow.CreateTablesFromExistingInput.peer:type_name -> peerdb_peers.Peer - 52, // 14: peerdb_flow.CreateTablesFromExistingInput.new_to_existing_table_mapping:type_name -> peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry - 53, // 15: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry - 63, // 16: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp + 64, // 13: peerdb_flow.CreateTablesFromExistingInput.peer:type_name -> peerdb_peers.Peer + 54, // 14: peerdb_flow.CreateTablesFromExistingInput.new_to_existing_table_mapping:type_name -> peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry + 55, // 15: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + 65, // 16: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp 14, // 17: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState 6, // 18: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs 12, // 19: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions - 54, // 20: peerdb_flow.StartFlowInput.relation_message_mapping:type_name -> peerdb_flow.StartFlowInput.RelationMessageMappingEntry + 56, // 20: peerdb_flow.StartFlowInput.relation_message_mapping:type_name -> peerdb_flow.StartFlowInput.RelationMessageMappingEntry 6, // 21: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 62, // 22: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 62, // 23: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 62, // 24: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 64, // 22: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 64, // 23: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer + 64, // 24: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer 20, // 25: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier 21, // 26: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier - 55, // 27: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 62, // 28: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 56, // 29: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 62, // 30: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 62, // 31: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 57, // 32: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry + 57, // 27: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 64, // 28: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 58, // 29: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 64, // 30: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 64, // 31: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 59, // 32: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry 0, // 33: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 58, // 34: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 62, // 35: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 59, // 36: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 62, // 37: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 60, // 34: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 64, // 35: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 61, // 36: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 64, // 37: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer 28, // 38: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema - 62, // 39: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 60, // 40: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 61, // 41: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 63, // 42: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 63, // 43: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp + 64, // 39: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 62, // 40: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 63, // 41: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 65, // 42: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 65, // 43: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp 37, // 44: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID 37, // 45: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID 35, // 46: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange 36, // 47: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange 38, // 48: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange 1, // 49: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType - 62, // 50: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 62, // 51: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer + 64, // 50: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 64, // 51: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer 0, // 52: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode 40, // 53: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode 39, // 54: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange @@ -4723,6 +4892,30 @@ func file_flow_proto_init() { return nil } } + file_flow_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PeerDBColumns); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetOpenConnectionsForUserResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_flow_proto_msgTypes[6].OneofWrappers = []interface{}{} file_flow_proto_msgTypes[19].OneofWrappers = []interface{}{ @@ -4739,7 +4932,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 60, + NumMessages: 62, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/go.mod b/flow/go.mod index a3575510c..69307cd2a 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -9,12 +9,13 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 - github.com/aws/aws-sdk-go v1.49.4 + github.com/aws/aws-sdk-go v1.49.9 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cockroachdb/pebble v0.0.0-20231210175914-b4d301aeb46a github.com/google/uuid v1.5.0 github.com/grafana/pyroscope-go v1.0.4 github.com/grpc-ecosystem/grpc-gateway/v2 v2.18.1 + github.com/jackc/pgerrcode v0.0.0-20220416144525-469b46aa5efa github.com/jackc/pglogrepl v0.0.0-20231111135425-1627ab1b5780 github.com/jackc/pgx/v5 v5.5.1 github.com/jmoiron/sqlx v1.3.5 @@ -24,6 +25,7 @@ require ( github.com/linkedin/goavro/v2 v2.12.0 github.com/microsoft/go-mssqldb v1.6.0 github.com/orcaman/concurrent-map/v2 v2.0.1 + github.com/slack-go/slack v0.12.3 github.com/snowflakedb/gosnowflake v1.7.1 github.com/stretchr/testify v1.8.4 github.com/twpayne/go-geos v0.14.0 @@ -37,8 +39,8 @@ require ( golang.org/x/sync v0.5.0 google.golang.org/api v0.154.0 google.golang.org/genproto/googleapis/api v0.0.0-20231212172506-995d672761c0 - google.golang.org/grpc v1.60.0 - google.golang.org/protobuf v1.31.0 + google.golang.org/grpc v1.60.1 + google.golang.org/protobuf v1.32.0 ) require ( @@ -51,8 +53,9 @@ require ( github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/getsentry/sentry-go v0.25.0 // indirect - github.com/go-logr/logr v1.3.0 // indirect + github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect + github.com/gorilla/websocket v1.5.1 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect @@ -88,8 +91,8 @@ require ( github.com/apache/thrift v0.19.0 // indirect github.com/aws/aws-sdk-go-v2 v1.24.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.16.12 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.7 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.16.13 // indirect + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.9 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.9 // indirect @@ -97,12 +100,12 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.9 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.9 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.47.5 // indirect + github.com/aws/aws-sdk-go-v2/service/s3 v1.47.7 // indirect github.com/aws/smithy-go v1.19.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.3 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dvsekhvalnov/jose2go v1.5.0 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect github.com/gabriel-vasile/mimetype v1.4.3 // indirect @@ -147,8 +150,8 @@ require ( github.com/xrash/smetrics v0.0.0-20231213231151-1d8dd44e695e // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.16.0 - golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611 + golang.org/x/crypto v0.17.0 + golang.org/x/exp v0.0.0-20231219180239-dc181d75b848 golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.19.0 // indirect golang.org/x/oauth2 v0.15.0 // indirect diff --git a/flow/go.sum b/flow/go.sum index cb1247fe0..360572bb2 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -62,20 +62,20 @@ github.com/apache/arrow/go/v12 v12.0.1 h1:JsR2+hzYYjgSUkBSaahpqCetqZMr76djX80fF/ github.com/apache/arrow/go/v12 v12.0.1/go.mod h1:weuTY7JvTG/HDPtMQxEUp7pU73vkLWMLpY67QwZ/WWw= github.com/apache/thrift v0.19.0 h1:sOqkWPzMj7w6XaYbJQG7m4sGqVolaW/0D28Ln7yPzMk= github.com/apache/thrift v0.19.0/go.mod h1:SUALL216IiaOw2Oy+5Vs9lboJ/t9g40C+G07Dc0QC1I= -github.com/aws/aws-sdk-go v1.49.4 h1:qiXsqEeLLhdLgUIyfr5ot+N/dGPWALmtM1SetRmbUlY= -github.com/aws/aws-sdk-go v1.49.4/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.49.9 h1:4xoyi707rsifB1yMsd5vGbAH21aBzwpL3gNRMSmjIyc= +github.com/aws/aws-sdk-go v1.49.9/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/aws/aws-sdk-go-v2 v1.24.0 h1:890+mqQ+hTpNuw0gGP6/4akolQkSToDJgHfQE7AwGuk= github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 h1:OCs21ST2LrepDfD3lwlQiOqIGp6JiEUqG84GzTDoyJs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4/go.mod h1:usURWEKSNNAcAZuzRn/9ZYPT8aZQkR7xcCtunK/LkJo= -github.com/aws/aws-sdk-go-v2/config v1.26.1 h1:z6DqMxclFGL3Zfo+4Q0rLnAZ6yVkzCRxhRMsiRQnD1o= -github.com/aws/aws-sdk-go-v2/config v1.26.1/go.mod h1:ZB+CuKHRbb5v5F0oJtGdhFTelmrxd4iWO1lf0rQwSAg= -github.com/aws/aws-sdk-go-v2/credentials v1.16.12 h1:v/WgB8NxprNvr5inKIiVVrXPuuTegM+K8nncFkr1usU= -github.com/aws/aws-sdk-go-v2/credentials v1.16.12/go.mod h1:X21k0FjEJe+/pauud82HYiQbEr9jRKY3kXEIQ4hXeTQ= +github.com/aws/aws-sdk-go-v2/config v1.26.2 h1:+RWLEIWQIGgrz2pBPAUoGgNGs1TOyF4Hml7hCnYj2jc= +github.com/aws/aws-sdk-go-v2/config v1.26.2/go.mod h1:l6xqvUxt0Oj7PI/SUXYLNyZ9T/yBPn3YTQcJLLOdtR8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.13 h1:WLABQ4Cp4vXtXfOWOS3MEZKr6AAYUpMczLhgKtAjQ/8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.13/go.mod h1:Qg6x82FXwW0sJHzYruxGiuApNo31UEtJvXVSZAXeWiw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.7 h1:FnLf60PtjXp8ZOzQfhJVsqF0OtYKQZWQfqOLshh8YXg= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.7/go.mod h1:tDVvl8hyU6E9B8TrnNrZQEVkQlB8hjJwcgpPhgtlnNg= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.9 h1:5zA8qVCXMPGt6YneFnll5B157SfdK2SewU85PH9/yM0= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.9/go.mod h1:t4gy210hPxkbtYM8xOzrWdxVq1PyekR76OOKXy3s0Vs= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 h1:v+HbZaCGmOwnTTVS86Fleq0vPzOd7tnJGbFhP0stNLs= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9/go.mod h1:Xjqy+Nyj7VDLBtCMkQYOw1QYfAEZCVLrfI0ezve8wd4= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 h1:N94sVhRACtXyVcjXxrwK1SKFIJrA9pOJ5yu2eSHnmls= @@ -92,14 +92,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 h1:Nf2sHxjMJ github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9/go.mod h1:idky4TER38YIjr2cADF1/ugFMKvZV7p//pVeV5LZbF0= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.9 h1:iEAeF6YC3l4FzlJPP9H3Ko1TXpdjdqWffxXjp8SY6uk= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.9/go.mod h1:kjsXoK23q9Z/tLBrckZLLyvjhZoS+AGrzqzUfEClvMM= -github.com/aws/aws-sdk-go-v2/service/s3 v1.47.5 h1:Keso8lIOS+IzI2MkPZyK6G0LYcK3My2LQ+T5bxghEAY= -github.com/aws/aws-sdk-go-v2/service/s3 v1.47.5/go.mod h1:vADO6Jn+Rq4nDtfwNjhgR84qkZwiC6FqCaXdw/kYwjA= +github.com/aws/aws-sdk-go-v2/service/s3 v1.47.7 h1:o0ASbVwUAIrfp/WcCac+6jioZt4Hd8k/1X8u7GJ/QeM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.47.7/go.mod h1:vADO6Jn+Rq4nDtfwNjhgR84qkZwiC6FqCaXdw/kYwjA= github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 h1:ldSFWz9tEHAwHNmjx2Cvy1MjP5/L9kNoR0skc6wyOOM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.5/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 h1:2k9KmFawS63euAkY4/ixVNsYYwrwnd5fIvgEKkfZFNM= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.5 h1:5UYvv8JUvllZsRnfrcMQ+hJ9jNICmcgKPAO1CER25Wg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.5/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 h1:HJeiuZ2fldpd0WqngyMR6KW7ofkXNLyOaHwEIGm39Cs= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.6/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -138,8 +138,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= -github.com/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= -github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= +github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY= +github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -164,13 +164,15 @@ github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3Bop github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY= -github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= +github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-test/deep v1.0.4 h1:u2CU3YKy9I2pmu9pX0eq50wCgjfGIt539SqR7FbHiho= +github.com/go-test/deep v1.0.4/go.mod h1:wGDj63lr65AM2AQyKZd/NYHGb0R+1RLqB8NKt3aSFNA= github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/godbus/dbus v0.0.0-20190726142602-4481cbc300e2 h1:ZpnhV/YsD2/4cESfV5+Hoeu/iUR3ruzNvZ+yQfO03a0= @@ -221,6 +223,7 @@ github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= @@ -235,6 +238,9 @@ github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfF github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= github.com/googleapis/gax-go/v2 v2.12.0/go.mod h1:y+aIqrI5eb1YGMVJfuV3185Ts/D7qKpsEkdD5+I6QGU= +github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/gorilla/websocket v1.5.1 h1:gmztn0JnHVt9JZquRuzLw3g4wouNVzKL15iLr/zn/QY= +github.com/gorilla/websocket v1.5.1/go.mod h1:x3kM2JMyaluk02fnUJpQuwD2dCS5NDG2ZHL0uE0tcaY= github.com/grafana/pyroscope-go v1.0.4 h1:oyQX0BOkL+iARXzHuCdIF5TQ7/sRSel1YFViMHC7Bm0= github.com/grafana/pyroscope-go v1.0.4/go.mod h1:0d7ftwSMBV/Awm7CCiYmHQEG8Y44Ma3YSjt+nWcWztY= github.com/grafana/pyroscope-go/godeltaprof v0.1.6 h1:nEdZ8louGAplSvIJi1HVp7kWvFvdiiYg3COLlTwJiFo= @@ -249,6 +255,8 @@ github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8 github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= github.com/hexops/gotextdiff v1.0.3 h1:gitA9+qJrrTCsiCl7+kh75nPqQt1cx4ZkudSTLoUqJM= github.com/hexops/gotextdiff v1.0.3/go.mod h1:pSWU5MAI3yDq+fZBTazCSJysOMbxWL1BSow5/V2vxeg= +github.com/jackc/pgerrcode v0.0.0-20220416144525-469b46aa5efa h1:s+4MhCQ6YrzisK6hFJUX53drDT4UsSW3DEhKn0ifuHw= +github.com/jackc/pgerrcode v0.0.0-20220416144525-469b46aa5efa/go.mod h1:a/s9Lp5W7n/DD0VrVoyJ00FbP2ytTPDVOivvn2bMlds= github.com/jackc/pgio v1.0.0 h1:g12B9UwVnzGhueNavwioyEEpAmqMe1E/BN9ES+8ovkE= github.com/jackc/pgio v1.0.0/go.mod h1:oP+2QK2wFfUWgr+gxjoBH9KGBb31Eio69xUb0w5bYf8= github.com/jackc/pglogrepl v0.0.0-20231111135425-1627ab1b5780 h1:pNK2AKKIRC1MMMvpa6UiNtdtOebpiIloX7q2JZDkfsk= @@ -351,6 +359,8 @@ github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/slack-go/slack v0.12.3 h1:92/dfFU8Q5XP6Wp5rr5/T5JHLM5c5Smtn53fhToAP88= +github.com/slack-go/slack v0.12.3/go.mod h1:hlGi5oXA+Gt+yWTPP0plCdRKmjsDxecdHxYQdlMQKOw= github.com/snowflakedb/gosnowflake v1.7.1 h1:c9JjyjjDlvxex9ud71TwKL+Wu54Vfx+39h4DAwbIdqU= github.com/snowflakedb/gosnowflake v1.7.1/go.mod h1:JI3eRZL8CpimPek6CJO0aTbDQjDGOt7Rxv9A/ti4f5c= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -423,11 +433,11 @@ golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220829220503-c86fa9a7ed90/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.16.0 h1:mMMrFzRSCF0GvB7Ne27XVtVAaXLrPmgPC7/v0tkwHaY= -golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611 h1:qCEDpW1G+vcj3Y7Fy52pEM1AWm3abj8WimGYejI3SC4= -golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= +golang.org/x/exp v0.0.0-20231219180239-dc181d75b848 h1:+iq7lrkxmFNBM7xx+Rae2W6uyPfhPeDWD+n+JgppptE= +golang.org/x/exp v0.0.0-20231219180239-dc181d75b848/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -548,8 +558,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k= -google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= +google.golang.org/grpc v1.60.1 h1:26+wFr+cNqSGFcOXcabYC0lUVJVRa2Sb2ortSK7VrEU= +google.golang.org/grpc v1.60.1/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -561,8 +571,8 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/flow/model/conversion_avro.go b/flow/model/conversion_avro.go index 2f43920ec..02a52b26d 100644 --- a/flow/model/conversion_avro.go +++ b/flow/model/conversion_avro.go @@ -71,7 +71,7 @@ func GetAvroSchemaDefinition( dstTableName string, qRecordSchema *QRecordSchema, ) (*QRecordAvroSchemaDefinition, error) { - avroFields := []QRecordAvroField{} + avroFields := make([]QRecordAvroField, 0, len(qRecordSchema.Fields)) nullableFields := make(map[string]struct{}) for _, qField := range qRecordSchema.Fields { @@ -80,16 +80,14 @@ func GetAvroSchemaDefinition( return nil, err } - consolidatedType := avroType.AvroLogicalSchema - if qField.Nullable { - consolidatedType = []interface{}{"null", consolidatedType} + avroType = []interface{}{"null", avroType} nullableFields[qField.Name] = struct{}{} } avroFields = append(avroFields, QRecordAvroField{ Name: qField.Name, - Type: consolidatedType, + Type: avroType, }) } diff --git a/flow/model/model.go b/flow/model/model.go index 02949f3c2..b579ccfb5 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -50,13 +50,15 @@ type PullRecordsRequest struct { RelationMessageMapping RelationMessageMapping // record batch for pushing changes into RecordStream *CDCRecordStream + // last offset may be forwarded while processing records + SetLastOffset func(int64) error } type Record interface { // GetCheckPointID returns the ID of the record. GetCheckPointID() int64 // get table name - GetTableName() string + GetDestinationTableName() string // get columns and values for the record GetItems() *RecordItems } @@ -242,7 +244,7 @@ func (r *InsertRecord) GetCheckPointID() int64 { return r.CheckPointID } -func (r *InsertRecord) GetTableName() string { +func (r *InsertRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -271,7 +273,7 @@ func (r *UpdateRecord) GetCheckPointID() int64 { } // Implement Record interface for UpdateRecord. -func (r *UpdateRecord) GetTableName() string { +func (r *UpdateRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -297,7 +299,7 @@ func (r *DeleteRecord) GetCheckPointID() int64 { return r.CheckPointID } -func (r *DeleteRecord) GetTableName() string { +func (r *DeleteRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -327,7 +329,7 @@ type CDCRecordStream struct { } func NewCDCRecordStream() *CDCRecordStream { - channelBuffer := peerdbenv.GetPeerDBCDCChannelBufferSize() + channelBuffer := peerdbenv.PeerDBCDCChannelBufferSize() return &CDCRecordStream{ records: make(chan Record, channelBuffer), // TODO (kaushik): more than 1024 schema deltas can cause problems! @@ -414,6 +416,11 @@ func (r *CDCRecordStream) GetRecords() chan Record { return r.records } +type SyncAndNormalizeBatchID struct { + SyncBatchID int64 + NormalizeBatchID int64 +} + type SyncRecordsRequest struct { Records *CDCRecordStream // FlowJobName is the name of the flow job. @@ -468,8 +475,8 @@ func (r *RelationRecord) GetCheckPointID() int64 { return r.CheckPointID } -func (r *RelationRecord) GetTableName() string { - return r.TableSchemaDelta.SrcTableName +func (r *RelationRecord) GetDestinationTableName() string { + return r.TableSchemaDelta.DstTableName } func (r *RelationRecord) GetItems() *RecordItems { diff --git a/flow/model/qschema.go b/flow/model/qschema.go index 5081b10d1..68243ef30 100644 --- a/flow/model/qschema.go +++ b/flow/model/qschema.go @@ -47,7 +47,7 @@ func (q *QRecordSchema) EqualNames(other *QRecordSchema) bool { // GetColumnNames returns a slice of column names. func (q *QRecordSchema) GetColumnNames() []string { - var names []string + names := make([]string, 0, len(q.Fields)) for _, field := range q.Fields { names = append(names, field.Name) } diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index 8fd206d22..4f9cbe2e4 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -2,18 +2,25 @@ package qvalue import ( "fmt" + "log/slog" "math/big" "time" "github.com/google/uuid" "github.com/linkedin/goavro/v2" - "golang.org/x/exp/slog" ) -// QValueKindAvroSchema defines a structure for representing Avro schemas. -// AvroLogicalSchema holds the Avro logical schema for a corresponding QValueKind. -type QValueKindAvroSchema struct { - AvroLogicalSchema interface{} +// https://avro.apache.org/docs/1.11.0/spec.html +type AvroSchemaArray struct { + Type string `json:"type"` + Items string `json:"items"` +} + +type AvroSchemaNumeric struct { + Type string `json:"type"` + LogicalType string `json:"logicalType"` + Precision int `json:"precision"` + Scale int `json:"scale"` } // GetAvroSchemaFromQValueKind returns the Avro schema for a given QValueKind. @@ -23,102 +30,63 @@ type QValueKindAvroSchema struct { // // For example, QValueKindInt64 would return an AvroLogicalSchema of "long". Unsupported QValueKinds // will return an error. -func GetAvroSchemaFromQValueKind(kind QValueKind) (QValueKindAvroSchema, error) { +func GetAvroSchemaFromQValueKind(kind QValueKind) (interface{}, error) { switch kind { case QValueKindString, QValueKindUUID: - return QValueKindAvroSchema{ - AvroLogicalSchema: "string", - }, nil + return "string", nil case QValueKindGeometry, QValueKindGeography, QValueKindPoint: - return QValueKindAvroSchema{ - AvroLogicalSchema: "string", - }, nil + return "string", nil case QValueKindInt16, QValueKindInt32, QValueKindInt64: - return QValueKindAvroSchema{ - AvroLogicalSchema: "long", - }, nil + return "long", nil case QValueKindFloat32: - return QValueKindAvroSchema{ - AvroLogicalSchema: "float", - }, nil + return "float", nil case QValueKindFloat64: - return QValueKindAvroSchema{ - AvroLogicalSchema: "double", - }, nil + return "double", nil case QValueKindBoolean: - return QValueKindAvroSchema{ - AvroLogicalSchema: "boolean", - }, nil + return "boolean", nil case QValueKindBytes, QValueKindBit: - return QValueKindAvroSchema{ - AvroLogicalSchema: "bytes", - }, nil + return "bytes", nil case QValueKindNumeric: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "bytes", - "logicalType": "decimal", - "precision": 38, - "scale": 9, - }, + return AvroSchemaNumeric{ + Type: "bytes", + LogicalType: "decimal", + Precision: 38, + Scale: 9, }, nil case QValueKindTime, QValueKindTimeTZ, QValueKindDate, QValueKindTimestamp, QValueKindTimestampTZ: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]string{ - "type": "string", - }, - }, nil + return "string", nil case QValueKindHStore, QValueKindJSON, QValueKindStruct: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "string", - "values": "string", - }, - }, nil + return "string", nil case QValueKindArrayFloat32: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "array", - "items": "float", - }, + return AvroSchemaArray{ + Type: "array", + Items: "float", }, nil case QValueKindArrayFloat64: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "array", - "items": "double", - }, + return AvroSchemaArray{ + Type: "array", + Items: "double", }, nil case QValueKindArrayInt32: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "array", - "items": "int", - }, + return AvroSchemaArray{ + Type: "array", + Items: "int", }, nil case QValueKindArrayInt64: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "array", - "items": "long", - }, + return AvroSchemaArray{ + Type: "array", + Items: "long", }, nil case QValueKindArrayString: - return QValueKindAvroSchema{ - AvroLogicalSchema: map[string]interface{}{ - "type": "array", - "items": "string", - }, + return AvroSchemaArray{ + Type: "array", + Items: "string", }, nil case QValueKindInvalid: // lets attempt to do invalid as a string - return QValueKindAvroSchema{ - AvroLogicalSchema: "string", - }, nil + return "string", nil default: - return QValueKindAvroSchema{ - AvroLogicalSchema: nil, - }, fmt.Errorf("unsupported QValueKind type: %s", kind) + return nil, fmt.Errorf("unsupported QValueKind type: %s", kind) } } diff --git a/flow/peerdbenv/config.go b/flow/peerdbenv/config.go index cf0e51c21..cdefa6a37 100644 --- a/flow/peerdbenv/config.go +++ b/flow/peerdbenv/config.go @@ -9,58 +9,79 @@ import ( // throughout the codebase. // PEERDB_VERSION_SHA_SHORT -func GetPeerDBVersionShaShort() string { +func PeerDBVersionShaShort() string { return getEnvString("PEERDB_VERSION_SHA_SHORT", "unknown") } // PEERDB_DEPLOYMENT_UID -func GetPeerDBDeploymentUID() string { +func PeerDBDeploymentUID() string { return getEnvString("PEERDB_DEPLOYMENT_UID", "") } // PEERDB_CDC_CHANNEL_BUFFER_SIZE -func GetPeerDBCDCChannelBufferSize() int { +func PeerDBCDCChannelBufferSize() int { return getEnvInt("PEERDB_CDC_CHANNEL_BUFFER_SIZE", 1<<18) } // PEERDB_EVENTHUB_FLUSH_TIMEOUT_SECONDS -func GetPeerDBEventhubFlushTimeoutSeconds() time.Duration { +func PeerDBEventhubFlushTimeoutSeconds() time.Duration { x := getEnvInt("PEERDB_EVENTHUB_FLUSH_TIMEOUT_SECONDS", 10) return time.Duration(x) * time.Second } // PEERDB_CDC_IDLE_TIMEOUT_SECONDS -func GetPeerDBCDCIdleTimeoutSeconds() time.Duration { +func PeerDBCDCIdleTimeoutSeconds() time.Duration { x := getEnvInt("PEERDB_CDC_IDLE_TIMEOUT_SECONDS", 60) return time.Duration(x) * time.Second } // PEERDB_CDC_DISK_SPILL_THRESHOLD -func GetPeerDBCDCDiskSpillThreshold() int { +func PeerDBCDCDiskSpillThreshold() int { return getEnvInt("PEERDB_CDC_DISK_SPILL_THRESHOLD", 1_000_000) } // PEERDB_CATALOG_HOST -func GetPeerDBCatalogHost() string { +func PeerDBCatalogHost() string { return getEnvString("PEERDB_CATALOG_HOST", "") } // PEERDB_CATALOG_PORT -func GetPeerDBCatalogPort() uint32 { +func PeerDBCatalogPort() uint32 { return getEnvUint32("PEERDB_CATALOG_PORT", 5432) } // PEERDB_CATALOG_USER -func GetPeerDBCatalogUser() string { +func PeerDBCatalogUser() string { return getEnvString("PEERDB_CATALOG_USER", "") } // PEERDB_CATALOG_PASSWORD -func GetPeerDBCatalogPassword() string { +func PeerDBCatalogPassword() string { return getEnvString("PEERDB_CATALOG_PASSWORD", "") } // PEERDB_CATALOG_DATABASE -func GetPeerDBCatalogDatabase() string { +func PeerDBCatalogDatabase() string { return getEnvString("PEERDB_CATALOG_DATABASE", "") } + +// PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD, 0 disables slot lag alerting entirely +func PeerDBSlotLagMBAlertThreshold() uint32 { + return getEnvUint32("PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD", 5000) +} + +// PEERDB_ALERTING_GAP_MINUTES, 0 disables all alerting entirely +func PeerDBAlertingGapMinutesAsDuration() time.Duration { + why := time.Duration(getEnvUint32("PEERDB_ALERTING_GAP_MINUTES", 15)) + return why * time.Minute +} + +// PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD, 0 disables open connections alerting entirely +func PeerDBOpenConnectionsAlertThreshold() uint32 { + return getEnvUint32("PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD", 5) +} + +// PEERDB_ENABLE_WAL_HEARTBEAT +func PeerDBEnableWALHeartbeat() bool { + return getEnvBool("PEERDB_ENABLE_WAL_HEARTBEAT", false) +} diff --git a/flow/peerdbenv/env.go b/flow/peerdbenv/env.go index 11e363d1e..3bba77c46 100644 --- a/flow/peerdbenv/env.go +++ b/flow/peerdbenv/env.go @@ -46,6 +46,23 @@ func getEnvUint32(name string, defaultValue uint32) uint32 { return uint32(i) } +// getEnvBool returns the value of the environment variable with the given name +// or defaultValue if the environment variable is not set or is not a valid +// boolean value. +func getEnvBool(name string, defaultValue bool) bool { + val, ok := getEnv(name) + if !ok { + return defaultValue + } + + b, err := strconv.ParseBool(val) + if err != nil { + return defaultValue + } + + return b +} + // GetEnvString returns the value of the environment variable with the given name // or defaultValue if the environment variable is not set. func getEnvString(name string, defaultValue string) string { diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go new file mode 100644 index 000000000..50608087d --- /dev/null +++ b/flow/shared/alerting/alerting.go @@ -0,0 +1,125 @@ +package alerting + +import ( + "context" + "encoding/json" + "fmt" + "log/slog" + "time" + + "github.com/PeerDB-io/peer-flow/peerdbenv" + "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgxpool" +) + +// alerting service, no cool name :( +type Alerter struct { + catalogPool *pgxpool.Pool + logger *slog.Logger +} + +func registerSendersFromPool(ctx context.Context, catalogPool *pgxpool.Pool) ([]*slackAlertSender, error) { + rows, err := catalogPool.Query(ctx, + "SELECT service_type,service_config FROM peerdb_stats.alerting_config") + if err != nil { + return nil, fmt.Errorf("failed to read alerter config from catalog: %w", err) + } + + var slackAlertSenders []*slackAlertSender + var serviceType, serviceConfig string + _, err = pgx.ForEachRow(rows, []any{&serviceType, &serviceConfig}, func() error { + switch serviceType { + case "slack": + var slackServiceConfig slackAlertConfig + err = json.Unmarshal([]byte(serviceConfig), &slackServiceConfig) + if err != nil { + return fmt.Errorf("failed to unmarshal Slack service config: %w", err) + } + + slackAlertSenders = append(slackAlertSenders, newSlackAlertSender(&slackServiceConfig)) + default: + return fmt.Errorf("unknown service type: %s", serviceType) + } + return nil + }) + + return slackAlertSenders, nil +} + +// doesn't take care of closing pool, needs to be done externally. +func NewAlerter(catalogPool *pgxpool.Pool) (*Alerter, error) { + logger := slog.Default() + if catalogPool == nil { + logger.Error("catalog pool is nil for Alerter") + return nil, fmt.Errorf("catalog pool is nil for Alerter") + } + + return &Alerter{ + catalogPool: catalogPool, + logger: logger, + }, nil +} + +// Only raises an alert if another alert with the same key hasn't been raised +// in the past X minutes, where X is configurable and defaults to 15 minutes +func (a *Alerter) AlertIf(ctx context.Context, alertKey string, alertMessage string) { + if peerdbenv.PeerDBAlertingGapMinutesAsDuration() == 0 { + a.logger.WarnContext(ctx, "Alerting disabled via environment variable, returning") + return + } + + var err error + row := a.catalogPool.QueryRow(ctx, + `SELECT created_timestamp FROM peerdb_stats.alerts_v1 WHERE alert_key=$1 + ORDER BY created_timestamp DESC LIMIT 1`, + alertKey) + var createdTimestamp time.Time + err = row.Scan(&createdTimestamp) + if err != nil && err != pgx.ErrNoRows { + a.logger.Warn("failed to send alert: %v", err) + return + } + + if time.Since(createdTimestamp) >= peerdbenv.PeerDBAlertingGapMinutesAsDuration() { + a.AddAlertToCatalog(ctx, alertKey, alertMessage) + a.AlertToSlack(ctx, alertKey, alertMessage) + } +} + +func (a *Alerter) AlertToSlack(ctx context.Context, alertKey string, alertMessage string) { + slackAlertSenders, err := registerSendersFromPool(ctx, a.catalogPool) + if err != nil { + a.logger.WarnContext(ctx, "failed to set Slack senders", slog.Any("error", err)) + return + } + + for _, slackAlertSender := range slackAlertSenders { + err = slackAlertSender.sendAlert(ctx, + fmt.Sprintf(":rotating_light:Alert:rotating_light:: %s", alertKey), alertMessage) + if err != nil { + a.logger.WarnContext(ctx, "failed to send alert", slog.Any("error", err)) + return + } + } +} + +func (a *Alerter) AddAlertToCatalog(ctx context.Context, alertKey string, alertMessage string) { + _, err := a.catalogPool.Exec(ctx, + "INSERT INTO peerdb_stats.alerts_v1(alert_key,alert_message) VALUES($1,$2)", + alertKey, alertMessage) + if err != nil { + a.logger.WarnContext(ctx, "failed to insert alert", slog.Any("error", err)) + return + } +} + +func (a *Alerter) LogFlowError(ctx context.Context, flowName string, err error) { + errorWithStack := fmt.Sprintf("%+v", err) + _, err = a.catalogPool.Exec(ctx, + "INSERT INTO peerdb_stats.flow_errors(flow_name,error_message,error_type) VALUES($1,$2,$3)", + flowName, errorWithStack, "error") + if err != nil { + a.logger.WarnContext(ctx, "failed to insert flow error", slog.Any("error", err)) + return + } +} diff --git a/flow/shared/alerting/slack_alert_sender.go b/flow/shared/alerting/slack_alert_sender.go new file mode 100644 index 000000000..04c9a4ad8 --- /dev/null +++ b/flow/shared/alerting/slack_alert_sender.go @@ -0,0 +1,38 @@ +package alerting + +import ( + "context" + "fmt" + + "github.com/slack-go/slack" +) + +type slackAlertSender struct { + client *slack.Client + channelIDs []string +} + +type slackAlertConfig struct { + AuthToken string `json:"auth_token"` + ChannelIDs []string `json:"channel_ids"` +} + +func newSlackAlertSender(config *slackAlertConfig) *slackAlertSender { + return &slackAlertSender{ + client: slack.New(config.AuthToken), + channelIDs: config.ChannelIDs, + } +} + +func (s *slackAlertSender) sendAlert(ctx context.Context, alertTitle string, alertMessage string) error { + for _, channelID := range s.channelIDs { + _, _, _, err := s.client.SendMessageContext(ctx, channelID, slack.MsgOptionBlocks( + slack.NewHeaderBlock(slack.NewTextBlockObject("plain_text", alertTitle, true, false)), + slack.NewSectionBlock(slack.NewTextBlockObject("mrkdwn", alertMessage, false, false), nil, nil), + )) + if err != nil { + return fmt.Errorf("failed to send message to Slack channel %s: %w", channelID, err) + } + } + return nil +} diff --git a/flow/shared/constants.go b/flow/shared/constants.go index 8589b5548..aa447520b 100644 --- a/flow/shared/constants.go +++ b/flow/shared/constants.go @@ -51,7 +51,7 @@ func GetPeerFlowTaskQueueName(taskQueueID TaskQueueID) (string, error) { } func prependUIDToTaskQueueName(taskQueueName string) string { - deploymentUID := peerdbenv.GetPeerDBDeploymentUID() + deploymentUID := peerdbenv.PeerDBDeploymentUID() if deploymentUID == "" { return taskQueueName } diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index a2cb9a388..b73f6945f 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -25,10 +25,6 @@ type CDCFlowLimits struct { // If 0, the number of sync flows will be continuously executed until the peer flow is cancelled. // This is typically non-zero for testing purposes. TotalSyncFlows int - // Number of normalize flows to execute in total. - // If 0, the number of sync flows will be continuously executed until the peer flow is cancelled. - // This is typically non-zero for testing purposes. - TotalNormalizeFlows int // Maximum number of rows in a sync flow batch. MaxBatchSize int // Rows synced after which we can say a test is done. @@ -160,6 +156,7 @@ func CDCFlowWorkflowWithConfig( return nil, fmt.Errorf("invalid connection configs") } + ctx = workflow.WithValue(ctx, "flowName", cfg.FlowJobName) w := NewCDCFlowWorkflowExecution(ctx) if limits.TotalSyncFlows == 0 { @@ -174,6 +171,10 @@ func CDCFlowWorkflowWithConfig( return state, fmt.Errorf("failed to set `%s` query handler: %w", CDCFlowStatusQuery, err) } + mirrorNameSearch := map[string]interface{}{ + shared.MirrorNameSearchAttribute: cfg.FlowJobName, + } + // we cannot skip SetupFlow if SnapshotFlow did not complete in cases where Resync is enabled // because Resync modifies TableMappings before Setup and also before Snapshot // for safety, rely on the idempotency of SetupFlow instead @@ -189,10 +190,6 @@ func CDCFlowWorkflowWithConfig( } } - mirrorNameSearch := map[string]interface{}{ - shared.MirrorNameSearchAttribute: cfg.FlowJobName, - } - // start the SetupFlow workflow as a child workflow, and wait for it to complete // it should return the table schema for the source peer setupFlowID, err := GetChildWorkflowID(ctx, "setup-flow", cfg.FlowJobName) @@ -208,7 +205,6 @@ func CDCFlowWorkflowWithConfig( SearchAttributes: mirrorNameSearch, } setupFlowCtx := workflow.WithChildOptions(ctx, childSetupFlowOpts) - setupFlowCtx = workflow.WithValue(setupFlowCtx, "flowName", cfg.FlowJobName) setupFlowFuture := workflow.ExecuteChildWorkflow(setupFlowCtx, SetupFlowWorkflow, cfg) if err := setupFlowFuture.Get(setupFlowCtx, &cfg); err != nil { return state, fmt.Errorf("failed to execute child workflow: %w", err) @@ -236,7 +232,6 @@ func CDCFlowWorkflowWithConfig( SearchAttributes: mirrorNameSearch, } snapshotFlowCtx := workflow.WithChildOptions(ctx, childSnapshotFlowOpts) - snapshotFlowCtx = workflow.WithValue(snapshotFlowCtx, "flowName", cfg.FlowJobName) snapshotFlowFuture := workflow.ExecuteChildWorkflow(snapshotFlowCtx, SnapshotFlowWorkflow, cfg) if err := snapshotFlowFuture.Get(snapshotFlowCtx, nil); err != nil { return state, fmt.Errorf("failed to execute child workflow: %w", err) @@ -270,7 +265,6 @@ func CDCFlowWorkflowWithConfig( StartToCloseTimeout: 12 * time.Hour, HeartbeatTimeout: 1 * time.Hour, }) - renameTablesCtx = workflow.WithValue(renameTablesCtx, "flowName", cfg.FlowJobName) renameTablesFuture := workflow.ExecuteActivity(renameTablesCtx, flowable.RenameTables, renameOpts) if err := renameTablesFuture.Get(renameTablesCtx, nil); err != nil { return state, fmt.Errorf("failed to execute rename tables activity: %w", err) @@ -279,6 +273,10 @@ func CDCFlowWorkflowWithConfig( state.SnapshotComplete = true state.Progress = append(state.Progress, "executed setup flow and snapshot flow") + // if initial_copy_only is opted for, we end the flow here. + if cfg.InitialCopyOnly { + return nil, nil + } } syncFlowOptions := &protos.SyncFlowOptions{ @@ -293,7 +291,10 @@ func CDCFlowWorkflowWithConfig( c.Receive(ctx, &batchSize) w.logger.Info("received batch size signal: ", batchSize) syncFlowOptions.BatchSize = batchSize + cfg.MaxBatchSize = uint32(batchSize) + limits.MaxBatchSize = int(batchSize) }) + batchSizeSelector.AddDefault(func() { w.logger.Info("no batch size signal received, batch size remains: ", syncFlowOptions.BatchSize) @@ -347,9 +348,6 @@ func CDCFlowWorkflowWithConfig( return state, err } - mirrorNameSearch := map[string]interface{}{ - shared.MirrorNameSearchAttribute: cfg.FlowJobName, - } // execute the sync flow as a child workflow childSyncFlowOpts := workflow.ChildWorkflowOptions{ WorkflowID: syncFlowID, @@ -359,18 +357,17 @@ func CDCFlowWorkflowWithConfig( }, SearchAttributes: mirrorNameSearch, } - ctx = workflow.WithChildOptions(ctx, childSyncFlowOpts) - ctx = workflow.WithValue(ctx, "flowName", cfg.FlowJobName) + syncCtx := workflow.WithChildOptions(ctx, childSyncFlowOpts) syncFlowOptions.RelationMessageMapping = *state.RelationMessageMapping childSyncFlowFuture := workflow.ExecuteChildWorkflow( - ctx, + syncCtx, SyncFlowWorkflow, cfg, syncFlowOptions, ) var childSyncFlowRes *model.SyncResponse - if err := childSyncFlowFuture.Get(ctx, &childSyncFlowRes); err != nil { + if err := childSyncFlowFuture.Get(syncCtx, &childSyncFlowRes); err != nil { w.logger.Error("failed to execute sync flow: ", err) state.SyncFlowErrors = append(state.SyncFlowErrors, err.Error()) } else { @@ -383,20 +380,6 @@ func CDCFlowWorkflowWithConfig( w.logger.Info("Total records synced: ", totalRecordsSynced) - normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) - if err != nil { - return state, err - } - - childNormalizeFlowOpts := workflow.ChildWorkflowOptions{ - WorkflowID: normalizeFlowID, - ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, - RetryPolicy: &temporal.RetryPolicy{ - MaximumAttempts: 20, - }, - SearchAttributes: mirrorNameSearch, - } - ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) var tableSchemaDeltas []*protos.TableSchemaDelta = nil if childSyncFlowRes != nil { tableSchemaDeltas = childSyncFlowRes.TableSchemaDeltas @@ -415,7 +398,6 @@ func CDCFlowWorkflowWithConfig( getModifiedSchemaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 5 * time.Minute, }) - getModifiedSchemaCtx = workflow.WithValue(getModifiedSchemaCtx, "flowName", cfg.FlowJobName) getModifiedSchemaFuture := workflow.ExecuteActivity(getModifiedSchemaCtx, flowable.GetTableSchema, &protos.GetTableSchemaBatchInput{ PeerConnectionConfig: cfg.Source, @@ -432,24 +414,34 @@ func CDCFlowWorkflowWithConfig( } } } - ctx = workflow.WithValue(ctx, "flowName", cfg.FlowJobName) + + normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) + if err != nil { + return state, err + } + + childNormalizeFlowOpts := workflow.ChildWorkflowOptions{ + WorkflowID: normalizeFlowID, + ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, + RetryPolicy: &temporal.RetryPolicy{ + MaximumAttempts: 20, + }, + SearchAttributes: mirrorNameSearch, + } + normCtx := workflow.WithChildOptions(ctx, childNormalizeFlowOpts) childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( - ctx, + normCtx, NormalizeFlowWorkflow, cfg, ) - selector := workflow.NewSelector(ctx) - selector.AddFuture(childNormalizeFlowFuture, func(f workflow.Future) { - var childNormalizeFlowRes *model.NormalizeResponse - if err := f.Get(ctx, &childNormalizeFlowRes); err != nil { - w.logger.Error("failed to execute normalize flow: ", err) - state.NormalizeFlowErrors = append(state.NormalizeFlowErrors, err.Error()) - } else { - state.NormalizeFlowStatuses = append(state.NormalizeFlowStatuses, childNormalizeFlowRes) - } - }) - selector.Select(ctx) + var childNormalizeFlowRes *model.NormalizeResponse + if err := childNormalizeFlowFuture.Get(normCtx, &childNormalizeFlowRes); err != nil { + w.logger.Error("failed to execute normalize flow: ", err) + state.NormalizeFlowErrors = append(state.NormalizeFlowErrors, err.Error()) + } else { + state.NormalizeFlowStatuses = append(state.NormalizeFlowStatuses, childNormalizeFlowRes) + } batchSizeSelector.Select(ctx) } diff --git a/flow/workflows/normalize_flow.go b/flow/workflows/normalize_flow.go new file mode 100644 index 000000000..39256eac1 --- /dev/null +++ b/flow/workflows/normalize_flow.go @@ -0,0 +1,65 @@ +package peerflow + +import ( + "fmt" + "time" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "github.com/PeerDB-io/peer-flow/model" + "go.temporal.io/sdk/log" + "go.temporal.io/sdk/workflow" +) + +type NormalizeFlowState struct { + CDCFlowName string + Progress []string +} + +type NormalizeFlowExecution struct { + NormalizeFlowState + executionID string + logger log.Logger +} + +func NewNormalizeFlowExecution(ctx workflow.Context, state *NormalizeFlowState) *NormalizeFlowExecution { + return &NormalizeFlowExecution{ + NormalizeFlowState: *state, + executionID: workflow.GetInfo(ctx).WorkflowExecution.ID, + logger: workflow.GetLogger(ctx), + } +} + +func NormalizeFlowWorkflow(ctx workflow.Context, + config *protos.FlowConnectionConfigs, +) (*model.NormalizeResponse, error) { + s := NewNormalizeFlowExecution(ctx, &NormalizeFlowState{ + CDCFlowName: config.FlowJobName, + Progress: []string{}, + }) + + return s.executeNormalizeFlow(ctx, config) +} + +func (s *NormalizeFlowExecution) executeNormalizeFlow( + ctx workflow.Context, + config *protos.FlowConnectionConfigs, +) (*model.NormalizeResponse, error) { + s.logger.Info("executing normalize flow - ", s.CDCFlowName) + + normalizeFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 7 * 24 * time.Hour, + HeartbeatTimeout: 5 * time.Minute, + }) + + startNormalizeInput := &protos.StartNormalizeInput{ + FlowConnectionConfigs: config, + } + fStartNormalize := workflow.ExecuteActivity(normalizeFlowCtx, flowable.StartNormalize, startNormalizeInput) + + var normalizeResponse *model.NormalizeResponse + if err := fStartNormalize.Get(normalizeFlowCtx, &normalizeResponse); err != nil { + return nil, fmt.Errorf("failed to flow: %w", err) + } + + return normalizeResponse, nil +} diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 2373427c8..644f61a61 100644 --- a/flow/workflows/qrep_flow.go +++ b/flow/workflows/qrep_flow.go @@ -125,6 +125,7 @@ func (q *QRepFlowExecution) SetupWatermarkTableOnDestination(ctx workflow.Contex TableNameSchemaMapping: map[string]*protos.TableSchema{ q.config.DestinationTableIdentifier: tblSchemaOutput.TableNameSchemaMapping[q.config.WatermarkTable], }, + SyncedAtColName: q.config.SyncedAtColName, } future = workflow.ExecuteActivity(ctx, flowable.CreateNormalizedTable, setupConfig) diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index ab8168191..03ed6dd3c 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -273,7 +273,7 @@ func (s *SetupFlowExecution) executeSetupFlow( func SetupFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionConfigs, ) (*protos.FlowConnectionConfigs, error) { - tblNameMapping := make(map[string]string) + tblNameMapping := make(map[string]string, len(config.TableMappings)) for _, v := range config.TableMappings { tblNameMapping[v.SourceTableIdentifier] = v.DestinationTableIdentifier } diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 527fde572..998317c52 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -176,6 +176,8 @@ func (s *SnapshotFlowExecution) cloneTable( NumRowsPerPartition: numRowsPerPartition, MaxParallelWorkers: numWorkers, StagingPath: s.config.SnapshotStagingPath, + SyncedAtColName: s.config.SyncedAtColName, + SoftDeleteColName: s.config.SoftDeleteColName, WriteMode: &protos.QRepWriteMode{ WriteType: protos.QRepWriteType_QREP_WRITE_MODE_APPEND, }, diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 0b207bc65..3ee45aecf 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -21,17 +21,6 @@ type SyncFlowExecution struct { logger log.Logger } -type NormalizeFlowState struct { - CDCFlowName string - Progress []string -} - -type NormalizeFlowExecution struct { - NormalizeFlowState - executionID string - logger log.Logger -} - // NewSyncFlowExecution creates a new instance of SyncFlowExecution. func NewSyncFlowExecution(ctx workflow.Context, state *SyncFlowState) *SyncFlowExecution { return &SyncFlowExecution{ @@ -41,14 +30,6 @@ func NewSyncFlowExecution(ctx workflow.Context, state *SyncFlowState) *SyncFlowE } } -func NewNormalizeFlowExecution(ctx workflow.Context, state *NormalizeFlowState) *NormalizeFlowExecution { - return &NormalizeFlowExecution{ - NormalizeFlowState: *state, - executionID: workflow.GetInfo(ctx).WorkflowExecution.ID, - logger: workflow.GetLogger(ctx), - } -} - // executeSyncFlow executes the sync flow. func (s *SyncFlowExecution) executeSyncFlow( ctx workflow.Context, @@ -131,39 +112,3 @@ func SyncFlowWorkflow(ctx workflow.Context, return s.executeSyncFlow(ctx, config, options, options.RelationMessageMapping) } - -func NormalizeFlowWorkflow(ctx workflow.Context, - config *protos.FlowConnectionConfigs, -) (*model.NormalizeResponse, error) { - s := NewNormalizeFlowExecution(ctx, &NormalizeFlowState{ - CDCFlowName: config.FlowJobName, - Progress: []string{}, - }) - - return s.executeNormalizeFlow(ctx, config) -} - -func (s *NormalizeFlowExecution) executeNormalizeFlow( - ctx workflow.Context, - config *protos.FlowConnectionConfigs, -) (*model.NormalizeResponse, error) { - s.logger.Info("executing normalize flow - ", s.CDCFlowName) - - normalizeFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ - StartToCloseTimeout: 7 * 24 * time.Hour, - HeartbeatTimeout: 5 * time.Minute, - }) - - // execute StartFlow on the peers to start the flow - startNormalizeInput := &protos.StartNormalizeInput{ - FlowConnectionConfigs: config, - } - fStartNormalize := workflow.ExecuteActivity(normalizeFlowCtx, flowable.StartNormalize, startNormalizeInput) - - var normalizeResponse *model.NormalizeResponse - if err := fStartNormalize.Get(normalizeFlowCtx, &normalizeResponse); err != nil { - return nil, fmt.Errorf("failed to flow: %w", err) - } - - return normalizeResponse, nil -} diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 3c243f295..09159e4d2 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -127,9 +127,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.75" +version = "1.0.76" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" +checksum = "59d2a3357dde987206219e78ecfbbb6e8dad06cbb65292758d3270e6254f7355" [[package]] name = "ar" @@ -170,7 +170,7 @@ checksum = "5fd55a5ba1179988837d24ab4c7cc8ed6efdeff578ede0416b4225a5fca35bd0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -192,18 +192,18 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] name = "async-trait" -version = "0.1.74" +version = "0.1.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" +checksum = "fdf6721fb0140e4f897002dd086c06f6c27775df19cfe1fccb21181a48fd2c98" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -362,7 +362,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", "syn_derive", ] @@ -548,7 +548,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -642,9 +642,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.9" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14c3242926edf34aec4ac3a77108ad4854bffaa2e4ddc1824124ce59231302d5" +checksum = "82a9b73a36529d9c47029b9fb3a6f0ea3cc916a261195352ba19e770fc1748b2" dependencies = [ "cfg-if", "crossbeam-utils", @@ -663,21 +663,20 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.16" +version = "0.9.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d2fe95351b870527a5d09bf563ed3c97c0cffb87cf1c78a591bf48bb218d9aa" +checksum = "0e3681d554572a651dda4186cd47240627c3d0114d45a95f6ad27f2f22e7548d" dependencies = [ "autocfg", "cfg-if", "crossbeam-utils", - "memoffset", ] [[package]] name = "crossbeam-utils" -version = "0.8.17" +version = "0.8.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c06d96137f14f244c37f989d9fff8f95e6c18b918e71f36638f8c49112e4c78f" +checksum = "c3a430a770ebd84726f584a90ee7f020d28db52c6d02138900f22341f866d39c" dependencies = [ "cfg-if", ] @@ -767,7 +766,7 @@ checksum = "d150dea618e920167e5973d70ae6ece4385b7164e0d799fe7c122dd0a5d912ad" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -782,12 +781,6 @@ dependencies = [ "subtle", ] -[[package]] -name = "doc-comment" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" - [[package]] name = "dotenvy" version = "0.15.7" @@ -897,11 +890,7 @@ dependencies = [ "anyhow", "catalog", "pt", - "reqwest", - "serde", "serde_json", - "serde_yaml", - "tokio", "tonic", "tonic-health", "tracing", @@ -945,9 +934,9 @@ checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" [[package]] name = "futures" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" +checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" dependencies = [ "futures-channel", "futures-core", @@ -960,9 +949,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" +checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" dependencies = [ "futures-core", "futures-sink", @@ -970,15 +959,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" +checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" [[package]] name = "futures-executor" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" +checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" dependencies = [ "futures-core", "futures-task", @@ -987,38 +976,38 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" +checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" [[package]] name = "futures-macro" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" +checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] name = "futures-sink" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" +checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" [[package]] name = "futures-task" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" +checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" [[package]] name = "futures-util" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" +checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" dependencies = [ "futures-channel", "futures-core", @@ -1286,19 +1275,6 @@ dependencies = [ "tokio-io-timeout", ] -[[package]] -name = "hyper-tls" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" -dependencies = [ - "bytes", - "hyper", - "native-tls", - "tokio", - "tokio-native-tls", -] - [[package]] name = "iana-time-zone" version = "0.1.58" @@ -1527,15 +1503,6 @@ version = "2.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" -[[package]] -name = "memoffset" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" -dependencies = [ - "autocfg", -] - [[package]] name = "mime" version = "0.3.17" @@ -1574,24 +1541,6 @@ version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" -[[package]] -name = "native-tls" -version = "0.2.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" -dependencies = [ - "lazy_static", - "libc", - "log", - "openssl", - "openssl-probe", - "openssl-sys", - "schannel", - "security-framework", - "security-framework-sys", - "tempfile", -] - [[package]] name = "nom" version = "7.1.3" @@ -1692,9 +1641,9 @@ dependencies = [ [[package]] name = "object" -version = "0.32.1" +version = "0.32.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" dependencies = [ "memchr", ] @@ -1707,9 +1656,9 @@ checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" [[package]] name = "openssl" -version = "0.10.61" +version = "0.10.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b8419dc8cc6d866deb801274bba2e6f8f6108c1bb7fcc10ee5ab864931dbb45" +checksum = "8cde4d2d9200ad5909f8dac647e29482e07c3a35de8a13fce7c9c7747ad9f671" dependencies = [ "bitflags 2.4.1", "cfg-if", @@ -1728,7 +1677,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -1737,24 +1686,14 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" -[[package]] -name = "openssl-src" -version = "300.2.1+3.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fe476c29791a5ca0d1273c697e96085bbabbbea2ef7afd5617e78a4b40332d3" -dependencies = [ - "cc", -] - [[package]] name = "openssl-sys" -version = "0.9.97" +version = "0.9.98" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3eaad34cdd97d81de97964fc7f29e2d104f483840d906ef56daa1912338460b" +checksum = "c1665caf8ab2dc9aef43d1c0023bd904633a6a05cb30b0ad59bec2ae986e57a7" dependencies = [ "cc", "libc", - "openssl-src", "pkg-config", "vcpkg", ] @@ -1847,7 +1786,6 @@ dependencies = [ "gcp-bigquery-client", "peer-connections", "peer-cursor", - "pgerror", "pgwire", "pt", "rust_decimal", @@ -1882,7 +1820,6 @@ dependencies = [ "anyhow", "async-trait", "futures", - "pgerror", "pgwire", "sqlparser", "tokio", @@ -1900,7 +1837,6 @@ dependencies = [ "futures", "peer-connections", "peer-cursor", - "pgerror", "pgwire", "postgres-connection", "postgres-inet", @@ -1932,10 +1868,7 @@ dependencies = [ "hex", "jsonwebtoken", "peer-cursor", - "pgerror", "pgwire", - "pkcs1", - "pkcs8", "pt", "reqwest", "rsa", @@ -1989,7 +1922,6 @@ dependencies = [ "peer-postgres", "peer-snowflake", "peerdb-parser", - "pgerror", "pgwire", "postgres", "prost", @@ -2041,18 +1973,11 @@ dependencies = [ "indexmap 2.1.0", ] -[[package]] -name = "pgerror" -version = "0.1.0" -dependencies = [ - "snafu", -] - [[package]] name = "pgwire" -version = "0.17.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f7f181d085a224ff2b2ea46bd2066b487b87e83dabbcdfe60bf3f027f5d0593" +checksum = "3b277432819ee6b76bf56de5e91eae578d6b332bd6f05f963ee81fc788bc886f" dependencies = [ "async-trait", "base64 0.21.5", @@ -2111,7 +2036,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2166,9 +2091,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.27" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" +checksum = "69d3587f8a9e599cc7ec2c00e331f71c4e69a5f9a4b8a6efd5b07466b9736f9a" [[package]] name = "postgres" @@ -2274,7 +2199,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ae005bd773ab59b4725093fd7df83fd7892f7d8eafb48dbd7de6e024e4215f9d" dependencies = [ "proc-macro2", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2313,9 +2238,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.70" +version = "1.0.71" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39278fbbf5fb4f646ce651690877f89d1c5811a3d4acb27700c1cb3cdb78fd3b" +checksum = "75cb1540fadbd5b8fbccc4dddad2734eba435053f725621c070711a14bb5f4b8" dependencies = [ "unicode-ident", ] @@ -2347,7 +2272,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.41", + "syn 2.0.43", "tempfile", "which", ] @@ -2362,7 +2287,7 @@ dependencies = [ "itertools 0.11.0", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2532,7 +2457,7 @@ dependencies = [ "quote", "refinery-core", "regex", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2590,9 +2515,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.22" +version = "0.11.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" +checksum = "37b1ae8d9ac08420c66222fb9096fc5de435c3c48542bc5336c51892cffafb41" dependencies = [ "async-compression", "base64 0.21.5", @@ -2605,12 +2530,10 @@ dependencies = [ "http-body", "hyper", "hyper-rustls", - "hyper-tls", "ipnet", "js-sys", "log", "mime", - "native-tls", "once_cell", "percent-encoding", "pin-project-lite", @@ -2621,7 +2544,6 @@ dependencies = [ "serde_urlencoded", "system-configuration", "tokio", - "tokio-native-tls", "tokio-rustls 0.24.1", "tokio-util", "tower-service", @@ -2782,9 +2704,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.0.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7673e0aa20ee4937c6aacfc12bb8341cfbf054cdd21df6bec5fd0629fe9339b" +checksum = "9e9d979b3ce68192e42760c7810125eb6cf2ea10efae545a156063e61f314e2a" [[package]] name = "rustls-webpki" @@ -2937,7 +2859,7 @@ checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2953,9 +2875,9 @@ dependencies = [ [[package]] name = "serde_spanned" -version = "0.6.4" +version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12022b835073e5b11e90a14f86838ceb1c8fb0325b72416845c487ac0fa95e80" +checksum = "eb3622f419d1296904700073ea6cc23ad690adbd66f13ea683df73298736f0c1" dependencies = [ "serde", ] @@ -2972,19 +2894,6 @@ dependencies = [ "serde", ] -[[package]] -name = "serde_yaml" -version = "0.9.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cc7a1570e38322cfe4154732e5110f887ea57e22b76f4bfd32b5bdd3368666c" -dependencies = [ - "indexmap 2.1.0", - "itoa", - "ryu", - "serde", - "unsafe-libyaml", -] - [[package]] name = "sha2" version = "0.10.8" @@ -3088,29 +2997,6 @@ version = "1.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4dccd0940a2dcdf68d092b8cbab7dc0ad8fa938bf95787e1b916b0e3d0e8e970" -[[package]] -name = "snafu" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" -dependencies = [ - "backtrace", - "doc-comment", - "snafu-derive", -] - -[[package]] -name = "snafu-derive" -version = "0.7.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "syn 1.0.109", -] - [[package]] name = "socket2" version = "0.5.5" @@ -3159,7 +3045,7 @@ source = "git+https://github.com/peerdb-io/sqlparser-rs.git#e2a651ace85013fca48b dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3198,9 +3084,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.41" +version = "2.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44c8b28c477cc3bf0e7966561e3460130e1255f7a1cf71931075f1c5e7a7e269" +checksum = "ee659fb5f3d355364e1f3e5bc10fb82068efbf824a1e9d1c9504244a6469ad53" dependencies = [ "proc-macro2", "quote", @@ -3216,7 +3102,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3302,7 +3188,7 @@ checksum = "01742297787513b79cf8e29d1056ede1313e2420b7b3b15d0a768b4921f549df" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3317,9 +3203,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" +checksum = "f657ba42c3f86e7680e53c8cd3af8abbe56b5491790b46e22e19c0d57463583e" dependencies = [ "deranged", "itoa", @@ -3339,9 +3225,9 @@ checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ad70d68dba9e1f8aceda7aa6711965dfec1cac869f311a51bd08b3a2ccbce20" +checksum = "26197e33420244aeb70c3e8c78376ca46571bc4e701e4791c2cd9f57dcb3a43f" dependencies = [ "time-core", ] @@ -3363,9 +3249,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.35.0" +version = "1.35.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "841d45b238a16291a4e1584e61820b8ae57d696cc5015c459c229ccc6990cc1c" +checksum = "c89b4efa943be685f629b149f53829423f8f5531ea21249408e8e2f8671ec104" dependencies = [ "backtrace", "bytes", @@ -3399,17 +3285,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", -] - -[[package]] -name = "tokio-native-tls" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbae76ab933c85776efabc971569dd6119c580d8f5d448769dec1764bf796ef2" -dependencies = [ - "native-tls", - "tokio", + "syn 2.0.43", ] [[package]] @@ -3671,7 +3547,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3758,12 +3634,6 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" -[[package]] -name = "unsafe-libyaml" -version = "0.2.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab4c90930b95a82d00dc9e9ac071b4991924390d46cbd0dfe566148667605e4b" - [[package]] name = "untrusted" version = "0.9.0" @@ -3904,7 +3774,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", "wasm-bindgen-shared", ] @@ -3938,7 +3808,7 @@ checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4161,9 +4031,9 @@ checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" [[package]] name = "winnow" -version = "0.5.28" +version = "0.5.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c830786f7720c2fd27a1a0e27a709dbd3c4d009b56d098fc742d4f4eab91fe2" +checksum = "9b5c3db89721d50d0e2a673f5043fc4722f76dcc352d7b1ab8b8288bed4ed2c5" dependencies = [ "memchr", ] @@ -4228,9 +4098,9 @@ dependencies = [ [[package]] name = "yup-oauth2" -version = "8.3.0" +version = "8.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "364ca376b5c04d9b2be9693054e3e0d2d146b363819d0f9a10c6ee66e4c8406b" +checksum = "24bea7df5a9a74a9a0de92f22e5ab3fb9505dd960c7f1f00de5b7231d9d97206" dependencies = [ "anyhow", "async-trait", @@ -4270,7 +4140,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] diff --git a/nexus/Cargo.toml b/nexus/Cargo.toml index a07c40ea4..2aaa406ac 100644 --- a/nexus/Cargo.toml +++ b/nexus/Cargo.toml @@ -9,7 +9,6 @@ members = [ "peer-cursor", "peer-postgres", "peer-snowflake", - "pgerror", "postgres-connection", "pt", "server", diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index f65fe4b2d..8a2a088f1 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -306,6 +306,11 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { _ => None, }; + let initial_copy_only = match raw_options.remove("initial_copy_only") { + Some(sqlparser::ast::Value::Boolean(b)) => *b, + _ => false, + }; + let flow_job = FlowJob { name: cdc.mirror_name.to_string().to_lowercase(), source_peer: cdc.source_peer.to_string().to_lowercase(), @@ -327,8 +332,13 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { resync, soft_delete_col_name, synced_at_col_name, + initial_copy_only, }; + if initial_copy_only && !do_initial_copy { + anyhow::bail!("initial_copy_only is set to true, but do_initial_copy is set to false"); + } + Ok(Some(PeerDDL::CreateMirrorForCDC { if_not_exists: *if_not_exists, flow_job, diff --git a/nexus/catalog/migrations/V15__schema_deltas_audit_log.sql b/nexus/catalog/migrations/V15__schema_deltas_audit_log.sql new file mode 100644 index 000000000..92fe24cab --- /dev/null +++ b/nexus/catalog/migrations/V15__schema_deltas_audit_log.sql @@ -0,0 +1,8 @@ +CREATE TABLE IF NOT EXISTS peerdb_stats.schema_deltas_audit_log ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + flow_job_name TEXT NOT NULL, + read_timestamp TIMESTAMP DEFAULT now(), + workflow_id TEXT NOT NULL, + run_id TEXT NOT NULL, + delta_info JSONB NOT NULL +); \ No newline at end of file diff --git a/nexus/catalog/migrations/V16__alerting_config_init.sql b/nexus/catalog/migrations/V16__alerting_config_init.sql new file mode 100644 index 000000000..de16cc340 --- /dev/null +++ b/nexus/catalog/migrations/V16__alerting_config_init.sql @@ -0,0 +1,13 @@ +CREATE TABLE IF NOT EXISTS peerdb_stats.alerting_config ( + id BIGINT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, + service_type TEXT NOT NULL CHECK (service_type IN ('slack')), + service_config JSONB NOT NULL +); + +CREATE TABLE IF NOT EXISTS peerdb_stats.alerts_v1 ( + id BIGINT PRIMARY KEY GENERATED BY DEFAULT AS IDENTITY, + alert_key TEXT NOT NULL, + alert_level TEXT NOT NULL CHECK (alert_level IN ('critical')) DEFAULT 'critical', + alert_message TEXT NOT NULL, + created_timestamp TIMESTAMP DEFAULT now() +); \ No newline at end of file diff --git a/nexus/catalog/migrations/V17__mirror_errors.sql b/nexus/catalog/migrations/V17__mirror_errors.sql new file mode 100644 index 000000000..06f2352ea --- /dev/null +++ b/nexus/catalog/migrations/V17__mirror_errors.sql @@ -0,0 +1,10 @@ +CREATE TABLE peerdb_stats.flow_errors ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + flow_name TEXT NOT NULL, + error_message TEXT NOT NULL, + error_type TEXT NOT NULL, + error_timestamp TIMESTAMP NOT NULL DEFAULT now(), + ack BOOLEAN NOT NULL DEFAULT FALSE +); + +CREATE INDEX idx_flow_errors_flow_name ON peerdb_stats.flow_errors (flow_name); diff --git a/nexus/flow-rs/Cargo.toml b/nexus/flow-rs/Cargo.toml index fc989a378..2fd4bb1cf 100644 --- a/nexus/flow-rs/Cargo.toml +++ b/nexus/flow-rs/Cargo.toml @@ -3,18 +3,8 @@ name = "flow-rs" version = "0.1.0" edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - [dependencies] -tokio = { version = "1", features = ["full"] } -serde = { version = "1.0", features = ["derive"] } -serde_yaml = "0.9" serde_json = "1.0" -reqwest = { version = "0.11", features = [ - "blocking", - "json", - "native-tls-vendored", -] } anyhow = "1.0" tracing = "0.1" tonic = "0.10" diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index 7b11a6a2d..7a051e3d2 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -195,6 +195,7 @@ impl FlowGrpcClient { resync: job.resync, soft_delete_col_name: job.soft_delete_col_name.clone().unwrap_or_default(), synced_at_col_name: job.synced_at_col_name.clone().unwrap_or_default(), + initial_copy_only: job.initial_copy_only, ..Default::default() }; diff --git a/nexus/parser/Cargo.toml b/nexus/parser/Cargo.toml index 2f4c1698d..0b1eae167 100644 --- a/nexus/parser/Cargo.toml +++ b/nexus/parser/Cargo.toml @@ -10,7 +10,7 @@ anyhow = "1" async-trait = "0.1" catalog = { path = "../catalog" } futures = { version = "0.3.28", features = ["executor"] } -pgwire = "0.17" +pgwire = "0.18" pt = { path = "../pt" } rand = "0.8" sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git" } diff --git a/nexus/peer-bigquery/Cargo.toml b/nexus/peer-bigquery/Cargo.toml index ff6a6c4e9..a7b570f9d 100644 --- a/nexus/peer-bigquery/Cargo.toml +++ b/nexus/peer-bigquery/Cargo.toml @@ -13,8 +13,7 @@ dashmap = "5.0" futures = { version = "0.3.28", features = ["executor"] } peer-cursor = { path = "../peer-cursor" } peer-connections = { path = "../peer-connections" } -pgerror = { path = "../pgerror" } -pgwire = "0.17" +pgwire = "0.18" pt = { path = "../pt" } rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } serde = { version = "1.0", features = ["derive"] } diff --git a/nexus/peer-bigquery/src/lib.rs b/nexus/peer-bigquery/src/lib.rs index c338518f4..29d58fb24 100644 --- a/nexus/peer-bigquery/src/lib.rs +++ b/nexus/peer-bigquery/src/lib.rs @@ -8,7 +8,6 @@ use gcp_bigquery_client::{ }; use peer_connections::PeerConnectionTracker; use peer_cursor::{CursorModification, QueryExecutor, QueryOutput, SchemaRef}; -use pgerror::PgError; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; use pt::peerdb_peers::BigqueryConfig; use sqlparser::ast::{CloseCursor, Expr, FetchDirection, Statement, Value}; @@ -76,9 +75,7 @@ impl BigQueryQueryExecutor { .await .map_err(|err| { tracing::error!("error tracking query: {}", err); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) + PgWireError::ApiError(err.into()) })?; let result_set = self @@ -88,16 +85,12 @@ impl BigQueryQueryExecutor { .await .map_err(|err| { tracing::error!("error running query: {}", err); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) + PgWireError::ApiError(err.into()) })?; token.end().await.map_err(|err| { tracing::error!("error closing tracking token: {}", err); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) + PgWireError::ApiError(err.into()) })?; Ok(result_set) @@ -116,11 +109,7 @@ impl QueryExecutor for BigQueryQueryExecutor { bq_ast .rewrite(&self.dataset_id, &mut query) .context("unable to rewrite query") - .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) - })?; + .map_err(|err| PgWireError::ApiError(err.into()))?; let query = query.to_string(); tracing::info!("bq rewritten query: {}", query); @@ -170,11 +159,7 @@ impl QueryExecutor for BigQueryQueryExecutor { // If parsing the count resulted in an error, return an internal error let count = match count { Ok(c) => c, - Err(err) => { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - }))) - } + Err(err) => return Err(PgWireError::ApiError(err.into())), }; tracing::info!("fetching {} rows", count); @@ -226,11 +211,7 @@ impl QueryExecutor for BigQueryQueryExecutor { bq_ast .rewrite(&self.dataset_id, &mut query) .context("unable to rewrite query") - .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) - })?; + .map_err(|err| PgWireError::ApiError(err.into()))?; // add LIMIT 0 to the root level query. // this is a workaround for the bigquery API not supporting DESCRIBE diff --git a/nexus/peer-bigquery/src/stream.rs b/nexus/peer-bigquery/src/stream.rs index 53e85c021..a831f6818 100644 --- a/nexus/peer-bigquery/src/stream.rs +++ b/nexus/peer-bigquery/src/stream.rs @@ -10,7 +10,6 @@ use gcp_bigquery_client::model::{ field_type::FieldType, query_response::ResultSet, table_field_schema::TableFieldSchema, }; use peer_cursor::{Record, RecordStream, Schema, SchemaRef}; -use pgerror::PgError; use pgwire::{ api::{ results::{FieldFormat, FieldInfo}, @@ -181,19 +180,13 @@ impl BqRecordStream { impl Stream for BqRecordStream { type Item = PgWireResult; - fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - let this = self.get_mut(); - match this.result_set.next_row() { - true => { - let record = this.convert_result_set_item(&this.result_set); - let result = record.map_err(|e| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error getting curent row: {}", e), - })) - }); - Poll::Ready(Some(result)) - } - false => Poll::Ready(None), + fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + if self.result_set.next_row() { + let record = self.convert_result_set_item(&self.result_set); + let result = record.map_err(|e| PgWireError::ApiError(e.into())); + Poll::Ready(Some(result)) + } else { + Poll::Ready(None) } } } diff --git a/nexus/peer-cursor/Cargo.toml b/nexus/peer-cursor/Cargo.toml index 921d0d52a..7623f8f00 100644 --- a/nexus/peer-cursor/Cargo.toml +++ b/nexus/peer-cursor/Cargo.toml @@ -9,8 +9,7 @@ edition = "2021" anyhow = "1.0" async-trait = "0.1" futures = "0.3" -pgerror = { path = "../pgerror" } -pgwire = "0.17" +pgwire = "0.18" sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git" } tokio = { version = "1.0", features = ["full"] } value = { path = "../value" } diff --git a/nexus/peer-cursor/src/util.rs b/nexus/peer-cursor/src/util.rs index e87478d67..e9b9d55b0 100644 --- a/nexus/peer-cursor/src/util.rs +++ b/nexus/peer-cursor/src/util.rs @@ -1,7 +1,6 @@ use std::sync::Arc; use futures::{stream, StreamExt}; -use pgerror::PgError; use pgwire::{ api::results::{DataRowEncoder, FieldInfo, QueryResponse, Response}, error::{PgWireError, PgWireResult}, @@ -48,14 +47,13 @@ fn encode_value(value: &Value, builder: &mut DataRowEncoder) -> PgWireResult<()> let s = u.to_string(); builder.encode_field(&s) } - Value::Enum(_) | Value::Hstore(_) => { - Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "cannot write value {:?} in postgres protocol: unimplemented", - &value - ), - }))) - } + Value::Enum(_) | Value::Hstore(_) => Err(PgWireError::ApiError( + format!( + "cannot write value {:?} in postgres protocol: unimplemented", + &value + ) + .into(), + )), } } diff --git a/nexus/peer-postgres/Cargo.toml b/nexus/peer-postgres/Cargo.toml index 130e8830f..9cc37b8fd 100644 --- a/nexus/peer-postgres/Cargo.toml +++ b/nexus/peer-postgres/Cargo.toml @@ -14,8 +14,7 @@ chrono = { version = "0.4", features = ["serde"] } futures = "0.3" peer-cursor = { path = "../peer-cursor" } peer-connections = { path = "../peer-connections" } -pgerror = { path = "../pgerror" } -pgwire = "0.17" +pgwire = "0.18" postgres-connection = { path = "../postgres-connection" } pt = { path = "../pt" } serde = { version = "1.0", features = ["derive"] } diff --git a/nexus/peer-postgres/src/lib.rs b/nexus/peer-postgres/src/lib.rs index 6617eeefd..7df2da191 100644 --- a/nexus/peer-postgres/src/lib.rs +++ b/nexus/peer-postgres/src/lib.rs @@ -1,7 +1,6 @@ use std::sync::Arc; use peer_cursor::{QueryExecutor, QueryOutput, Schema, SchemaRef}; -use pgerror::PgError; use pgwire::{ api::results::{FieldFormat, FieldInfo}, error::{PgWireError, PgWireResult}, @@ -71,9 +70,7 @@ impl QueryExecutor for PostgresQueryExecutor { .await .map_err(|e| { tracing::error!("error getting schema: {}", e); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error getting schema: {}", e), - })) + PgWireError::ApiError(format!("error getting schema: {}", e).into()) })?; tracing::info!("[peer-postgres] rewritten query: {}", rewritten_query); @@ -86,9 +83,7 @@ impl QueryExecutor for PostgresQueryExecutor { .await .map_err(|e| { tracing::error!("error executing query: {}", e); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error executing query: {}", e), - })) + PgWireError::ApiError(format!("error executing query: {}", e).into()) })?; // log that raw query execution has completed @@ -101,9 +96,7 @@ impl QueryExecutor for PostgresQueryExecutor { let mut rewritten_stmt = stmt.clone(); ast.rewrite_statement(&mut rewritten_stmt).map_err(|e| { tracing::error!("error rewriting statement: {}", e); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error rewriting statement: {}", e), - })) + PgWireError::ApiError(format!("error rewriting statement: {}", e).into()) })?; let rewritten_query = rewritten_stmt.to_string(); tracing::info!("[peer-postgres] rewritten statement: {}", rewritten_query); @@ -113,9 +106,7 @@ impl QueryExecutor for PostgresQueryExecutor { .await .map_err(|e| { tracing::error!("error executing query: {}", e); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error executing query: {}", e), - })) + PgWireError::ApiError(format!("error executing query: {}", e).into()) })?; Ok(QueryOutput::AffectedRows(rows_affected as usize)) } @@ -130,9 +121,7 @@ impl QueryExecutor for PostgresQueryExecutor { .await .map_err(|e| { tracing::error!("error getting schema: {}", e); - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error getting schema: {}", e), - })) + PgWireError::ApiError(format!("error getting schema: {}", e).into()) })?; Ok(Some(schema)) } diff --git a/nexus/peer-postgres/src/stream.rs b/nexus/peer-postgres/src/stream.rs index e7195a2ae..21905c1cc 100644 --- a/nexus/peer-postgres/src/stream.rs +++ b/nexus/peer-postgres/src/stream.rs @@ -2,7 +2,6 @@ use bytes::Bytes; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Utc}; use futures::Stream; use peer_cursor::{Record, RecordStream, SchemaRef}; -use pgerror::PgError; use pgwire::error::{PgWireError, PgWireResult}; use postgres_inet::MaskedIpAddr; use rust_decimal::Decimal; @@ -268,10 +267,7 @@ impl Stream for PgRecordStream { Poll::Ready(Some(Ok(record))) } Poll::Ready(Some(Err(e))) => { - let err = Box::new(PgError::Internal { - err_msg: e.to_string(), - }); - let err = PgWireError::ApiError(err); + let err = PgWireError::ApiError(Box::new(e)); Poll::Ready(Some(Err(err))) } Poll::Ready(None) => Poll::Ready(None), diff --git a/nexus/peer-snowflake/Cargo.toml b/nexus/peer-snowflake/Cargo.toml index a878be492..7017ced42 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -11,23 +11,20 @@ peer-cursor = { path = "../peer-cursor" } sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git" } value = { path = "../value" } tracing = "0.1" -pgerror = { path = "../pgerror" } secrecy = { version = "0.8.0" } async-trait = "0.1.57" jsonwebtoken = { version = "9.0", features = ["use_pem"] } base64 = "0.21" dashmap = "5.0" -pgwire = "0.17" +pgwire = "0.18" sha2 = "0.10" pt = { path = "../pt" } -pkcs8 = { version = "0.10.2", features = ["std", "pem", "encryption"] } -pkcs1 = "0.7.5" -rsa = "0.9.2" +rsa = { version = "0.9.2", features = ["pem", "pkcs5"] } serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" futures = "0.3" ureq = { version = "2", features = ["json", "charset"] } -reqwest = { version = "0.11", features = ["json", "gzip"] } +reqwest = { version = "0.11", default-features = false, features = ["json", "gzip", "rustls-tls"] } anyhow = "1.0" tokio = { version = "1.21", features = ["full"] } hex = "0.4" diff --git a/nexus/peer-snowflake/src/auth.rs b/nexus/peer-snowflake/src/auth.rs index 482272aec..4ec1a9082 100644 --- a/nexus/peer-snowflake/src/auth.rs +++ b/nexus/peer-snowflake/src/auth.rs @@ -6,9 +6,9 @@ use std::{ use anyhow::Context; use base64::prelude::{Engine as _, BASE64_STANDARD}; use jsonwebtoken::{encode as jwt_encode, Algorithm, EncodingKey, Header}; -use pkcs1::EncodeRsaPrivateKey; -use pkcs8::{DecodePrivateKey, EncodePublicKey}; -use rsa::{RsaPrivateKey, RsaPublicKey}; +use rsa::pkcs1::EncodeRsaPrivateKey; +use rsa::pkcs8::{DecodePrivateKey, EncodePublicKey}; +use rsa::RsaPrivateKey; use secrecy::{Secret, SecretString}; use serde::Serialize; use sha2::{Digest, Sha256}; @@ -91,7 +91,7 @@ impl SnowflakeAuth { #[tracing::instrument(name = "peer_sflake::gen_public_key_fp", skip_all)] fn gen_public_key_fp(private_key: &RsaPrivateKey) -> anyhow::Result { - let public_key = EncodePublicKey::to_public_key_der(&RsaPublicKey::from(private_key))?; + let public_key = private_key.to_public_key().to_public_key_der()?; let res = format!( "SHA256:{}", BASE64_STANDARD.encode(Sha256::new_with_prefix(public_key.as_bytes()).finalize()) @@ -101,9 +101,8 @@ impl SnowflakeAuth { #[tracing::instrument(name = "peer_sflake::auth_refresh_jwt", skip_all)] fn refresh_jwt(&mut self) -> anyhow::Result<()> { - let private_key_jwt: EncodingKey = EncodingKey::from_rsa_der( - EncodeRsaPrivateKey::to_pkcs1_der(&self.private_key)?.as_bytes(), - ); + let private_key_jwt: EncodingKey = + EncodingKey::from_rsa_der(self.private_key.to_pkcs1_der()?.as_bytes()); self.last_refreshed = SystemTime::now().duration_since(UNIX_EPOCH)?.as_secs(); info!( "Refreshing SnowFlake JWT for account: {} and user: {} at time {}", diff --git a/nexus/peer-snowflake/src/lib.rs b/nexus/peer-snowflake/src/lib.rs index 9fe3a8536..ac4d0154d 100644 --- a/nexus/peer-snowflake/src/lib.rs +++ b/nexus/peer-snowflake/src/lib.rs @@ -2,7 +2,6 @@ use anyhow::Context; use async_recursion::async_recursion; use cursor::SnowflakeCursorManager; use peer_cursor::{CursorModification, QueryExecutor, QueryOutput, SchemaRef}; -use pgerror::PgError; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; use sqlparser::dialect::GenericDialect; use sqlparser::parser; @@ -209,11 +208,10 @@ impl SnowflakeQueryExecutor { let query_str: String = query.to_string(); info!("Processing SnowFlake query: {}", query_str); - let result_set = self.process_query(&query_str).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) - })?; + let result_set = self + .process_query(&query_str) + .await + .map_err(|err| PgWireError::ApiError(err.into()))?; Ok(result_set) } @@ -309,11 +307,7 @@ impl QueryExecutor for SnowflakeQueryExecutor { snowflake_ast .rewrite(&mut new_query) .context("unable to rewrite query") - .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) - })?; + .map_err(|err| PgWireError::ApiError(err.into()))?; let result_set = self.query(&query.clone()).await?; @@ -361,11 +355,7 @@ impl QueryExecutor for SnowflakeQueryExecutor { // If parsing the count resulted in an error, return an internal error let count = match count { Ok(c) => c, - Err(err) => { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - }))) - } + Err(err) => return Err(PgWireError::ApiError(err.into())), }; tracing::info!("fetching {} rows", count); @@ -413,11 +403,7 @@ impl QueryExecutor for SnowflakeQueryExecutor { sf_ast .rewrite(&mut new_query) .context("unable to rewrite query") - .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: err.to_string(), - })) - })?; + .map_err(|err| PgWireError::ApiError(err.into()))?; // new_query.limit = Some(Expr::Value(Value::Number("1".to_owned(), false))); diff --git a/nexus/peer-snowflake/src/stream.rs b/nexus/peer-snowflake/src/stream.rs index 3434b70df..4740270d1 100644 --- a/nexus/peer-snowflake/src/stream.rs +++ b/nexus/peer-snowflake/src/stream.rs @@ -3,7 +3,6 @@ use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; use futures::Stream; use peer_cursor::Schema; use peer_cursor::{Record, RecordStream, SchemaRef}; -use pgerror::PgError; use pgwire::{ api::{ results::{FieldFormat, FieldInfo}, @@ -236,25 +235,15 @@ impl SnowflakeRecordStream { impl Stream for SnowflakeRecordStream { type Item = PgWireResult; - fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - let this = self.get_mut(); - - match this.advance() { + fn poll_next(mut self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + match self.advance() { Ok(true) => { - let record = this.convert_result_set_item(); - let result = record.map_err(|e| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("error getting current row: {}", e), - })) - }); + let record = self.convert_result_set_item(); + let result = record.map_err(|e| PgWireError::ApiError(e.into())); Poll::Ready(Some(result)) } Ok(false) => Poll::Ready(None), - Err(err) => Poll::Ready(Some(Err(PgWireError::ApiError(Box::new( - PgError::Internal { - err_msg: format!("Checking for next row in result set failed: {}", err), - }, - ))))), + Err(err) => Poll::Ready(Some(Err(PgWireError::ApiError(err.into())))), } } } diff --git a/nexus/pgerror/Cargo.toml b/nexus/pgerror/Cargo.toml deleted file mode 100644 index 2dca877fe..000000000 --- a/nexus/pgerror/Cargo.toml +++ /dev/null @@ -1,9 +0,0 @@ -[package] -name = "pgerror" -version = "0.1.0" -edition = "2021" - -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - -[dependencies] -snafu = { version = "0.7", features = ["backtraces"] } diff --git a/nexus/pgerror/src/lib.rs b/nexus/pgerror/src/lib.rs deleted file mode 100644 index bf842f778..000000000 --- a/nexus/pgerror/src/lib.rs +++ /dev/null @@ -1,8 +0,0 @@ -use snafu::Snafu; - -#[derive(Debug, Snafu)] -#[snafu(visibility(pub))] -pub enum PgError { - #[snafu(display("Internal error: {}", err_msg))] - Internal { err_msg: String }, -} diff --git a/nexus/postgres-connection/src/lib.rs b/nexus/postgres-connection/src/lib.rs index 3eb6558a2..dfa165b7d 100644 --- a/nexus/postgres-connection/src/lib.rs +++ b/nexus/postgres-connection/src/lib.rs @@ -1,7 +1,7 @@ -use std::fmt::Write; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; use pt::peerdb_peers::PostgresConfig; +use std::fmt::Write; pub fn get_pg_connection_string(config: &PostgresConfig) -> String { let mut connection_string = String::from("postgres://"); @@ -13,7 +13,12 @@ pub fn get_pg_connection_string(config: &PostgresConfig) -> String { } // Add the timeout as a query parameter, sslmode changes here appear to be useless - write!(connection_string, "@{}:{}/{}?connect_timeout=15", config.host, config.port, config.database).ok(); + write!( + connection_string, + "@{}:{}/{}?connect_timeout=15", + config.host, config.port, config.database + ) + .ok(); connection_string } diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 5c8b1e385..9a39485e2 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -70,6 +70,7 @@ pub struct FlowJob { pub resync: bool, pub soft_delete_col_name: Option, pub synced_at_col_name: Option, + pub initial_copy_only: bool, } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 798e09c99..86ec33a2b 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -100,6 +100,8 @@ pub struct FlowConnectionConfigs { pub soft_delete_col_name: ::prost::alloc::string::String, #[prost(string, tag="25")] pub synced_at_col_name: ::prost::alloc::string::String, + #[prost(bool, tag="26")] + pub initial_copy_only: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -472,6 +474,10 @@ pub struct QRepConfig { /// to be used after the old mirror is dropped #[prost(bool, tag="18")] pub dst_table_full_resync: bool, + #[prost(string, tag="19")] + pub synced_at_col_name: ::prost::alloc::string::String, + #[prost(string, tag="20")] + pub soft_delete_col_name: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -541,6 +547,24 @@ pub struct QRepFlowState { #[prost(bool, tag="4")] pub disable_wait_for_new_rows: bool, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PeerDbColumns { + #[prost(string, tag="1")] + pub soft_delete_col_name: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub synced_at_col_name: ::prost::alloc::string::String, + #[prost(bool, tag="3")] + pub soft_delete: bool, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetOpenConnectionsForUserResult { + #[prost(string, tag="1")] + pub user_name: ::prost::alloc::string::String, + #[prost(int64, tag="2")] + pub current_open_connections: i64, +} /// protos for qrep #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 0436bf334..a5a601c80 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -1229,6 +1229,9 @@ impl serde::Serialize for FlowConnectionConfigs { if !self.synced_at_col_name.is_empty() { len += 1; } + if self.initial_copy_only { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.FlowConnectionConfigs", len)?; if let Some(v) = self.source.as_ref() { struct_ser.serialize_field("source", v)?; @@ -1309,6 +1312,9 @@ impl serde::Serialize for FlowConnectionConfigs { if !self.synced_at_col_name.is_empty() { struct_ser.serialize_field("syncedAtColName", &self.synced_at_col_name)?; } + if self.initial_copy_only { + struct_ser.serialize_field("initialCopyOnly", &self.initial_copy_only)?; + } struct_ser.end() } } @@ -1366,6 +1372,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "softDeleteColName", "synced_at_col_name", "syncedAtColName", + "initial_copy_only", + "initialCopyOnly", ]; #[allow(clippy::enum_variant_names)] @@ -1395,6 +1403,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { Resync, SoftDeleteColName, SyncedAtColName, + InitialCopyOnly, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1442,6 +1451,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "resync" => Ok(GeneratedField::Resync), "softDeleteColName" | "soft_delete_col_name" => Ok(GeneratedField::SoftDeleteColName), "syncedAtColName" | "synced_at_col_name" => Ok(GeneratedField::SyncedAtColName), + "initialCopyOnly" | "initial_copy_only" => Ok(GeneratedField::InitialCopyOnly), _ => Ok(GeneratedField::__SkipField__), } } @@ -1486,6 +1496,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { let mut resync__ = None; let mut soft_delete_col_name__ = None; let mut synced_at_col_name__ = None; + let mut initial_copy_only__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Source => { @@ -1655,6 +1666,12 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { } synced_at_col_name__ = Some(map.next_value()?); } + GeneratedField::InitialCopyOnly => { + if initial_copy_only__.is_some() { + return Err(serde::de::Error::duplicate_field("initialCopyOnly")); + } + initial_copy_only__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1686,6 +1703,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { resync: resync__.unwrap_or_default(), soft_delete_col_name: soft_delete_col_name__.unwrap_or_default(), synced_at_col_name: synced_at_col_name__.unwrap_or_default(), + initial_copy_only: initial_copy_only__.unwrap_or_default(), }) } } @@ -1806,6 +1824,122 @@ impl<'de> serde::Deserialize<'de> for GetLastSyncedIdInput { deserializer.deserialize_struct("peerdb_flow.GetLastSyncedIDInput", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for GetOpenConnectionsForUserResult { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.user_name.is_empty() { + len += 1; + } + if self.current_open_connections != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.GetOpenConnectionsForUserResult", len)?; + if !self.user_name.is_empty() { + struct_ser.serialize_field("userName", &self.user_name)?; + } + if self.current_open_connections != 0 { + struct_ser.serialize_field("currentOpenConnections", ToString::to_string(&self.current_open_connections).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for GetOpenConnectionsForUserResult { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "user_name", + "userName", + "current_open_connections", + "currentOpenConnections", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + UserName, + CurrentOpenConnections, + __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 { + "userName" | "user_name" => Ok(GeneratedField::UserName), + "currentOpenConnections" | "current_open_connections" => Ok(GeneratedField::CurrentOpenConnections), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GetOpenConnectionsForUserResult; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.GetOpenConnectionsForUserResult") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut user_name__ = None; + let mut current_open_connections__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::UserName => { + if user_name__.is_some() { + return Err(serde::de::Error::duplicate_field("userName")); + } + user_name__ = Some(map.next_value()?); + } + GeneratedField::CurrentOpenConnections => { + if current_open_connections__.is_some() { + return Err(serde::de::Error::duplicate_field("currentOpenConnections")); + } + current_open_connections__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(GetOpenConnectionsForUserResult { + user_name: user_name__.unwrap_or_default(), + current_open_connections: current_open_connections__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.GetOpenConnectionsForUserResult", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for GetTableSchemaBatchInput { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -2476,6 +2610,138 @@ impl<'de> serde::Deserialize<'de> for PartitionRange { deserializer.deserialize_struct("peerdb_flow.PartitionRange", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for PeerDbColumns { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.soft_delete_col_name.is_empty() { + len += 1; + } + if !self.synced_at_col_name.is_empty() { + len += 1; + } + if self.soft_delete { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.PeerDBColumns", len)?; + if !self.soft_delete_col_name.is_empty() { + struct_ser.serialize_field("softDeleteColName", &self.soft_delete_col_name)?; + } + if !self.synced_at_col_name.is_empty() { + struct_ser.serialize_field("syncedAtColName", &self.synced_at_col_name)?; + } + if self.soft_delete { + struct_ser.serialize_field("softDelete", &self.soft_delete)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for PeerDbColumns { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "soft_delete_col_name", + "softDeleteColName", + "synced_at_col_name", + "syncedAtColName", + "soft_delete", + "softDelete", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + SoftDeleteColName, + SyncedAtColName, + SoftDelete, + __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 { + "softDeleteColName" | "soft_delete_col_name" => Ok(GeneratedField::SoftDeleteColName), + "syncedAtColName" | "synced_at_col_name" => Ok(GeneratedField::SyncedAtColName), + "softDelete" | "soft_delete" => Ok(GeneratedField::SoftDelete), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PeerDbColumns; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.PeerDBColumns") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut soft_delete_col_name__ = None; + let mut synced_at_col_name__ = None; + let mut soft_delete__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::SoftDeleteColName => { + if soft_delete_col_name__.is_some() { + return Err(serde::de::Error::duplicate_field("softDeleteColName")); + } + soft_delete_col_name__ = Some(map.next_value()?); + } + GeneratedField::SyncedAtColName => { + if synced_at_col_name__.is_some() { + return Err(serde::de::Error::duplicate_field("syncedAtColName")); + } + synced_at_col_name__ = Some(map.next_value()?); + } + GeneratedField::SoftDelete => { + if soft_delete__.is_some() { + return Err(serde::de::Error::duplicate_field("softDelete")); + } + soft_delete__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(PeerDbColumns { + soft_delete_col_name: soft_delete_col_name__.unwrap_or_default(), + synced_at_col_name: synced_at_col_name__.unwrap_or_default(), + soft_delete: soft_delete__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.PeerDBColumns", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for PostgresTableIdentifier { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -2636,6 +2902,12 @@ impl serde::Serialize for QRepConfig { if self.dst_table_full_resync { len += 1; } + if !self.synced_at_col_name.is_empty() { + len += 1; + } + if !self.soft_delete_col_name.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.QRepConfig", len)?; if !self.flow_job_name.is_empty() { struct_ser.serialize_field("flowJobName", &self.flow_job_name)?; @@ -2693,6 +2965,12 @@ impl serde::Serialize for QRepConfig { if self.dst_table_full_resync { struct_ser.serialize_field("dstTableFullResync", &self.dst_table_full_resync)?; } + if !self.synced_at_col_name.is_empty() { + struct_ser.serialize_field("syncedAtColName", &self.synced_at_col_name)?; + } + if !self.soft_delete_col_name.is_empty() { + struct_ser.serialize_field("softDeleteColName", &self.soft_delete_col_name)?; + } struct_ser.end() } } @@ -2738,6 +3016,10 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { "setupWatermarkTableOnDestination", "dst_table_full_resync", "dstTableFullResync", + "synced_at_col_name", + "syncedAtColName", + "soft_delete_col_name", + "softDeleteColName", ]; #[allow(clippy::enum_variant_names)] @@ -2760,6 +3042,8 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { NumRowsPerPartition, SetupWatermarkTableOnDestination, DstTableFullResync, + SyncedAtColName, + SoftDeleteColName, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -2800,6 +3084,8 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { "numRowsPerPartition" | "num_rows_per_partition" => Ok(GeneratedField::NumRowsPerPartition), "setupWatermarkTableOnDestination" | "setup_watermark_table_on_destination" => Ok(GeneratedField::SetupWatermarkTableOnDestination), "dstTableFullResync" | "dst_table_full_resync" => Ok(GeneratedField::DstTableFullResync), + "syncedAtColName" | "synced_at_col_name" => Ok(GeneratedField::SyncedAtColName), + "softDeleteColName" | "soft_delete_col_name" => Ok(GeneratedField::SoftDeleteColName), _ => Ok(GeneratedField::__SkipField__), } } @@ -2837,6 +3123,8 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { let mut num_rows_per_partition__ = None; let mut setup_watermark_table_on_destination__ = None; let mut dst_table_full_resync__ = None; + let mut synced_at_col_name__ = None; + let mut soft_delete_col_name__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::FlowJobName => { @@ -2957,6 +3245,18 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { } dst_table_full_resync__ = Some(map.next_value()?); } + GeneratedField::SyncedAtColName => { + if synced_at_col_name__.is_some() { + return Err(serde::de::Error::duplicate_field("syncedAtColName")); + } + synced_at_col_name__ = Some(map.next_value()?); + } + GeneratedField::SoftDeleteColName => { + if soft_delete_col_name__.is_some() { + return Err(serde::de::Error::duplicate_field("softDeleteColName")); + } + soft_delete_col_name__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -2981,6 +3281,8 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { num_rows_per_partition: num_rows_per_partition__.unwrap_or_default(), setup_watermark_table_on_destination: setup_watermark_table_on_destination__.unwrap_or_default(), dst_table_full_resync: dst_table_full_resync__.unwrap_or_default(), + synced_at_col_name: synced_at_col_name__.unwrap_or_default(), + soft_delete_col_name: soft_delete_col_name__.unwrap_or_default(), }) } } diff --git a/nexus/server/Cargo.toml b/nexus/server/Cargo.toml index 36f7c65b4..33d6d1492 100644 --- a/nexus/server/Cargo.toml +++ b/nexus/server/Cargo.toml @@ -47,7 +47,7 @@ peer-cursor = { path = "../peer-cursor" } peer-postgres = { path = "../peer-postgres" } peer-snowflake = { path = "../peer-snowflake" } peerdb-parser = { path = "../parser" } -pgwire = "0.17" +pgwire = "0.18" prost = "0.12" pt = { path = "../pt" } sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git", features = ["visitor"] } @@ -60,7 +60,6 @@ tracing-appender = "0.2" tracing-subscriber = "0.3" uuid = "1.0" cargo-deb = "2.0" -pgerror = { path = "../pgerror" } [dev-dependencies] postgres = "0.19.4" diff --git a/nexus/server/src/main.rs b/nexus/server/src/main.rs index c5011bd4f..9aa4dcb9a 100644 --- a/nexus/server/src/main.rs +++ b/nexus/server/src/main.rs @@ -20,7 +20,6 @@ use peer_cursor::{ QueryExecutor, QueryOutput, SchemaRef, }; use peerdb_parser::{NexusParsedStatement, NexusQueryParser, NexusStatement}; -use pgerror::PgError; use pgwire::{ api::{ auth::{ @@ -30,7 +29,6 @@ use pgwire::{ portal::{Format, Portal}, query::{ExtendedQueryHandler, SimpleQueryHandler, StatementOrPortal}, results::{DescribeResponse, Response, Tag}, - store::MemPortalStore, ClientInfo, MakeHandler, Type, }, error::{ErrorInfo, PgWireError, PgWireResult}, @@ -82,7 +80,6 @@ impl AuthSource for FixedPasswordAuthSource { pub struct NexusBackend { catalog: Arc>, peer_connections: PeerConnectionTracker, - portal_store: Arc>, query_parser: NexusQueryParser, peer_cursors: Mutex, executors: DashMap>, @@ -101,7 +98,6 @@ impl NexusBackend { Self { catalog, peer_connections, - portal_store: Arc::new(MemPortalStore::new()), query_parser, peer_cursors: Mutex::new(PeerCursors::new()), executors: DashMap::new(), @@ -172,9 +168,9 @@ impl NexusBackend { .get_workflow_details_for_flow_job(flow_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to query catalog for job metadata: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to query catalog for job metadata: {:?}", err).into(), + ) })?; Ok(workflow_details) } @@ -184,9 +180,7 @@ impl NexusBackend { peer_name: &str, ) -> PgWireResult { let peer = catalog.get_peer(peer_name).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to get peer {:?}: {:?}", peer_name, err), - })) + PgWireError::ApiError(format!("unable to get peer {:?}: {:?}", peer_name, err).into()) })?; Ok(peer) } @@ -227,9 +221,7 @@ impl NexusBackend { .validate_peer(&validate_request) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to check peer validity: {:?}", err), - })) + PgWireError::ApiError(format!("unable to check peer validity: {:?}", err).into()) })?; if let PeerValidationResult::Invalid(validation_err) = validity { Err(PgWireError::UserError(Box::new(ErrorInfo::new( @@ -254,9 +246,9 @@ impl NexusBackend { flow_job_name, } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let catalog = self.catalog.lock().await; @@ -269,12 +261,10 @@ impl NexusBackend { .get_workflow_details_for_flow_job(flow_job_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to query catalog for job metadata: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to query catalog for job metadata: {:?}", err) + .into(), + ) })?; tracing::info!( "got workflow id: {:?}", @@ -286,17 +276,17 @@ impl NexusBackend { .shutdown_flow_job(flow_job_name, workflow_details) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to shutdown flow job: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to shutdown flow job: {:?}", err).into(), + ) })?; catalog .delete_flow_job_entry(flow_job_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to delete job metadata: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to delete job metadata: {:?}", err).into(), + ) })?; let drop_mirror_success = format!("DROP MIRROR {}", flow_job_name); Ok(vec![Response::Execution(Tag::new_for_execution( @@ -334,9 +324,9 @@ impl NexusBackend { qrep_flow_job, } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let mirror_details; { @@ -351,12 +341,10 @@ impl NexusBackend { .create_qrep_flow_job_entry(qrep_flow_job) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to create mirror job entry: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to create mirror job entry: {:?}", err) + .into(), + ) })?; } @@ -424,9 +412,9 @@ impl NexusBackend { flow_job, } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let catalog = self.catalog.lock().await; let mirror_details = Self::check_for_mirror(&catalog, &flow_job.name).await?; @@ -438,24 +426,22 @@ impl NexusBackend { let mut destinations = HashSet::with_capacity(table_mappings_count); for tm in flow_job.table_mappings.iter() { if !sources.insert(tm.source_table_identifier.as_str()) { - return Err(PgWireError::ApiError(Box::new( - PgError::Internal { - err_msg: format!( - "Duplicate source table identifier {}", - tm.source_table_identifier - ), - }, - ))); + return Err(PgWireError::ApiError( + format!( + "Duplicate source table identifier {}", + tm.source_table_identifier + ) + .into(), + )); } if !destinations.insert(tm.destination_table_identifier.as_str()) { - return Err(PgWireError::ApiError(Box::new( - PgError::Internal { - err_msg: format!( - "Duplicate destination table identifier {}", - tm.destination_table_identifier - ), - }, - ))); + return Err(PgWireError::ApiError( + format!( + "Duplicate destination table identifier {}", + tm.destination_table_identifier + ) + .into(), + )); } } } @@ -464,12 +450,9 @@ impl NexusBackend { .create_cdc_flow_job_entry(flow_job) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to create mirror job entry: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to create mirror job entry: {:?}", err).into(), + ) })?; // get source and destination peers @@ -484,18 +467,18 @@ impl NexusBackend { .start_peer_flow_job(flow_job, src_peer, dst_peer) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to submit job: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to submit job: {:?}", err).into(), + ) })?; catalog .update_workflow_id_for_flow_job(&flow_job.name, &workflow_id) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to save job metadata: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to save job metadata: {:?}", err).into(), + ) })?; let create_mirror_success = format!("CREATE MIRROR {}", flow_job.name); @@ -512,9 +495,9 @@ impl NexusBackend { } PeerDDL::ExecuteMirrorForSelect { flow_job_name } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } if let Some(job) = { @@ -523,9 +506,9 @@ impl NexusBackend { .get_qrep_flow_job_by_name(flow_job_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to get qrep flow job: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to get qrep flow job: {:?}", err).into(), + ) })? } { let workflow_id = self.run_qrep_mirror(&job).await?; @@ -548,9 +531,9 @@ impl NexusBackend { peer_name, } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let catalog = self.catalog.lock().await; @@ -560,20 +543,15 @@ impl NexusBackend { if_exists ); let peer_exists = catalog.check_peer_entry(peer_name).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to query catalog for peer metadata: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to query catalog for peer metadata: {:?}", err).into(), + ) })?; tracing::info!("peer exist count: {}", peer_exists); if peer_exists != 0 { let mut flow_handler = self.flow_handler.as_ref().unwrap().lock().await; flow_handler.drop_peer(peer_name).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to drop peer: {:?}", err), - })) + PgWireError::ApiError(format!("unable to drop peer: {:?}", err).into()) })?; let drop_peer_success = format!("DROP PEER {}", peer_name); Ok(vec![Response::Execution(Tag::new_for_execution( @@ -601,9 +579,9 @@ impl NexusBackend { .. } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let qrep_config = { @@ -613,12 +591,9 @@ impl NexusBackend { .get_qrep_config_proto(mirror_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "error while getting QRep flow job: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("error while getting QRep flow job: {:?}", err).into(), + ) })? }; @@ -647,12 +622,10 @@ impl NexusBackend { .start_query_replication_flow(&qrep_config) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "error while starting new QRep job: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("error while starting new QRep job: {:?}", err) + .into(), + ) })?; // relock catalog, DROP MIRROR is done with it now let catalog = self.catalog.lock().await; @@ -663,12 +636,13 @@ impl NexusBackend { ) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( + PgWireError::ApiError( + format!( "unable to update workflow for flow job: {:?}", err - ), - })) + ) + .into(), + ) })?; let resync_mirror_success = format!("RESYNC MIRROR {}", mirror_name); @@ -691,9 +665,9 @@ impl NexusBackend { flow_job_name, } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let catalog = self.catalog.lock().await; @@ -706,12 +680,10 @@ impl NexusBackend { .get_workflow_details_for_flow_job(flow_job_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to query catalog for job metadata: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to query catalog for job metadata: {:?}", err) + .into(), + ) })?; tracing::info!( "[PAUSE MIRROR] got workflow id: {:?}", @@ -724,9 +696,9 @@ impl NexusBackend { .flow_state_change(flow_job_name, &workflow_details.workflow_id, true) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to shutdown flow job: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to shutdown flow job: {:?}", err).into(), + ) })?; let drop_mirror_success = format!("PAUSE MIRROR {}", flow_job_name); Ok(vec![Response::Execution(Tag::new_for_execution( @@ -752,9 +724,9 @@ impl NexusBackend { flow_job_name, } => { if self.flow_handler.is_none() { - return Err(PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: "flow service is not configured".to_owned(), - }))); + return Err(PgWireError::ApiError( + "flow service is not configured".into(), + )); } let catalog = self.catalog.lock().await; @@ -767,12 +739,10 @@ impl NexusBackend { .get_workflow_details_for_flow_job(flow_job_name) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to query catalog for job metadata: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to query catalog for job metadata: {:?}", err) + .into(), + ) })?; tracing::info!( "[RESUME MIRROR] got workflow id: {:?}", @@ -785,9 +755,9 @@ impl NexusBackend { .flow_state_change(flow_job_name, &workflow_details.workflow_id, false) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to shutdown flow job: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to shutdown flow job: {:?}", err).into(), + ) })?; let drop_mirror_success = format!("RESUME MIRROR {}", flow_job_name); Ok(vec![Response::Execution(Tag::new_for_execution( @@ -816,9 +786,9 @@ impl NexusBackend { tracing::info!("handling peer[{}] query: {}", peer.name, stmt); peer_holder = Some(peer.clone()); self.get_peer_executor(&peer).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to get peer executor: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err).into(), + ) })? } QueryAssociation::Catalog => { @@ -852,9 +822,9 @@ impl NexusBackend { Arc::clone(catalog.get_executor()) } Some(peer) => self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to get peer executor: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err).into(), + ) })?, } }; @@ -875,18 +845,14 @@ impl NexusBackend { .get_peer(&qrep_flow_job.source_peer) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to get source peer: {:?}", err), - })) + PgWireError::ApiError(format!("unable to get source peer: {:?}", err).into()) })?; let dst_peer = catalog .get_peer(&qrep_flow_job.target_peer) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to get destination peer: {:?}", err), - })) + PgWireError::ApiError(format!("unable to get destination peer: {:?}", err).into()) })?; // make a request to the flow service to start the job. @@ -895,18 +861,16 @@ impl NexusBackend { .start_qrep_flow_job(qrep_flow_job, src_peer, dst_peer) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to submit job: {:?}", err), - })) + PgWireError::ApiError(format!("unable to submit job: {:?}", err).into()) })?; catalog .update_workflow_id_for_flow_job(&qrep_flow_job.name, &workflow_id) .await .map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!("unable to update workflow for flow job: {:?}", err), - })) + PgWireError::ApiError( + format!("unable to update workflow for flow job: {:?}", err).into(), + ) })?; Ok(workflow_id) @@ -1008,13 +972,8 @@ fn parameter_to_string(portal: &Portal, idx: usize) -> PgW #[async_trait] impl ExtendedQueryHandler for NexusBackend { type Statement = NexusParsedStatement; - type PortalStore = MemPortalStore; type QueryParser = NexusQueryParser; - fn portal_store(&self) -> Arc { - self.portal_store.clone() - } - fn query_parser(&self) -> Arc { Arc::new(self.query_parser.clone()) } @@ -1081,36 +1040,30 @@ impl ExtendedQueryHandler for NexusBackend { Some(Config::BigqueryConfig(_)) => { let executor = self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to get peer executor: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err) + .into(), + ) })?; executor.describe(stmt).await? } Some(Config::PostgresConfig(_)) => { let executor = self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to get peer executor: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err) + .into(), + ) })?; executor.describe(stmt).await? } Some(Config::SnowflakeConfig(_)) => { let executor = self.get_peer_executor(peer).await.map_err(|err| { - PgWireError::ApiError(Box::new(PgError::Internal { - err_msg: format!( - "unable to get peer executor: {:?}", - err - ), - })) + PgWireError::ApiError( + format!("unable to get peer executor: {:?}", err) + .into(), + ) })?; executor.describe(stmt).await? } diff --git a/nexus/value/Cargo.toml b/nexus/value/Cargo.toml index 921865f63..d4fdf39cb 100644 --- a/nexus/value/Cargo.toml +++ b/nexus/value/Cargo.toml @@ -14,7 +14,7 @@ serde_json = "1.0" postgres-inet = "0.19.0" chrono = { version = "0.4", features = ["serde"] } hex = "0.4" -pgwire = "0.17" +pgwire = "0.18" postgres = { version = "0.19", features = ["with-chrono-0_4"] } postgres-types = { version = "0.2.5", features = ["array-impls"] } uuid = { version = "1.0", features = ["serde", "v4"] } diff --git a/protos/flow.proto b/protos/flow.proto index 88d87b983..cd09ddf8c 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -71,6 +71,8 @@ message FlowConnectionConfigs { string soft_delete_col_name = 24; string synced_at_col_name = 25; + + bool initial_copy_only = 26; } message RenameTableOption { @@ -321,6 +323,9 @@ message QRepConfig { // create new tables with "_peerdb_resync" suffix, perform initial load and then swap the new table with the old ones // to be used after the old mirror is dropped bool dst_table_full_resync = 18; + + string synced_at_col_name = 19; + string soft_delete_col_name = 20; } message QRepPartition { @@ -364,3 +369,14 @@ message QRepFlowState { bool needs_resync = 3; bool disable_wait_for_new_rows = 4; } + +message PeerDBColumns { + string soft_delete_col_name = 1; + string synced_at_col_name = 2; + bool soft_delete = 3; +} + +message GetOpenConnectionsForUserResult { + string user_name = 1; + int64 current_open_connections = 2; +} \ No newline at end of file diff --git a/ui/app/api/mirrors/alerts/route.ts b/ui/app/api/mirrors/alerts/route.ts new file mode 100644 index 000000000..13cc61250 --- /dev/null +++ b/ui/app/api/mirrors/alerts/route.ts @@ -0,0 +1,40 @@ +import prisma from '@/app/utils/prisma'; + +export const dynamic = 'force-dynamic'; + +export async function POST(request: Request) { + const { flowName } = await request.json(); + const errCount = await prisma.flow_errors.count({ + where: { + flow_name: flowName, + error_type: 'error', + ack: false, + }, + }); + let mirrorStatus: 'healthy' | 'failed'; + if (errCount > 0) { + mirrorStatus = 'failed'; + } else { + mirrorStatus = 'healthy'; + } + return new Response(JSON.stringify(mirrorStatus)); +} + +// We accept a list here in preparation for a Select All feature in UI +export async function PUT(request: Request) { + const { mirrorIDStringList } = await request.json(); + const mirrorIDList: bigint[] = mirrorIDStringList.map((id: string) => + BigInt(id) + ); + const success = await prisma.flow_errors.updateMany({ + where: { + id: { + in: mirrorIDList, + }, + }, + data: { + ack: true, + }, + }); + return new Response(JSON.stringify(success.count)); +} diff --git a/ui/app/api/mirrors/route.ts b/ui/app/api/mirrors/route.ts new file mode 100644 index 000000000..65e5e3fe6 --- /dev/null +++ b/ui/app/api/mirrors/route.ts @@ -0,0 +1,25 @@ +import { getTruePeer } from '@/app/api/peers/getTruePeer'; +import prisma from '@/app/utils/prisma'; + +export const dynamic = 'force-dynamic'; + +export async function GET(request: Request) { + const mirrors = await prisma.flows.findMany({ + distinct: 'name', + include: { + sourcePeer: true, + destinationPeer: true, + }, + }); + + // using any as type because of the way prisma returns data + const flows = mirrors?.map((mirror: any) => { + let newMirror: any = { + ...mirror, + sourcePeer: getTruePeer(mirror.sourcePeer), + destinationPeer: getTruePeer(mirror.destinationPeer), + }; + return newMirror; + }); + return new Response(JSON.stringify(flows)); +} diff --git a/ui/app/api/peers/getTruePeer.ts b/ui/app/api/peers/getTruePeer.ts new file mode 100644 index 000000000..1af4155de --- /dev/null +++ b/ui/app/api/peers/getTruePeer.ts @@ -0,0 +1,60 @@ +import { CatalogPeer } from '@/app/dto/PeersDTO'; +import { + BigqueryConfig, + EventHubConfig, + EventHubGroupConfig, + Peer, + PostgresConfig, + S3Config, + SnowflakeConfig, + SqlServerConfig, +} from '@/grpc_generated/peers'; + +export const getTruePeer = (peer: CatalogPeer) => { + const newPeer: Peer = { + name: peer.name, + type: peer.type, + }; + const options = peer.options; + let config: + | BigqueryConfig + | SnowflakeConfig + | PostgresConfig + | EventHubConfig + | S3Config + | SqlServerConfig + | EventHubGroupConfig; + switch (peer.type) { + case 0: + config = BigqueryConfig.decode(options); + newPeer.bigqueryConfig = config; + break; + case 1: + config = SnowflakeConfig.decode(options); + newPeer.snowflakeConfig = config; + break; + case 3: + config = PostgresConfig.decode(options); + newPeer.postgresConfig = config; + break; + case 4: + config = EventHubConfig.decode(options); + newPeer.eventhubConfig = config; + break; + case 5: + config = S3Config.decode(options); + newPeer.s3Config = config; + break; + case 6: + config = SqlServerConfig.decode(options); + newPeer.sqlserverConfig = config; + break; + case 7: + config = EventHubGroupConfig.decode(options); + newPeer.eventhubGroupConfig = config; + break; + default: + return newPeer; + } + return newPeer; +}; diff --git a/ui/app/api/peers/route.ts b/ui/app/api/peers/route.ts index c865979ef..03aa98ae4 100644 --- a/ui/app/api/peers/route.ts +++ b/ui/app/api/peers/route.ts @@ -1,3 +1,4 @@ +import { getTruePeer } from '@/app/api/peers/getTruePeer'; import { CatalogPeer, PeerConfig, @@ -8,13 +9,10 @@ import prisma from '@/app/utils/prisma'; import { BigqueryConfig, DBType, - EventHubConfig, - EventHubGroupConfig, Peer, PostgresConfig, S3Config, SnowflakeConfig, - SqlServerConfig, } from '@/grpc_generated/peers'; import { CreatePeerRequest, @@ -63,6 +61,8 @@ const constructPeer = ( } }; +export const dynamic = 'force-dynamic'; + export async function POST(request: Request) { const body = await request.json(); console.log('POST Validate Peer:', body); @@ -117,55 +117,6 @@ export async function POST(request: Request) { } } -export const getTruePeer = (peer: CatalogPeer) => { - const newPeer: Peer = { - name: peer.name, - type: peer.type, - }; - const options = peer.options; - let config: - | BigqueryConfig - | SnowflakeConfig - | PostgresConfig - | EventHubConfig - | S3Config - | SqlServerConfig - | EventHubGroupConfig; - switch (peer.type) { - case 0: - config = BigqueryConfig.decode(options); - newPeer.bigqueryConfig = config; - break; - case 1: - config = SnowflakeConfig.decode(options); - newPeer.snowflakeConfig = config; - break; - case 3: - config = PostgresConfig.decode(options); - newPeer.postgresConfig = config; - break; - case 4: - config = EventHubConfig.decode(options); - newPeer.eventhubConfig = config; - break; - case 5: - config = S3Config.decode(options); - newPeer.s3Config = config; - break; - case 6: - config = SqlServerConfig.decode(options); - newPeer.sqlserverConfig = config; - break; - case 7: - config = EventHubGroupConfig.decode(options); - newPeer.eventhubGroupConfig = config; - break; - default: - return newPeer; - } - return newPeer; -}; - // GET all the peers from the database export async function GET(request: Request) { const peers = await prisma.peers.findMany(); diff --git a/ui/app/api/version/route.ts b/ui/app/api/version/route.ts new file mode 100644 index 000000000..1197ca204 --- /dev/null +++ b/ui/app/api/version/route.ts @@ -0,0 +1,23 @@ +import { UVersionResponse } from '@/app/dto/VersionDTO'; +import { PeerDBVersionResponse } from '@/grpc_generated/route'; +import { GetFlowHttpAddressFromEnv } from '@/rpc/http'; + +export const dynamic = 'force-dynamic'; + +export async function GET() { + const flowServiceAddr = GetFlowHttpAddressFromEnv(); + try { + const versionResponse: PeerDBVersionResponse = await fetch( + `${flowServiceAddr}/v1/version` + ).then((res) => { + return res.json(); + }); + let response: UVersionResponse = { + version: versionResponse.version, + }; + return new Response(JSON.stringify(response)); + } catch (error) { + console.error('Error getting version:', error); + return new Response(JSON.stringify({ error: error })); + } +} diff --git a/ui/app/dto/MirrorsDTO.ts b/ui/app/dto/MirrorsDTO.ts index 4a76200fd..977f7f353 100644 --- a/ui/app/dto/MirrorsDTO.ts +++ b/ui/app/dto/MirrorsDTO.ts @@ -28,3 +28,12 @@ export type SyncStatusRow = { endTime: Date | null; numRows: number; }; + +export type AlertErr = { + id: bigint; + flow_name: string; + error_message: string; + error_type: string; + error_timestamp: Date; + ack: boolean; +}; diff --git a/ui/app/dto/VersionDTO.ts b/ui/app/dto/VersionDTO.ts new file mode 100644 index 000000000..cb0e4b189 --- /dev/null +++ b/ui/app/dto/VersionDTO.ts @@ -0,0 +1,3 @@ +export type UVersionResponse = { + version: string; +}; diff --git a/ui/app/login/page.tsx b/ui/app/login/page.tsx index 41c00c0dd..1172b0e81 100644 --- a/ui/app/login/page.tsx +++ b/ui/app/login/page.tsx @@ -13,9 +13,7 @@ export default function Login() { const searchParams = useSearchParams(); const [pass, setPass] = useState(''); const [show, setShow] = useState(false); - const [error, setError] = useState(() => - searchParams.has('reject') ? 'Authentication failed, please login' : '' - ); + const [error, setError] = useState(() => ''); const login = () => { fetch('/api/login', { method: 'POST', diff --git a/ui/app/mirrors/create/cdc/cdc.tsx b/ui/app/mirrors/create/cdc/cdc.tsx index 65f2158ee..63155acdf 100644 --- a/ui/app/mirrors/create/cdc/cdc.tsx +++ b/ui/app/mirrors/create/cdc/cdc.tsx @@ -66,12 +66,6 @@ export default function CDCConfigForm({ if (mirrorConfig.source != undefined && mirrorConfig.destination != undefined) return ( <> - {normalSettings.map((setting, id) => { return ( paramDisplayCondition(setting) && ( @@ -112,6 +106,13 @@ export default function CDCConfigForm({ /> ); })} + + ); } diff --git a/ui/app/mirrors/create/cdc/columnbox.tsx b/ui/app/mirrors/create/cdc/columnbox.tsx new file mode 100644 index 000000000..b68560419 --- /dev/null +++ b/ui/app/mirrors/create/cdc/columnbox.tsx @@ -0,0 +1,80 @@ +'use client'; +import { TableMapRow } from '@/app/dto/MirrorsDTO'; +import { Checkbox } from '@/lib/Checkbox'; +import { Label } from '@/lib/Label'; +import { RowWithCheckbox } from '@/lib/Layout'; +import { Dispatch, SetStateAction } from 'react'; + +interface ColumnProps { + columns: string[]; + tableRow: TableMapRow; + rows: TableMapRow[]; + setRows: Dispatch>; +} +export default function ColumnBox({ + columns, + tableRow, + rows, + setRows, +}: ColumnProps) { + const handleColumnExclusion = ( + source: string, + column: string, + include: boolean + ) => { + const currRows = [...rows]; + const rowOfSource = currRows.find((row) => row.source === source); + if (rowOfSource) { + if (include) { + const updatedExclude = rowOfSource.exclude.filter( + (col) => col !== column + ); + rowOfSource.exclude = updatedExclude; + } else { + rowOfSource.exclude.push(column); + } + } + setRows(currRows); + }; + + const columnExclusion = new Set(tableRow.exclude); + return columns.map((column) => { + const [columnName, columnType, isPkeyStr] = column.split(':'); + const isPkey = isPkeyStr === 'true'; + return ( + + {columnName} +

+ {columnType} +

+ + } + action={ + + handleColumnExclusion(tableRow.source, columnName, state) + } + /> + } + /> + ); + }); +} diff --git a/ui/app/mirrors/create/cdc/schemabox.tsx b/ui/app/mirrors/create/cdc/schemabox.tsx index 4195fae83..d9b15a703 100644 --- a/ui/app/mirrors/create/cdc/schemabox.tsx +++ b/ui/app/mirrors/create/cdc/schemabox.tsx @@ -7,9 +7,16 @@ import { Label } from '@/lib/Label'; import { RowWithCheckbox } from '@/lib/Layout'; import { SearchField } from '@/lib/SearchField'; import { TextField } from '@/lib/TextField'; -import { Dispatch, SetStateAction, useCallback, useState } from 'react'; +import { + Dispatch, + SetStateAction, + useCallback, + useMemo, + useState, +} from 'react'; import { BarLoader } from 'react-spinners/'; import { fetchColumns, fetchTables } from '../handlers'; +import ColumnBox from './columnbox'; import { expandableStyle, schemaBoxStyle, tableBoxStyle } from './styles'; interface SchemaBoxProps { @@ -36,6 +43,20 @@ const SchemaBox = ({ const [columnsLoading, setColumnsLoading] = useState(false); const [expandedSchemas, setExpandedSchemas] = useState([]); const [tableQuery, setTableQuery] = useState(''); + const [schemaLoadedSet, setSchemaLoadedSet] = useState>( + new Set() + ); + + const [handlingAll, setHandlingAll] = useState(false); + + const searchedTables = useMemo(() => { + const tableQueryLower = tableQuery.toLowerCase(); + return rows.filter( + (row) => + row.schema === schema && + row.source.toLowerCase().includes(tableQueryLower) + ); + }, [schema, rows, tableQuery]); const schemaIsExpanded = useCallback( (schema: string) => { @@ -74,11 +95,13 @@ const SchemaBox = ({ const addTableColumns = (table: string) => { const schemaName = table.split('.')[0]; const tableName = table.split('.')[1]; + fetchColumns(sourcePeer, schemaName, tableName, setColumnsLoading).then( - (res) => + (res) => { setTableColumns((prev) => { return [...prev, { tableName: table, columns: res }]; - }) + }); + } ); }; @@ -93,47 +116,34 @@ const SchemaBox = ({ ?.columns; }; - const handleColumnExclusion = ( - source: string, - column: string, - include: boolean - ) => { - const currRows = [...rows]; - const rowOfSource = currRows.find((row) => row.source === source); - if (rowOfSource) { - if (include) { - const updatedExclude = rowOfSource.exclude.filter( - (col) => col !== column - ); - rowOfSource.exclude = updatedExclude; - } else { - rowOfSource.exclude.push(column); - } - } - setRows(currRows); - }; - const handleSelectAll = ( - e: React.MouseEvent + e: React.MouseEvent, + schemaName: string ) => { + setHandlingAll(true); const newRows = [...rows]; for (const row of newRows) { - row.selected = e.currentTarget.checked; - if (e.currentTarget.checked) addTableColumns(row.source); - else removeTableColumns(row.source); + if (row.schema === schemaName) { + row.selected = e.currentTarget.checked; + if (e.currentTarget.checked) addTableColumns(row.source); + else removeTableColumns(row.source); + } } setRows(newRows); + setHandlingAll(false); }; const handleSchemaClick = (schemaName: string) => { if (!schemaIsExpanded(schemaName)) { - setTablesLoading(true); setExpandedSchemas((curr) => [...curr, schemaName]); - fetchTables(sourcePeer, schemaName, peerType).then((tableRows) => { - const newRows = [...rows, ...tableRows]; - setRows(newRows); - setTablesLoading(false); - }); + if (!schemaLoadedSet.has(schemaName)) { + setTablesLoading(true); + setSchemaLoadedSet((loaded) => new Set(loaded).add(schemaName)); + fetchTables(sourcePeer, schemaName, peerType).then((tableRows) => { + setRows((value) => [...value, ...tableRows]); + setTablesLoading(false); + }); + } } else { setExpandedSchemas((curr) => curr.filter((expandedSchema) => expandedSchema != schemaName) @@ -158,7 +168,10 @@ const SchemaBox = ({
- handleSelectAll(e)} /> + handleSelectAll(e, schema)} + /> @@ -173,139 +186,96 @@ const SchemaBox = ({ />
- {schemaIsExpanded(schema) && ( + {/* TABLE BOX */} + {handlingAll && } + {!handlingAll && schemaIsExpanded(schema) && (
- {rows.filter((row) => row.schema === schema).length ? ( - rows - .filter( - (row) => - row.schema === schema && - row.source.toLowerCase().includes(tableQuery.toLowerCase()) - ) - .map((row) => { - const columns = getTableColumns(row.source); - return ( -
+ {searchedTables.length ? ( + searchedTables.map((row) => { + const columns = getTableColumns(row.source); + return ( +
+
+ + {row.source} + + } + action={ + + handleTableSelect(state, row.source) + } + /> + } + /> +
- - {row.source} - - } - action={ - - handleTableSelect(state, row.source) - } - /> +

Target Table:

+ ) => + updateDestination(row.source, e.target.value) } /> +
+
-
+ + {columns ? ( + -
-
- {row.selected && ( -
+ ) : columnsLoading ? ( + + ) : ( - {columns ? ( - columns.map((column) => { - const columnName = column.split(':')[0]; - const columnType = column.split(':')[1]; - const isPkey = column.split(':')[2] === 'true'; - return ( - - {columnName} -

- {columnType} -

- - } - action={ - col == columnName - ) - } - onCheckedChange={(state: boolean) => - handleColumnExclusion( - row.source, - columnName, - state - ) - } - /> - } - /> - ); - }) - ) : columnsLoading ? ( - - ) : ( - - )} -
- )} -
- ); - }) + )} +
+ )} + + ); + }) ) : tablesLoading ? ( ) : ( diff --git a/ui/app/mirrors/create/cdc/tablemapping.tsx b/ui/app/mirrors/create/cdc/tablemapping.tsx index 85c889cb4..2cccea321 100644 --- a/ui/app/mirrors/create/cdc/tablemapping.tsx +++ b/ui/app/mirrors/create/cdc/tablemapping.tsx @@ -55,7 +55,7 @@ const TableMapping = ({ /> -
+
{allSchemas ? ( allSchemas ?.filter((schema) => { diff --git a/ui/app/mirrors/create/handlers.ts b/ui/app/mirrors/create/handlers.ts index 7c0b28ae0..81f36bd73 100644 --- a/ui/app/mirrors/create/handlers.ts +++ b/ui/app/mirrors/create/handlers.ts @@ -73,45 +73,33 @@ const validateCDCFields = ( } | undefined )[], - setMsg: Dispatch>, config: CDCConfig -): boolean => { +): string | undefined => { let validationErr: string | undefined; const tablesValidity = tableMappingSchema.safeParse(tableMapping); if (!tablesValidity.success) { validationErr = tablesValidity.error.issues[0].message; - setMsg({ ok: false, msg: validationErr }); - return false; } const configValidity = cdcSchema.safeParse(config); if (!configValidity.success) { validationErr = configValidity.error.issues[0].message; - setMsg({ ok: false, msg: validationErr }); - return false; } - setMsg({ ok: true, msg: '' }); - return true; + return validationErr; }; const validateQRepFields = ( query: string, - setMsg: Dispatch>, config: QRepConfig -): boolean => { +): string | undefined => { if (query.length < 5) { - setMsg({ ok: false, msg: 'Query is invalid' }); - return false; + return 'Query is invalid'; } - let validationErr: string | undefined; const configValidity = qrepSchema.safeParse(config); if (!configValidity.success) { validationErr = configValidity.error.issues[0].message; - setMsg({ ok: false, msg: validationErr }); - return false; } - setMsg({ ok: true, msg: '' }); - return true; + return validationErr; }; interface TableMapping { @@ -140,25 +128,23 @@ export const handleCreateCDC = async ( flowJobName: string, rows: TableMapRow[], config: CDCConfig, - setMsg: Dispatch< - SetStateAction<{ - ok: boolean; - msg: string; - }> - >, + notify: (msg: string) => void, setLoading: Dispatch>, route: RouteCallback ) => { const flowNameValid = flowNameSchema.safeParse(flowJobName); if (!flowNameValid.success) { const flowNameErr = flowNameValid.error.issues[0].message; - setMsg({ ok: false, msg: flowNameErr }); + notify(flowNameErr); return; } const tableNameMapping = reformattedTableMapping(rows); - const isValid = validateCDCFields(tableNameMapping, setMsg, config); - if (!isValid) return; + const fieldErr = validateCDCFields(tableNameMapping, config); + if (fieldErr) { + notify(fieldErr); + return; + } config['tableMappings'] = tableNameMapping as TableMapping[]; config['flowJobName'] = flowJobName; @@ -170,6 +156,12 @@ export const handleCreateCDC = async ( config.cdcSyncMode = QRepSyncMode.QREP_SYNC_MODE_STORAGE_AVRO; config.snapshotSyncMode = QRepSyncMode.QREP_SYNC_MODE_STORAGE_AVRO; } + + if (config.doInitialCopy == false && config.initialCopyOnly == true) { + notify('Initial Copy Only cannot be true if Initial Copy is false.'); + return; + } + setLoading(true); const statusMessage: UCreateMirrorResponse = await fetch('/api/mirrors/cdc', { method: 'POST', @@ -178,11 +170,11 @@ export const handleCreateCDC = async ( }), }).then((res) => res.json()); if (!statusMessage.created) { - setMsg({ ok: false, msg: 'unable to create mirror.' }); + notify('unable to create mirror.'); setLoading(false); return; } - setMsg({ ok: true, msg: 'CDC Mirror created successfully' }); + notify('CDC Mirror created successfully'); route(); setLoading(false); }; @@ -200,12 +192,7 @@ export const handleCreateQRep = async ( flowJobName: string, query: string, config: QRepConfig, - setMsg: Dispatch< - SetStateAction<{ - ok: boolean; - msg: string; - }> - >, + notify: (msg: string) => void, setLoading: Dispatch>, route: RouteCallback, xmin?: boolean @@ -213,7 +200,7 @@ export const handleCreateQRep = async ( const flowNameValid = flowNameSchema.safeParse(flowJobName); if (!flowNameValid.success) { const flowNameErr = flowNameValid.error.issues[0].message; - setMsg({ ok: false, msg: flowNameErr }); + notify(flowNameErr); return; } @@ -228,16 +215,17 @@ export const handleCreateQRep = async ( if ( config.writeMode?.writeType == QRepWriteType.QREP_WRITE_MODE_UPSERT && - !config.writeMode?.upsertKeyColumns + (!config.writeMode?.upsertKeyColumns || + config.writeMode?.upsertKeyColumns.length == 0) ) { - setMsg({ - ok: false, - msg: 'For upsert mode, unique key columns cannot be empty.', - }); + notify('For upsert mode, unique key columns cannot be empty.'); + return; + } + const fieldErr = validateQRepFields(query, config); + if (fieldErr) { + notify(fieldErr); return; } - const isValid = validateQRepFields(query, setMsg, config); - if (!isValid) return; config.flowJobName = flowJobName; config.query = query; @@ -258,11 +246,11 @@ export const handleCreateQRep = async ( } ).then((res) => res.json()); if (!statusMessage.created) { - setMsg({ ok: false, msg: 'unable to create mirror.' }); + notify('unable to create mirror.'); setLoading(false); return; } - setMsg({ ok: true, msg: 'Query Replication Mirror created successfully' }); + notify('Query Replication Mirror created successfully'); route(); setLoading(false); }; diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index d24e9bd9f..88d39a6dc 100644 --- a/ui/app/mirrors/create/helpers/cdc.ts +++ b/ui/app/mirrors/create/helpers/cdc.ts @@ -103,4 +103,15 @@ export const cdcSettings: MirrorSetting[] = [ default: 'SQL', type: 'switch', }, + { + label: 'Initial Copy Only', + stateHandler: (value, setter) => + setter((curr: CDCConfig) => ({ + ...curr, + initialCopyOnly: (value as boolean) || false, + })), + tips: 'If set, PeerDB will only perform initial load and will not perform CDC sync.', + type: 'switch', + advanced: true, + }, ]; diff --git a/ui/app/mirrors/create/helpers/common.ts b/ui/app/mirrors/create/helpers/common.ts index 063588b2c..15f060792 100644 --- a/ui/app/mirrors/create/helpers/common.ts +++ b/ui/app/mirrors/create/helpers/common.ts @@ -45,6 +45,7 @@ export const blankCDCSetting: FlowConnectionConfigs = { resync: false, softDeleteColName: '', syncedAtColName: '', + initialCopyOnly: false, }; export const blankQRepSetting = { diff --git a/ui/app/mirrors/create/helpers/qrep.ts b/ui/app/mirrors/create/helpers/qrep.ts index 654d5c7ff..fca1f9fcb 100644 --- a/ui/app/mirrors/create/helpers/qrep.ts +++ b/ui/app/mirrors/create/helpers/qrep.ts @@ -112,8 +112,9 @@ export const qrepSettings: MirrorSetting[] = [ writeMode: currWriteMode, }; }), - tips: `Comma separated string column names. Needed when write mode is set to UPSERT. + tips: `Needed when write mode is set to UPSERT. These columns need to be unique and are used for updates.`, + type: 'select', }, { label: 'Initial Copy Only', diff --git a/ui/app/mirrors/create/page.tsx b/ui/app/mirrors/create/page.tsx index a075432bb..497d3aea4 100644 --- a/ui/app/mirrors/create/page.tsx +++ b/ui/app/mirrors/create/page.tsx @@ -4,17 +4,18 @@ import { RequiredIndicator } from '@/components/RequiredIndicator'; import { QRepConfig } from '@/grpc_generated/flow'; import { DBType, Peer } from '@/grpc_generated/peers'; import { Button } from '@/lib/Button'; -import { ButtonGroup } from '@/lib/ButtonGroup'; +import { Icon } from '@/lib/Icon'; import { Label } from '@/lib/Label'; import { RowWithSelect, RowWithTextField } from '@/lib/Layout'; import { Panel } from '@/lib/Panel'; import { TextField } from '@/lib/TextField'; import { Divider } from '@tremor/react'; import Image from 'next/image'; -import Link from 'next/link'; import { useRouter } from 'next/navigation'; import { useEffect, useState } from 'react'; import ReactSelect from 'react-select'; +import { ToastContainer, toast } from 'react-toastify'; +import 'react-toastify/dist/ReactToastify.css'; import { InfoPopover } from '../../../components/InfoPopover'; import { CDCConfig, TableMapRow } from '../../dto/MirrorsDTO'; import CDCConfigForm from './cdc/cdc'; @@ -47,22 +48,23 @@ function getPeerLabel(peer: Peer) { ); } +const notifyErr = (errMsg: string) => { + toast.error(errMsg, { + position: toast.POSITION.BOTTOM_CENTER, + }); +}; export default function CreateMirrors() { const router = useRouter(); const [mirrorName, setMirrorName] = useState(''); const [mirrorType, setMirrorType] = useState(''); - const [formMessage, setFormMessage] = useState<{ ok: boolean; msg: string }>({ - ok: true, - msg: '', - }); const [loading, setLoading] = useState(false); const [config, setConfig] = useState(blankCDCSetting); const [peers, setPeers] = useState([]); const [rows, setRows] = useState([]); const [qrepQuery, setQrepQuery] = useState(`-- Here's a sample template: - SELECT * FROM - WHERE + SELECT * FROM + WHERE BETWEEN {{.start}} AND {{.end}}`); useEffect(() => { @@ -183,15 +185,7 @@ export default function CreateMirrors() { Configuration )} - {!loading && formMessage.msg.length > 0 && ( - - )} + {!loading && } {mirrorType === '' ? ( <> ) : mirrorType === 'CDC' ? ( @@ -213,36 +207,41 @@ export default function CreateMirrors() { {mirrorType && ( - - - - + )}
diff --git a/ui/app/mirrors/create/qrep/qrep.tsx b/ui/app/mirrors/create/qrep/qrep.tsx index 6b5b7b4b3..9dd7943b8 100644 --- a/ui/app/mirrors/create/qrep/qrep.tsx +++ b/ui/app/mirrors/create/qrep/qrep.tsx @@ -14,6 +14,7 @@ import { MirrorSetter } from '../../../dto/MirrorsDTO'; import { defaultSyncMode } from '../cdc/cdc'; import { fetchAllTables, fetchColumns } from '../handlers'; import { MirrorSetting, blankQRepSetting } from '../helpers/common'; +import UpsertColsDisplay from './upsertcols'; interface QRepConfigProps { settings: MirrorSetting[]; @@ -29,10 +30,6 @@ interface QRepConfigProps { xmin?: boolean; } -const SyncModes = ['AVRO', 'Copy with Binary'].map((value) => ({ - label: value, - value, -})); const WriteModes = ['Append', 'Upsert', 'Overwrite'].map((value) => ({ label: value, value, @@ -50,6 +47,7 @@ export default function QRepConfigForm({ const [watermarkColumns, setWatermarkColumns] = useState< { value: string; label: string }[] >([]); + const [loading, setLoading] = useState(false); const handleChange = (val: string | boolean, setting: MirrorSetting) => { @@ -220,6 +218,13 @@ export default function QRepConfigForm({ } options={WriteModes} /> + ) : setting.label === 'Upsert Key Columns' ? ( + ) : ( { + const [uniqueColumnsSet, setUniqueColumnsSet] = useState>( + new Set() + ); + + const handleUniqueColumns = ( + col: string, + setting: MirrorSetting, + action: 'add' | 'remove' + ) => { + if (action === 'add') setUniqueColumnsSet((prev) => new Set(prev).add(col)); + else if (action === 'remove') { + setUniqueColumnsSet((prev) => { + const newSet = new Set(prev); + newSet.delete(col); + return newSet; + }); + } + const uniqueColsArr = Array.from(uniqueColumnsSet); + setting.stateHandler(uniqueColsArr, setter); + }; + + useEffect(() => { + const uniqueColsArr = Array.from(uniqueColumnsSet); + setter((curr) => { + let defaultMode: QRepWriteMode = { + writeType: QRepWriteType.QREP_WRITE_MODE_APPEND, + upsertKeyColumns: [], + }; + let currWriteMode = (curr as QRepConfig).writeMode || defaultMode; + currWriteMode.upsertKeyColumns = uniqueColsArr as string[]; + return { + ...curr, + writeMode: currWriteMode, + }; + }); + }, [uniqueColumnsSet, setter]); + return ( + <> + { + val && handleUniqueColumns(val.value, setting, 'add'); + }} + isLoading={loading} + options={columns} + /> +
+ {Array.from(uniqueColumnsSet).map((col: string) => { + return ( + + {col} + + + ); + })} +
+ + ); +}; + +export default UpsertColsDisplay; diff --git a/ui/app/mirrors/edit/[mirrorId]/cdc.tsx b/ui/app/mirrors/edit/[mirrorId]/cdc.tsx index 7f54d227a..8c33fa7c9 100644 --- a/ui/app/mirrors/edit/[mirrorId]/cdc.tsx +++ b/ui/app/mirrors/edit/[mirrorId]/cdc.tsx @@ -13,12 +13,11 @@ import { Label } from '@/lib/Label'; import { ProgressBar } from '@/lib/ProgressBar'; import { SearchField } from '@/lib/SearchField'; import { Table, TableCell, TableRow } from '@/lib/Table'; -import * as Tabs from '@radix-ui/react-tabs'; +import { Tab, TabGroup, TabList, TabPanel, TabPanels } from '@tremor/react'; import moment, { Duration, Moment } from 'moment'; import Link from 'next/link'; import { useEffect, useMemo, useState } from 'react'; import ReactSelect from 'react-select'; -import styled from 'styled-components'; import CdcDetails from './cdcDetails'; class TableCloneSummary { @@ -264,21 +263,6 @@ export const SnapshotStatusTable = ({ status }: SnapshotStatusProps) => { ); }; -const Trigger = styled( - ({ isActive, ...props }: { isActive?: boolean } & Tabs.TabsTriggerProps) => ( - - ) -)<{ isActive?: boolean }>` - background-color: ${({ theme, isActive }) => - isActive ? theme.colors.accent.surface.selected : 'white'}; - - font-weight: ${({ isActive }) => (isActive ? 'bold' : 'normal')}; - - &:hover { - color: ${({ theme }) => theme.colors.accent.text.highContrast}; - } -`; - type CDCMirrorStatusProps = { cdc: CDCMirrorStatus; rows: SyncStatusRow[]; @@ -298,11 +282,6 @@ export function CDCMirror({ snapshot = ; } - const handleTab = (tabVal: string) => { - localStorage.setItem('mirrortab', tabVal); - setSelectedTab(tabVal); - }; - useEffect(() => { if (typeof window !== 'undefined') { setSelectedTab(localStorage?.getItem('mirrortab') || 'tab1'); @@ -310,48 +289,26 @@ export function CDCMirror({ }, []); return ( - handleTab(val)} - style={{ marginTop: '2rem' }} - > - - - Overview - - - Sync Status - - - Initial Copy - - - - - - - {syncStatusChild} - - - {snapshot} - - + + + Overview + Sync Status + Initial Copy + + + + + + {syncStatusChild} + {snapshot} + + ); } diff --git a/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx b/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx index 32992f871..738218c64 100644 --- a/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx +++ b/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx @@ -5,10 +5,9 @@ import PeerButton from '@/components/PeerComponent'; import TimeLabel from '@/components/TimeComponent'; import { FlowConnectionConfigs } from '@/grpc_generated/flow'; import { dBTypeFromJSON } from '@/grpc_generated/peers'; -import { Badge } from '@/lib/Badge'; -import { Icon } from '@/lib/Icon'; import { Label } from '@/lib/Label'; import moment from 'moment'; +import { MirrorError } from '../../mirror-status'; import MirrorValues from './configValues'; import TablePairs from './tablePairs'; @@ -33,11 +32,8 @@ function CdcDetails({ syncs, createdAt, mirrorConfig }: props) {
-
diff --git a/ui/app/mirrors/edit/[mirrorId]/tablePairs.tsx b/ui/app/mirrors/edit/[mirrorId]/tablePairs.tsx index 5289e77a0..516b8dd3c 100644 --- a/ui/app/mirrors/edit/[mirrorId]/tablePairs.tsx +++ b/ui/app/mirrors/edit/[mirrorId]/tablePairs.tsx @@ -15,7 +15,7 @@ const TablePairs = ({ tables }: { tables?: TableMapping[] }) => { }, [tables, searchQuery]); if (tables) return ( - <> +
{ }} />
- - - - - + - Destination Table - - - - - {shownTables?.map((table) => ( - - - + + - ))} - -
- Source Table - +
- {table.sourceTableIdentifier} - - {table.destinationTableIdentifier} - + Source Table + + Destination Table +
- + + + {shownTables?.map((table) => ( + + + {table.sourceTableIdentifier} + + + {table.destinationTableIdentifier} + + + ))} + + +
+ ); }; diff --git a/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx b/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx new file mode 100644 index 000000000..b6d14eed8 --- /dev/null +++ b/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx @@ -0,0 +1,57 @@ +'use client'; +import { Button } from '@/lib/Button'; +import { Label } from '@/lib/Label'; +import { ProgressCircle } from '@/lib/ProgressCircle'; +import { useState } from 'react'; +import { toast } from 'react-toastify'; + +const notifyErr = (errMsg: string) => { + toast.error(errMsg, { + position: toast.POSITION.BOTTOM_CENTER, + }); +}; + +const AckButton = ({ ack, id }: { ack: boolean; id: number | bigint }) => { + const [loading, setLoading] = useState(false); + const [updated, setUpdated] = useState(false); + // handleAck updates ack to true for the given mirrorID + const handleAck = async (mirrorID: bigint | number) => { + setLoading(true); + const updateResResult = await fetch('/api/mirrors/alerts', { + method: 'PUT', + body: JSON.stringify({ + mirrorIDStringList: [mirrorID.toString()], + }), + }); + const updateRes = await updateResResult.json(); + setLoading(false); + if (!updateRes) { + notifyErr('Something went wrong when trying to acknowledge'); + return; + } + setUpdated(true); + }; + return ( + <> + {ack !== true && updated !== true ? ( + + ) : ( + + )} + + ); +}; + +export default AckButton; diff --git a/ui/app/mirrors/errors/[mirrorName]/page.tsx b/ui/app/mirrors/errors/[mirrorName]/page.tsx new file mode 100644 index 000000000..899b25a49 --- /dev/null +++ b/ui/app/mirrors/errors/[mirrorName]/page.tsx @@ -0,0 +1,101 @@ +import { AlertErr } from '@/app/dto/MirrorsDTO'; +import prisma from '@/app/utils/prisma'; +import TimeLabel from '@/components/TimeComponent'; +import { Label } from '@/lib/Label'; +import { Table, TableCell, TableRow } from '@/lib/Table'; +import { ToastContainer } from 'react-toastify'; +import 'react-toastify/dist/ReactToastify.css'; +import AckButton from './ackbutton'; + +type MirrorErrorProps = { + params: { mirrorName: string }; +}; + +const MirrorError = async ({ params: { mirrorName } }: MirrorErrorProps) => { + const mirrorErrors: AlertErr[] = await prisma.flow_errors.findMany({ + where: { + flow_name: mirrorName, + error_type: 'error', + }, + orderBy: { + error_timestamp: 'desc', + }, + }); + + return ( + <> +
+ +
+
+ + + +
+ +
+ +
+ + Type + + + + Message + + + } + > + {mirrorErrors.map((mirrorError) => ( + + + {mirrorError.error_type.toUpperCase()} + + + + + + {mirrorError.error_message} + + + + + + ))} +
+
+
+
+ + + ); +}; + +export default MirrorError; diff --git a/ui/app/mirrors/mirror-status.tsx b/ui/app/mirrors/mirror-status.tsx new file mode 100644 index 000000000..b64d87ff2 --- /dev/null +++ b/ui/app/mirrors/mirror-status.tsx @@ -0,0 +1,99 @@ +'use client'; + +import { Button } from '@/lib/Button'; +import { Icon } from '@/lib/Icon'; +import { Label } from '@/lib/Label'; +import { ProgressCircle } from '@/lib/ProgressCircle'; +import Link from 'next/link'; +import { useRouter } from 'next/navigation'; +import { useEffect, useState } from 'react'; +export const ErrorModal = ({ flowName }: { flowName: string }) => { + const router = useRouter(); + return ( + + + + ); +}; + +export const MirrorError = ({ flowName }: { flowName: string }) => { + const [flowStatus, setFlowStatus] = useState(); + const [isLoading, setIsLoading] = useState(true); + const [error, setError] = useState(null); + + useEffect(() => { + const fetchData = async () => { + setIsLoading(true); + try { + const response = await fetch(`/api/mirrors/alerts`, { + method: 'POST', + headers: { + 'Content-Type': 'application/json', + }, + body: JSON.stringify({ flowName }), + }); + + if (!response.ok) { + throw new Error('Network response was not ok'); + } + + const flowStatus = await response.json(); + setFlowStatus(flowStatus); + } catch (err: any) { + setError(err.message); + } finally { + setIsLoading(false); + } + }; + + fetchData(); + }, [flowName]); + + if (isLoading) { + return ( +
+ +
+ ); + } + + if (error) { + return ( +
+ +
+ ); + } + + if (flowStatus === 'healthy') { + return ( + + + + ); + } + + return ; +}; diff --git a/ui/app/mirrors/page.tsx b/ui/app/mirrors/page.tsx index 717af7b95..c98b40285 100644 --- a/ui/app/mirrors/page.tsx +++ b/ui/app/mirrors/page.tsx @@ -1,3 +1,5 @@ +'use client'; + import { QRepConfig } from '@/grpc_generated/flow'; import { Button } from '@/lib/Button'; import { Header } from '@/lib/Header'; @@ -5,63 +7,45 @@ import { Icon } from '@/lib/Icon'; import { Label } from '@/lib/Label'; import { LayoutMain } from '@/lib/Layout'; import { Panel } from '@/lib/Panel'; +import { ProgressCircle } from '@/lib/ProgressCircle'; import Link from 'next/link'; -import { getTruePeer } from '../api/peers/route'; -import prisma from '../utils/prisma'; +import useSWR from 'swr'; import { CDCFlows, QRepFlows } from './tables'; export const dynamic = 'force-dynamic'; -const stringifyConfig = (flowArray: any[]) => { - flowArray.forEach((flow) => { - if (flow.config_proto) { - flow.config_proto = new TextDecoder().decode(flow.config_proto); - } - }); -}; - -export default async function Mirrors() { - let mirrors = await prisma.flows.findMany({ - distinct: 'name', - include: { - sourcePeer: true, - destinationPeer: true, - }, - }); +const fetcher = (...args: [any]) => fetch(...args).then((res) => res.json()); - const flows = mirrors.map((mirror) => { - let newMirror: any = { - ...mirror, - sourcePeer: getTruePeer(mirror.sourcePeer), - destinationPeer: getTruePeer(mirror.destinationPeer), - }; - return newMirror; - }); +export default function Mirrors() { + const { + data: flows, + error, + isLoading, + }: { data: [any]; error: any; isLoading: boolean } = useSWR( + '/api/mirrors', + fetcher + ); - let cdcFlows = flows.filter((flow) => { + let cdcFlows = flows?.filter((flow) => { return !flow.query_string; }); - let qrepFlows = flows.filter((flow) => { + let qrepFlows = flows?.filter((flow) => { if (flow.config_proto && flow.query_string) { - let config = QRepConfig.decode(flow.config_proto); + let config = QRepConfig.decode(flow.config_proto.data); const watermarkCol = config.watermarkColumn.toLowerCase(); return watermarkCol !== 'xmin' && watermarkCol !== 'ctid'; } return false; }); - let xminFlows = flows.filter((flow) => { + let xminFlows = flows?.filter((flow) => { if (flow.config_proto && flow.query_string) { - let config = QRepConfig.decode(flow.config_proto); + let config = QRepConfig.decode(flow.config_proto.data); return config.watermarkColumn.toLowerCase() === 'xmin'; } return false; }); - stringifyConfig(cdcFlows); - stringifyConfig(qrepFlows); - stringifyConfig(xminFlows); - return ( @@ -84,15 +68,28 @@ export default async function Mirrors() { Mirrors - - - - - - - - - + {isLoading && ( + +
+ +
+
+ )} + {!isLoading && ( + + + + )} + {!isLoading && ( + + + + )} + {!isLoading && ( + + + + )}
); } diff --git a/ui/app/mirrors/tables.tsx b/ui/app/mirrors/tables.tsx index 743bbc28a..6c1289bef 100644 --- a/ui/app/mirrors/tables.tsx +++ b/ui/app/mirrors/tables.tsx @@ -7,6 +7,7 @@ import { SearchField } from '@/lib/SearchField'; import { Table, TableCell, TableRow } from '@/lib/Table'; import Link from 'next/link'; import { useMemo, useState } from 'react'; +import { MirrorError } from './mirror-status'; export function CDCFlows({ cdcFlows }: { cdcFlows: any }) { const [searchQuery, setSearchQuery] = useState(''); @@ -43,15 +44,26 @@ export function CDCFlows({ cdcFlows }: { cdcFlows: any }) { }} header={ - {['Name', 'Source', 'Destination', 'Start Time', ''].map( - (heading, index) => ( - - - - ) - )} + {[ + 'Name', + 'Source', + 'Destination', + 'Start Time', + 'Status', + '', + ].map((heading, index) => ( + + + + ))} } > @@ -77,6 +89,9 @@ export function CDCFlows({ cdcFlows }: { cdcFlows: any }) { + + + fetch(...args).then((res) => res.json()); export default function SidebarComponent(props: { logout?: boolean }) { const timezones = ['UTC', 'Local', 'Relative']; const setZone = useTZStore((state) => state.setZone); const zone = useTZStore((state) => state.timezone); + + const { + data: version, + error, + isLoading, + }: { data: UVersionResponse; error: any; isLoading: boolean } = useSWR( + '/api/version', + fetcher + ); + return ( -
+
Timezone:} @@ -63,7 +75,15 @@ export default function SidebarComponent(props: { logout?: boolean }) { {props.logout && } } - bottomLabel={} + bottomLabel={ +
+ +
+ } > >> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.softDeleteColName = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.syncedAtColName = reader.string(); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.softDelete = reader.bool(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): PeerDBColumns { + return { + softDeleteColName: isSet(object.softDeleteColName) ? String(object.softDeleteColName) : "", + syncedAtColName: isSet(object.syncedAtColName) ? String(object.syncedAtColName) : "", + softDelete: isSet(object.softDelete) ? Boolean(object.softDelete) : false, + }; + }, + + toJSON(message: PeerDBColumns): unknown { + const obj: any = {}; + if (message.softDeleteColName !== "") { + obj.softDeleteColName = message.softDeleteColName; + } + if (message.syncedAtColName !== "") { + obj.syncedAtColName = message.syncedAtColName; + } + if (message.softDelete === true) { + obj.softDelete = message.softDelete; + } + return obj; + }, + + create, I>>(base?: I): PeerDBColumns { + return PeerDBColumns.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): PeerDBColumns { + const message = createBasePeerDBColumns(); + message.softDeleteColName = object.softDeleteColName ?? ""; + message.syncedAtColName = object.syncedAtColName ?? ""; + message.softDelete = object.softDelete ?? false; + return message; + }, +}; + +function createBaseGetOpenConnectionsForUserResult(): GetOpenConnectionsForUserResult { + return { userName: "", currentOpenConnections: 0 }; +} + +export const GetOpenConnectionsForUserResult = { + encode(message: GetOpenConnectionsForUserResult, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.userName !== "") { + writer.uint32(10).string(message.userName); + } + if (message.currentOpenConnections !== 0) { + writer.uint32(16).int64(message.currentOpenConnections); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): GetOpenConnectionsForUserResult { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseGetOpenConnectionsForUserResult(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.userName = reader.string(); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.currentOpenConnections = longToNumber(reader.int64() as Long); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): GetOpenConnectionsForUserResult { + return { + userName: isSet(object.userName) ? String(object.userName) : "", + currentOpenConnections: isSet(object.currentOpenConnections) ? Number(object.currentOpenConnections) : 0, + }; + }, + + toJSON(message: GetOpenConnectionsForUserResult): unknown { + const obj: any = {}; + if (message.userName !== "") { + obj.userName = message.userName; + } + if (message.currentOpenConnections !== 0) { + obj.currentOpenConnections = Math.round(message.currentOpenConnections); + } + return obj; + }, + + create, I>>(base?: I): GetOpenConnectionsForUserResult { + return GetOpenConnectionsForUserResult.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): GetOpenConnectionsForUserResult { + const message = createBaseGetOpenConnectionsForUserResult(); + message.userName = object.userName ?? ""; + message.currentOpenConnections = object.currentOpenConnections ?? 0; + return message; + }, +}; + declare const self: any | undefined; declare const window: any | undefined; declare const global: any | undefined; diff --git a/ui/package-lock.json b/ui/package-lock.json index a9ace2b05..bde9fe3a3 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -10,7 +10,7 @@ "dependencies": { "@grpc/grpc-js": "^1.9.13", "@monaco-editor/react": "^4.6.0", - "@prisma/client": "^5.7.0", + "@prisma/client": "^5.7.1", "@radix-ui/react-checkbox": "^1.0.4", "@radix-ui/react-collapsible": "^1.0.3", "@radix-ui/react-dialog": "^1.0.5", @@ -29,10 +29,10 @@ "@types/node": "^20.10.5", "@types/react": "^18.2.45", "@types/react-dom": "^18.2.18", - "classnames": "^2.3.2", + "classnames": "^2.3.3", "clsx": "^2.0.0", "long": "^5.2.3", - "lucide-react": "^0.298.0", + "lucide-react": "^0.302.0", "material-symbols": "^0.14.3", "moment": "^2.29.4", "moment-timezone": "^0.5.43", @@ -43,18 +43,19 @@ "react-dom": "18.2.0", "react-select": "^5.8.0", "react-spinners": "^0.13.8", - "styled-components": "^6.1.1", + "react-toastify": "^9.1.3", + "styled-components": "^6.1.3", "swr": "^2.2.4", "zod": "^3.22.4", "zustand": "^4.4.7" }, "devDependencies": { - "@storybook/addon-essentials": "^7.6.5", - "@storybook/addon-interactions": "^7.6.5", - "@storybook/addon-links": "^7.6.5", + "@storybook/addon-essentials": "^7.6.6", + "@storybook/addon-interactions": "^7.6.6", + "@storybook/addon-links": "^7.6.6", "@storybook/addon-styling": "^1.3.7", "@storybook/blocks": "^7.3.0", - "@storybook/nextjs": "^7.6.5", + "@storybook/nextjs": "^7.6.6", "@storybook/react": "^7.3.0", "@storybook/testing-library": "^0.2.2", "autoprefixer": "^10.4.16", @@ -63,15 +64,15 @@ "eslint-config-next": "^14.0.4", "eslint-config-prettier": "^9.1.0", "eslint-plugin-storybook": "^0.6.15", - "gh-pages": "^6.1.0", + "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.32", "prettier": "^3.1.1", "prettier-plugin-organize-imports": "^3.2.4", - "prisma": "^5.7.0", - "storybook": "^7.6.5", + "prisma": "^5.7.1", + "storybook": "^7.6.6", "string-width": "^7.0.0", - "tailwindcss": "^3.3.6", + "tailwindcss": "^3.4.0", "tailwindcss-animate": "^1.0.7", "typescript": "^5.3.3", "webpack": "^5.89.0" @@ -2354,14 +2355,14 @@ "integrity": "sha512-W2P2c/VRW1/1tLox0mVUalvnWXxavmv/Oum2aPsRcoDJuob75FC3Y8FbpfLwUegRcxINtGUMPq0tFCvYNTBXNA==" }, "node_modules/@emotion/react": { - "version": "11.11.1", - "resolved": "https://registry.npmjs.org/@emotion/react/-/react-11.11.1.tgz", - "integrity": "sha512-5mlW1DquU5HaxjLkfkGN1GA/fvVGdyHURRiX/0FHl2cfIfRxSOfmxEH5YS43edp0OldZrZ+dkBKbngxcNCdZvA==", + "version": "11.11.3", + "resolved": "https://registry.npmjs.org/@emotion/react/-/react-11.11.3.tgz", + "integrity": "sha512-Cnn0kuq4DoONOMcnoVsTOR8E+AdnKFf//6kUWc4LCdnxj31pZWn7rIULd6Y7/Js1PiPHzn7SKCM9vB/jBni8eA==", "dependencies": { "@babel/runtime": "^7.18.3", "@emotion/babel-plugin": "^11.11.0", "@emotion/cache": "^11.11.0", - "@emotion/serialize": "^1.1.2", + "@emotion/serialize": "^1.1.3", "@emotion/use-insertion-effect-with-fallbacks": "^1.0.1", "@emotion/utils": "^1.2.1", "@emotion/weak-memoize": "^0.3.1", @@ -2377,9 +2378,9 @@ } }, "node_modules/@emotion/serialize": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@emotion/serialize/-/serialize-1.1.2.tgz", - "integrity": "sha512-zR6a/fkFP4EAcCMQtLOhIgpprZOwNmCldtpaISpvz348+DP4Mz8ZoKaGGCQpbzepNIUWbq4w6hNZkwDyKoS+HA==", + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/@emotion/serialize/-/serialize-1.1.3.tgz", + "integrity": "sha512-iD4D6QVZFDhcbH0RAG1uVu1CwVLMWUkCvAqqlewO/rxf8+87yIBAlt4+AxMiiKPLs5hFc0owNk/sLLAOROw3cA==", "dependencies": { "@emotion/hash": "^0.9.1", "@emotion/memoize": "^0.8.1", @@ -3027,7 +3028,6 @@ "version": "8.0.2", "resolved": "https://registry.npmjs.org/@isaacs/cliui/-/cliui-8.0.2.tgz", "integrity": "sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA==", - "dev": true, "dependencies": { "string-width": "^5.1.2", "string-width-cjs": "npm:string-width@^4.2.0", @@ -3044,7 +3044,6 @@ "version": "6.0.1", "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-6.0.1.tgz", "integrity": "sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA==", - "dev": true, "engines": { "node": ">=12" }, @@ -3056,7 +3055,6 @@ "version": "5.1.2", "resolved": "https://registry.npmjs.org/string-width/-/string-width-5.1.2.tgz", "integrity": "sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA==", - "dev": true, "dependencies": { "eastasianwidth": "^0.2.0", "emoji-regex": "^9.2.2", @@ -3073,7 +3071,6 @@ "version": "7.1.0", "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-7.1.0.tgz", "integrity": "sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ==", - "dev": true, "dependencies": { "ansi-regex": "^6.0.1" }, @@ -3580,7 +3577,6 @@ "version": "0.11.0", "resolved": "https://registry.npmjs.org/@pkgjs/parseargs/-/parseargs-0.11.0.tgz", "integrity": "sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg==", - "dev": true, "optional": true, "engines": { "node": ">=14" @@ -3651,9 +3647,9 @@ } }, "node_modules/@prisma/client": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/@prisma/client/-/client-5.7.0.tgz", - "integrity": "sha512-cZmglCrfNbYpzUtz7HscVHl38e9CrUs31nrVoGUK1nIPXGgt8hT4jj2s657UXcNdQ/jBUxDgGmHyu2Nyrq1txg==", + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/@prisma/client/-/client-5.7.1.tgz", + "integrity": "sha512-TUSa4nUcC4nf/e7X3jyO1pEd6XcI/TLRCA0KjkA46RDIpxUaRsBYEOqITwXRW2c0bMFyKcCRXrH4f7h4q9oOlg==", "hasInstallScript": true, "engines": { "node": ">=16.13" @@ -3668,48 +3664,48 @@ } }, "node_modules/@prisma/debug": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/@prisma/debug/-/debug-5.7.0.tgz", - "integrity": "sha512-tZ+MOjWlVvz1kOEhNYMa4QUGURY+kgOUBqLHYIV8jmCsMuvA1tWcn7qtIMLzYWCbDcQT4ZS8xDgK0R2gl6/0wA==", + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/@prisma/debug/-/debug-5.7.1.tgz", + "integrity": "sha512-yrVSO/YZOxdeIxcBtZ5BaNqUfPrZkNsAKQIQg36cJKMxj/VYK3Vk5jMKkI+gQLl0KReo1YvX8GWKfV788SELjw==", "devOptional": true }, "node_modules/@prisma/engines": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/@prisma/engines/-/engines-5.7.0.tgz", - "integrity": "sha512-TkOMgMm60n5YgEKPn9erIvFX2/QuWnl3GBo6yTRyZKk5O5KQertXiNnrYgSLy0SpsKmhovEPQb+D4l0SzyE7XA==", + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/@prisma/engines/-/engines-5.7.1.tgz", + "integrity": "sha512-R+Pqbra8tpLP2cvyiUpx+SIKglav3nTCpA+rn6826CThviQ8yvbNG0s8jNpo51vS9FuZO3pOkARqG062vKX7uA==", "devOptional": true, "hasInstallScript": true, "dependencies": { - "@prisma/debug": "5.7.0", - "@prisma/engines-version": "5.7.0-41.79fb5193cf0a8fdbef536e4b4a159cad677ab1b9", - "@prisma/fetch-engine": "5.7.0", - "@prisma/get-platform": "5.7.0" + "@prisma/debug": "5.7.1", + "@prisma/engines-version": "5.7.1-1.0ca5ccbcfa6bdc81c003cf549abe4269f59c41e5", + "@prisma/fetch-engine": "5.7.1", + "@prisma/get-platform": "5.7.1" } }, "node_modules/@prisma/engines-version": { - "version": "5.7.0-41.79fb5193cf0a8fdbef536e4b4a159cad677ab1b9", - "resolved": "https://registry.npmjs.org/@prisma/engines-version/-/engines-version-5.7.0-41.79fb5193cf0a8fdbef536e4b4a159cad677ab1b9.tgz", - "integrity": "sha512-V6tgRVi62jRwTm0Hglky3Scwjr/AKFBFtS+MdbsBr7UOuiu1TKLPc6xfPiyEN1+bYqjEtjxwGsHgahcJsd1rNg==", + "version": "5.7.1-1.0ca5ccbcfa6bdc81c003cf549abe4269f59c41e5", + "resolved": "https://registry.npmjs.org/@prisma/engines-version/-/engines-version-5.7.1-1.0ca5ccbcfa6bdc81c003cf549abe4269f59c41e5.tgz", + "integrity": "sha512-dIR5IQK/ZxEoWRBDOHF87r1Jy+m2ih3Joi4vzJRP+FOj5yxCwS2pS5SBR3TWoVnEK1zxtLI/3N7BjHyGF84fgw==", "devOptional": true }, "node_modules/@prisma/fetch-engine": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/@prisma/fetch-engine/-/fetch-engine-5.7.0.tgz", - "integrity": "sha512-zIn/qmO+N/3FYe7/L9o+yZseIU8ivh4NdPKSkQRIHfg2QVTVMnbhGoTcecbxfVubeTp+DjcbjS0H9fCuM4W04w==", + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/@prisma/fetch-engine/-/fetch-engine-5.7.1.tgz", + "integrity": "sha512-9ELauIEBkIaEUpMIYPRlh5QELfoC6pyHolHVQgbNxglaINikZ9w9X7r1TIePAcm05pCNp2XPY1ObQIJW5nYfBQ==", "devOptional": true, "dependencies": { - "@prisma/debug": "5.7.0", - "@prisma/engines-version": "5.7.0-41.79fb5193cf0a8fdbef536e4b4a159cad677ab1b9", - "@prisma/get-platform": "5.7.0" + "@prisma/debug": "5.7.1", + "@prisma/engines-version": "5.7.1-1.0ca5ccbcfa6bdc81c003cf549abe4269f59c41e5", + "@prisma/get-platform": "5.7.1" } }, "node_modules/@prisma/get-platform": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/@prisma/get-platform/-/get-platform-5.7.0.tgz", - "integrity": "sha512-ZeV/Op4bZsWXuw5Tg05WwRI8BlKiRFhsixPcAM+5BKYSiUZiMKIi713tfT3drBq8+T0E1arNZgYSA9QYcglWNA==", + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/@prisma/get-platform/-/get-platform-5.7.1.tgz", + "integrity": "sha512-eDlswr3a1m5z9D/55Iyt/nZqS5UpD+DZ9MooBB3hvrcPhDQrcf9m4Tl7buy4mvAtrubQ626ECtb8c6L/f7rGSQ==", "devOptional": true, "dependencies": { - "@prisma/debug": "5.7.0" + "@prisma/debug": "5.7.1" } }, "node_modules/@protobufjs/aspromise": { @@ -4926,12 +4922,12 @@ "dev": true }, "node_modules/@storybook/addon-actions": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-actions/-/addon-actions-7.6.5.tgz", - "integrity": "sha512-lW/m9YcaNfBZk+TZLxyzHdd563mBWpsUIveOKYjcPdl/q0FblWWZrRsFHqwLK1ldZ4AZXs8J/47G8CBr6Ew2uQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-actions/-/addon-actions-7.6.6.tgz", + "integrity": "sha512-mLJip9Evb2Chj7ymKbpaybe5NgDy3Du7oSWeURPy/0qXJ2cBqHWnhZ8CTK2DasrstsUhQSJaZVXHhaENT+fn+g==", "dev": true, "dependencies": { - "@storybook/core-events": "7.6.5", + "@storybook/core-events": "7.6.6", "@storybook/global": "^5.0.0", "@types/uuid": "^9.0.1", "dequal": "^2.0.2", @@ -4944,9 +4940,9 @@ } }, "node_modules/@storybook/addon-backgrounds": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-backgrounds/-/addon-backgrounds-7.6.5.tgz", - "integrity": "sha512-wZZOL19vg4TTRtOTl71XKqPe5hQx3XUh9Fle0wOi91FiFrBdqusrppnyS89wPS8RQG5lXEOFEUvYcMmdCcdZfw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-backgrounds/-/addon-backgrounds-7.6.6.tgz", + "integrity": "sha512-w5dZ/0cOe55M2G/Lx9f+Ptk4txUPb+Ng+KqEvTaTNqHoh0Xw4QxEn/ciJwmh1u1g3aMZsOgOvwPwug7ykmLgsA==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", @@ -4959,12 +4955,12 @@ } }, "node_modules/@storybook/addon-controls": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-controls/-/addon-controls-7.6.5.tgz", - "integrity": "sha512-EdSZ2pYf74mOXZGGJ22lrDvdvL0YKc95iWv9FFEhUFOloMy/0OZPB2ybYmd2KVCy3SeIE4Zfeiw8pDXdCUniOQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-controls/-/addon-controls-7.6.6.tgz", + "integrity": "sha512-VAXXfPLi1M3RXhBf3uIBZ2hrD9UPDe7yvXHIlCzgj1HIJELODCFyUc+RtvN0mPc/nnlEfzhGfJtenZou5LYwIw==", "dev": true, "dependencies": { - "@storybook/blocks": "7.6.5", + "@storybook/blocks": "7.6.6", "lodash": "^4.17.21", "ts-dedent": "^2.0.0" }, @@ -4974,26 +4970,26 @@ } }, "node_modules/@storybook/addon-docs": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-docs/-/addon-docs-7.6.5.tgz", - "integrity": "sha512-D9tZyD41IujCHiPYdfS2bKtZRJPNwO4EydzyqODXppomluhFbY3uTEaf0H1UFnJLQxWNXZ7rr3aS0V3O6yu8pA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-docs/-/addon-docs-7.6.6.tgz", + "integrity": "sha512-l4gtoNTn1wHE11x44te1cDkqfm+/w+eNonHe56bwgSqETclS5z18wvM9bQZF32G6C9fpSefaJW3cxVvcuJL1fg==", "dev": true, "dependencies": { "@jest/transform": "^29.3.1", "@mdx-js/react": "^2.1.5", - "@storybook/blocks": "7.6.5", - "@storybook/client-logger": "7.6.5", - "@storybook/components": "7.6.5", - "@storybook/csf-plugin": "7.6.5", - "@storybook/csf-tools": "7.6.5", + "@storybook/blocks": "7.6.6", + "@storybook/client-logger": "7.6.6", + "@storybook/components": "7.6.6", + "@storybook/csf-plugin": "7.6.6", + "@storybook/csf-tools": "7.6.6", "@storybook/global": "^5.0.0", "@storybook/mdx2-csf": "^1.0.0", - "@storybook/node-logger": "7.6.5", - "@storybook/postinstall": "7.6.5", - "@storybook/preview-api": "7.6.5", - "@storybook/react-dom-shim": "7.6.5", - "@storybook/theming": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/node-logger": "7.6.6", + "@storybook/postinstall": "7.6.6", + "@storybook/preview-api": "7.6.6", + "@storybook/react-dom-shim": "7.6.6", + "@storybook/theming": "7.6.6", + "@storybook/types": "7.6.6", "fs-extra": "^11.1.0", "remark-external-links": "^8.0.0", "remark-slug": "^6.0.0", @@ -5009,24 +5005,24 @@ } }, "node_modules/@storybook/addon-essentials": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-essentials/-/addon-essentials-7.6.5.tgz", - "integrity": "sha512-VCLj1JAEpGoqF5iFJOo1CZFFck/tg4m/98DLdQuNuXvxT6jqaF0NI9UUQuJLIGteDCR7NKRbTFc1hV3/Ev+Ziw==", - "dev": true, - "dependencies": { - "@storybook/addon-actions": "7.6.5", - "@storybook/addon-backgrounds": "7.6.5", - "@storybook/addon-controls": "7.6.5", - "@storybook/addon-docs": "7.6.5", - "@storybook/addon-highlight": "7.6.5", - "@storybook/addon-measure": "7.6.5", - "@storybook/addon-outline": "7.6.5", - "@storybook/addon-toolbars": "7.6.5", - "@storybook/addon-viewport": "7.6.5", - "@storybook/core-common": "7.6.5", - "@storybook/manager-api": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/preview-api": "7.6.5", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-essentials/-/addon-essentials-7.6.6.tgz", + "integrity": "sha512-OQ8A6r06mg/HvyIk/j2Gt9DK5Qtqgtwq2Ydm5IgVW6gZsuRnv1FAeUG6okf8oXowAzpYoHdsDmCVwNOAGWGO7w==", + "dev": true, + "dependencies": { + "@storybook/addon-actions": "7.6.6", + "@storybook/addon-backgrounds": "7.6.6", + "@storybook/addon-controls": "7.6.6", + "@storybook/addon-docs": "7.6.6", + "@storybook/addon-highlight": "7.6.6", + "@storybook/addon-measure": "7.6.6", + "@storybook/addon-outline": "7.6.6", + "@storybook/addon-toolbars": "7.6.6", + "@storybook/addon-viewport": "7.6.6", + "@storybook/core-common": "7.6.6", + "@storybook/manager-api": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/preview-api": "7.6.6", "ts-dedent": "^2.0.0" }, "funding": { @@ -5039,9 +5035,9 @@ } }, "node_modules/@storybook/addon-highlight": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-highlight/-/addon-highlight-7.6.5.tgz", - "integrity": "sha512-CxzmIb30F9nLPQwT0lCPYhOAwGlGF4IkgkO8hYA7VfGCGUkJZEyyN/YkP/ZCUSdCIRChDBouR3KiFFd4mDFKzg==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-highlight/-/addon-highlight-7.6.6.tgz", + "integrity": "sha512-B85UaCts2uMpa0yHBSnupzy2WCdW4vfB+lfaBug9beyOyZQdel07BumblE0KwSJftYgdCNPUZ5MRlqEDzMLTWQ==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0" @@ -5052,13 +5048,13 @@ } }, "node_modules/@storybook/addon-interactions": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-interactions/-/addon-interactions-7.6.5.tgz", - "integrity": "sha512-8Hzt9u1DQzFvtGER/hCGIvGpCoVwzVoqpM98f2KAIVx/NMFmRW7UyKihXzw1j2t4q2ZaF2jZDYWCBqlP+iwILA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-interactions/-/addon-interactions-7.6.6.tgz", + "integrity": "sha512-EJWx6ciJPgv1c75tB/M4smWDpPDGM/L24v4DZxGpl1eV3oQOSQCKImG5btwoy6QcIi68ozroUHdUti/kzCKS1w==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", - "@storybook/types": "7.6.5", + "@storybook/types": "7.6.6", "jest-mock": "^27.0.6", "polished": "^4.2.2", "ts-dedent": "^2.2.0" @@ -5069,9 +5065,9 @@ } }, "node_modules/@storybook/addon-links": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-links/-/addon-links-7.6.5.tgz", - "integrity": "sha512-Lx4Ng+iXt0YpIrKGr+nOZlpN9ypOoEDoP/7bZ6m7GXuVAkDm3JrRCBp7e2ZKSKcTxPdjPuO9HVKkIjtqjINlpw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-links/-/addon-links-7.6.6.tgz", + "integrity": "sha512-NEcqOz6zZ1dJnCcVmYdaQTAMAGIb8NFAZGnr9DU0q+t4B1fTaWUgqLtBM5V6YqIrXGSC/oKLpjWUkS5UpswlHA==", "dev": true, "dependencies": { "@storybook/csf": "^0.1.2", @@ -5092,9 +5088,9 @@ } }, "node_modules/@storybook/addon-measure": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-measure/-/addon-measure-7.6.5.tgz", - "integrity": "sha512-tlUudVQSrA+bwI4dhO8J7nYHtYdylcBZ86ybnqMmdTthsnyc7jnaFVQwbb6bbQJpPxvEvoNds5bVGUFocuvymQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-measure/-/addon-measure-7.6.6.tgz", + "integrity": "sha512-b4hyCudlhsbYN1We8pfZHZJ0i0sfC8+GJvrqZQqdSqGicUmA00mggY1GE+gEoHziQ5/4auxFRS3HfUgaQWUNjg==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", @@ -5106,9 +5102,9 @@ } }, "node_modules/@storybook/addon-outline": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-outline/-/addon-outline-7.6.5.tgz", - "integrity": "sha512-P7X4+Z9L/l/RZW9UvvM+iuK2SUHD22KPc+dbYOifRXDovUqhfmcKVh1CUqTDMyZrg2ZAbropehMz1eI9BlQfxg==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-outline/-/addon-outline-7.6.6.tgz", + "integrity": "sha512-BMjpjzNEnN8LC7JK92WCXyWgmJwAaEQjRDinr7eD4cBt4Uas5kbciw1g8PtTnh0GbYUsImKao0nzakSVObAdzg==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0", @@ -5188,9 +5184,9 @@ } }, "node_modules/@storybook/addon-toolbars": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-toolbars/-/addon-toolbars-7.6.5.tgz", - "integrity": "sha512-/zqWbVNE/SHc8I5Prnd2Q8U57RGEIYvHfeXjfkuLcE2Quc4Iss4x/9eU7SKu4jm+IOO2s0wlN6HcqI3XEf2XxA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-toolbars/-/addon-toolbars-7.6.6.tgz", + "integrity": "sha512-sQm5+FcoSMSGn1ioXHoukO6OhUlcNZil0/fonAY50uvp6Z4DyI0FTU7BKIm/NoMqAExQk3sZRfAC/nZZ9Epb0Q==", "dev": true, "funding": { "type": "opencollective", @@ -5198,9 +5194,9 @@ } }, "node_modules/@storybook/addon-viewport": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/addon-viewport/-/addon-viewport-7.6.5.tgz", - "integrity": "sha512-9ghKTaduIUvQ6oShmWLuwMeTjtMR4RgKeKHrTJ7THMqvE/ydDPCYeL7ugF65ocXZSEz/QmxdK7uL686ZMKsqNA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/addon-viewport/-/addon-viewport-7.6.6.tgz", + "integrity": "sha512-/ijbzDf1Iq30LvZW2NE8cO4TeHusw0N+jTDUK1+vhxGNMFo9DUIgRkAi6VpFEfS0aQ5d82523WSWzVso7b/Hmg==", "dev": true, "dependencies": { "memoizerific": "^1.11.3" @@ -5211,13 +5207,13 @@ } }, "node_modules/@storybook/api": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/api/-/api-7.6.5.tgz", - "integrity": "sha512-o1RH47iIgG4ie4hjJP1HgsCiuTKlGW0egaAy6E6Np3bDmujy5udWEf8vnXbcaBerc5ZSrQs45kfSWugHy2a4FA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/api/-/api-7.6.6.tgz", + "integrity": "sha512-e3k45k7twP3z5ZJ+rCCaHI+jmYm5yoFo2eLjYmnYFUv2V3vvYPgqD2CiT0crne7uWmhpRxP49aU9DEvQaEZtdA==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.5", - "@storybook/manager-api": "7.6.5" + "@storybook/client-logger": "7.6.6", + "@storybook/manager-api": "7.6.6" }, "funding": { "type": "opencollective", @@ -5225,22 +5221,22 @@ } }, "node_modules/@storybook/blocks": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/blocks/-/blocks-7.6.5.tgz", - "integrity": "sha512-/NjuYkPks5w9lKn47KLgVC5cBkwfc+ERAp0CY0Xe//BQJkP+bcI8lE8d9Qc9IXFbOTvYEULeQrFgCkesk5BmLg==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/blocks/-/blocks-7.6.6.tgz", + "integrity": "sha512-QLqkiSNrtGnh8RK9ipD63jVAUenkRu+72xR31DViZWRV9V8G2hzky5E/RoZWPEx+DfmBIUJ7Tcef6cCRcxEj9A==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.5", - "@storybook/client-logger": "7.6.5", - "@storybook/components": "7.6.5", - "@storybook/core-events": "7.6.5", + "@storybook/channels": "7.6.6", + "@storybook/client-logger": "7.6.6", + "@storybook/components": "7.6.6", + "@storybook/core-events": "7.6.6", "@storybook/csf": "^0.1.2", - "@storybook/docs-tools": "7.6.5", + "@storybook/docs-tools": "7.6.6", "@storybook/global": "^5.0.0", - "@storybook/manager-api": "7.6.5", - "@storybook/preview-api": "7.6.5", - "@storybook/theming": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/manager-api": "7.6.6", + "@storybook/preview-api": "7.6.6", + "@storybook/theming": "7.6.6", + "@storybook/types": "7.6.6", "@types/lodash": "^4.14.167", "color-convert": "^2.0.1", "dequal": "^2.0.2", @@ -5264,15 +5260,15 @@ } }, "node_modules/@storybook/builder-manager": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/builder-manager/-/builder-manager-7.6.5.tgz", - "integrity": "sha512-FQyI+tfzMam2XKXq7k921YVafIJs9Vqvos5qx8vyRnRffo55UU8tgunwjGn0PswtbMm6sThVqE0C0ZzVr7RG8A==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/builder-manager/-/builder-manager-7.6.6.tgz", + "integrity": "sha512-96vmtUqh016H2n80xhvBZU2w5flTOzY7S0nW9nfxbY4UY4b39WajgwJ5wpg8l0YmCwQTrxCwY9/VE2Pd6CCqPA==", "dev": true, "dependencies": { "@fal-works/esbuild-plugin-global-externals": "^2.1.2", - "@storybook/core-common": "7.6.5", - "@storybook/manager": "7.6.5", - "@storybook/node-logger": "7.6.5", + "@storybook/core-common": "7.6.6", + "@storybook/manager": "7.6.6", + "@storybook/node-logger": "7.6.6", "@types/ejs": "^3.1.1", "@types/find-cache-dir": "^3.2.1", "@yarnpkg/esbuild-plugin-pnp": "^3.0.0-rc.10", @@ -5292,20 +5288,20 @@ } }, "node_modules/@storybook/builder-webpack5": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/builder-webpack5/-/builder-webpack5-7.6.5.tgz", - "integrity": "sha512-Lf4jVHGTQRSLIcgXHG2webiFlNwEV8uo2CmDucU2IDV9p3NdloyOmCou40G6Du1hobBTflx8Zj2j9n3A5/+0GA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/builder-webpack5/-/builder-webpack5-7.6.6.tgz", + "integrity": "sha512-V3pVdhlhqBdVhrE1M3ip07X8Ma6FXBeTtcZsT+oFVX7A+wxMfA9fhGWVjL6yOB/E+87jF5SvlvYbo+18Xw6hqg==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", - "@storybook/channels": "7.6.5", - "@storybook/client-logger": "7.6.5", - "@storybook/core-common": "7.6.5", - "@storybook/core-events": "7.6.5", - "@storybook/core-webpack": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/preview": "7.6.5", - "@storybook/preview-api": "7.6.5", + "@storybook/channels": "7.6.6", + "@storybook/client-logger": "7.6.6", + "@storybook/core-common": "7.6.6", + "@storybook/core-events": "7.6.6", + "@storybook/core-webpack": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/preview": "7.6.6", + "@storybook/preview-api": "7.6.6", "@swc/core": "^1.3.82", "@types/node": "^18.0.0", "@types/semver": "^7.3.4", @@ -5388,13 +5384,13 @@ "dev": true }, "node_modules/@storybook/channels": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/channels/-/channels-7.6.5.tgz", - "integrity": "sha512-FIlNkyfQy9uHoJfAFL2/wO3ASGJELFvBzURBE2rcEF/TS7GcUiqWnBfiDxAbwSEjSOm2F0eEq3UXhaZEjpJHDw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/channels/-/channels-7.6.6.tgz", + "integrity": "sha512-vvo7fBe2WffPonNNOA7Xx7jcHAto8qJYlq+VMysfheXrsRRbhHl3WQOA18Vm8hV9txtqdqk0hwQiXOWvhYVpeQ==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.5", - "@storybook/core-events": "7.6.5", + "@storybook/client-logger": "7.6.6", + "@storybook/core-events": "7.6.6", "@storybook/global": "^5.0.0", "qs": "^6.10.0", "telejson": "^7.2.0", @@ -5406,23 +5402,23 @@ } }, "node_modules/@storybook/cli": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/cli/-/cli-7.6.5.tgz", - "integrity": "sha512-w+Y8dx5oCLQVESOVmpsQuFksr/ewARKrnSKl9kwnVMN4sMgjOgoZ3zmV66J7SKexvwyuwlOjf840pmEglGdPPg==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/cli/-/cli-7.6.6.tgz", + "integrity": "sha512-FLmWrbmGOqe1VYwqyIWxU2lJcYPssORmSbSVVPw6OqQIXx3NrNBrmZDLncMwbVCDQ8eU54J1zb+MyDmSqMbVFg==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", "@babel/preset-env": "^7.23.2", "@babel/types": "^7.23.0", "@ndelangen/get-tarball": "^3.0.7", - "@storybook/codemod": "7.6.5", - "@storybook/core-common": "7.6.5", - "@storybook/core-events": "7.6.5", - "@storybook/core-server": "7.6.5", - "@storybook/csf-tools": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/telemetry": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/codemod": "7.6.6", + "@storybook/core-common": "7.6.6", + "@storybook/core-events": "7.6.6", + "@storybook/core-server": "7.6.6", + "@storybook/csf-tools": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/telemetry": "7.6.6", + "@storybook/types": "7.6.6", "@types/semver": "^7.3.4", "@yarnpkg/fslib": "2.10.3", "@yarnpkg/libzip": "2.3.0", @@ -5540,9 +5536,9 @@ "dev": true }, "node_modules/@storybook/client-logger": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/client-logger/-/client-logger-7.6.5.tgz", - "integrity": "sha512-S5aROWgssqg7tcs9lgW5wmCAz4SxMAtioiyVj5oFecmPCbQtFVIAREYzeoxE4GfJL+plrfRkum4BzziANn8EhQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/client-logger/-/client-logger-7.6.6.tgz", + "integrity": "sha512-WEvVyuQR5oNF8jcMmGA13zDjxP/l46kOBBvB6JSc8toUdtLZ/kZWSnU0ioNM8+ECpFqXHjBcF2K6uSJOEb6YEg==", "dev": true, "dependencies": { "@storybook/global": "^5.0.0" @@ -5553,18 +5549,18 @@ } }, "node_modules/@storybook/codemod": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/codemod/-/codemod-7.6.5.tgz", - "integrity": "sha512-K5C9ltBClZ0aSyujGt3RJFtRicrUZy8nzhHrcADUj27rrQD26jH/p+Y05jWKj9JcI8SyMg978GN5X/1aw2Y31A==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/codemod/-/codemod-7.6.6.tgz", + "integrity": "sha512-6QwW6T6ZgwwbTkEoZ7CAoX7lUUob7Sy7bRkMHhSjJe2++wEVFOYLvzHcLUJCupK59+WhmsJU9PpUMlXEKi40TQ==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", "@babel/preset-env": "^7.23.2", "@babel/types": "^7.23.0", "@storybook/csf": "^0.1.2", - "@storybook/csf-tools": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/csf-tools": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/types": "7.6.6", "@types/cross-spawn": "^6.0.2", "cross-spawn": "^7.0.3", "globby": "^11.0.2", @@ -5614,18 +5610,18 @@ } }, "node_modules/@storybook/components": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/components/-/components-7.6.5.tgz", - "integrity": "sha512-w4ZucbBBZ+NKMWlJKVj2I/bMBBq7gzDp9lzc4+8QaQ3vUPXKqc1ilIPYo/7UR5oxwDVMZocmMSgl9L8lvf7+Mw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/components/-/components-7.6.6.tgz", + "integrity": "sha512-FSfcRxdmV4+LJHjMk0eodGVnZdb2qrKKmbtsn0O/434z586zPA287/wJJsm4JS/Xr1WS9oTvU6mYMDChkcxgeQ==", "dev": true, "dependencies": { "@radix-ui/react-select": "^1.2.2", "@radix-ui/react-toolbar": "^1.0.4", - "@storybook/client-logger": "7.6.5", + "@storybook/client-logger": "7.6.6", "@storybook/csf": "^0.1.2", "@storybook/global": "^5.0.0", - "@storybook/theming": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/theming": "7.6.6", + "@storybook/types": "7.6.6", "memoizerific": "^1.11.3", "use-resize-observer": "^9.1.0", "util-deprecate": "^1.0.2" @@ -5640,13 +5636,13 @@ } }, "node_modules/@storybook/core-client": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/core-client/-/core-client-7.6.5.tgz", - "integrity": "sha512-6FtyJcz8MSl+JYwNJZ53FM6rkT27pFHWcJPdtw/9229Ec8as9RpkNeZ/NBZjRTeDkn9Ki0VOiVAefNie9tZ/8Q==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/core-client/-/core-client-7.6.6.tgz", + "integrity": "sha512-P100aNf+WpvzlfULZp1NPd60/nxsppLmft2DdIyAx1j4QPMZvUJyJB+hdBMzTFiPEhIUncIMoIVf2R3UXC5DfA==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.5", - "@storybook/preview-api": "7.6.5" + "@storybook/client-logger": "7.6.6", + "@storybook/preview-api": "7.6.6" }, "funding": { "type": "opencollective", @@ -5654,14 +5650,14 @@ } }, "node_modules/@storybook/core-common": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/core-common/-/core-common-7.6.5.tgz", - "integrity": "sha512-z4EgzZSIVbID6Ib0jhh3jimKeaDWU8OOhoZYfn3galFmgQWowWOv1oMgipWiXfRLWw9DaLFQiCHIdLANH+VO2g==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/core-common/-/core-common-7.6.6.tgz", + "integrity": "sha512-DpbFSYw8LHuwpeU2ec5uWryxrSqslFJnWTfNA7AvpzCviWXkz4kq+YYrDee9XExF6OozNwILmG6m52SnraysBA==", "dev": true, "dependencies": { - "@storybook/core-events": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/core-events": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/types": "7.6.6", "@types/find-cache-dir": "^3.2.1", "@types/node": "^18.0.0", "@types/node-fetch": "^2.6.4", @@ -5698,9 +5694,9 @@ } }, "node_modules/@storybook/core-events": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/core-events/-/core-events-7.6.5.tgz", - "integrity": "sha512-zk2q/qicYXAzHA4oV3GDbIql+Kd4TOHUgDE8e4jPCOPp856z2ScqEKUAbiJizs6eEJOH4nW9Db1kuzgrBVEykQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/core-events/-/core-events-7.6.6.tgz", + "integrity": "sha512-7+q9HiZiLxaQcwpaSLQrLdjHNHBoOoUY9ZcZXI9iNFSopOgb/ItDnzzlpv08NC7CbKae1hVKJM/t5aSTl7tCMw==", "dev": true, "dependencies": { "ts-dedent": "^2.0.0" @@ -5711,26 +5707,26 @@ } }, "node_modules/@storybook/core-server": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/core-server/-/core-server-7.6.5.tgz", - "integrity": "sha512-BfKzK/ObTjUcPvE5/r1pogCifM/4nLRhOUYJl7XekwHkOQwn19e6H3/ku1W3jDoYXBu642Dc9X7l/ERjKTqxFg==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/core-server/-/core-server-7.6.6.tgz", + "integrity": "sha512-QFVahaExgGtq9swBXgQAMUiCqpCcyVXOiKTIy1j+1uAhPVqhpCxBkkFoXruih5hbIMZyohE4mLPCAr/ivicoDg==", "dev": true, "dependencies": { "@aw-web-design/x-default-browser": "1.4.126", "@discoveryjs/json-ext": "^0.5.3", - "@storybook/builder-manager": "7.6.5", - "@storybook/channels": "7.6.5", - "@storybook/core-common": "7.6.5", - "@storybook/core-events": "7.6.5", + "@storybook/builder-manager": "7.6.6", + "@storybook/channels": "7.6.6", + "@storybook/core-common": "7.6.6", + "@storybook/core-events": "7.6.6", "@storybook/csf": "^0.1.2", - "@storybook/csf-tools": "7.6.5", + "@storybook/csf-tools": "7.6.6", "@storybook/docs-mdx": "^0.1.0", "@storybook/global": "^5.0.0", - "@storybook/manager": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/preview-api": "7.6.5", - "@storybook/telemetry": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/manager": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/preview-api": "7.6.6", + "@storybook/telemetry": "7.6.6", + "@storybook/types": "7.6.6", "@types/detect-port": "^1.3.0", "@types/node": "^18.0.0", "@types/pretty-hrtime": "^1.0.0", @@ -5826,14 +5822,14 @@ "dev": true }, "node_modules/@storybook/core-webpack": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/core-webpack/-/core-webpack-7.6.5.tgz", - "integrity": "sha512-if5ixN2W3e8vwYvgFHq+k0FOSVwgolbPRLDeOToPXHAJjH/TmgGEANZLFAVVwEzsS4KOfRGJQ48KzF0knTsqzA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/core-webpack/-/core-webpack-7.6.6.tgz", + "integrity": "sha512-VQyA6nb7K5uAWp/bqntW9Dtv2mVQWOp99v52ilHI/aCR9gEIC2XKBB7PTs/l5L9PKTcES6UJTcFJmOp+s7kFOA==", "dev": true, "dependencies": { - "@storybook/core-common": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/core-common": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/types": "7.6.6", "@types/node": "^18.0.0", "ts-dedent": "^2.0.0" }, @@ -5861,12 +5857,12 @@ } }, "node_modules/@storybook/csf-plugin": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/csf-plugin/-/csf-plugin-7.6.5.tgz", - "integrity": "sha512-iQ8Y/Qq1IUhHRddjDVicWJA2sM7OZA1FR97OvWUT2240WjCuQSCfy32JD8TQlYjqXgEolJeLPv3zW4qH5om4LQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/csf-plugin/-/csf-plugin-7.6.6.tgz", + "integrity": "sha512-SqdffT14+XNpf+7vA29Elur28VArXtFv4cXMlsCbswbRuY+a0A8vYNwVIfCUy9u4WHTcQX1/tUkDAMh80lrVRQ==", "dev": true, "dependencies": { - "@storybook/csf-tools": "7.6.5", + "@storybook/csf-tools": "7.6.6", "unplugin": "^1.3.1" }, "funding": { @@ -5875,9 +5871,9 @@ } }, "node_modules/@storybook/csf-tools": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/csf-tools/-/csf-tools-7.6.5.tgz", - "integrity": "sha512-1iaCh7nt+WE7Q5UwRhLLc5flMNoAV/vBr0tvDSCKiHaO+D3dZzlZOe/U+S6wegdyN2QNcvT2xs179CcrX6Qp6w==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/csf-tools/-/csf-tools-7.6.6.tgz", + "integrity": "sha512-VXOZCzfSVJL832u17pPhFu1x3PPaAN9d8VXNFX+t/2raga7tK3T7Qhe7lWfP7EZcrVvSCEEp0aMRz2EzzDGVtw==", "dev": true, "dependencies": { "@babel/generator": "^7.23.0", @@ -5885,7 +5881,7 @@ "@babel/traverse": "^7.23.2", "@babel/types": "^7.23.0", "@storybook/csf": "^0.1.2", - "@storybook/types": "7.6.5", + "@storybook/types": "7.6.6", "fs-extra": "^11.1.0", "recast": "^0.23.1", "ts-dedent": "^2.0.0" @@ -5902,14 +5898,14 @@ "dev": true }, "node_modules/@storybook/docs-tools": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/docs-tools/-/docs-tools-7.6.5.tgz", - "integrity": "sha512-UyHkHu5Af6jMpYsR4lZ69D32GQGeA0pLAn7jaBbQndgAjBdK1ykZcifiUC7Wz1hG7+YpuYspEGuDEddOh+X8FQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/docs-tools/-/docs-tools-7.6.6.tgz", + "integrity": "sha512-nc5ZjN2s8SC2PtsZoFf9Wm6gD8TcSlkYbF/mjtyLCGN+Fi+k5B5iudqoa65H19hwiLlzBdcnpQ8C89AiK33J9Q==", "dev": true, "dependencies": { - "@storybook/core-common": "7.6.5", - "@storybook/preview-api": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/core-common": "7.6.6", + "@storybook/preview-api": "7.6.6", + "@storybook/types": "7.6.6", "@types/doctrine": "^0.0.3", "assert": "^2.1.0", "doctrine": "^3.0.0", @@ -5927,9 +5923,9 @@ "dev": true }, "node_modules/@storybook/manager": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/manager/-/manager-7.6.5.tgz", - "integrity": "sha512-y1KLH0O1PGPyMxGMvOhppzFSO7r4ibjTve5iqsI0JZwxUjNuBKRLYbrhXdAyC2iacvxYNrHgevae1k9XdD+FQw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/manager/-/manager-7.6.6.tgz", + "integrity": "sha512-Ga3LcSu/xxSyg+cLlO9AS8QjW+D667V+c9qQPmsFyU6qfFc6m6mVqcRLSmFVD5e7P/o0FL7STOf9jAKkDcW8xw==", "dev": true, "funding": { "type": "opencollective", @@ -5937,19 +5933,19 @@ } }, "node_modules/@storybook/manager-api": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/manager-api/-/manager-api-7.6.5.tgz", - "integrity": "sha512-tE3OShOcs6A3XtI3NJd6hYQOZLaP++Fn0dCtowBwYh/vS1EN/AyroVmL97tsxn1DZTyoRt0GidwbB6dvLMBOwA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/manager-api/-/manager-api-7.6.6.tgz", + "integrity": "sha512-euRAbSZAUzHDt6z1Pq/g45N/RNqta9RaQAym18zt/oLWiYOIrkLmdf7kCuFYsmuA5XQBytiJqwkAD7uF1aLe0g==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.5", - "@storybook/client-logger": "7.6.5", - "@storybook/core-events": "7.6.5", + "@storybook/channels": "7.6.6", + "@storybook/client-logger": "7.6.6", + "@storybook/core-events": "7.6.6", "@storybook/csf": "^0.1.2", "@storybook/global": "^5.0.0", - "@storybook/router": "7.6.5", - "@storybook/theming": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/router": "7.6.6", + "@storybook/theming": "7.6.6", + "@storybook/types": "7.6.6", "dequal": "^2.0.2", "lodash": "^4.17.21", "memoizerific": "^1.11.3", @@ -6003,9 +5999,9 @@ "dev": true }, "node_modules/@storybook/nextjs": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/nextjs/-/nextjs-7.6.5.tgz", - "integrity": "sha512-IZomWGALmcCdW4hgFQZnp6VZ2jBW7lPFKOA+ug3jORmlD7zA4deYXqLKnOr7yx0/MJYR1/nfd+pz5NCAtdAlQA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/nextjs/-/nextjs-7.6.6.tgz", + "integrity": "sha512-evSTvIvrcuDJCFtiEgr4VVCW28JHY8Cz+8O9QR9Mytj+hRd2DaO67xtBRuKYtqSWsyrV9UpnteiKnuiUgiqGtw==", "dev": true, "dependencies": { "@babel/core": "^7.23.2", @@ -6021,14 +6017,14 @@ "@babel/preset-react": "^7.22.15", "@babel/preset-typescript": "^7.23.2", "@babel/runtime": "^7.23.2", - "@storybook/addon-actions": "7.6.5", - "@storybook/builder-webpack5": "7.6.5", - "@storybook/core-common": "7.6.5", - "@storybook/core-events": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/preset-react-webpack": "7.6.5", - "@storybook/preview-api": "7.6.5", - "@storybook/react": "7.6.5", + "@storybook/addon-actions": "7.6.6", + "@storybook/builder-webpack5": "7.6.6", + "@storybook/core-common": "7.6.6", + "@storybook/core-events": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/preset-react-webpack": "7.6.6", + "@storybook/preview-api": "7.6.6", + "@storybook/react": "7.6.6", "@types/node": "^18.0.0", "css-loader": "^6.7.3", "find-up": "^5.0.0", @@ -6156,9 +6152,9 @@ "dev": true }, "node_modules/@storybook/node-logger": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/node-logger/-/node-logger-7.6.5.tgz", - "integrity": "sha512-xKw6IH1wLkIssekdBv3bd13xYKUF1t8EwqDR8BYcN8AVjZlqJMTifssqG4bYV+G/B7J3tz4ugJ5nmtWg6RQ0Qw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/node-logger/-/node-logger-7.6.6.tgz", + "integrity": "sha512-b2OF9GRNI01MlBlnDGS8S6/yOpBNl8eH/0ONafuMPzFEZs5PouHGsFflJvQwwcdVTknMjF5uVS2eSmnLZ8spvA==", "dev": true, "funding": { "type": "opencollective", @@ -6166,9 +6162,9 @@ } }, "node_modules/@storybook/postinstall": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/postinstall/-/postinstall-7.6.5.tgz", - "integrity": "sha512-12WxfpqGKsk7GQ3KWiZSbamsYK8vtRmhOTkavZ9IQkcJ/zuVfmqK80/Mds+njJMudUPzuREuSFGWACczo17EDA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/postinstall/-/postinstall-7.6.6.tgz", + "integrity": "sha512-jamn7QNTJPZiu22nu25LqfSTJohugFhCu4b48yqP+pdMdkQ3qVd3NdDYhBlgkH/Btar+kppiJP1gRtoiJF761w==", "dev": true, "funding": { "type": "opencollective", @@ -6176,18 +6172,18 @@ } }, "node_modules/@storybook/preset-react-webpack": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/preset-react-webpack/-/preset-react-webpack-7.6.5.tgz", - "integrity": "sha512-Lf9bYUkH1DoWjgptkNgMtA0IkRqGhWE/EnmOvq4otV+Cr/BaVQw3/sfKQ76qdpTQZ4Qg4dG+nBWrwaDkfqxKDA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/preset-react-webpack/-/preset-react-webpack-7.6.6.tgz", + "integrity": "sha512-spOEPePPiKJQIVFARVUPfJ3cT8mBWFMlb3iS54MO4IW55aQRPWw1HQYt7uZ3NwZVT49Npwn4D1x81rWMu9ikPg==", "dev": true, "dependencies": { "@babel/preset-flow": "^7.22.15", "@babel/preset-react": "^7.22.15", "@pmmmwh/react-refresh-webpack-plugin": "^0.5.11", - "@storybook/core-webpack": "7.6.5", - "@storybook/docs-tools": "7.6.5", - "@storybook/node-logger": "7.6.5", - "@storybook/react": "7.6.5", + "@storybook/core-webpack": "7.6.6", + "@storybook/docs-tools": "7.6.6", + "@storybook/node-logger": "7.6.6", + "@storybook/react": "7.6.6", "@storybook/react-docgen-typescript-plugin": "1.0.6--canary.9.0c3f3b7.0", "@types/node": "^18.0.0", "@types/semver": "^7.3.4", @@ -6263,9 +6259,9 @@ "dev": true }, "node_modules/@storybook/preview": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/preview/-/preview-7.6.5.tgz", - "integrity": "sha512-zmLa7C7yFGTYhgGZXoecdww9rx0Z5HpNi/GDBRWoNSK+FEdE8Jj2jF5NJ2ncldtYIyegz9ku29JFMKbhMj9K5Q==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/preview/-/preview-7.6.6.tgz", + "integrity": "sha512-Rl+Pom2bNwzc0MdlbFALmvxsbCkbIwlpTZlRZZTh5Ah8JViV9htQgP9e8uwo3NZA2BhjbDLkmnZeloWxQnI5Ig==", "dev": true, "funding": { "type": "opencollective", @@ -6273,17 +6269,17 @@ } }, "node_modules/@storybook/preview-api": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/preview-api/-/preview-api-7.6.5.tgz", - "integrity": "sha512-9XzuDXXgNuA6dDZ3DXsUwEG6ElxeTbzLuYuzcjtS1FusSICZ2iYmxfS0GfSud9MjPPYOJYoSOvMdIHjorjgByA==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/preview-api/-/preview-api-7.6.6.tgz", + "integrity": "sha512-Bt6xIAR5yZ/JWc90X4BbLOA97iL65glZ1SOBgFFv2mHrdZ1lcdKhAlQr2aeJAf1mLvBtalPjvKzi9EuVY3FZ4w==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.5", - "@storybook/client-logger": "7.6.5", - "@storybook/core-events": "7.6.5", + "@storybook/channels": "7.6.6", + "@storybook/client-logger": "7.6.6", + "@storybook/core-events": "7.6.6", "@storybook/csf": "^0.1.2", "@storybook/global": "^5.0.0", - "@storybook/types": "7.6.5", + "@storybook/types": "7.6.6", "@types/qs": "^6.9.5", "dequal": "^2.0.2", "lodash": "^4.17.21", @@ -6299,18 +6295,18 @@ } }, "node_modules/@storybook/react": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/react/-/react-7.6.5.tgz", - "integrity": "sha512-z0l5T+gL//VekMXnHi+lW5qr7OQ8X7WoeIRMk38e62ppSpGUZRfoxRmmhU/9YcIFAlCgMaoLSYmhOceKGRZuVw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/react/-/react-7.6.6.tgz", + "integrity": "sha512-pE6GJ4hPGJIsX6AREjW6HibshwZE6rFhWRtjeX5MV0eKMmQgoRWRgiRfg9/YB6Z0tRtuptI83Uaszimmif1BKg==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.5", - "@storybook/core-client": "7.6.5", - "@storybook/docs-tools": "7.6.5", + "@storybook/client-logger": "7.6.6", + "@storybook/core-client": "7.6.6", + "@storybook/docs-tools": "7.6.6", "@storybook/global": "^5.0.0", - "@storybook/preview-api": "7.6.5", - "@storybook/react-dom-shim": "7.6.5", - "@storybook/types": "7.6.5", + "@storybook/preview-api": "7.6.6", + "@storybook/react-dom-shim": "7.6.6", + "@storybook/types": "7.6.6", "@types/escodegen": "^0.0.6", "@types/estree": "^0.0.51", "@types/node": "^18.0.0", @@ -6364,9 +6360,9 @@ } }, "node_modules/@storybook/react-dom-shim": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/react-dom-shim/-/react-dom-shim-7.6.5.tgz", - "integrity": "sha512-Qp3N3zENdvx20ikHmz5yI03z+mAWF8bUAwUofqXarVtZUkBNtvfTfUwgAezOAF0eClClH+ktIziIKd976tLSPw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/react-dom-shim/-/react-dom-shim-7.6.6.tgz", + "integrity": "sha512-WWNlXtCVoBWXX/kLNulUeMgzmlAEHi2aBrdIv2jz0DScPf0YxeWAkWmgK7F0zMot9mdwYncr+pk1AILbTBJSyg==", "dev": true, "funding": { "type": "opencollective", @@ -6387,12 +6383,12 @@ } }, "node_modules/@storybook/router": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/router/-/router-7.6.5.tgz", - "integrity": "sha512-QiTC86gRuoepzzmS6HNJZTwfz/n27NcqtaVEIxJi1Yvsx2/kLa9NkRhylNkfTuZ1gEry9stAlKWanMsB2aKyjQ==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/router/-/router-7.6.6.tgz", + "integrity": "sha512-dkn81MtxrG7JMDbOHEcVZkTDVKsneg72CyqJ8ELZfC81iKQcDMQkV9mdmnMl45aKn6UrscudI4K23OxQmsevkw==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.5", + "@storybook/client-logger": "7.6.6", "memoizerific": "^1.11.3", "qs": "^6.10.0" }, @@ -6402,14 +6398,14 @@ } }, "node_modules/@storybook/telemetry": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/telemetry/-/telemetry-7.6.5.tgz", - "integrity": "sha512-FiLRh9k9LoGphqgBqPYySWdGqplihiZyDwqdo+Qs19RcQ/eiKg0W7fdA09nStcdcsHmDl/1cMfRhz9KUiMtwOw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/telemetry/-/telemetry-7.6.6.tgz", + "integrity": "sha512-2WdDcrMrt1bPVgdMVO0tFmVxT6YIjiPRfKbH/7wwYMOGmV75m4mJ9Ha2gzZc/oXTSK1M4/fiK12IgW+S3ErcMg==", "dev": true, "dependencies": { - "@storybook/client-logger": "7.6.5", - "@storybook/core-common": "7.6.5", - "@storybook/csf-tools": "7.6.5", + "@storybook/client-logger": "7.6.6", + "@storybook/core-common": "7.6.6", + "@storybook/csf-tools": "7.6.6", "chalk": "^4.1.0", "detect-package-manager": "^2.0.1", "fetch-retry": "^5.0.2", @@ -6433,13 +6429,13 @@ } }, "node_modules/@storybook/theming": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/theming/-/theming-7.6.5.tgz", - "integrity": "sha512-RpcWT0YEgiobO41McVPDfQQHHFnjyr1sJnNTPJIvOUgSfURdgSj17mQVxtD5xcXcPWUdle5UhIOrCixHbL/NNw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/theming/-/theming-7.6.6.tgz", + "integrity": "sha512-hNZOOxaF55iAGUEM0dvAIP6LfGMgPKCJQIk/qyotFk+SKkg3PBqzph89XfFl9yCD3KiX5cryqarULgVuNawLJg==", "dev": true, "dependencies": { "@emotion/use-insertion-effect-with-fallbacks": "^1.0.0", - "@storybook/client-logger": "7.6.5", + "@storybook/client-logger": "7.6.6", "@storybook/global": "^5.0.0", "memoizerific": "^1.11.3" }, @@ -6453,12 +6449,12 @@ } }, "node_modules/@storybook/types": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/@storybook/types/-/types-7.6.5.tgz", - "integrity": "sha512-Q757v+fYZZSaEpks/zDL5YgXRozxkgKakXFc+BoQHK5q5sVhJ+0jvpLJiAQAniIIaMIkqY/G24Kd6Uo6UdKBCg==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/@storybook/types/-/types-7.6.6.tgz", + "integrity": "sha512-77vbQp3GX93OD8UzFkY4a0fAmkZrqLe61XVo6yABrwbVDY0EcAwaCF5gcXRhOHldlH7KYbLfEQkDkkKTBjX7ow==", "dev": true, "dependencies": { - "@storybook/channels": "7.6.5", + "@storybook/channels": "7.6.6", "@types/babel__core": "^7.0.0", "@types/express": "^4.7.0", "file-system-cache": "2.3.0" @@ -6897,9 +6893,9 @@ "dev": true }, "node_modules/@types/eslint": { - "version": "8.44.9", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.44.9.tgz", - "integrity": "sha512-6yBxcvwnnYoYT1Uk2d+jvIfsuP4mb2EdIxFnrPABj5a/838qe5bGkNLFOiipX4ULQ7XVQvTxOh7jO+BTAiqsEw==", + "version": "8.56.0", + "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.56.0.tgz", + "integrity": "sha512-FlsN0p4FhuYRjIxpbdXovvHQhtlG05O1GG/RNWvdAxTboR438IOTwmrY/vLA+Xfgg06BTkP045M3vpFwTMv1dg==", "dev": true, "dependencies": { "@types/estree": "*", @@ -7074,9 +7070,9 @@ "integrity": "sha512-ga8y9v9uyeiLdpKddhxYQkxNDrfvuPrlFb0N1qnZZByvcElJaXthF1UhvCh9TLWJBEHeNtdnbysW7Y6Uq8CVng==" }, "node_modules/@types/qs": { - "version": "6.9.10", - "resolved": "https://registry.npmjs.org/@types/qs/-/qs-6.9.10.tgz", - "integrity": "sha512-3Gnx08Ns1sEoCrWssEgTSJs/rsT2vhGP+Ja9cnnk9k4ALxinORlQneLXFeFKOTJMOeZUFD1s7w+w2AphTpvzZw==", + "version": "6.9.11", + "resolved": "https://registry.npmjs.org/@types/qs/-/qs-6.9.11.tgz", + "integrity": "sha512-oGk0gmhnEJK4Yyk+oI7EfXsLayXatCWPHary1MtcmbAifkobT9cM9yutG/hZKIseOU0MqbIwQ/u2nn/Gb+ltuQ==", "dev": true }, "node_modules/@types/range-parser": { @@ -7150,9 +7146,9 @@ } }, "node_modules/@types/stylis": { - "version": "4.2.4", - "resolved": "https://registry.npmjs.org/@types/stylis/-/stylis-4.2.4.tgz", - "integrity": "sha512-36ZrGJ8fgtBr6nwNnuJ9jXIj+bn/pF6UoqmrQT7+Y99+tFFeHHsoR54+194dHdyhPjgbeoNz3Qru0oRt0l6ASQ==" + "version": "4.2.0", + "resolved": "https://registry.npmjs.org/@types/stylis/-/stylis-4.2.0.tgz", + "integrity": "sha512-n4sx2bqL0mW1tvDf/loQ+aMX7GQD3lc3fkCMC55VFNDu/vBOabO+LTIeXKM14xK0ppk5TUGcWRjiSpIlUpghKw==" }, "node_modules/@types/unist": { "version": "2.0.10", @@ -7182,15 +7178,15 @@ "dev": true }, "node_modules/@typescript-eslint/parser": { - "version": "6.14.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-6.14.0.tgz", - "integrity": "sha512-QjToC14CKacd4Pa7JK4GeB/vHmWFJckec49FR4hmIRf97+KXole0T97xxu9IFiPxVQ1DBWrQ5wreLwAGwWAVQA==", + "version": "6.15.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-6.15.0.tgz", + "integrity": "sha512-MkgKNnsjC6QwcMdlNAel24jjkEO/0hQaMDLqP4S9zq5HBAUJNQB6y+3DwLjX7b3l2b37eNAxMPLwb3/kh8VKdA==", "dev": true, "dependencies": { - "@typescript-eslint/scope-manager": "6.14.0", - "@typescript-eslint/types": "6.14.0", - "@typescript-eslint/typescript-estree": "6.14.0", - "@typescript-eslint/visitor-keys": "6.14.0", + "@typescript-eslint/scope-manager": "6.15.0", + "@typescript-eslint/types": "6.15.0", + "@typescript-eslint/typescript-estree": "6.15.0", + "@typescript-eslint/visitor-keys": "6.15.0", "debug": "^4.3.4" }, "engines": { @@ -7210,13 +7206,13 @@ } }, "node_modules/@typescript-eslint/scope-manager": { - "version": "6.14.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-6.14.0.tgz", - "integrity": "sha512-VT7CFWHbZipPncAZtuALr9y3EuzY1b1t1AEkIq2bTXUPKw+pHoXflGNG5L+Gv6nKul1cz1VH8fz16IThIU0tdg==", + "version": "6.15.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-6.15.0.tgz", + "integrity": "sha512-+BdvxYBltqrmgCNu4Li+fGDIkW9n//NrruzG9X1vBzaNK+ExVXPoGB71kneaVw/Jp+4rH/vaMAGC6JfMbHstVg==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.14.0", - "@typescript-eslint/visitor-keys": "6.14.0" + "@typescript-eslint/types": "6.15.0", + "@typescript-eslint/visitor-keys": "6.15.0" }, "engines": { "node": "^16.0.0 || >=18.0.0" @@ -7227,9 +7223,9 @@ } }, "node_modules/@typescript-eslint/types": { - "version": "6.14.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-6.14.0.tgz", - "integrity": "sha512-uty9H2K4Xs8E47z3SnXEPRNDfsis8JO27amp2GNCnzGETEW3yTqEIVg5+AI7U276oGF/tw6ZA+UesxeQ104ceA==", + "version": "6.15.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-6.15.0.tgz", + "integrity": "sha512-yXjbt//E4T/ee8Ia1b5mGlbNj9fB9lJP4jqLbZualwpP2BCQ5is6BcWwxpIsY4XKAhmdv3hrW92GdtJbatC6dQ==", "dev": true, "engines": { "node": "^16.0.0 || >=18.0.0" @@ -7240,13 +7236,13 @@ } }, "node_modules/@typescript-eslint/typescript-estree": { - "version": "6.14.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-6.14.0.tgz", - "integrity": "sha512-yPkaLwK0yH2mZKFE/bXkPAkkFgOv15GJAUzgUVonAbv0Hr4PK/N2yaA/4XQbTZQdygiDkpt5DkxPELqHguNvyw==", + "version": "6.15.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-6.15.0.tgz", + "integrity": "sha512-7mVZJN7Hd15OmGuWrp2T9UvqR2Ecg+1j/Bp1jXUEY2GZKV6FXlOIoqVDmLpBiEiq3katvj/2n2mR0SDwtloCew==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.14.0", - "@typescript-eslint/visitor-keys": "6.14.0", + "@typescript-eslint/types": "6.15.0", + "@typescript-eslint/visitor-keys": "6.15.0", "debug": "^4.3.4", "globby": "^11.1.0", "is-glob": "^4.0.3", @@ -7495,12 +7491,12 @@ "dev": true }, "node_modules/@typescript-eslint/visitor-keys": { - "version": "6.14.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-6.14.0.tgz", - "integrity": "sha512-fB5cw6GRhJUz03MrROVuj5Zm/Q+XWlVdIsFj+Zb1Hvqouc8t+XP2H5y53QYU/MGtd2dPg6/vJJlhoX3xc2ehfw==", + "version": "6.15.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-6.15.0.tgz", + "integrity": "sha512-1zvtdC1a9h5Tb5jU9x3ADNXO9yjP8rXlaoChu0DQX40vf5ACVpYIVIZhIMZ6d5sDXH7vq4dsZBT1fEGj8D2n2w==", "dev": true, "dependencies": { - "@typescript-eslint/types": "6.14.0", + "@typescript-eslint/types": "6.15.0", "eslint-visitor-keys": "^3.4.1" }, "engines": { @@ -7820,15 +7816,12 @@ } }, "node_modules/agent-base": { - "version": "7.1.0", - "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-7.1.0.tgz", - "integrity": "sha512-o/zjMZRhJxny7OyEF+Op8X+efiELC7k7yOjMzgfzVqOzXqkBkWI79YoTdOtsuWd5BWhAGAuOY/Xa6xpiaWXiNg==", + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-5.1.1.tgz", + "integrity": "sha512-TMeqbNl2fMW0nMjTEPOwe3J/PRFP4vqeoNuQMG0HlMrtm5QxKqdvAkZ1pRBQ/ulIyDD5Yq0nJ7YbdD8ey0TO3g==", "dev": true, - "dependencies": { - "debug": "^4.3.4" - }, "engines": { - "node": ">= 14" + "node": ">= 6.0.0" } }, "node_modules/aggregate-error": { @@ -8754,6 +8747,7 @@ "version": "1.1.11", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz", "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==", + "dev": true, "dependencies": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -9042,9 +9036,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001570", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001570.tgz", - "integrity": "sha512-+3e0ASu4sw1SWaoCtvPeyXp+5PsjigkSt8OXZbF9StH5pQWbxEjLAZE3n8Aup5udop1uRiKA7a4utUk/uoSpUw==", + "version": "1.0.30001571", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001571.tgz", + "integrity": "sha512-tYq/6MoXhdezDLFZuCO/TKboTzuQ/xR5cFdgXPfDtM7/kchBO3b4VWghE/OAi/DV7tTdhmLjZiZBZi1fA/GheQ==", "funding": [ { "type": "opencollective", @@ -9165,10 +9159,19 @@ "safe-buffer": "^5.0.1" } }, + "node_modules/citty": { + "version": "0.1.5", + "resolved": "https://registry.npmjs.org/citty/-/citty-0.1.5.tgz", + "integrity": "sha512-AS7n5NSc0OQVMV9v6wt3ByujNIrne0/cTjiC2MYqhvao57VNfiuVksTSr2p17nVOhEr2KtqiAkGwHcgMC/qUuQ==", + "dev": true, + "dependencies": { + "consola": "^3.2.3" + } + }, "node_modules/classnames": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/classnames/-/classnames-2.3.2.tgz", - "integrity": "sha512-CSbhY4cFEJRe6/GQzIk5qXZ4Jeg5pcsP7b5peFSDpffpe1cqjASH/n9UTjBwOp6XpMSTwQ8Za2K5V02ueA7Tmw==" + "version": "2.3.3", + "resolved": "https://registry.npmjs.org/classnames/-/classnames-2.3.3.tgz", + "integrity": "sha512-1inzZmicIFcmUya7PGtUQeXtcF7zZpPnxtQoYOrz0uiOBGlLFa4ik4361seYL2JCcRDIyfdFHiwQolESFlw+Og==" }, "node_modules/clean-css": { "version": "5.3.3", @@ -9495,7 +9498,8 @@ "node_modules/concat-map": { "version": "0.0.1", "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz", - "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==" + "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==", + "dev": true }, "node_modules/concat-stream": { "version": "1.6.2", @@ -9548,6 +9552,15 @@ "safe-buffer": "~5.1.0" } }, + "node_modules/consola": { + "version": "3.2.3", + "resolved": "https://registry.npmjs.org/consola/-/consola-3.2.3.tgz", + "integrity": "sha512-I5qxpzLv+sJhTVEoLYNcTW+bThDCPsit0vLNKShZx6rLtpilNpmmeTPaeqJb9ZE9dV3DGaeby6Vuhrw38WjeyQ==", + "dev": true, + "engines": { + "node": "^14.18.0 || >=16.10.0" + } + }, "node_modules/console-browserify": { "version": "1.2.0", "resolved": "https://registry.npmjs.org/console-browserify/-/console-browserify-1.2.0.tgz", @@ -9783,7 +9796,6 @@ "version": "7.0.3", "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz", "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==", - "dev": true, "dependencies": { "path-key": "^3.1.0", "shebang-command": "^2.0.0", @@ -10611,8 +10623,7 @@ "node_modules/eastasianwidth": { "version": "0.2.0", "resolved": "https://registry.npmjs.org/eastasianwidth/-/eastasianwidth-0.2.0.tgz", - "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==", - "dev": true + "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==" }, "node_modules/ee-first": { "version": "1.1.1", @@ -10636,9 +10647,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.614", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.614.tgz", - "integrity": "sha512-X4ze/9Sc3QWs6h92yerwqv7aB/uU8vCjZcrMjA8N9R1pjMFRe44dLsck5FzLilOYvcXuDn93B+bpGYyufc70gQ==", + "version": "1.4.616", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.616.tgz", + "integrity": "sha512-1n7zWYh8eS0L9Uy+GskE0lkBUNK83cXTVJI0pU3mGprFsbfSdAc15VTFbo+A+Bq4pwstmL30AVcEU3Fo463lNg==", "dev": true }, "node_modules/elliptic": { @@ -10671,8 +10682,7 @@ "node_modules/emoji-regex": { "version": "9.2.2", "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-9.2.2.tgz", - "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==", - "dev": true + "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==" }, "node_modules/emojis-list": { "version": "3.0.0", @@ -11815,9 +11825,9 @@ "dev": true }, "node_modules/fastq": { - "version": "1.15.0", - "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.15.0.tgz", - "integrity": "sha512-wBrocU2LCXXa+lWBt8RoIRD89Fi8OdABODa/kEnyeyjS5aZO5/GNvI5sEINADqP/h8M29UHTHUb53sUu5Ihqdw==", + "version": "1.16.0", + "resolved": "https://registry.npmjs.org/fastq/-/fastq-1.16.0.tgz", + "integrity": "sha512-ifCoaXsDrsdkWTtiNJX5uzHDsrck5TzfKKDcuFFTIrrc/BS076qgEIfoIy1VeZqViznfKiysPYTh/QeHtnIsYA==", "dependencies": { "reusify": "^1.0.4" } @@ -12114,9 +12124,9 @@ "dev": true }, "node_modules/flow-parser": { - "version": "0.224.0", - "resolved": "https://registry.npmjs.org/flow-parser/-/flow-parser-0.224.0.tgz", - "integrity": "sha512-S1P78o0VLB1FZvkoGSIpaRiiTUQ3xDhm9I4Z1qc3lglmkjehfR2sjM0vhwKS7UC1G12VT4Leb/GGV/KlactqjA==", + "version": "0.225.1", + "resolved": "https://registry.npmjs.org/flow-parser/-/flow-parser-0.225.1.tgz", + "integrity": "sha512-50fjR6zbLQcpq5IFNkheUSY/AFPxVeeLiBM5B3NQBSKId2G0cUuExOlDDOguxc49dl9lnh8hI1xcYlPJWNp4KQ==", "dev": true, "engines": { "node": ">=0.4.0" @@ -12135,7 +12145,6 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/foreground-child/-/foreground-child-3.1.1.tgz", "integrity": "sha512-TMKDUnIte6bfb5nWv7V/caI169OHgvwjb7V4WkeUvbQQdjr5rWKqHFiKWb/fcOwB+CzBT+qbWjvj+DVwRskpIg==", - "dev": true, "dependencies": { "cross-spawn": "^7.0.0", "signal-exit": "^4.0.1" @@ -12326,7 +12335,8 @@ "node_modules/fs.realpath": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", - "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==" + "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==", + "dev": true }, "node_modules/fsevents": { "version": "2.3.3", @@ -12499,9 +12509,9 @@ } }, "node_modules/gh-pages": { - "version": "6.1.0", - "resolved": "https://registry.npmjs.org/gh-pages/-/gh-pages-6.1.0.tgz", - "integrity": "sha512-MdXigvqN3I66Y+tAZsQJMzpBWQOI1snD6BYuECmP+GEdryYMMOQvzn4AConk/+qNg/XIuQhB1xNGrl3Rmj1iow==", + "version": "6.1.1", + "resolved": "https://registry.npmjs.org/gh-pages/-/gh-pages-6.1.1.tgz", + "integrity": "sha512-upnohfjBwN5hBP9w2dPE7HO5JJTHzSGMV1JrLrHvNuqmjoYHg6TBrCcnEoorjG/e0ejbuvnwyKMdTyM40PEByw==", "dev": true, "dependencies": { "async": "^3.2.4", @@ -12569,16 +12579,17 @@ } }, "node_modules/giget": { - "version": "1.1.3", - "resolved": "https://registry.npmjs.org/giget/-/giget-1.1.3.tgz", - "integrity": "sha512-zHuCeqtfgqgDwvXlR84UNgnJDuUHQcNI5OqWqFxxuk2BshuKbYhJWdxBsEo4PvKqoGh23lUAIvBNpChMLv7/9Q==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/giget/-/giget-1.2.1.tgz", + "integrity": "sha512-4VG22mopWtIeHwogGSy1FViXVo0YT+m6BrqZfz0JJFwbSsePsCdOzdLIIli5BtMp7Xe8f/o2OmBpQX2NBOC24g==", "dev": true, "dependencies": { - "colorette": "^2.0.20", - "defu": "^6.1.2", - "https-proxy-agent": "^7.0.2", - "mri": "^1.2.0", - "node-fetch-native": "^1.4.0", + "citty": "^0.1.5", + "consola": "^3.2.3", + "defu": "^6.1.3", + "node-fetch-native": "^1.6.1", + "nypm": "^0.3.3", + "ohash": "^1.1.3", "pathe": "^1.1.1", "tar": "^6.2.0" }, @@ -12602,7 +12613,6 @@ "version": "10.3.10", "resolved": "https://registry.npmjs.org/glob/-/glob-10.3.10.tgz", "integrity": "sha512-fa46+tv1Ak0UPK1TOy/pZrIybNNt4HCv7SDzwyfiOZkvZLEbjsZkJBPtDHVshZjbecAoAGSC20MjLDG/qr679g==", - "dev": true, "dependencies": { "foreground-child": "^3.1.0", "jackspeak": "^2.3.5", @@ -12640,7 +12650,6 @@ "version": "2.0.1", "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-2.0.1.tgz", "integrity": "sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA==", - "dev": true, "dependencies": { "balanced-match": "^1.0.0" } @@ -12649,7 +12658,6 @@ "version": "9.0.3", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", - "dev": true, "dependencies": { "brace-expansion": "^2.0.1" }, @@ -13016,9 +13024,9 @@ } }, "node_modules/html-webpack-plugin": { - "version": "5.5.4", - "resolved": "https://registry.npmjs.org/html-webpack-plugin/-/html-webpack-plugin-5.5.4.tgz", - "integrity": "sha512-3wNSaVVxdxcu0jd4FpQFoICdqgxs4zIQQvj+2yQKFfBOnLETQ6X5CDWdeasuGlSsooFlMkEioWDTqBv1wvw5Iw==", + "version": "5.6.0", + "resolved": "https://registry.npmjs.org/html-webpack-plugin/-/html-webpack-plugin-5.6.0.tgz", + "integrity": "sha512-iwaY4wzbe48AfKLZ/Cc8k0L+FKG6oSNRaZ8x5A/T/IVDGyXcbHncM9TdDa93wn0FsSm82FhTKW7f3vS61thXAw==", "dev": true, "dependencies": { "@types/html-minifier-terser": "^6.0.0", @@ -13035,7 +13043,16 @@ "url": "https://opencollective.com/html-webpack-plugin" }, "peerDependencies": { + "@rspack/core": "0.x || 1.x", "webpack": "^5.20.0" + }, + "peerDependenciesMeta": { + "@rspack/core": { + "optional": true + }, + "webpack": { + "optional": true + } } }, "node_modules/htmlparser2": { @@ -13080,16 +13097,16 @@ "dev": true }, "node_modules/https-proxy-agent": { - "version": "7.0.2", - "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-7.0.2.tgz", - "integrity": "sha512-NmLNjm6ucYwtcUmL7JQC1ZQ57LmHP4lT15FQ8D61nak1rO6DH+fz5qNK2Ap5UN4ZapYICE3/0KodcLYSPsPbaA==", + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-4.0.0.tgz", + "integrity": "sha512-zoDhWrkR3of1l9QAL8/scJZyLu8j/gBkcwcaQOZh7Gyh/+uJQzGVETdgT30akuwkpL8HTRfssqI3BZuV18teDg==", "dev": true, "dependencies": { - "agent-base": "^7.0.2", + "agent-base": "5", "debug": "4" }, "engines": { - "node": ">= 14" + "node": ">= 6.0.0" } }, "node_modules/human-signals": { @@ -13214,6 +13231,7 @@ "version": "1.0.6", "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz", "integrity": "sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==", + "dev": true, "dependencies": { "once": "^1.3.0", "wrappy": "1" @@ -13222,7 +13240,8 @@ "node_modules/inherits": { "version": "2.0.4", "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz", - "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==" + "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==", + "dev": true }, "node_modules/ini": { "version": "1.3.8", @@ -13758,8 +13777,7 @@ "node_modules/isexe": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz", - "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==", - "dev": true + "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==" }, "node_modules/isobject": { "version": "3.0.1", @@ -13812,7 +13830,6 @@ "version": "2.3.6", "resolved": "https://registry.npmjs.org/jackspeak/-/jackspeak-2.3.6.tgz", "integrity": "sha512-N3yCS/NegsOBokc8GAdM8UcmfsKiSS8cipheD/nivzr700H+nsMOxJjQnvwOcRYVuFkdH0wGUvW2WbXGmrZGbQ==", - "dev": true, "dependencies": { "@isaacs/cliui": "^8.0.2" }, @@ -14428,9 +14445,9 @@ } }, "node_modules/lucide-react": { - "version": "0.298.0", - "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.298.0.tgz", - "integrity": "sha512-tWoxZ663Zf/n8VxXTHnTJsU/w1ysWT1LORnIL1pzqElFdSqBhWbZeJ3sLdCZ5FpzpbkpkYEtluhuTyG2BTDYNQ==", + "version": "0.302.0", + "resolved": "https://registry.npmjs.org/lucide-react/-/lucide-react-0.302.0.tgz", + "integrity": "sha512-JZX+1fjpqxvQmEgItvPOAwRlqf0Eg9dSZMxljA2/V2M6dluOhQCPBhewIlSJWgkNu0M36kViOgmTAMnDaAMOFw==", "peerDependencies": { "react": "^16.5.1 || ^17.0.0 || ^18.0.0" } @@ -14711,6 +14728,7 @@ "version": "3.1.2", "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz", "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==", + "dev": true, "dependencies": { "brace-expansion": "^1.1.7" }, @@ -14731,7 +14749,6 @@ "version": "7.0.4", "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.0.4.tgz", "integrity": "sha512-jYofLM5Dam9279rdkWzqHozUo4ybjdZmCsDHePy5V/PbBcVMiSZR97gmAy45aqi8CK1lG2ECd356FU86avfwUQ==", - "dev": true, "engines": { "node": ">=16 || 14 >=14.17" } @@ -14810,15 +14827,6 @@ "integrity": "sha512-mjv1G1ZzfEE3k9HZN0dQ2olMdwIfaeAAjFiwNprLfYNRSz7ctv9XuCT7gPtBGrMUeV1/iZzYKj17Khu1hxoHOA==", "peer": true }, - "node_modules/mri": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/mri/-/mri-1.2.0.tgz", - "integrity": "sha512-tzzskb3bG8LvYGFF/mDTpq3jpI6Q9wc3LEmBaghu+DdCssd1FakN7Bc0hVNmEyGq1bq3RgfkCb3cmQLpNPOroA==", - "dev": true, - "engines": { - "node": ">=4" - } - }, "node_modules/ms": { "version": "2.1.2", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", @@ -15082,9 +15090,9 @@ } }, "node_modules/node-fetch-native": { - "version": "1.4.1", - "resolved": "https://registry.npmjs.org/node-fetch-native/-/node-fetch-native-1.4.1.tgz", - "integrity": "sha512-NsXBU0UgBxo2rQLOeWNZqS3fvflWePMECr8CoSWoSTqCqGbVVsvl9vZu1HfQicYN0g5piV9Gh8RTEvo/uP752w==", + "version": "1.6.1", + "resolved": "https://registry.npmjs.org/node-fetch-native/-/node-fetch-native-1.6.1.tgz", + "integrity": "sha512-bW9T/uJDPAJB2YNYEpWzE54U5O3MQidXsOyTfnbKYtTtFexRvGzb1waphBN4ZwP6EcIvYYEOwW0b72BpAqydTw==", "dev": true }, "node_modules/node-int64": { @@ -15200,6 +15208,146 @@ "url": "https://github.com/fb55/nth-check?sponsor=1" } }, + "node_modules/nypm": { + "version": "0.3.3", + "resolved": "https://registry.npmjs.org/nypm/-/nypm-0.3.3.tgz", + "integrity": "sha512-FHoxtTscAE723e80d2M9cJRb4YVjL82Ra+ZV+YqC6rfNZUWahi+ZhPF+krnR+bdMvibsfHCtgKXnZf5R6kmEPA==", + "dev": true, + "dependencies": { + "citty": "^0.1.4", + "execa": "^8.0.1", + "pathe": "^1.1.1", + "ufo": "^1.3.0" + }, + "bin": { + "nypm": "dist/cli.mjs" + }, + "engines": { + "node": "^14.16.0 || >=16.10.0" + } + }, + "node_modules/nypm/node_modules/execa": { + "version": "8.0.1", + "resolved": "https://registry.npmjs.org/execa/-/execa-8.0.1.tgz", + "integrity": "sha512-VyhnebXciFV2DESc+p6B+y0LjSm0krU4OgJN44qFAhBY0TJ+1V61tYD2+wHusZ6F9n5K+vl8k0sTy7PEfV4qpg==", + "dev": true, + "dependencies": { + "cross-spawn": "^7.0.3", + "get-stream": "^8.0.1", + "human-signals": "^5.0.0", + "is-stream": "^3.0.0", + "merge-stream": "^2.0.0", + "npm-run-path": "^5.1.0", + "onetime": "^6.0.0", + "signal-exit": "^4.1.0", + "strip-final-newline": "^3.0.0" + }, + "engines": { + "node": ">=16.17" + }, + "funding": { + "url": "https://github.com/sindresorhus/execa?sponsor=1" + } + }, + "node_modules/nypm/node_modules/get-stream": { + "version": "8.0.1", + "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-8.0.1.tgz", + "integrity": "sha512-VaUJspBffn/LMCJVoMvSAdmscJyS1auj5Zulnn5UoYcY531UWmdwhRWkcGKnGU93m5HSXP9LP2usOryrBtQowA==", + "dev": true, + "engines": { + "node": ">=16" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/nypm/node_modules/human-signals": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/human-signals/-/human-signals-5.0.0.tgz", + "integrity": "sha512-AXcZb6vzzrFAUE61HnN4mpLqd/cSIwNQjtNWR0euPm6y0iqx3G4gOXaIDdtdDwZmhwe82LA6+zinmW4UBWVePQ==", + "dev": true, + "engines": { + "node": ">=16.17.0" + } + }, + "node_modules/nypm/node_modules/is-stream": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-3.0.0.tgz", + "integrity": "sha512-LnQR4bZ9IADDRSkvpqMGvt/tEJWclzklNgSw48V5EAaAeDd6qGvN8ei6k5p0tvxSR171VmGyHuTiAOfxAbr8kA==", + "dev": true, + "engines": { + "node": "^12.20.0 || ^14.13.1 || >=16.0.0" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/nypm/node_modules/mimic-fn": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-4.0.0.tgz", + "integrity": "sha512-vqiC06CuhBTUdZH+RYl8sFrL096vA45Ok5ISO6sE/Mr1jRbGH4Csnhi8f3wKVl7x8mO4Au7Ir9D3Oyv1VYMFJw==", + "dev": true, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/nypm/node_modules/npm-run-path": { + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-5.2.0.tgz", + "integrity": "sha512-W4/tgAXFqFA0iL7fk0+uQ3g7wkL8xJmx3XdK0VGb4cHW//eZTtKGvFBBoRKVTpY7n6ze4NL9ly7rgXcHufqXKg==", + "dev": true, + "dependencies": { + "path-key": "^4.0.0" + }, + "engines": { + "node": "^12.20.0 || ^14.13.1 || >=16.0.0" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/nypm/node_modules/onetime": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/onetime/-/onetime-6.0.0.tgz", + "integrity": "sha512-1FlR+gjXK7X+AsAHso35MnyN5KqGwJRi/31ft6x0M194ht7S+rWAvd7PHss9xSKMzE0asv1pyIHaJYq+BbacAQ==", + "dev": true, + "dependencies": { + "mimic-fn": "^4.0.0" + }, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/nypm/node_modules/path-key": { + "version": "4.0.0", + "resolved": "https://registry.npmjs.org/path-key/-/path-key-4.0.0.tgz", + "integrity": "sha512-haREypq7xkM7ErfgIyA0z+Bj4AGKlMSdlQE2jvJo6huWD1EdkKYV+G/T4nq0YEF2vgTT8kqMFKo1uHn950r4SQ==", + "dev": true, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/nypm/node_modules/strip-final-newline": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/strip-final-newline/-/strip-final-newline-3.0.0.tgz", + "integrity": "sha512-dOESqjYr96iWYylGObzd39EuNTa5VJxyvVAEm5Jnh7KGo75V43Hk1odPQkNDyXNmUR6k+gEiDVXnjB8HJ3crXw==", + "dev": true, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, "node_modules/object-assign": { "version": "4.1.1", "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", @@ -15347,6 +15495,12 @@ "integrity": "sha512-eJJDYkhJFFbBBAxeh8xW+weHlkI28n2ZdQV/J/DNfWfSKlGEf2xcfAbZTv3riEXHAhL9SVOTs2pRmXiSTf78xg==", "dev": true }, + "node_modules/ohash": { + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/ohash/-/ohash-1.1.3.tgz", + "integrity": "sha512-zuHHiGTYTA1sYJ/wZN+t5HKZaH23i4yI1HMwbuXm24Nid7Dv0KcuRlKoNKS9UNfAVSBlnGLcuQrnOKWOZoEGaw==", + "dev": true + }, "node_modules/on-finished": { "version": "2.4.1", "resolved": "https://registry.npmjs.org/on-finished/-/on-finished-2.4.1.tgz", @@ -15372,6 +15526,7 @@ "version": "1.4.0", "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", "integrity": "sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==", + "dev": true, "dependencies": { "wrappy": "1" } @@ -15612,6 +15767,7 @@ "version": "1.0.1", "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz", "integrity": "sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg==", + "dev": true, "engines": { "node": ">=0.10.0" } @@ -15620,7 +15776,6 @@ "version": "3.1.1", "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz", "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==", - "dev": true, "engines": { "node": ">=8" } @@ -15634,7 +15789,6 @@ "version": "1.10.1", "resolved": "https://registry.npmjs.org/path-scurry/-/path-scurry-1.10.1.tgz", "integrity": "sha512-MkhCqzzBEpPvxxQ71Md0b1Kk51W01lrYvlMzSUaIzNsODdd7mqhiimSZlr+VegAz5Z6Vzt9Xg2ttE//XBhH3EQ==", - "dev": true, "dependencies": { "lru-cache": "^9.1.1 || ^10.0.0", "minipass": "^5.0.0 || ^6.0.2 || ^7.0.0" @@ -15650,7 +15804,6 @@ "version": "10.1.0", "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-10.1.0.tgz", "integrity": "sha512-/1clY/ui8CzjKFyjdvwPWJUYKiFVXG2I2cY0ssG7h4+hwk+XOIX7ZSG9Q7TW8TW3Kp3BUSqgFWBLgL4PJ+Blag==", - "dev": true, "engines": { "node": "14 || >=16.14" } @@ -16037,9 +16190,9 @@ } }, "node_modules/postcss-modules-scope": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-3.0.0.tgz", - "integrity": "sha512-hncihwFA2yPath8oZ15PZqvWGkWf+XUfQgUGamS4LqoP1anQLOsOJw0vr7J7IwLpoY9fatA2qiGUGmuZL0Iqlg==", + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/postcss-modules-scope/-/postcss-modules-scope-3.1.0.tgz", + "integrity": "sha512-SaIbK8XW+MZbd0xHPf7kdfA/3eOt7vxJ72IRecn3EzuZVLr1r0orzf0MX/pN8m+NMDoo6X/SQd8oeKqGZd8PXg==", "dev": true, "dependencies": { "postcss-selector-parser": "^6.0.4" @@ -16265,13 +16418,13 @@ } }, "node_modules/prisma": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/prisma/-/prisma-5.7.0.tgz", - "integrity": "sha512-0rcfXO2ErmGAtxnuTNHQT9ztL0zZheQjOI/VNJzdq87C3TlGPQtMqtM+KCwU6XtmkoEr7vbCQqA7HF9IY0ST+Q==", + "version": "5.7.1", + "resolved": "https://registry.npmjs.org/prisma/-/prisma-5.7.1.tgz", + "integrity": "sha512-ekho7ziH0WEJvC4AxuJz+ewRTMskrebPcrKuBwcNzVDniYxx+dXOGcorNeIb9VEMO5vrKzwNYvhD271Ui2jnNw==", "devOptional": true, "hasInstallScript": true, "dependencies": { - "@prisma/engines": "5.7.0" + "@prisma/engines": "5.7.1" }, "bin": { "prisma": "build/index.js" @@ -16462,15 +16615,6 @@ "node": ">=8.16.0" } }, - "node_modules/puppeteer-core/node_modules/agent-base": { - "version": "5.1.1", - "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-5.1.1.tgz", - "integrity": "sha512-TMeqbNl2fMW0nMjTEPOwe3J/PRFP4vqeoNuQMG0HlMrtm5QxKqdvAkZ1pRBQ/ulIyDD5Yq0nJ7YbdD8ey0TO3g==", - "dev": true, - "engines": { - "node": ">= 6.0.0" - } - }, "node_modules/puppeteer-core/node_modules/glob": { "version": "7.2.3", "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz", @@ -16491,19 +16635,6 @@ "url": "https://github.com/sponsors/isaacs" } }, - "node_modules/puppeteer-core/node_modules/https-proxy-agent": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/https-proxy-agent/-/https-proxy-agent-4.0.0.tgz", - "integrity": "sha512-zoDhWrkR3of1l9QAL8/scJZyLu8j/gBkcwcaQOZh7Gyh/+uJQzGVETdgT30akuwkpL8HTRfssqI3BZuV18teDg==", - "dev": true, - "dependencies": { - "agent-base": "5", - "debug": "4" - }, - "engines": { - "node": ">= 6.0.0" - } - }, "node_modules/puppeteer-core/node_modules/mime": { "version": "2.6.0", "resolved": "https://registry.npmjs.org/mime/-/mime-2.6.0.tgz", @@ -16694,15 +16825,15 @@ } }, "node_modules/react-day-picker": { - "version": "8.9.1", - "resolved": "https://registry.npmjs.org/react-day-picker/-/react-day-picker-8.9.1.tgz", - "integrity": "sha512-W0SPApKIsYq+XCtfGeMYDoU0KbsG3wfkYtlw8l+vZp6KoBXGOlhzBUp4tNx1XiwiOZwhfdGOlj7NGSCKGSlg5Q==", + "version": "8.10.0", + "resolved": "https://registry.npmjs.org/react-day-picker/-/react-day-picker-8.10.0.tgz", + "integrity": "sha512-mz+qeyrOM7++1NCb1ARXmkjMkzWVh2GL9YiPbRjKe0zHccvekk4HE+0MPOZOrosn8r8zTHIIeOUXTmXRqmkRmg==", "funding": { "type": "individual", "url": "https://github.com/sponsors/gpbl" }, "peerDependencies": { - "date-fns": "^2.28.0", + "date-fns": "^2.28.0 || ^3.0.0", "react": "^16.8.0 || ^17.0.0 || ^18.0.0" } }, @@ -16928,6 +17059,26 @@ } } }, + "node_modules/react-toastify": { + "version": "9.1.3", + "resolved": "https://registry.npmjs.org/react-toastify/-/react-toastify-9.1.3.tgz", + "integrity": "sha512-fPfb8ghtn/XMxw3LkxQBk3IyagNpF/LIKjOBflbexr2AWxAH1MJgvnESwEwBn9liLFXgTKWgBSdZpw9m4OTHTg==", + "dependencies": { + "clsx": "^1.1.1" + }, + "peerDependencies": { + "react": ">=16", + "react-dom": ">=16" + } + }, + "node_modules/react-toastify/node_modules/clsx": { + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/clsx/-/clsx-1.2.1.tgz", + "integrity": "sha512-EcR6r5a8bj6pu3ycsa/E/cKVGuTgZJZdsyUYHOksG/UHIiKfjxzRxYJpyVBwYaQeOvghal9fcc4PidlgzugAQg==", + "engines": { + "node": ">=6" + } + }, "node_modules/react-transition-group": { "version": "4.4.5", "resolved": "https://registry.npmjs.org/react-transition-group/-/react-transition-group-4.4.5.tgz", @@ -17054,9 +17205,9 @@ } }, "node_modules/readable-stream": { - "version": "4.5.0", - "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-4.5.0.tgz", - "integrity": "sha512-AeYh93VyUwnNI/HCB4XdAaP4N/yGgg3rci3ISEUSM0jN95yWpbL9tSuRIwHzCq7e6TzYwJ6Vn7viUYTsfIxBlQ==", + "version": "4.5.1", + "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-4.5.1.tgz", + "integrity": "sha512-uQjbf34vmf/asGnOHQEw07Q4llgMACQZTWWa4MmICS0IKJoHbLwKCy71H3eR99Dw5iYejc6W+pqZZEeqRtUFAw==", "dev": true, "dependencies": { "abort-controller": "^3.0.0", @@ -17845,7 +17996,6 @@ "version": "2.0.0", "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz", "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==", - "dev": true, "dependencies": { "shebang-regex": "^3.0.0" }, @@ -17857,7 +18007,6 @@ "version": "3.0.0", "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz", "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==", - "dev": true, "engines": { "node": ">=8" } @@ -17880,7 +18029,6 @@ "version": "4.1.0", "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-4.1.0.tgz", "integrity": "sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw==", - "dev": true, "engines": { "node": ">=14" }, @@ -18131,12 +18279,12 @@ "dev": true }, "node_modules/storybook": { - "version": "7.6.5", - "resolved": "https://registry.npmjs.org/storybook/-/storybook-7.6.5.tgz", - "integrity": "sha512-uHPrL+g/0v6iIVtDA8J0uWd3jDZcdr51lCR/vPXTkrCY1uVaFjswzl8EMy5PR05I7jMpKUzkJWZtFbgbh9e1Bw==", + "version": "7.6.6", + "resolved": "https://registry.npmjs.org/storybook/-/storybook-7.6.6.tgz", + "integrity": "sha512-PmJxpjGdLvDOHaRzqLOvcJ3ALQPaNeW6D5Lv7rPPVbuO24wdDzd/75dPRP7gJKYcGE0NnDZ6cLQq3UlCfbkIBA==", "dev": true, "dependencies": { - "@storybook/cli": "7.6.5" + "@storybook/cli": "7.6.6" }, "bin": { "sb": "index.js", @@ -18252,7 +18400,6 @@ "version": "4.2.3", "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", - "dev": true, "dependencies": { "emoji-regex": "^8.0.0", "is-fullwidth-code-point": "^3.0.0", @@ -18265,8 +18412,7 @@ "node_modules/string-width-cjs/node_modules/emoji-regex": { "version": "8.0.0", "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", - "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==", - "dev": true + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==" }, "node_modules/string-width/node_modules/ansi-regex": { "version": "6.0.1", @@ -18382,7 +18528,6 @@ "version": "6.0.1", "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", - "dev": true, "dependencies": { "ansi-regex": "^5.0.1" }, @@ -18473,19 +18618,19 @@ } }, "node_modules/styled-components": { - "version": "6.1.1", - "resolved": "https://registry.npmjs.org/styled-components/-/styled-components-6.1.1.tgz", - "integrity": "sha512-cpZZP5RrKRIClBW5Eby4JM1wElLVP4NQrJbJ0h10TidTyJf4SIIwa3zLXOoPb4gJi8MsJ8mjq5mu2IrEhZIAcQ==", - "dependencies": { - "@emotion/is-prop-valid": "^1.2.1", - "@emotion/unitless": "^0.8.0", - "@types/stylis": "^4.0.2", - "css-to-react-native": "^3.2.0", - "csstype": "^3.1.2", - "postcss": "^8.4.31", - "shallowequal": "^1.1.0", - "stylis": "^4.3.0", - "tslib": "^2.5.0" + "version": "6.1.3", + "resolved": "https://registry.npmjs.org/styled-components/-/styled-components-6.1.3.tgz", + "integrity": "sha512-kLerFjTAABuEZ870O4q4dyT/VCOJC/HA08+VeIGhkiOKkwJLP17HAWHCiqZWnUMV19m3axlOKR/+/EbCbuJAZg==", + "dependencies": { + "@emotion/is-prop-valid": "1.2.1", + "@emotion/unitless": "0.8.0", + "@types/stylis": "4.2.0", + "css-to-react-native": "3.2.0", + "csstype": "3.1.2", + "postcss": "8.4.31", + "shallowequal": "1.1.0", + "stylis": "4.3.0", + "tslib": "2.5.0" }, "engines": { "node": ">= 16" @@ -18499,11 +18644,53 @@ "react-dom": ">= 16.8.0" } }, + "node_modules/styled-components/node_modules/@emotion/unitless": { + "version": "0.8.0", + "resolved": "https://registry.npmjs.org/@emotion/unitless/-/unitless-0.8.0.tgz", + "integrity": "sha512-VINS5vEYAscRl2ZUDiT3uMPlrFQupiKgHz5AA4bCH1miKBg4qtwkim1qPmJj/4WG6TreYMY111rEFsjupcOKHw==" + }, + "node_modules/styled-components/node_modules/csstype": { + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/csstype/-/csstype-3.1.2.tgz", + "integrity": "sha512-I7K1Uu0MBPzaFKg4nI5Q7Vs2t+3gWWW648spaF+Rg7pI9ds18Ugn+lvg4SHczUdKlHI5LWBXyqfS8+DufyBsgQ==" + }, + "node_modules/styled-components/node_modules/postcss": { + "version": "8.4.31", + "resolved": "https://registry.npmjs.org/postcss/-/postcss-8.4.31.tgz", + "integrity": "sha512-PS08Iboia9mts/2ygV3eLpY5ghnUcfLV/EXTOW1E2qYxJKGGBUtNjN76FYHnMs36RmARn41bC0AZmn+rR0OVpQ==", + "funding": [ + { + "type": "opencollective", + "url": "https://opencollective.com/postcss/" + }, + { + "type": "tidelift", + "url": "https://tidelift.com/funding/github/npm/postcss" + }, + { + "type": "github", + "url": "https://github.com/sponsors/ai" + } + ], + "dependencies": { + "nanoid": "^3.3.6", + "picocolors": "^1.0.0", + "source-map-js": "^1.0.2" + }, + "engines": { + "node": "^10 || ^12 || >=14" + } + }, "node_modules/styled-components/node_modules/stylis": { "version": "4.3.0", "resolved": "https://registry.npmjs.org/stylis/-/stylis-4.3.0.tgz", "integrity": "sha512-E87pIogpwUsUwXw7dNyU4QDjdgVMy52m+XEOPEKUn161cCzWjjhPSQhByfd1CcNvrOLnXQ6OnnZDwnJrz/Z4YQ==" }, + "node_modules/styled-components/node_modules/tslib": { + "version": "2.5.0", + "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.5.0.tgz", + "integrity": "sha512-336iVw3rtn2BUK7ORdIAHTyxHGRIHVReokCR3XjbckJMK7ms8FysBfhLR8IXnAgy7T0PTPNBWKiH514FOW/WSg==" + }, "node_modules/styled-jsx": { "version": "5.1.1", "resolved": "https://registry.npmjs.org/styled-jsx/-/styled-jsx-5.1.1.tgz", @@ -18532,13 +18719,13 @@ "integrity": "sha512-Orov6g6BB1sDfYgzWfTHDOxamtX1bE/zo104Dh9e6fqJ3PooipYyfJ0pUmrZO2wAvO8YbEyeFrkV91XTsGMSrw==" }, "node_modules/sucrase": { - "version": "3.34.0", - "resolved": "https://registry.npmjs.org/sucrase/-/sucrase-3.34.0.tgz", - "integrity": "sha512-70/LQEZ07TEcxiU2dz51FKaE6hCTWC6vr7FOk3Gr0U60C3shtAN+H+BFr9XlYe5xqf3RA8nrc+VIwzCfnxuXJw==", + "version": "3.35.0", + "resolved": "https://registry.npmjs.org/sucrase/-/sucrase-3.35.0.tgz", + "integrity": "sha512-8EbVDiu9iN/nESwxeSxDKe0dunta1GOlHufmSSXxMD2z2/tMZpDMpvXQGsc+ajGo8y2uYUmixaSRUc/QPoQ0GA==", "dependencies": { "@jridgewell/gen-mapping": "^0.3.2", "commander": "^4.0.0", - "glob": "7.1.6", + "glob": "^10.3.10", "lines-and-columns": "^1.1.6", "mz": "^2.7.0", "pirates": "^4.0.1", @@ -18549,7 +18736,7 @@ "sucrase-node": "bin/sucrase-node" }, "engines": { - "node": ">=8" + "node": ">=16 || 14 >=14.17" } }, "node_modules/sucrase/node_modules/commander": { @@ -18560,25 +18747,6 @@ "node": ">= 6" } }, - "node_modules/sucrase/node_modules/glob": { - "version": "7.1.6", - "resolved": "https://registry.npmjs.org/glob/-/glob-7.1.6.tgz", - "integrity": "sha512-LwaxwyZ72Lk7vZINtNNrywX0ZuLyStrdDtabefZKAY5ZGJhVtgdznluResxNmPitE0SAO+O26sWTHeKSI2wMBA==", - "dependencies": { - "fs.realpath": "^1.0.0", - "inflight": "^1.0.4", - "inherits": "2", - "minimatch": "^3.0.4", - "once": "^1.3.0", - "path-is-absolute": "^1.0.0" - }, - "engines": { - "node": "*" - }, - "funding": { - "url": "https://github.com/sponsors/isaacs" - } - }, "node_modules/supports-color": { "version": "7.2.0", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz", @@ -18645,9 +18813,9 @@ } }, "node_modules/tailwindcss": { - "version": "3.3.6", - "resolved": "https://registry.npmjs.org/tailwindcss/-/tailwindcss-3.3.6.tgz", - "integrity": "sha512-AKjF7qbbLvLaPieoKeTjG1+FyNZT6KaJMJPFeQyLfIp7l82ggH1fbHJSsYIvnbTFQOlkh+gBYpyby5GT1LIdLw==", + "version": "3.4.0", + "resolved": "https://registry.npmjs.org/tailwindcss/-/tailwindcss-3.4.0.tgz", + "integrity": "sha512-VigzymniH77knD1dryXbyxR+ePHihHociZbXnLZHUyzf2MMs2ZVqlUrZ3FvpXP8pno9JzmILt1sZPD19M3IxtA==", "dependencies": { "@alloc/quick-lru": "^5.2.0", "arg": "^5.0.2", @@ -19104,9 +19272,9 @@ } }, "node_modules/tocbot": { - "version": "4.23.0", - "resolved": "https://registry.npmjs.org/tocbot/-/tocbot-4.23.0.tgz", - "integrity": "sha512-5DWuSZXsqG894mkGb8ZsQt9myyQyVxE50AiGRZ0obV0BVUTVkaZmc9jbgpknaAAPUm4FIrzGkEseD6FuQJYJDQ==", + "version": "4.25.0", + "resolved": "https://registry.npmjs.org/tocbot/-/tocbot-4.25.0.tgz", + "integrity": "sha512-kE5wyCQJ40hqUaRVkyQ4z5+4juzYsv/eK+aqD97N62YH0TxFhzJvo22RUQQZdO3YnXAk42ZOfOpjVdy+Z0YokA==", "dev": true }, "node_modules/toidentifier": { @@ -19378,6 +19546,12 @@ "node": ">=14.17" } }, + "node_modules/ufo": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/ufo/-/ufo-1.3.2.tgz", + "integrity": "sha512-o+ORpgGwaYQXgqGDwd+hkS4PuZ3QnmqMMxRuajK/a38L6fTpcE5GPIfrf+L/KemFzfUpeUQc1rRS1iDBozvnFA==", + "dev": true + }, "node_modules/uglify-js": { "version": "3.17.4", "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.17.4.tgz", @@ -20034,7 +20208,6 @@ "version": "2.0.2", "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz", "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==", - "dev": true, "dependencies": { "isexe": "^2.0.0" }, @@ -20131,7 +20304,6 @@ "version": "8.1.0", "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-8.1.0.tgz", "integrity": "sha512-si7QWI6zUMq56bESFvagtmzMdGOtoxfR+Sez11Mobfc7tm+VkUckk9bW2UeffTGVUbOksxmSw0AA2gs8g71NCQ==", - "dev": true, "dependencies": { "ansi-styles": "^6.1.0", "string-width": "^5.0.1", @@ -20149,7 +20321,6 @@ "version": "7.0.0", "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", - "dev": true, "dependencies": { "ansi-styles": "^4.0.0", "string-width": "^4.1.0", @@ -20165,14 +20336,12 @@ "node_modules/wrap-ansi-cjs/node_modules/emoji-regex": { "version": "8.0.0", "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz", - "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==", - "dev": true + "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==" }, "node_modules/wrap-ansi-cjs/node_modules/string-width": { "version": "4.2.3", "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", - "dev": true, "dependencies": { "emoji-regex": "^8.0.0", "is-fullwidth-code-point": "^3.0.0", @@ -20186,7 +20355,6 @@ "version": "6.0.1", "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-6.0.1.tgz", "integrity": "sha512-n5M855fKb2SsfMIiFFoVrABHJC8QtHwVx+mHWP3QcEqBHYienj5dHSgjbxtC0WEZXYt4wcD6zrQElDPhFuZgfA==", - "dev": true, "engines": { "node": ">=12" }, @@ -20198,7 +20366,6 @@ "version": "6.2.1", "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-6.2.1.tgz", "integrity": "sha512-bN798gFfQX+viw3R7yrGWRqnrN2oRkEkUjjl4JNn4E8GxxbjtG3FbrEIIY3l8/hrwUwIeCZvi4QuOTP4MErVug==", - "dev": true, "engines": { "node": ">=12" }, @@ -20210,7 +20377,6 @@ "version": "5.1.2", "resolved": "https://registry.npmjs.org/string-width/-/string-width-5.1.2.tgz", "integrity": "sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA==", - "dev": true, "dependencies": { "eastasianwidth": "^0.2.0", "emoji-regex": "^9.2.2", @@ -20227,7 +20393,6 @@ "version": "7.1.0", "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-7.1.0.tgz", "integrity": "sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ==", - "dev": true, "dependencies": { "ansi-regex": "^6.0.1" }, @@ -20241,7 +20406,8 @@ "node_modules/wrappy": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", - "integrity": "sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==" + "integrity": "sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==", + "dev": true }, "node_modules/write-file-atomic": { "version": "4.0.2", diff --git a/ui/package.json b/ui/package.json index 35fdd5f17..bfd883e69 100644 --- a/ui/package.json +++ b/ui/package.json @@ -16,7 +16,7 @@ "dependencies": { "@grpc/grpc-js": "^1.9.13", "@monaco-editor/react": "^4.6.0", - "@prisma/client": "^5.7.0", + "@prisma/client": "^5.7.1", "@radix-ui/react-checkbox": "^1.0.4", "@radix-ui/react-collapsible": "^1.0.3", "@radix-ui/react-dialog": "^1.0.5", @@ -35,10 +35,10 @@ "@types/node": "^20.10.5", "@types/react": "^18.2.45", "@types/react-dom": "^18.2.18", - "classnames": "^2.3.2", + "classnames": "^2.3.3", "clsx": "^2.0.0", "long": "^5.2.3", - "lucide-react": "^0.298.0", + "lucide-react": "^0.302.0", "material-symbols": "^0.14.3", "moment": "^2.29.4", "moment-timezone": "^0.5.43", @@ -49,18 +49,19 @@ "react-dom": "18.2.0", "react-select": "^5.8.0", "react-spinners": "^0.13.8", - "styled-components": "^6.1.1", + "react-toastify": "^9.1.3", + "styled-components": "^6.1.3", "swr": "^2.2.4", "zod": "^3.22.4", "zustand": "^4.4.7" }, "devDependencies": { - "@storybook/addon-essentials": "^7.6.5", - "@storybook/addon-interactions": "^7.6.5", - "@storybook/addon-links": "^7.6.5", + "@storybook/addon-essentials": "^7.6.6", + "@storybook/addon-interactions": "^7.6.6", + "@storybook/addon-links": "^7.6.6", "@storybook/addon-styling": "^1.3.7", "@storybook/blocks": "^7.3.0", - "@storybook/nextjs": "^7.6.5", + "@storybook/nextjs": "^7.6.6", "@storybook/react": "^7.3.0", "@storybook/testing-library": "^0.2.2", "autoprefixer": "^10.4.16", @@ -69,15 +70,15 @@ "eslint-config-next": "^14.0.4", "eslint-config-prettier": "^9.1.0", "eslint-plugin-storybook": "^0.6.15", - "gh-pages": "^6.1.0", + "gh-pages": "^6.1.1", "less": "^4.2.0", "postcss": "^8.4.32", "prettier": "^3.1.1", "prettier-plugin-organize-imports": "^3.2.4", - "prisma": "^5.7.0", - "storybook": "^7.6.5", + "prisma": "^5.7.1", + "storybook": "^7.6.6", "string-width": "^7.0.0", - "tailwindcss": "^3.3.6", + "tailwindcss": "^3.4.0", "tailwindcss-animate": "^1.0.7", "typescript": "^5.3.3", "webpack": "^5.89.0" diff --git a/ui/prisma/schema.prisma b/ui/prisma/schema.prisma index 3e71753d4..3337588e2 100644 --- a/ui/prisma/schema.prisma +++ b/ui/prisma/schema.prisma @@ -155,7 +155,51 @@ model peer_slot_size { confirmed_flush_lsn String? slot_size BigInt? updated_at DateTime @default(now()) @db.Timestamp(6) + wal_status String? @@index([slot_name], map: "index_slot_name") @@schema("peerdb_stats") } + +/// This table contains check constraints and requires additional setup for migrations. Visit https://pris.ly/d/check-constraints for more info. +model alerting_config { + id BigInt @id @default(autoincrement()) + service_type String + service_config Json + + @@schema("peerdb_stats") +} + +/// This table contains check constraints and requires additional setup for migrations. Visit https://pris.ly/d/check-constraints for more info. +model alerts_v1 { + id BigInt @id @default(autoincrement()) + alert_key String + alert_level String @default("critical") + alert_message String + created_timestamp DateTime? @default(now()) @db.Timestamp(6) + + @@schema("peerdb_stats") +} + +model flow_errors { + id BigInt @id @default(autoincrement()) + flow_name String + error_message String + error_type String + error_timestamp DateTime @default(now()) @db.Timestamp(6) + ack Boolean @default(false) + + @@index([flow_name], map: "idx_flow_errors_flow_name") + @@schema("peerdb_stats") +} + +model schema_deltas_audit_log { + id BigInt @id @default(autoincrement()) + flow_job_name String + read_timestamp DateTime? @default(now()) @db.Timestamp(6) + workflow_id String + run_id String + delta_info Json + + @@schema("peerdb_stats") +}