From a77979aaedcc59e45edf39569bafc44c9bd72e59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 18 Dec 2023 17:29:29 +0000 Subject: [PATCH 01/52] golangci-lint: add prealloc (#844) --- flow/.golangci.yml | 3 ++- flow/connectors/snowflake/qrep_avro_sync.go | 4 ++-- flow/connectors/sql/query_executor.go | 2 +- flow/e2e/bigquery/bigquery_helper.go | 4 ++-- flow/e2e/test_utils.go | 2 +- flow/model/qschema.go | 2 +- 6 files changed, 9 insertions(+), 8 deletions(-) diff --git a/flow/.golangci.yml b/flow/.golangci.yml index fa679ccf56..2c8032f91d 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -7,6 +7,7 @@ linters: - dupl - gofumpt - gosec + - gosimple - misspell - nakedret - stylecheck @@ -14,7 +15,7 @@ linters: - unparam - whitespace - errcheck - - gosimple + - prealloc - staticcheck - ineffassign - unused diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 8c926acb94..eb83b554b1 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -306,8 +306,8 @@ func (c *SnowflakeConnector) GetCopyTransformation( 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)) switch colType { diff --git a/flow/connectors/sql/query_executor.go b/flow/connectors/sql/query_executor.go index 729e86b64b..b95eb511fd 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/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 54b3a180c1..fb9dadb9ba 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -285,7 +285,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 +433,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 { diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index abd0076d54..7c12240580 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -366,7 +366,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/model/qschema.go b/flow/model/qschema.go index 5081b10d12..68243ef30f 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) } From 2500b34cfd8c848eaf31e25ce60c7b6011c02bc8 Mon Sep 17 00:00:00 2001 From: pankaj-peerdb <149565017+pankaj-peerdb@users.noreply.github.com> Date: Mon, 18 Dec 2023 23:47:56 +0530 Subject: [PATCH 02/52] Improve loading 2, mirrors page (#841) Convert mirrors page to render on client --------- Co-authored-by: Kaushik Iska --- ui/app/api/mirrors/route.ts | 33 ++++++++++++++ ui/app/api/peers/getTruePeer.ts | 60 ++++++++++++++++++++++++ ui/app/api/peers/route.ts | 55 ++-------------------- ui/app/mirrors/page.tsx | 81 ++++++++++++++++----------------- 4 files changed, 135 insertions(+), 94 deletions(-) create mode 100644 ui/app/api/mirrors/route.ts create mode 100644 ui/app/api/peers/getTruePeer.ts diff --git a/ui/app/api/mirrors/route.ts b/ui/app/api/mirrors/route.ts new file mode 100644 index 0000000000..f9ca2f1911 --- /dev/null +++ b/ui/app/api/mirrors/route.ts @@ -0,0 +1,33 @@ +import { getTruePeer } from '@/app/api/peers/getTruePeer'; +import prisma from '@/app/utils/prisma'; + +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); + } + }); + return flowArray; +}; + +export async function GET(request: Request) { + const mirrors = await prisma.flows.findMany({ + distinct: 'name', + include: { + sourcePeer: true, + destinationPeer: true, + }, + }); + + const flows = mirrors?.map((mirror) => { + let newMirror: any = { + ...mirror, + sourcePeer: getTruePeer(mirror.sourcePeer), + destinationPeer: getTruePeer(mirror.destinationPeer), + }; + return newMirror; + }); + return new Response(JSON.stringify(stringifyConfig(flows))); +} diff --git a/ui/app/api/peers/getTruePeer.ts b/ui/app/api/peers/getTruePeer.ts new file mode 100644 index 0000000000..1af4155dec --- /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 c865979efe..03aa98ae4a 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/mirrors/page.tsx b/ui/app/mirrors/page.tsx index 717af7b956..7cfe894294 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,43 +7,29 @@ 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); const watermarkCol = config.watermarkColumn.toLowerCase(); @@ -50,7 +38,7 @@ export default async function Mirrors() { 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); return config.watermarkColumn.toLowerCase() === 'xmin'; @@ -58,10 +46,6 @@ export default async function Mirrors() { return false; }); - stringifyConfig(cdcFlows); - stringifyConfig(qrepFlows); - stringifyConfig(xminFlows); - return ( @@ -84,15 +68,28 @@ export default async function Mirrors() { Mirrors - - - - - - - - - + {isLoading && ( + +
+ +
+
+ )} + {!isLoading && ( + + + + )} + {!isLoading && ( + + + + )} + {!isLoading && ( + + + + )}
); } From 6dbdf61a4e2553a5dfe0a1c70e79ede0cbffb029 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Tue, 19 Dec 2023 00:10:25 +0530 Subject: [PATCH 03/52] logs schema deltas to catalog as soon as they are read (#842) Currently stores the following info: ``` 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 ); ``` delta_info is the marshaled version of the `RelationRecord`. Store happens in the same function where `RelationMessages` are processed. --- flow/connectors/postgres/cdc.go | 63 +++++++++++++++---- flow/connectors/postgres/client.go | 2 +- flow/connectors/postgres/postgres.go | 5 +- .../V15__schema_deltas_audit_log.sql | 8 +++ 4 files changed, 63 insertions(+), 15 deletions(-) create mode 100644 nexus/catalog/migrations/V15__schema_deltas_audit_log.sql diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 031ae5a8e3..979723f930 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 { @@ -38,6 +40,10 @@ type PostgresCDCSource struct { // 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,8 @@ type PostgresCDCConfig struct { SrcTableIDNameMapping map[uint32]string TableNameMapping map[string]model.NameAndExclude RelationMessageMapping model.RelationMessageMapping + CatalogPool *pgxpool.Pool + FlowJobName string } // Create a new PostgresCDCSource @@ -71,6 +79,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 } @@ -186,7 +196,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 +210,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) }) @@ -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,7 +359,8 @@ 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) } @@ -464,7 +476,8 @@ func (p *PostgresCDCSource) consumeStream( } } -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 +516,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 +762,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,10 +840,11 @@ 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, diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index dbae60b9e6..77a5413de7 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -383,7 +383,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) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index da65b09c0d..20dd2a5a71 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -236,6 +236,8 @@ func (c *PostgresConnector) PullRecords(catalogPool *pgxpool.Pool, req *model.Pu Publication: publicationName, TableNameMapping: req.TableNameMapping, RelationMessageMapping: req.RelationMessageMapping, + CatalogPool: catalogPool, + FlowJobName: req.FlowJobName, }, c.customTypesMapping) if err != nil { return fmt.Errorf("failed to create cdc source: %w", err) @@ -365,7 +367,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 { 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 0000000000..92fe24cab9 --- /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 From 476549b5b05b24337f3d9ef053fc60707e70f74a Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 19 Dec 2023 13:40:43 +0530 Subject: [PATCH 04/52] PG,BQ,SF CDC: PeerDB Columns (#845) Irons out `_PEERDB_IS_DELETED` (Soft Delete feature) and `_PEERDB_SYNCED_AT` columns for CDC in BQ, SF, and PG. --- flow/connectors/bigquery/bigquery.go | 23 +- .../bigquery/merge_statement_generator.go | 54 ++- .../bigquery/merge_stmt_generator_test.go | 30 +- flow/connectors/postgres/client.go | 116 +++++- flow/connectors/postgres/postgres.go | 9 +- flow/connectors/snowflake/snowflake.go | 21 +- flow/e2e/bigquery/peer_flow_bq_test.go | 101 +++++ flow/e2e/congen.go | 6 +- flow/e2e/postgres/peer_flow_pg_test.go | 86 ++++ flow/e2e/snowflake/peer_flow_sf_test.go | 11 +- flow/generated/protos/flow.pb.go | 370 +++++++++++------- nexus/pt/src/peerdb_flow.rs | 12 + nexus/pt/src/peerdb_flow.serde.rs | 150 +++++++ protos/flow.proto | 8 + ui/grpc_generated/flow.ts | 112 ++++++ 15 files changed, 924 insertions(+), 185 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 6806445f3d..5f966ecf7f 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -793,6 +793,11 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) SyncBatchID: syncBatchID, NormalizeBatchID: 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() @@ -961,7 +966,7 @@ func (c *BigQueryConnector) SetupNormalizedTables( } // 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{ @@ -972,6 +977,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}) diff --git a/flow/connectors/bigquery/merge_statement_generator.go b/flow/connectors/bigquery/merge_statement_generator.go index 2a37ef5ecb..149825c2cf 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -26,6 +26,8 @@ type mergeStmtGenerator struct { NormalizedTableSchema *protos.TableSchema // array of toast column combinations that are unchanged UnchangedToastColumns []string + // _PEERDB_IS_DELETED and _SYNCED_AT columns + peerdbCols *protos.PeerDBColumns } // GenerateMergeStmt generates a merge statements. @@ -39,7 +41,7 @@ func (m *mergeStmtGenerator) generateMergeStmts() []string { "CREATE TEMP TABLE %s AS (%s, %s);", tempTable, flattenedCTE, deDupedCTE) - mergeStmt := m.generateMergeStmt(tempTable) + mergeStmt := m.generateMergeStmt(tempTable, m.peerdbCols) dropTempTableStmt := fmt.Sprintf("DROP TABLE %s;", tempTable) @@ -127,7 +129,7 @@ func (m *mergeStmtGenerator) generateDeDupedCTE() string { } // generateMergeStmt generates a merge statement. -func (m *mergeStmtGenerator) generateMergeStmt(tempTable string) string { +func (m *mergeStmtGenerator) generateMergeStmt(tempTable string, peerdbCols *protos.PeerDBColumns) string { // comma separated list of column names backtickColNames := make([]string, 0, len(m.NormalizedTableSchema.Columns)) pureColNames := make([]string, 0, len(m.NormalizedTableSchema.Columns)) @@ -136,8 +138,19 @@ func (m *mergeStmtGenerator) generateMergeStmt(tempTable string) string { pureColNames = append(pureColNames, colName) } csep := strings.Join(backtickColNames, ", ") - - updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, m.UnchangedToastColumns) + insertColumnsSQL := csep + fmt.Sprintf(", `%s`", peerdbCols.SyncedAtColName) + insertValuesSQL := csep + ",CURRENT_TIMESTAMP" + + updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, + m.UnchangedToastColumns, peerdbCols) + if m.peerdbCols.SoftDelete { + softDeleteInsertColumnsSQL := insertColumnsSQL + fmt.Sprintf(", `%s`", 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)) @@ -148,6 +161,16 @@ func (m *mergeStmtGenerator) generateMergeStmt(tempTable string) string { // _peerdb_target. = _peerdb_deduped. AND _peerdb_target. = _peerdb_deduped. ... pkeySelectSQL := strings.Join(pkeySelectSQLArray, " AND ") + 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) + } + } + return fmt.Sprintf(` MERGE %s.%s _peerdb_target USING %s _peerdb_deduped ON %s @@ -155,8 +178,9 @@ func (m *mergeStmtGenerator) generateMergeStmt(tempTable string) string { 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.Dataset, m.NormalizedTable, tempTable, pkeySelectSQL, insertColumnsSQL, insertValuesSQL, + updateStringToastCols, deletePart) } /* @@ -174,7 +198,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,6 +212,18 @@ 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' diff --git a/flow/connectors/bigquery/merge_stmt_generator_test.go b/flow/connectors/bigquery/merge_stmt_generator_test.go index 41e54114e6..47705167d6 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) { @@ -16,21 +18,28 @@ func TestGenerateUpdateStatement_WithUnchangedToastCols(t *testing.T) { " AND _peerdb_unchanged_toast_columns='' " + "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='col2, col3' " + - "THEN UPDATE SET `col1` = _peerdb_deduped.col1", + "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'" + "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + - " `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='col3'" + "THEN UPDATE SET `col1` = _peerdb_deduped.col1," + - " `col2` = _peerdb_deduped.col2", + " `col2` = _peerdb_deduped.col2," + "`synced_at`=CURRENT_TIMESTAMP," + "`deleted`=FALSE", } - 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]) @@ -53,10 +62,17 @@ 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", } - 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/postgres/client.go b/flow/connectors/postgres/client.go index 77a5413de7..9aa05131c7 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -58,7 +58,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,7 +71,7 @@ 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" @@ -346,15 +346,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)) @@ -523,17 +536,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 +584,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 +642,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,13 +709,24 @@ 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.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) } - return strings.Join(updateStmts, "\n") + return updateStmts } func (c *PostgresConnector) getCurrentLSN() (pglogrepl.LSN, error) { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 20dd2a5a71..82426b3e3f 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -442,8 +442,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 { @@ -634,7 +639,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) diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 90b29e4765..f92ed3e33e 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -751,7 +751,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)) @@ -847,17 +847,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) @@ -866,10 +870,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)) diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index de3ddae7e5..30e203aeba 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -9,6 +9,7 @@ import ( "time" "github.com/PeerDB-io/peer-flow/e2e" + "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 +52,43 @@ func (s PeerFlowE2ETestSuiteBQ) attachSuffix(input string) string { return fmt.Sprintf("%s_%s", input, s.bqSuffix) } +func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, rowID int8) error { + qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, dstQualified) + query := fmt.Sprintf("SELECT `_PEERDB_IS_DELETED`,`_PEERDB_SYNCED_AT` FROM %s WHERE id = %d", + qualifiedTableName, rowID) + + 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 != 2 { + return fmt.Errorf("peerdb column failed: _PEERDB_IS_DELETED or _PEERDB_SYNCED_AT not present") + } + + return nil +} + // setupBigQuery sets up the bigquery connection. func setupBigQuery(t *testing.T) *BigQueryTestHelper { bqHelper, err := NewBigQueryTestHelper() @@ -1095,3 +1133,66 @@ 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, 1) + require.NoError(s.t, err) + + env.AssertExpectations(s.t) +} diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index ac28879f45..e881dd5ead 100644 --- a/flow/e2e/congen.go +++ b/flow/e2e/congen.go @@ -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 2720891fb6..da050ccf64 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/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 3e6f0c2bc0..d4ff50751f 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -1176,8 +1176,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 +1222,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 +1261,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 +1348,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 +1431,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 +1517,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/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index a4a8ff581c..4be91a690f 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -2669,7 +2669,8 @@ 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"` } func (x *QRepConfig) Reset() { @@ -2830,6 +2831,13 @@ func (x *QRepConfig) GetDstTableFullResync() bool { return false } +func (x *QRepConfig) GetSyncedAtColName() string { + if x != nil { + return x.SyncedAtColName + } + return "" +} + type QRepPartition struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3286,6 +3294,69 @@ 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 +} + var File_flow_proto protoreflect.FileDescriptor var file_flow_proto_rawDesc = []byte{ @@ -3837,7 +3908,7 @@ var file_flow_proto_rawDesc = []byte{ 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, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xc6, 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, @@ -3895,94 +3966,106 @@ var file_flow_proto_rawDesc = []byte{ 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, - 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, + 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, 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, 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, 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 +4081,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, 61) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType @@ -4050,74 +4133,75 @@ 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 + nil, // 51: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 52: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 53: peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry + nil, // 54: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + nil, // 55: peerdb_flow.StartFlowInput.RelationMessageMappingEntry + nil, // 56: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + nil, // 57: peerdb_flow.SetupReplicationInput.TableNameMappingEntry + nil, // 58: peerdb_flow.CreateRawTableInput.TableNameMappingEntry + nil, // 59: peerdb_flow.TableSchema.ColumnsEntry + nil, // 60: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + nil, // 61: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + nil, // 62: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + (*Peer)(nil), // 63: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 64: 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 + 63, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 63, // 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 + 51, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + 52, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + 63, // 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 + 63, // 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 + 63, // 13: peerdb_flow.CreateTablesFromExistingInput.peer:type_name -> peerdb_peers.Peer + 53, // 14: peerdb_flow.CreateTablesFromExistingInput.new_to_existing_table_mapping:type_name -> peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry + 54, // 15: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + 64, // 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 + 55, // 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 + 63, // 22: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 63, // 23: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer + 63, // 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 + 56, // 27: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 63, // 28: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 57, // 29: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 63, // 30: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 63, // 31: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 58, // 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 + 59, // 34: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 63, // 35: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 60, // 36: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 63, // 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 + 63, // 39: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 61, // 40: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 62, // 41: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 64, // 42: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 64, // 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 + 63, // 50: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 63, // 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 +4807,18 @@ 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[6].OneofWrappers = []interface{}{} file_flow_proto_msgTypes[19].OneofWrappers = []interface{}{ @@ -4739,7 +4835,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 60, + NumMessages: 61, NumExtensions: 0, NumServices: 0, }, diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 798e09c99b..50b1541e0d 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -472,6 +472,8 @@ 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, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -541,6 +543,16 @@ 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, +} /// 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 0436bf3345..1ebf981cd4 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -2476,6 +2476,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 +2768,9 @@ impl serde::Serialize for QRepConfig { if self.dst_table_full_resync { len += 1; } + if !self.synced_at_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 +2828,9 @@ 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)?; + } struct_ser.end() } } @@ -2738,6 +2876,8 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { "setupWatermarkTableOnDestination", "dst_table_full_resync", "dstTableFullResync", + "synced_at_col_name", + "syncedAtColName", ]; #[allow(clippy::enum_variant_names)] @@ -2760,6 +2900,7 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { NumRowsPerPartition, SetupWatermarkTableOnDestination, DstTableFullResync, + SyncedAtColName, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -2800,6 +2941,7 @@ 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), _ => Ok(GeneratedField::__SkipField__), } } @@ -2837,6 +2979,7 @@ 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; while let Some(k) = map.next_key()? { match k { GeneratedField::FlowJobName => { @@ -2957,6 +3100,12 @@ 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::__SkipField__ => { let _ = map.next_value::()?; } @@ -2981,6 +3130,7 @@ 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(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index 88d87b9835..57ceef506f 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -321,6 +321,8 @@ 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; } message QRepPartition { @@ -364,3 +366,9 @@ 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; +} \ No newline at end of file diff --git a/ui/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index 845b4d627f..094d97765a 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -430,6 +430,7 @@ export interface QRepConfig { * to be used after the old mirror is dropped */ dstTableFullResync: boolean; + syncedAtColName: string; } export interface QRepPartition { @@ -474,6 +475,12 @@ export interface QRepFlowState { disableWaitForNewRows: boolean; } +export interface PeerDBColumns { + softDeleteColName: string; + syncedAtColName: string; + softDelete: boolean; +} + function createBaseTableNameMapping(): TableNameMapping { return { sourceTableName: "", destinationTableName: "" }; } @@ -5301,6 +5308,7 @@ function createBaseQRepConfig(): QRepConfig { numRowsPerPartition: 0, setupWatermarkTableOnDestination: false, dstTableFullResync: false, + syncedAtColName: "", }; } @@ -5360,6 +5368,9 @@ export const QRepConfig = { if (message.dstTableFullResync === true) { writer.uint32(144).bool(message.dstTableFullResync); } + if (message.syncedAtColName !== "") { + writer.uint32(154).string(message.syncedAtColName); + } return writer; }, @@ -5496,6 +5507,13 @@ export const QRepConfig = { message.dstTableFullResync = reader.bool(); continue; + case 19: + if (tag !== 154) { + break; + } + + message.syncedAtColName = reader.string(); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -5529,6 +5547,7 @@ export const QRepConfig = { ? Boolean(object.setupWatermarkTableOnDestination) : false, dstTableFullResync: isSet(object.dstTableFullResync) ? Boolean(object.dstTableFullResync) : false, + syncedAtColName: isSet(object.syncedAtColName) ? String(object.syncedAtColName) : "", }; }, @@ -5588,6 +5607,9 @@ export const QRepConfig = { if (message.dstTableFullResync === true) { obj.dstTableFullResync = message.dstTableFullResync; } + if (message.syncedAtColName !== "") { + obj.syncedAtColName = message.syncedAtColName; + } return obj; }, @@ -5620,6 +5642,7 @@ export const QRepConfig = { message.numRowsPerPartition = object.numRowsPerPartition ?? 0; message.setupWatermarkTableOnDestination = object.setupWatermarkTableOnDestination ?? false; message.dstTableFullResync = object.dstTableFullResync ?? false; + message.syncedAtColName = object.syncedAtColName ?? ""; return message; }, }; @@ -6257,6 +6280,95 @@ export const QRepFlowState = { }, }; +function createBasePeerDBColumns(): PeerDBColumns { + return { softDeleteColName: "", syncedAtColName: "", softDelete: false }; +} + +export const PeerDBColumns = { + encode(message: PeerDBColumns, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.softDeleteColName !== "") { + writer.uint32(10).string(message.softDeleteColName); + } + if (message.syncedAtColName !== "") { + writer.uint32(18).string(message.syncedAtColName); + } + if (message.softDelete === true) { + writer.uint32(24).bool(message.softDelete); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): PeerDBColumns { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBasePeerDBColumns(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 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; + }, +}; + declare const self: any | undefined; declare const window: any | undefined; declare const global: any | undefined; From 51bc1a26deaa265a730246ec3a1852172065f27b Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 19 Dec 2023 17:53:50 +0530 Subject: [PATCH 05/52] UI: Better mirror fetching (#852) The mirror fetch API was sending data in the wrong format, causing the mirrors page to crash when qrep mirrors exist. This PR restores functionality of the mirrors page --- ui/app/api/mirrors/route.ts | 14 +++----------- ui/app/mirrors/page.tsx | 4 ++-- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/ui/app/api/mirrors/route.ts b/ui/app/api/mirrors/route.ts index f9ca2f1911..65e5e3fe6c 100644 --- a/ui/app/api/mirrors/route.ts +++ b/ui/app/api/mirrors/route.ts @@ -3,15 +3,6 @@ import prisma from '@/app/utils/prisma'; 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); - } - }); - return flowArray; -}; - export async function GET(request: Request) { const mirrors = await prisma.flows.findMany({ distinct: 'name', @@ -21,7 +12,8 @@ export async function GET(request: Request) { }, }); - const flows = mirrors?.map((mirror) => { + // 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), @@ -29,5 +21,5 @@ export async function GET(request: Request) { }; return newMirror; }); - return new Response(JSON.stringify(stringifyConfig(flows))); + return new Response(JSON.stringify(flows)); } diff --git a/ui/app/mirrors/page.tsx b/ui/app/mirrors/page.tsx index 7cfe894294..c98b402857 100644 --- a/ui/app/mirrors/page.tsx +++ b/ui/app/mirrors/page.tsx @@ -31,7 +31,7 @@ export default function Mirrors() { 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'; } @@ -40,7 +40,7 @@ export default function Mirrors() { 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; From 6a08210fea811864c028aa75214c8e4f6086665c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Dec 2023 15:21:16 +0000 Subject: [PATCH 06/52] Update go dependencies: CVE-2023-48795 (#850) Fixed by crypto 0.17 --- flow/go.mod | 10 +++++----- flow/go.sum | 20 ++++++++++---------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/flow/go.mod b/flow/go.mod index a3575510c5..356eedb562 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -9,7 +9,7 @@ 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.5 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cockroachdb/pebble v0.0.0-20231210175914-b4d301aeb46a github.com/google/uuid v1.5.0 @@ -37,7 +37,7 @@ 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/grpc v1.60.1 google.golang.org/protobuf v1.31.0 ) @@ -89,7 +89,7 @@ require ( 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/feature/s3/manager v1.15.8 // 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,7 +97,7 @@ 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.6 // 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 @@ -147,7 +147,7 @@ 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/crypto v0.17.0 golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611 golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.19.0 // indirect diff --git a/flow/go.sum b/flow/go.sum index cb1247fe0c..98802c7e49 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -62,8 +62,8 @@ 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.5 h1:y2yfBlwjPDi3/sBVKeznYEdDy6wIhjA2L5NCBMLUIYA= +github.com/aws/aws-sdk-go v1.49.5/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= @@ -74,8 +74,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.12 h1:v/WgB8NxprNvr5inKIiVVrXPuuT github.com/aws/aws-sdk-go-v2/credentials v1.16.12/go.mod h1:X21k0FjEJe+/pauud82HYiQbEr9jRKY3kXEIQ4hXeTQ= 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.8 h1:7wCngExMTAW2Bjf0Y92uWap6ZUcenLLWI5T3VJiQneU= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.8/go.mod h1:XVrAWYYM4ZRwOCOuLoUiao5hbLqNutEdqwCR3ZvkXgc= 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,8 +92,8 @@ 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.6 h1:bkmlzokzTJyrFNA0J+EPlsF8x4/wp+9D45HTHO/ZUiY= +github.com/aws/aws-sdk-go-v2/service/s3 v1.47.6/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= @@ -423,8 +423,8 @@ 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= @@ -548,8 +548,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= From 8e15e28ca77de19a2f9eb059fd66e827daac1c70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Dec 2023 15:31:53 +0000 Subject: [PATCH 07/52] Only run nexus/flow/ui CI when PR affects their directory (#848) --- .github/workflows/ci.yml | 4 +--- .github/workflows/flow.yml | 3 +-- .github/workflows/golang-lint.yml | 4 ++-- .github/workflows/rust-lint.yml | 4 ++-- .github/workflows/ui-build.yml | 7 +++---- .github/workflows/ui-lint.yml | 7 +++---- 6 files changed, 12 insertions(+), 17 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index dad2551196..5a5291f822 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 2ddff33bd0..70f65df13f 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] diff --git a/.github/workflows/golang-lint.yml b/.github/workflows/golang-lint.yml index 48a0510b20..5b4b31441d 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 8a56cac05e..c4a43ad791 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 ec59e13969..752bb9bd7f 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 84fe76b0bc..48428e1572 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 From 77dea98543e1172493ef0f691f2d0895c04df7c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 19 Dec 2023 16:29:33 +0000 Subject: [PATCH 08/52] Remove direct dependency on pkcs1/pkcs8 (#853) Refactoring while attempting to remove pkcs1 to get around https://people.redhat.com/~hkario/marvin --- nexus/Cargo.lock | 18 ++++++++---------- nexus/peer-snowflake/Cargo.toml | 4 +--- nexus/peer-snowflake/src/auth.rs | 10 +++++----- 3 files changed, 14 insertions(+), 18 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 3c243f2958..652a9ebd11 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -1934,8 +1934,6 @@ dependencies = [ "peer-cursor", "pgerror", "pgwire", - "pkcs1", - "pkcs8", "pt", "reqwest", "rsa", @@ -2590,9 +2588,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", @@ -3317,9 +3315,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 +3337,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", ] @@ -4161,9 +4159,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", ] diff --git a/nexus/peer-snowflake/Cargo.toml b/nexus/peer-snowflake/Cargo.toml index a878be492a..6e16ff63cf 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -20,9 +20,7 @@ dashmap = "5.0" pgwire = "0.17" 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" diff --git a/nexus/peer-snowflake/src/auth.rs b/nexus/peer-snowflake/src/auth.rs index 482272aecf..64bb0f0458 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::RsaPrivateKey; +use rsa::pkcs1::EncodeRsaPrivateKey; +use rsa::pkcs8::{DecodePrivateKey, EncodePublicKey}; 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()) @@ -102,7 +102,7 @@ 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(), + self.private_key.to_pkcs1_der()?.as_bytes(), ); self.last_refreshed = SystemTime::now().duration_since(UNIX_EPOCH)?.as_secs(); info!( From 531e1c224a7762a58611eb51ad6d9cac95373a7b Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 20 Dec 2023 01:32:44 +0530 Subject: [PATCH 09/52] SyncedAt Column for QRep (#854) Implements `_PEERDB_SYNCED_AT` for Query Replication mirrors for PG -> [PG, BQ, SF] --- flow/cmd/handler.go | 7 ++ flow/connectors/bigquery/qrep.go | 3 +- flow/connectors/bigquery/qrep_avro_sync.go | 17 ++++- flow/connectors/postgres/qrep.go | 3 +- flow/connectors/postgres/qrep_sync_method.go | 26 +++++-- flow/connectors/snowflake/qrep.go | 4 +- flow/connectors/snowflake/qrep_avro_sync.go | 7 +- flow/e2e/bigquery/peer_flow_bq_test.go | 19 +++-- flow/e2e/bigquery/qrep_flow_bq_test.go | 39 +++++++++- flow/e2e/postgres/qrep_flow_pg_test.go | 79 +++++++++++++++++++- flow/e2e/s3/qrep_flow_s3_test.go | 6 +- flow/e2e/snowflake/qrep_flow_sf_test.go | 57 +++++++++++++- flow/e2e/snowflake/snowflake_helper.go | 23 ++++++ flow/e2e/test_utils.go | 6 +- flow/workflows/qrep_flow.go | 1 + flow/workflows/snapshot_flow.go | 1 + 16 files changed, 271 insertions(+), 27 deletions(-) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index 9efdc59fb5..7b03c9de67 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -268,6 +268,13 @@ func (h *FlowRequestHandler) CreateQRepFlow( } 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/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index a353d432eb..bf1c603d43 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -46,7 +46,8 @@ func (c *BigQueryConnector) SyncQRepRecords( partition.PartitionId, destTable)) avroSync := &QRepAvroSyncMethod{connector: c, gcsBucket: config.StagingPath} - return avroSync.SyncQRepRecords(config.FlowJobName, destTable, partition, tblMetadata, stream) + return avroSync.SyncQRepRecords(config.FlowJobName, destTable, partition, + tblMetadata, stream, config.SyncedAtColName) } func (c *BigQueryConnector) replayTableSchemaDeltasQRep(config *protos.QRepConfig, partition *protos.QRepPartition, diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 9bb01157fe..882f11a8c2 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -48,7 +48,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( flowJobName, dstTableName, syncBatchID), ) // You will need to define your Avro schema as a string - avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata) + avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata, "") if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } @@ -107,6 +107,7 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( partition *protos.QRepPartition, dstTableMetadata *bigquery.TableMetadata, stream *model.QRecordStream, + syncedAtCol string, ) (int, error) { startTime := time.Now() flowLog := slog.Group("sync_metadata", @@ -115,7 +116,7 @@ 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) if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } @@ -137,9 +138,13 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( // Start a transaction stmts := []string{"BEGIN TRANSACTION;"} + selector := "*" + 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) + insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT %s FROM `%s.%s`;", + datasetID, dstTableName, selector, datasetID, stagingTable) stmts = append(stmts, insertStmt) @@ -181,11 +186,15 @@ type AvroSchema struct { func DefineAvroSchema(dstTableName string, dstTableMetadata *bigquery.TableMetadata, + syncedAtCol string, ) (*model.QRecordAvroSchemaDefinition, error) { avroFields := []AvroField{} nullableFields := make(map[string]struct{}) for _, bqField := range dstTableMetadata.Schema { + if bqField.Name == syncedAtCol { + continue + } avroType, err := GetAvroType(bqField) if err != nil { return nil, err diff --git a/flow/connectors/postgres/qrep.go b/flow/connectors/postgres/qrep.go index be8daa903d..ce114b702b 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 a54769e3d8..6725032411 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/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index b099a54ff4..98d20b63ff 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -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,7 +272,7 @@ 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) } diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index eb83b554b1..7184898ae3 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -300,6 +300,7 @@ 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 { @@ -310,6 +311,10 @@ func (c *SnowflakeConnector) GetCopyTransformation( 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, @@ -354,7 +359,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) } diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 30e203aeba..b28577f4d3 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -52,10 +52,14 @@ func (s PeerFlowE2ETestSuiteBQ) attachSuffix(input string) string { return fmt.Sprintf("%s_%s", input, s.bqSuffix) } -func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, rowID int8) error { +func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, softDelete bool) error { qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, dstQualified) - query := fmt.Sprintf("SELECT `_PEERDB_IS_DELETED`,`_PEERDB_SYNCED_AT` FROM %s WHERE id = %d", - qualifiedTableName, rowID) + 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 { @@ -63,6 +67,7 @@ func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, rowID i } recordCount := 0 + for _, record := range recordBatch.Records { for _, entry := range record.Entries { if entry.Kind == qvalue.QValueKindBoolean { @@ -78,12 +83,14 @@ func (s *PeerFlowE2ETestSuiteBQ) checkPeerdbColumns(dstQualified string, rowID i if !ok { return fmt.Errorf("peerdb column failed: _PEERDB_SYNCED_AT is not valid") } + recordCount += 1 } } } - if recordCount != 2 { - return fmt.Errorf("peerdb column failed: _PEERDB_IS_DELETED or _PEERDB_SYNCED_AT not present") + + if recordCount == 0 { + return fmt.Errorf("peerdb column check failed: no records found") } return nil @@ -1191,7 +1198,7 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { // allow only continue as new error require.Contains(s.t, err.Error(), "continue as new") - err = s.checkPeerdbColumns(dstTableName, 1) + err = s.checkPeerdbColumns(dstTableName, true) require.NoError(s.t, err) env.AssertExpectations(s.t) diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index f520014b04..ca74a412c3 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/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 192863e397..1c86c973b9 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" @@ -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/qrep_flow_s3_test.go b/flow/e2e/s3/qrep_flow_s3_test.go index 62523e1adf..fda57ced09 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/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index 49ed3614b9..3ac7fee713 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -13,7 +13,7 @@ import ( ) 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 +77,8 @@ func (s PeerFlowE2ETestSuiteSF) Test_Complete_QRep_Flow_Avro_SF() { query, s.sfHelper.Peer, "", + false, + "", ) require.NoError(s.t, err) @@ -116,6 +118,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 +163,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 +205,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 +255,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 +274,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 38fefeddc0..0401d34f58 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 7c12240580..0aa1c12242 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -123,7 +123,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 +287,8 @@ func CreateQRepWorkflowConfig( query string, dest *protos.Peer, stagingPath string, + setupDst bool, + syncedAtCol string, ) (*protos.QRepConfig, error) { connectionGen := QRepFlowConnectionGenerationConfig{ FlowJobName: flowJobName, @@ -304,6 +306,8 @@ func CreateQRepWorkflowConfig( return nil, err } qrepConfig.InitialCopyOnly = true + qrepConfig.SyncedAtColName = syncedAtCol + qrepConfig.SetupWatermarkTableOnDestination = setupDst return qrepConfig, nil } diff --git a/flow/workflows/qrep_flow.go b/flow/workflows/qrep_flow.go index 2373427c8e..644f61a611 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/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 527fde5720..bc8448b4b7 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -176,6 +176,7 @@ func (s *SnapshotFlowExecution) cloneTable( NumRowsPerPartition: numRowsPerPartition, MaxParallelWorkers: numWorkers, StagingPath: s.config.SnapshotStagingPath, + SyncedAtColName: s.config.SyncedAtColName, WriteMode: &protos.QRepWriteMode{ WriteType: protos.QRepWriteType_QREP_WRITE_MODE_APPEND, }, From d5baa48b5e6c00916318c5eb8705a1a82359f569 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 20 Dec 2023 19:59:30 +0530 Subject: [PATCH 10/52] UI: Improve table picker component (#859) - Fixes SelectAll and SchemaExpand issues - Moves Column view to separate component for readability - Moves table picker below the settings for CDC and has more height - useMemo for table search - overall code improvements --- ui/app/mirrors/create/cdc/cdc.tsx | 13 +- ui/app/mirrors/create/cdc/columnbox.tsx | 80 ++++++ ui/app/mirrors/create/cdc/schemabox.tsx | 272 +++++++++------------ ui/app/mirrors/create/cdc/tablemapping.tsx | 2 +- 4 files changed, 209 insertions(+), 158 deletions(-) create mode 100644 ui/app/mirrors/create/cdc/columnbox.tsx diff --git a/ui/app/mirrors/create/cdc/cdc.tsx b/ui/app/mirrors/create/cdc/cdc.tsx index 65f2158ee6..63155acdf2 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 0000000000..b68560419f --- /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 4195fae83d..d9b15a703c 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 85c889cb4b..2cccea321f 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) => { From 8e4e7baaa66f27623d8a8854a33d07297f27b0e7 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Wed, 20 Dec 2023 09:44:31 -0500 Subject: [PATCH 11/52] Refactor replica identity type and primary key column retrieval in Postgres (#860) This is to support `USING INDEX` replica identity types in Postgres. Treating the index columns as primary key columns for now as it is the fastest way for us to support replica identity index. --- flow/connectors/postgres/client.go | 87 ++++++++++++++++++------- flow/connectors/postgres/postgres.go | 12 ++-- flow/e2e/snowflake/peer_flow_sf_test.go | 66 +++++++++++++++++++ 3 files changed, 137 insertions(+), 28 deletions(-) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 9aa05131c7..5e00ca8a7f 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" ) @@ -77,6 +78,15 @@ const ( deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=$1" ) +type ReplicaIdentityType rune + +const ( + ReplicaIdentityDefault ReplicaIdentityType = 'd' + ReplicaIdentityFull = 'f' + ReplicaIdentityIndex = 'i' + ReplicaIdentityNothing = 'n' +) + // getRelIDForTable returns the relation ID for a table. func (c *PostgresConnector) getRelIDForTable(schemaTable *utils.SchemaTable) (uint32, error) { var relID pgtype.Uint32 @@ -92,10 +102,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 +113,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) { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 82426b3e3f..cad914706f 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -558,12 +558,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) } @@ -581,7 +581,7 @@ func (c *PostgresConnector) getTableSchemaForTable( TableIdentifier: tableName, Columns: make(map[string]string), PrimaryKeyColumns: pKeyCols, - IsReplicaIdentityFull: isFullReplica, + IsReplicaIdentityFull: replicaIdentityType == ReplicaIdentityFull, } for _, fieldDescription := range rows.FieldDescriptions() { @@ -731,18 +731,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) } diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index d4ff50751f..8d521dbb72 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) From 952199444159308fda7e6011a07f37a6f62f1a96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Dec 2023 15:04:01 +0000 Subject: [PATCH 12/52] postgres cdc: update mirror lsn_offset when wal processing raises consumedXLogPos (#823) Not doing so could cause failures if consumption restarted, since retrieval of mirror lsn_offset would start before updated lsn sent to pg, making possible that pg had gotten rid of that wal in meantime Also always use `GREATEST` to make sure lsn_offset is monotonic --- flow/activities/flowable.go | 3 ++ flow/connectors/bigquery/bigquery.go | 17 ++++++++++ flow/connectors/core.go | 3 ++ flow/connectors/eventhub/eventhub.go | 6 ++-- flow/connectors/external_metadata/store.go | 4 +-- flow/connectors/postgres/cdc.go | 38 ++++++++++++++-------- flow/connectors/postgres/client.go | 4 ++- flow/connectors/postgres/postgres.go | 14 +++++++- flow/connectors/s3/s3.go | 4 +-- flow/connectors/snowflake/snowflake.go | 13 +++++++- flow/model/model.go | 2 ++ 11 files changed, 85 insertions(+), 23 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 69628c28d5..11a6d37895 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -264,6 +264,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, RelationMessageMapping: input.RelationMessageMapping, RecordStream: recordBatch, + SetLastOffset: func(lastOffset int64) error { + return dstConn.SetLastOffset(input.FlowConnectionConfigs.FlowJobName, lastOffset) + }, }) }) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 5f966ecf7f..2749566ced 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -340,6 +340,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) diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 707a7f0b11..c3fb138398 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -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) diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 4be57309f2..027d3027fa 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)) @@ -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 ef2cf5e45b..eee1d4ef66 100644 --- a/flow/connectors/external_metadata/store.go +++ b/flow/connectors/external_metadata/store.go @@ -146,7 +146,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 +197,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 979723f930..f2eda2e5f4 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -30,10 +30,10 @@ 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 @@ -56,6 +56,7 @@ type PostgresCDCConfig struct { RelationMessageMapping model.RelationMessageMapping CatalogPool *pgxpool.Pool FlowJobName string + SetLastOffset func(int64) error } // Create a new PostgresCDCSource @@ -72,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(), @@ -152,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 @@ -181,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}) @@ -194,6 +197,7 @@ func (p *PostgresCDCSource) consumeStream( return fmt.Errorf("[initial-flush] SendStandbyStatusUpdate failed: %w", err) } } + proposedConsumedXLogPos := consumedXLogPos var standByLastLogged time.Time cdcRecordsStorage := cdc_records.NewCDCRecordsStore(p.flowJobName) @@ -252,19 +256,27 @@ func (p *PostgresCDCSource) consumeStream( if pkmRequiresResponse { // Update XLogPos to the last processed position, we can only confirm // that this is the last row committed on the destination. + if proposedConsumedXLogPos > consumedXLogPos { + p.logger.Info(fmt.Sprintf("Heartbeat adjusting lsn from %d to %d", consumedXLogPos, proposedConsumedXLogPos)) + consumedXLogPos = proposedConsumedXLogPos + err := p.SetLastOffset(int64(consumedXLogPos)) + if err != nil { + return fmt.Errorf("storing updated LSN failed: %w", err) + } + } + 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 { @@ -469,7 +481,7 @@ func (p *PostgresCDCSource) consumeStream( 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 + proposedConsumedXLogPos = clientXLogPos records.UpdateLatestCheckpoint(int64(clientXLogPos)) } } diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 5e00ca8a7f..d709866239 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -34,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, @@ -486,6 +487,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 } diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index cad914706f..b848c5a5b5 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -185,13 +185,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() { @@ -238,6 +249,7 @@ func (c *PostgresConnector) PullRecords(catalogPool *pgxpool.Pool, req *model.Pu 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) diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 96d16930cc..c40ef05bd4 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/snowflake.go b/flow/connectors/snowflake/snowflake.go index f92ed3e33e..ad4d952547 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -73,6 +73,7 @@ const ( 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=?" + 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=?" getLastNormalizeBatchID_SQL = "SELECT NORMALIZE_BATCH_ID FROM %s.%s WHERE MIRROR_JOB_NAME=?" dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" @@ -301,7 +302,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 @@ -311,10 +312,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) diff --git a/flow/model/model.go b/flow/model/model.go index 02949f3c2e..487616c531 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -50,6 +50,8 @@ 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 { From 9618c5e3cb0e3b08eb4b93c4b16af0bb3a874bce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Wed, 20 Dec 2023 15:28:41 +0000 Subject: [PATCH 13/52] Replace QValueKindAvroSchema with interface{} (#846) Instead of being either a string or a map[string]string, create structs for the different avro schema definitions Remove no longer applicable comments, along with unused nullable parameter --- flow/model/conversion_avro.go | 10 +-- flow/model/qvalue/avro_converter.go | 120 ++++++++++------------------ 2 files changed, 46 insertions(+), 84 deletions(-) diff --git a/flow/model/conversion_avro.go b/flow/model/conversion_avro.go index 6818299073..02a52b26d6 100644 --- a/flow/model/conversion_avro.go +++ b/flow/model/conversion_avro.go @@ -71,25 +71,23 @@ 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 { - avroType, err := qvalue.GetAvroSchemaFromQValueKind(qField.Type, qField.Nullable) + avroType, err := qvalue.GetAvroSchemaFromQValueKind(qField.Type) if err != nil { 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/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index 2cd6fe2e52..1bffa9d130 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -10,10 +10,17 @@ import ( "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,104 +30,61 @@ type QValueKindAvroSchema struct { // // For example, QValueKindInt64 would return an AvroLogicalSchema of "long". Unsupported QValueKinds // will return an error. -// -// The function currently does not support the following QValueKinds: -// - QValueKindBit -// -// Please note that for QValueKindNumeric and QValueKindETime, RespectNull is always -// set to false, regardless of the nullable value passed in. -func GetAvroSchemaFromQValueKind(kind QValueKind, nullable bool) (*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 nil, fmt.Errorf("unsupported QValueKind type: %s", kind) } From 6b4e0e317d557d865869f39eabdf87aabebf073e Mon Sep 17 00:00:00 2001 From: Sai Srirampur Date: Wed, 20 Dec 2023 09:33:15 -0800 Subject: [PATCH 14/52] Replication should now work from PG 16 Standbys (#858) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit I tried PeerDB on Read Replicas in PG 16: 1. Initial Load worked as expected 2. SyncFlows were erroring out: ``` ERROR: recovery is in progress HINT: WAL control functions cannot be executed during recovery. ``` The reason was `pg_current_wal_lsn()` cannot be run on a standby. I changed that function to `pg_last_wal_replay_lsn()` if Replica. For now `CREATE PUBLICATION` needs to be run on primary by user Co-authored-by: Philip Dubé --- flow/connectors/postgres/client.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index d709866239..1d7d4d5fb5 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -253,7 +253,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 @@ -775,7 +776,8 @@ func (c *PostgresConnector) generateUpdateStatement(allCols []string, } 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 { From 9ec86d48f86b4243caa68b85f3035bb5b28db778 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Thu, 21 Dec 2023 00:24:41 +0530 Subject: [PATCH 15/52] Fix: Soft Delete for Bigquery Snapshot (#862) BigQuery uses destination table schema to derive AVRO schema. We create two columns on the destination which makes the AVRO writing fail. This PR fixes that by skipping the soft delete column (the synced_at column was already being skipped) for AVRO writing --- flow/connectors/bigquery/qrep.go | 2 +- flow/connectors/bigquery/qrep_avro_sync.go | 13 +- flow/generated/protos/flow.pb.go | 209 +++++++++++---------- flow/workflows/snapshot_flow.go | 1 + nexus/pt/src/peerdb_flow.rs | 2 + nexus/pt/src/peerdb_flow.serde.rs | 18 ++ protos/flow.proto | 1 + ui/grpc_generated/flow.ts | 17 ++ 8 files changed, 159 insertions(+), 104 deletions(-) diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index bf1c603d43..df771e50a2 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -47,7 +47,7 @@ func (c *BigQueryConnector) SyncQRepRecords( avroSync := &QRepAvroSyncMethod{connector: c, gcsBucket: config.StagingPath} return avroSync.SyncQRepRecords(config.FlowJobName, destTable, partition, - tblMetadata, stream, config.SyncedAtColName) + tblMetadata, stream, config.SyncedAtColName, config.SoftDeleteColName) } func (c *BigQueryConnector) replayTableSchemaDeltasQRep(config *protos.QRepConfig, partition *protos.QRepPartition, diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 882f11a8c2..6a83d23ae8 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -48,7 +48,7 @@ func (s *QRepAvroSyncMethod) SyncRecords( flowJobName, dstTableName, syncBatchID), ) // You will need to define your Avro schema as a string - avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata, "") + avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata, "", "") if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } @@ -108,6 +108,7 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( dstTableMetadata *bigquery.TableMetadata, stream *model.QRecordStream, syncedAtCol string, + softDeleteCol string, ) (int, error) { startTime := time.Now() flowLog := slog.Group("sync_metadata", @@ -116,7 +117,7 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( slog.String("destinationTable", dstTableName), ) // You will need to define your Avro schema as a string - avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata, syncedAtCol) + avroSchema, err := DefineAvroSchema(dstTableName, dstTableMetadata, syncedAtCol, softDeleteCol) if err != nil { return 0, fmt.Errorf("failed to define Avro schema: %w", err) } @@ -139,8 +140,11 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( stmts := []string{"BEGIN TRANSACTION;"} selector := "*" + if softDeleteCol != "" { // PeerDB column + selector += ", FALSE" + } if syncedAtCol != "" { // PeerDB column - selector = "*, CURRENT_TIMESTAMP" + selector += ", CURRENT_TIMESTAMP" } // Insert the records from the staging table into the destination table insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT %s FROM `%s.%s`;", @@ -187,12 +191,13 @@ 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 { + if bqField.Name == syncedAtCol || bqField.Name == softDeleteCol { continue } avroType, err := GetAvroType(bqField) diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 4be91a690f..3cb146df01 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -2671,6 +2671,7 @@ type QRepConfig struct { // 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"` 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() { @@ -2838,6 +2839,13 @@ func (x *QRepConfig) GetSyncedAtColName() string { return "" } +func (x *QRepConfig) GetSoftDeleteColName() string { + if x != nil { + return x.SoftDeleteColName + } + return "" +} + type QRepPartition struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3908,7 +3916,7 @@ var file_flow_proto_rawDesc = []byte{ 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, 0xc6, 0x07, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 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, @@ -3968,104 +3976,107 @@ var file_flow_proto_rawDesc = []byte{ 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, 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, 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, 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, + 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, 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, 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 ( diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index bc8448b4b7..998317c520 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -177,6 +177,7 @@ func (s *SnapshotFlowExecution) cloneTable( 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/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 50b1541e0d..4163aa0363 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -474,6 +474,8 @@ pub struct QRepConfig { 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)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 1ebf981cd4..3374e09b99 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -2771,6 +2771,9 @@ impl serde::Serialize for QRepConfig { 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)?; @@ -2831,6 +2834,9 @@ impl serde::Serialize for QRepConfig { 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() } } @@ -2878,6 +2884,8 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { "dstTableFullResync", "synced_at_col_name", "syncedAtColName", + "soft_delete_col_name", + "softDeleteColName", ]; #[allow(clippy::enum_variant_names)] @@ -2901,6 +2909,7 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { SetupWatermarkTableOnDestination, DstTableFullResync, SyncedAtColName, + SoftDeleteColName, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -2942,6 +2951,7 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { "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__), } } @@ -2980,6 +2990,7 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { 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 => { @@ -3106,6 +3117,12 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { } 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::()?; } @@ -3131,6 +3148,7 @@ impl<'de> serde::Deserialize<'de> for QRepConfig { 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/protos/flow.proto b/protos/flow.proto index 57ceef506f..965d8d0fce 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -323,6 +323,7 @@ message QRepConfig { bool dst_table_full_resync = 18; string synced_at_col_name = 19; + string soft_delete_col_name = 20; } message QRepPartition { diff --git a/ui/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index 094d97765a..1ba651b45f 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -431,6 +431,7 @@ export interface QRepConfig { */ dstTableFullResync: boolean; syncedAtColName: string; + softDeleteColName: string; } export interface QRepPartition { @@ -5309,6 +5310,7 @@ function createBaseQRepConfig(): QRepConfig { setupWatermarkTableOnDestination: false, dstTableFullResync: false, syncedAtColName: "", + softDeleteColName: "", }; } @@ -5371,6 +5373,9 @@ export const QRepConfig = { if (message.syncedAtColName !== "") { writer.uint32(154).string(message.syncedAtColName); } + if (message.softDeleteColName !== "") { + writer.uint32(162).string(message.softDeleteColName); + } return writer; }, @@ -5514,6 +5519,13 @@ export const QRepConfig = { message.syncedAtColName = reader.string(); continue; + case 20: + if (tag !== 162) { + break; + } + + message.softDeleteColName = reader.string(); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -5548,6 +5560,7 @@ export const QRepConfig = { : false, dstTableFullResync: isSet(object.dstTableFullResync) ? Boolean(object.dstTableFullResync) : false, syncedAtColName: isSet(object.syncedAtColName) ? String(object.syncedAtColName) : "", + softDeleteColName: isSet(object.softDeleteColName) ? String(object.softDeleteColName) : "", }; }, @@ -5610,6 +5623,9 @@ export const QRepConfig = { if (message.syncedAtColName !== "") { obj.syncedAtColName = message.syncedAtColName; } + if (message.softDeleteColName !== "") { + obj.softDeleteColName = message.softDeleteColName; + } return obj; }, @@ -5643,6 +5659,7 @@ export const QRepConfig = { message.setupWatermarkTableOnDestination = object.setupWatermarkTableOnDestination ?? false; message.dstTableFullResync = object.dstTableFullResync ?? false; message.syncedAtColName = object.syncedAtColName ?? ""; + message.softDeleteColName = object.softDeleteColName ?? ""; return message; }, }; From e1b59cf7d75e26914008827c2b80fa35c21f63a9 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Thu, 21 Dec 2023 09:52:08 -0500 Subject: [PATCH 16/52] Debug S3 Test Failures (#864) --- flow/connectors/utils/avro/avro_writer.go | 5 +++-- flow/e2e/s3/cdc_s3_test.go | 5 +++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/flow/connectors/utils/avro/avro_writer.go b/flow/connectors/utils/avro/avro_writer.go index 7e194f2d34..90c016b404 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/e2e/s3/cdc_s3_test.go b/flow/e2e/s3/cdc_s3_test.go index bfa19f866b..a938f673b3 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") From e0ace30d7817839b515bf53143c538eae3476f83 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Thu, 21 Dec 2023 15:04:15 +0000 Subject: [PATCH 17/52] Don't use exp/slog (#865) --- flow/model/qvalue/avro_converter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/model/qvalue/avro_converter.go b/flow/model/qvalue/avro_converter.go index 1bffa9d130..4f9cbe2e47 100644 --- a/flow/model/qvalue/avro_converter.go +++ b/flow/model/qvalue/avro_converter.go @@ -2,12 +2,12 @@ package qvalue import ( "fmt" + "log/slog" "math/big" "time" "github.com/google/uuid" "github.com/linkedin/goavro/v2" - "golang.org/x/exp/slog" ) // https://avro.apache.org/docs/1.11.0/spec.html From 65c585aa3bf1da03185a7dde99a17f4588e1039c Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Thu, 21 Dec 2023 22:16:46 +0530 Subject: [PATCH 18/52] Soft Delete: Patch update statements (#867) Fixes the update statements we generate during normalize merge for bigquery and postgres. Also adds comments for SF,PG and BQ for generating update statements --- .../bigquery/merge_statement_generator.go | 13 +++++ .../bigquery/merge_stmt_generator_test.go | 50 +++++++++++-------- flow/connectors/postgres/client.go | 15 +++++- flow/connectors/snowflake/snowflake.go | 4 ++ 4 files changed, 61 insertions(+), 21 deletions(-) diff --git a/flow/connectors/bigquery/merge_statement_generator.go b/flow/connectors/bigquery/merge_statement_generator.go index 149825c2cf..22161c434b 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -229,6 +229,19 @@ func (m *mergeStmtGenerator) generateUpdateStatements( (_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 47705167d6..37dd3e07ed 100644 --- a/flow/connectors/bigquery/merge_stmt_generator_test.go +++ b/flow/connectors/bigquery/merge_stmt_generator_test.go @@ -14,25 +14,31 @@ 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," + - "`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`=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`=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`=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`=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, &protos.PeerDBColumns{ @@ -47,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) } } @@ -65,6 +71,10 @@ func TestGenerateUpdateStatement_NoUnchangedToastCols(t *testing.T) { " `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, diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 1d7d4d5fb5..9f516a49c6 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -761,7 +761,7 @@ func (c *PostgresConnector) generateUpdateStatement(allCols []string, peerdbCols.SyncedAtColName)) } // set soft-deleted to false, tackles insert after soft-delete - if peerdbCols.SoftDeleteColName != "" { + if peerdbCols.SoftDelete && (peerdbCols.SoftDeleteColName != "") { tmpArray = append(tmpArray, fmt.Sprintf(`"%s" = FALSE`, peerdbCols.SoftDeleteColName)) } @@ -771,6 +771,19 @@ func (c *PostgresConnector) generateUpdateStatement(allCols []string, 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 updateStmts } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index ad4d952547..8cd8240f11 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -1064,6 +1064,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, ", ") From 08861aaa54f9e5ee1f22849f2bbf72f518120522 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Thu, 21 Dec 2023 23:52:19 +0530 Subject: [PATCH 19/52] basic alerting, refactored to use slack-go instead (#866) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit For now, an alerting function logs an entry in an alerts table in catalog, and also dispatches notifications to one or more slack channels. Events are staggered to be atleast 15 minutes apart by default, as long as the same "alert key" is used. This delay can be configured via the `PEERDB_ALERTING_GAP_MINUTES` environment variable, and alerts can be disabled by setting it to 0. Logs alerts in the following 2 cases for now: 1) When computed slot size of a peer exceeds a threshold, which is `5000MB` by default. This limit is configurable via the environment variable `PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD`. Setting it to 0 should disable this type of alerts entirely. 2) When the number of connections from the configured user for a peer exceed a threshold, which is 5 by default. This limit is configurable by the environment variable `PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD`. Setting it to 0 should disable this type of alerts entirely. To configure alerting, insert a row into the `peerdb_stats.alerting_config` table like this: ``` INSERT INTO peerdb_stats.alerting_config(service_type,service_config) VALUES('slack','{"auth_token": "SLACK_AUTH_TOKEN", "channel_ids": ["SLACK_CHANNEL","IDS_IN_ARRAY"]}'); ``` --------- Co-authored-by: Philip Dubé Co-authored-by: Kaushik Iska --- docker-compose-dev.yml | 2 +- docker-compose.yml | 2 +- flow/activities/flowable.go | 40 +++- flow/cmd/worker.go | 2 + flow/connectors/core.go | 3 + flow/connectors/postgres/client.go | 5 +- flow/connectors/postgres/postgres.go | 18 ++ flow/e2e/test_utils.go | 6 +- flow/generated/protos/flow.pb.go | 211 ++++++++++++------ flow/go.mod | 2 + flow/go.sum | 8 + flow/peerdbenv/config.go | 16 ++ flow/shared/alerting/alerting.go | 101 +++++++++ flow/shared/alerting/slack_alert_sender.go | 38 ++++ .../migrations/V16__alerting_config_init.sql | 13 ++ nexus/pt/src/peerdb_flow.rs | 8 + nexus/pt/src/peerdb_flow.serde.rs | 116 ++++++++++ protos/flow.proto | 5 + ui/grpc_generated/flow.ts | 81 +++++++ 19 files changed, 602 insertions(+), 75 deletions(-) create mode 100644 flow/shared/alerting/alerting.go create mode 100644 flow/shared/alerting/slack_alert_sender.go create mode 100644 nexus/catalog/migrations/V16__alerting_config_init.sql diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index f5d8942ba1..158483a7eb 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -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 diff --git a/docker-compose.yml b/docker-compose.yml index 5645f2b1e4..add4bb2db1 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -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 diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 11a6d37895..b5cf8e8973 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. @@ -174,10 +176,38 @@ func (a *FlowableActivity) handleSlotInfo( ) error { slotInfo, err := srcConn.GetSlotInfo(slotName) if err != nil { - slog.Warn("warning: failed to get slot info", slog.Any("error", err)) + slog.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) return err } + deploymentUIDPrefix := "" + if peerdbenv.GetPeerDBDeploymentUID() != "" { + deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.GetPeerDBDeploymentUID()) + } + + slotLagInMBThreshold := peerdbenv.GetPeerDBSlotLagMBAlertThreshold() + 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.GetPeerDBOpenConnectionsAlertThreshold() + 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]) } @@ -190,7 +220,13 @@ func (a *FlowableActivity) recordSlotSizePeriodically( slotName string, peerName string, ) { - timeout := 10 * time.Minute + // 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() diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index ba6e0d0e18..a42ac76b47 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" @@ -133,6 +134,7 @@ func WorkerMain(opts *WorkerOptions) error { w.RegisterWorkflow(peerflow.HeartbeatFlowWorkflow) w.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, + Alerter: alerting.NewAlerter(conn), }) err = w.Run(worker.InterruptCh()) diff --git a/flow/connectors/core.go b/flow/connectors/core.go index c3fb138398..e5efec63ca 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -48,6 +48,9 @@ type CDCPullConnector interface { // 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 { diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 9f516a49c6..dc604d5631 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -75,8 +75,9 @@ const ( ) %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 diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index b848c5a5b5..a46005fe01 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -895,3 +895,21 @@ func (c *PostgresConnector) SendWALHeartbeat() error { return nil } + +// 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 nil, fmt.Errorf("error while reading result row: %w", err) + } + + return &protos.GetOpenConnectionsForUserResult{ + UserName: c.config.User, + CurrentOpenConnections: result.Int64, + }, nil +} diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 0aa1c12242..13ca8044e5 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,10 @@ 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}) + env.RegisterActivity(&activities.FlowableActivity{ + CatalogPool: conn, + Alerter: alerting.NewAlerter(conn), + }) env.RegisterActivity(&activities.SnapshotActivity{}) } diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 3cb146df01..27d2bb5c04 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -3365,6 +3365,61 @@ func (x *PeerDBColumns) GetSoftDelete() bool { 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{ @@ -4056,27 +4111,34 @@ var file_flow_proto_rawDesc = []byte{ 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, 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, + 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 ( @@ -4092,7 +4154,7 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 61) +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 @@ -4145,74 +4207,75 @@ var file_flow_proto_goTypes = []interface{}{ (*ReplayTableSchemaDeltaInput)(nil), // 48: peerdb_flow.ReplayTableSchemaDeltaInput (*QRepFlowState)(nil), // 49: peerdb_flow.QRepFlowState (*PeerDBColumns)(nil), // 50: peerdb_flow.PeerDBColumns - nil, // 51: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 52: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 53: peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry - nil, // 54: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry - nil, // 55: peerdb_flow.StartFlowInput.RelationMessageMappingEntry - nil, // 56: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 57: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 58: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 59: peerdb_flow.TableSchema.ColumnsEntry - nil, // 60: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 61: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 62: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 63: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 64: google.protobuf.Timestamp + (*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 - 63, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 63, // 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 - 51, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 52, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 63, // 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 - 63, // 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 - 63, // 13: peerdb_flow.CreateTablesFromExistingInput.peer:type_name -> peerdb_peers.Peer - 53, // 14: peerdb_flow.CreateTablesFromExistingInput.new_to_existing_table_mapping:type_name -> peerdb_flow.CreateTablesFromExistingInput.NewToExistingTableMappingEntry - 54, // 15: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry - 64, // 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 - 55, // 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 - 63, // 22: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 63, // 23: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 63, // 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 - 56, // 27: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 63, // 28: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 57, // 29: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 63, // 30: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 63, // 31: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 58, // 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 - 59, // 34: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 63, // 35: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 60, // 36: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 63, // 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 - 63, // 39: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 61, // 40: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 62, // 41: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 64, // 42: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 64, // 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 - 63, // 50: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 63, // 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 @@ -4830,6 +4893,18 @@ func file_flow_proto_init() { 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{}{ @@ -4846,7 +4921,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 61, + NumMessages: 62, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/go.mod b/flow/go.mod index 356eedb562..dd59212b71 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -24,6 +24,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 @@ -53,6 +54,7 @@ require ( github.com/getsentry/sentry-go v0.25.0 // indirect github.com/go-logr/logr v1.3.0 // 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 diff --git a/flow/go.sum b/flow/go.sum index 98802c7e49..e9a2d2b5eb 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -171,6 +171,8 @@ github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre 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= @@ -351,6 +357,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= diff --git a/flow/peerdbenv/config.go b/flow/peerdbenv/config.go index cf0e51c21a..970be3455d 100644 --- a/flow/peerdbenv/config.go +++ b/flow/peerdbenv/config.go @@ -64,3 +64,19 @@ func GetPeerDBCatalogPassword() string { func GetPeerDBCatalogDatabase() string { return getEnvString("PEERDB_CATALOG_DATABASE", "") } + +// PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD, 0 disables slot lag alerting entirely +func GetPeerDBSlotLagMBAlertThreshold() uint32 { + return getEnvUint32("PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD", 5000) +} + +// PEERDB_ALERTING_GAP_MINUTES, 0 disables all alerting entirely +func GetPeerDBAlertingGapMinutesAsDuration() 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 GetPeerDBOpenConnectionsAlertThreshold() uint32 { + return getEnvUint32("PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD", 5) +} diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go new file mode 100644 index 0000000000..7dc3cb489f --- /dev/null +++ b/flow/shared/alerting/alerting.go @@ -0,0 +1,101 @@ +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(catalogPool *pgxpool.Pool) ([]*slackAlertSender, error) { + rows, err := catalogPool.Query(context.Background(), + "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 { + return &Alerter{ + catalogPool: catalogPool, + logger: slog.Default(), + } +} + +// 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.GetPeerDBAlertingGapMinutesAsDuration() == 0 { + a.logger.WarnContext(ctx, "Alerting disabled via environment variable, returning") + return + } + + if a.catalogPool != nil { + slackAlertSenders, err := registerSendersFromPool(a.catalogPool) + if err != nil { + a.logger.WarnContext(ctx, "failed to set Slack senders", slog.Any("error", err)) + return + } + if len(slackAlertSenders) == 0 { + a.logger.WarnContext(ctx, "no Slack senders configured, returning") + return + } + + row := a.catalogPool.QueryRow(context.Background(), + `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.GetPeerDBAlertingGapMinutesAsDuration() { + for _, slackAlertSender := range slackAlertSenders { + err = slackAlertSender.sendAlert(context.Background(), + 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 + } + _, _ = a.catalogPool.Exec(context.Background(), + "INSERT INTO peerdb_stats.alerts_v1(alert_key,alert_message) VALUES($1,$2)", + alertKey, alertMessage) + } + } + } +} diff --git a/flow/shared/alerting/slack_alert_sender.go b/flow/shared/alerting/slack_alert_sender.go new file mode 100644 index 0000000000..04c9a4ad86 --- /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/nexus/catalog/migrations/V16__alerting_config_init.sql b/nexus/catalog/migrations/V16__alerting_config_init.sql new file mode 100644 index 0000000000..de16cc340a --- /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/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 4163aa0363..a58fcbb95d 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -555,6 +555,14 @@ pub struct PeerDbColumns { #[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 3374e09b99..0e079f82fa 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -1806,6 +1806,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 diff --git a/protos/flow.proto b/protos/flow.proto index 965d8d0fce..430968bcba 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -372,4 +372,9 @@ 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/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index 1ba651b45f..ba881795ed 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -482,6 +482,11 @@ export interface PeerDBColumns { softDelete: boolean; } +export interface GetOpenConnectionsForUserResult { + userName: string; + currentOpenConnections: number; +} + function createBaseTableNameMapping(): TableNameMapping { return { sourceTableName: "", destinationTableName: "" }; } @@ -6386,6 +6391,82 @@ export const PeerDBColumns = { }, }; +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; From 5d32604b4b738af2ded63e9e1779ebd906d92c3d Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 22 Dec 2023 18:25:36 +0530 Subject: [PATCH 20/52] only initialize replication pool when needed in CDCPull (#872) --- flow/cmd/peer_data.go | 4 +-- flow/connectors/core.go | 15 ++++++---- flow/connectors/postgres/postgres.go | 30 ++++++++++--------- .../connectors/postgres/postgres_repl_test.go | 2 +- .../postgres/postgres_schema_delta_test.go | 2 +- flow/e2e/postgres/qrep_flow_pg_test.go | 2 +- 6 files changed, 30 insertions(+), 25 deletions(-) diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 454dceb20e..34f31219ed 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -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/connectors/core.go b/flow/connectors/core.go index e5efec63ca..1b823b04ae 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -140,7 +140,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 } @@ -150,7 +150,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: @@ -172,7 +172,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: @@ -186,7 +186,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: @@ -198,7 +198,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: @@ -219,7 +219,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/postgres/postgres.go b/flow/connectors/postgres/postgres.go index a46005fe01..2a6406cfea 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" diff --git a/flow/connectors/postgres/postgres_repl_test.go b/flow/connectors/postgres/postgres_repl_test.go index b50a1f89fc..df3a7de13f 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 98a6a47b99..8a919eb214 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/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 1c86c973b9..0bb886f9a3 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -55,7 +55,7 @@ func (s *PeerFlowE2ETestSuitePG) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, false) s.NoError(err) } From fd9aebe35dd740428b6a372cfa6d7b89a893a1c2 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 22 Dec 2023 18:27:03 +0530 Subject: [PATCH 21/52] remove SendWALHeartbeat from the CDCPullConnector interface (#873) Since this is done on the activity level now for all Postgres peers Co-authored-by: Kaushik Iska --- flow/connectors/core.go | 3 --- flow/connectors/postgres/postgres.go | 16 ---------------- 2 files changed, 19 deletions(-) diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 1b823b04ae..477b7cf46b 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -43,9 +43,6 @@ 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) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 2a6406cfea..1152493b01 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -882,22 +882,6 @@ 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) - if err != nil { - return fmt.Errorf("error bumping wal position: %w", err) - } - - return nil -} - // GetLastOffset returns the last synced offset for a job. func (c *PostgresConnector) GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) { row := c.pool. From 45a420573f560ef8fc3ce6eda8f2ec7fabd2394f Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 22 Dec 2023 18:27:28 +0530 Subject: [PATCH 22/52] changing naming convention for getters in peerdbenv (#875) the Go convention is not to have `Get` in the function name, `GetPeer` vs `Peer`. Can be changed in other places on a case by case basis. Co-authored-by: Kaushik Iska --- flow/activities/flowable.go | 10 +++---- flow/cmd/version.go | 2 +- flow/connectors/eventhub/eventhub.go | 2 +- flow/connectors/utils/catalog/env.go | 10 +++---- .../utils/cdc_records/cdc_records_storage.go | 2 +- flow/model/model.go | 2 +- flow/peerdbenv/config.go | 28 +++++++++---------- flow/shared/alerting/alerting.go | 4 +-- flow/shared/constants.go | 2 +- 9 files changed, 31 insertions(+), 31 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index b5cf8e8973..982364ba58 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -181,11 +181,11 @@ func (a *FlowableActivity) handleSlotInfo( } deploymentUIDPrefix := "" - if peerdbenv.GetPeerDBDeploymentUID() != "" { - deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.GetPeerDBDeploymentUID()) + if peerdbenv.PeerDBDeploymentUID() != "" { + deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.PeerDBDeploymentUID()) } - slotLagInMBThreshold := peerdbenv.GetPeerDBSlotLagMBAlertThreshold() + 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! @@ -194,7 +194,7 @@ cc: `, } // Also handles alerts for PeerDB user connections exceeding a given limit here - maxOpenConnectionsThreshold := peerdbenv.GetPeerDBOpenConnectionsAlertThreshold() + maxOpenConnectionsThreshold := peerdbenv.PeerDBOpenConnectionsAlertThreshold() res, err := srcConn.GetOpenConnectionsForUser() if err != nil { slog.WarnContext(ctx, "warning: failed to get current open connections", slog.Any("error", err)) @@ -294,7 +294,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, 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, diff --git a/flow/cmd/version.go b/flow/cmd/version.go index 94210ea9cb..3338a20e6a 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/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 027d3027fa..05347a4263 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -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() diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go index cdd85535b9..f5c8e0507d 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 3147045a76..66722b69ed 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/model/model.go b/flow/model/model.go index 487616c531..581b57178b 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -329,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! diff --git a/flow/peerdbenv/config.go b/flow/peerdbenv/config.go index 970be3455d..33b1058066 100644 --- a/flow/peerdbenv/config.go +++ b/flow/peerdbenv/config.go @@ -9,74 +9,74 @@ 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 GetPeerDBSlotLagMBAlertThreshold() uint32 { +func PeerDBSlotLagMBAlertThreshold() uint32 { return getEnvUint32("PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD", 5000) } // PEERDB_ALERTING_GAP_MINUTES, 0 disables all alerting entirely -func GetPeerDBAlertingGapMinutesAsDuration() time.Duration { +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 GetPeerDBOpenConnectionsAlertThreshold() uint32 { +func PeerDBOpenConnectionsAlertThreshold() uint32 { return getEnvUint32("PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD", 5) } diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go index 7dc3cb489f..a3a3d6c6e0 100644 --- a/flow/shared/alerting/alerting.go +++ b/flow/shared/alerting/alerting.go @@ -57,7 +57,7 @@ func NewAlerter(catalogPool *pgxpool.Pool) *Alerter { // 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.GetPeerDBAlertingGapMinutesAsDuration() == 0 { + if peerdbenv.PeerDBAlertingGapMinutesAsDuration() == 0 { a.logger.WarnContext(ctx, "Alerting disabled via environment variable, returning") return } @@ -84,7 +84,7 @@ func (a *Alerter) AlertIf(ctx context.Context, alertKey string, alertMessage str return } - if time.Since(createdTimestamp) >= peerdbenv.GetPeerDBAlertingGapMinutesAsDuration() { + if time.Since(createdTimestamp) >= peerdbenv.PeerDBAlertingGapMinutesAsDuration() { for _, slackAlertSender := range slackAlertSenders { err = slackAlertSender.sendAlert(context.Background(), fmt.Sprintf(":rotating_light:Alert:rotating_light:: %s", alertKey), alertMessage) diff --git a/flow/shared/constants.go b/flow/shared/constants.go index 8589b55487..aa447520b5 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 } From 60044de384a0f7244410233b54f659d356290f2f Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Fri, 22 Dec 2023 18:28:34 +0530 Subject: [PATCH 23/52] making all Snowflake DB access use a context (#876) Closes #766 Co-authored-by: Kaushik Iska --- flow/connectors/snowflake/qrep.go | 4 ++-- flow/connectors/snowflake/qrep_avro_sync.go | 16 +++++++--------- flow/connectors/snowflake/snowflake.go | 5 +++-- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 98d20b63ff..def870c183 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) } @@ -294,7 +294,7 @@ func (c *SnowflakeConnector) getColsFromTable(tableName string) (*model.ColumnIn WHERE UPPER(table_name) = '%s' AND UPPER(table_schema) = '%s' `, components.tableIdentifier, components.schemaIdentifier) - 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 7184898ae3..30834c2554 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -112,7 +112,6 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( s.connector.logger.Info("sync function called and schema acquired", partitionLog) err = s.addMissingColumns( - config.FlowJobName, schema, dstTableSchema, dstTableName, @@ -152,7 +151,6 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( } func (s *SnowflakeAvroSyncMethod) addMissingColumns( - flowJobName string, schema *model.QRecordSchema, dstTableSchema []*sql.ColumnType, dstTableName string, @@ -197,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) } } @@ -290,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) } @@ -395,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) @@ -434,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) } @@ -518,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) @@ -526,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) } @@ -538,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 8cd8240f11..e61830db88 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -480,8 +480,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) From 7f97c0010a7ff2e75ce59a063b05b8d837019467 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 22 Dec 2023 08:55:47 -0500 Subject: [PATCH 24/52] Make alerter independent of slack (#877) This makes the alerter log to catalog whether slack based alerting is setup or not. Also fixes usage of Context in the alerting class. --- flow/cmd/worker.go | 8 ++- flow/e2e/test_utils.go | 8 ++- flow/shared/alerting/alerting.go | 85 ++++++++++++++++++-------------- 3 files changed, 63 insertions(+), 38 deletions(-) diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index a42ac76b47..eea0e9184f 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -132,9 +132,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: alerting.NewAlerter(conn), + Alerter: alerter, }) err = w.Run(worker.InterruptCh()) diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 13ca8044e5..8bea8cf984 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -60,9 +60,15 @@ func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment, t *t env.RegisterWorkflow(peerflow.QRepFlowWorkflow) env.RegisterWorkflow(peerflow.XminFlowWorkflow) env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) + + alerter, err := alerting.NewAlerter(conn) + if err != nil { + t.Fatalf("unable to create alerter: %v", err) + } + env.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, - Alerter: alerting.NewAlerter(conn), + Alerter: alerter, }) env.RegisterActivity(&activities.SnapshotActivity{}) } diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go index a3a3d6c6e0..394623825d 100644 --- a/flow/shared/alerting/alerting.go +++ b/flow/shared/alerting/alerting.go @@ -18,8 +18,8 @@ type Alerter struct { logger *slog.Logger } -func registerSendersFromPool(catalogPool *pgxpool.Pool) ([]*slackAlertSender, error) { - rows, err := catalogPool.Query(context.Background(), +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) @@ -47,11 +47,17 @@ func registerSendersFromPool(catalogPool *pgxpool.Pool) ([]*slackAlertSender, er } // doesn't take care of closing pool, needs to be done externally. -func NewAlerter(catalogPool *pgxpool.Pool) *Alerter { +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: slog.Default(), - } + logger: logger, + }, nil } // Only raises an alert if another alert with the same key hasn't been raised @@ -62,40 +68,47 @@ func (a *Alerter) AlertIf(ctx context.Context, alertKey string, alertMessage str return } - if a.catalogPool != nil { - slackAlertSenders, err := registerSendersFromPool(a.catalogPool) - if err != nil { - a.logger.WarnContext(ctx, "failed to set Slack senders", slog.Any("error", err)) - return - } - if len(slackAlertSenders) == 0 { - a.logger.WarnContext(ctx, "no Slack senders configured, returning") - return - } - - row := a.catalogPool.QueryRow(context.Background(), - `SELECT created_timestamp FROM peerdb_stats.alerts_v1 WHERE alert_key=$1 + 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) + 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 } + } +} - if time.Since(createdTimestamp) >= peerdbenv.PeerDBAlertingGapMinutesAsDuration() { - for _, slackAlertSender := range slackAlertSenders { - err = slackAlertSender.sendAlert(context.Background(), - 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 - } - _, _ = a.catalogPool.Exec(context.Background(), - "INSERT INTO peerdb_stats.alerts_v1(alert_key,alert_message) VALUES($1,$2)", - alertKey, alertMessage) - } - } +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 } } From 68616141d708263b5f15194134a691672f0a3aa6 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 22 Dec 2023 09:41:02 -0500 Subject: [PATCH 25/52] log common errors to catalog for user acknowledgement (#878) --- flow/activities/flowable.go | 122 ++++++------------ flow/activities/slot.go | 89 +++++++++++++ flow/shared/alerting/alerting.go | 11 ++ .../catalog/migrations/V17__mirror_errors.sql | 10 ++ 4 files changed, 147 insertions(+), 85 deletions(-) create mode 100644 flow/activities/slot.go create mode 100644 nexus/catalog/migrations/V17__mirror_errors.sql diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 982364ba58..8fe8e4be2b 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -73,6 +73,7 @@ func (a *FlowableActivity) SetupMetadataTables(ctx context.Context, config *prot defer connectors.CloseConnector(dstConn) if err := dstConn.SetupMetadataTables(); err != nil { + a.Alerter.LogFlowError(ctx, config.Name, err) return fmt.Errorf("failed to setup metadata tables: %w", err) } @@ -111,6 +112,7 @@ func (a *FlowableActivity) EnsurePullability( output, err := srcConn.EnsurePullability(config) if err != nil { + a.Alerter.LogFlowError(ctx, config.PeerConnectionConfig.Name, err) return nil, fmt.Errorf("failed to ensure pullability: %w", err) } @@ -165,84 +167,13 @@ 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.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) - return err - } - - deploymentUIDPrefix := "" - if peerdbenv.PeerDBDeploymentUID() != "" { - deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.PeerDBDeploymentUID()) + a.Alerter.LogFlowError(ctx, config.PeerConnectionConfig.Name, err) + return nil, fmt.Errorf("failed to setup normalized tables: %w", err) } - 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) - } + return setupNormalizedTablesOutput, nil } // StartFlow implements StartFlow. @@ -256,6 +187,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 { @@ -268,10 +200,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 { @@ -287,9 +215,12 @@ 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, @@ -301,7 +232,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, RelationMessageMapping: input.RelationMessageMapping, RecordStream: recordBatch, SetLastOffset: func(lastOffset int64) error { - return dstConn.SetLastOffset(input.FlowConnectionConfigs.FlowJobName, lastOffset) + return dstConn.SetLastOffset(flowName, lastOffset) }, }) }) @@ -309,12 +240,12 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, hasRecords := !recordBatch.WaitAndCheckEmpty() slog.InfoContext(ctx, fmt.Sprintf("the current sync flow has records: %v", hasRecords)) if a.CatalogPool != nil && hasRecords { - syncBatchID, err := dstConn.GetLastSyncBatchID(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, @@ -322,6 +253,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, StartTime: startTime, }) if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, err } } @@ -330,6 +262,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") @@ -358,11 +291,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) } @@ -465,6 +400,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) } @@ -502,7 +438,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. @@ -513,7 +455,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. @@ -538,6 +486,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 { @@ -578,6 +527,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 } } @@ -717,6 +667,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 } @@ -1017,6 +968,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 0000000000..cf1375b4e4 --- /dev/null +++ b/flow/activities/slot.go @@ -0,0 +1,89 @@ +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 + } + + 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/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go index 394623825d..50608087d2 100644 --- a/flow/shared/alerting/alerting.go +++ b/flow/shared/alerting/alerting.go @@ -112,3 +112,14 @@ func (a *Alerter) AddAlertToCatalog(ctx context.Context, alertKey string, alertM 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/nexus/catalog/migrations/V17__mirror_errors.sql b/nexus/catalog/migrations/V17__mirror_errors.sql new file mode 100644 index 0000000000..06f2352ea2 --- /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); From 8dbe7c932a53760dd3925b5acd67a24312095db2 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 22 Dec 2023 21:27:19 +0530 Subject: [PATCH 26/52] Fix: Add placeholder for wal_status (#880) Co-authored-by: Kaushik Iska --- flow/connectors/utils/monitoring/monitoring.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 7815372277..7e9263cf26 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, From 0cb7abef3deaadf35b94356701a647d3da603d97 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Fri, 22 Dec 2023 23:56:32 +0530 Subject: [PATCH 27/52] Parameter for cdc with only snapshot (#884) This PR introduces a parameter - `initial_copy_only` which when set, will initiate CDC and exit after Snapshot Flow is complete. No Sync Flows will be called. By default this is set to false. Screenshot 2023-12-22 at 10 15 30 PM In the UI, this parameter is listed in the Advanced Settings section: Screenshot 2023-12-22 at 10 51 13 PM Validation for this parameter is set in both query layer and UI: ```bash amogh=> CREATE MIRROR sfnosnappy1 amogh-> FROM postgres_local TO sf_peer amogh-> WITH TABLE MAPPING ( amogh(> public.oss2:public.oss2nexusso6, amogh(> public.oss2ud1:public.oss2ud1nexusso6 amogh(> ) amogh-> WITH( amogh(> do_initial_copy = false, amogh(> snapshot_staging_path = '', amogh(> initial_copy_only = true amogh(> ); ERROR: initial_copy_only is set to true, but do_initial_copy is set to false ``` Screenshot 2023-12-22 at 10 57 46 PM This has been tested via UI and query layer and the happy path where it is not set it also tested. --- flow/generated/protos/flow.pb.go | 1153 ++++++++++++----------- flow/workflows/cdc_flow.go | 4 + nexus/analyzer/src/lib.rs | 10 + nexus/flow-rs/src/grpc.rs | 1 + nexus/pt/src/flow_model.rs | 1 + nexus/pt/src/peerdb_flow.rs | 2 + nexus/pt/src/peerdb_flow.serde.rs | 18 + protos/flow.proto | 2 + ui/app/mirrors/create/handlers.ts | 9 + ui/app/mirrors/create/helpers/cdc.ts | 11 + ui/app/mirrors/create/helpers/common.ts | 1 + ui/grpc_generated/flow.ts | 17 + 12 files changed, 658 insertions(+), 571 deletions(-) diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 27d2bb5c04..22b8ad06c4 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 @@ -3463,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, @@ -3552,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, @@ -3802,343 +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, - 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, + 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, - 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, - 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, + 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, 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, + 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, 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, 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, + 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, 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, 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, + 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, 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 ( diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index a2cb9a388e..e9091a6c3f 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -279,6 +279,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{ diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index f65fe4b2d0..8a2a088f1a 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/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index 7b11a6a2dd..7a051e3d23 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/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 5c8b1e3857..9a39485e20 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 a58fcbb95d..86ec33a2b0 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)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 0e079f82fa..a5a601c807 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(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index 430968bcba..cd09ddf8cc 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 { diff --git a/ui/app/mirrors/create/handlers.ts b/ui/app/mirrors/create/handlers.ts index 7c0b28ae07..039766171d 100644 --- a/ui/app/mirrors/create/handlers.ts +++ b/ui/app/mirrors/create/handlers.ts @@ -170,6 +170,15 @@ 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) { + setMsg({ + ok: false, + msg: 'Initial Copy Only cannot be true if Initial Copy is false.', + }); + return; + } + setLoading(true); const statusMessage: UCreateMirrorResponse = await fetch('/api/mirrors/cdc', { method: 'POST', diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index d24e9bd9f4..88d39a6dc8 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 063588b2c1..15f0607920 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/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index ba881795ed..58b916ff41 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -142,6 +142,7 @@ export interface FlowConnectionConfigs { resync: boolean; softDeleteColName: string; syncedAtColName: string; + initialCopyOnly: boolean; } export interface FlowConnectionConfigs_SrcTableIdNameMappingEntry { @@ -872,6 +873,7 @@ function createBaseFlowConnectionConfigs(): FlowConnectionConfigs { resync: false, softDeleteColName: "", syncedAtColName: "", + initialCopyOnly: false, }; } @@ -954,6 +956,9 @@ export const FlowConnectionConfigs = { if (message.syncedAtColName !== "") { writer.uint32(202).string(message.syncedAtColName); } + if (message.initialCopyOnly === true) { + writer.uint32(208).bool(message.initialCopyOnly); + } return writer; }, @@ -1145,6 +1150,13 @@ export const FlowConnectionConfigs = { message.syncedAtColName = reader.string(); continue; + case 26: + if (tag !== 208) { + break; + } + + message.initialCopyOnly = reader.bool(); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -1199,6 +1211,7 @@ export const FlowConnectionConfigs = { resync: isSet(object.resync) ? Boolean(object.resync) : false, softDeleteColName: isSet(object.softDeleteColName) ? String(object.softDeleteColName) : "", syncedAtColName: isSet(object.syncedAtColName) ? String(object.syncedAtColName) : "", + initialCopyOnly: isSet(object.initialCopyOnly) ? Boolean(object.initialCopyOnly) : false, }; }, @@ -1291,6 +1304,9 @@ export const FlowConnectionConfigs = { if (message.syncedAtColName !== "") { obj.syncedAtColName = message.syncedAtColName; } + if (message.initialCopyOnly === true) { + obj.initialCopyOnly = message.initialCopyOnly; + } return obj; }, @@ -1346,6 +1362,7 @@ export const FlowConnectionConfigs = { message.resync = object.resync ?? false; message.softDeleteColName = object.softDeleteColName ?? ""; message.syncedAtColName = object.syncedAtColName ?? ""; + message.initialCopyOnly = object.initialCopyOnly ?? false; return message; }, }; From 9c4f9cc66221017eebd9c22323cd7901602896f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Fri, 22 Dec 2023 20:23:33 +0000 Subject: [PATCH 28/52] Split normalize flow logic out of sync_flow.go (#886) Refactoring to split up #754 --- .github/workflows/flow.yml | 2 +- flow/activities/flowable.go | 1 - flow/connectors/bigquery/bigquery.go | 9 ++-- flow/e2e/congen.go | 2 +- flow/workflows/normalize_flow.go | 66 ++++++++++++++++++++++++++++ flow/workflows/sync_flow.go | 55 ----------------------- 6 files changed, 72 insertions(+), 63 deletions(-) create mode 100644 flow/workflows/normalize_flow.go diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 70f65df13f..dffa4e0840 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -95,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/flow/activities/flowable.go b/flow/activities/flowable.go index 8fe8e4be2b..1c71c26b21 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -176,7 +176,6 @@ func (a *FlowableActivity) CreateNormalizedTable( return setupNormalizedTablesOutput, nil } -// StartFlow implements StartFlow. func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlowInput, ) (*model.SyncResponse, error) { diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 2749566ced..b8831be301 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -500,16 +500,15 @@ func (r StagingBQRecord) Save() (map[string]bigquery.Value, string, error) { } // 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) diff --git a/flow/e2e/congen.go b/flow/e2e/congen.go index e881dd5ead..df1ff17c13 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' diff --git a/flow/workflows/normalize_flow.go b/flow/workflows/normalize_flow.go new file mode 100644 index 0000000000..af14e11b8f --- /dev/null +++ b/flow/workflows/normalize_flow.go @@ -0,0 +1,66 @@ +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, + }) + + // 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/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 0b207bc65f..3ee45aecf9 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 -} From c478a37c4cde6a50d4f04b69ed3ae4ca93eaada4 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 22 Dec 2023 15:53:01 -0500 Subject: [PATCH 29/52] Show alerts in UI (#883) Screenshot 2023-12-23 at 2 15 37 AM Screenshot 2023-12-23 at 2 00 26 AM --------- Co-authored-by: Amogh-Bharadwaj --- ui/app/api/mirrors/alerts/route.ts | 21 ++++++ ui/app/dto/MirrorsDTO.ts | 9 +++ ui/app/mirrors/errors/[mirrorName]/page.tsx | 75 +++++++++++++++++++ ui/app/mirrors/mirror-status.tsx | 83 +++++++++++++++++++++ ui/app/mirrors/tables.tsx | 33 +++++--- ui/prisma/schema.prisma | 44 +++++++++++ 6 files changed, 256 insertions(+), 9 deletions(-) create mode 100644 ui/app/api/mirrors/alerts/route.ts create mode 100644 ui/app/mirrors/errors/[mirrorName]/page.tsx create mode 100644 ui/app/mirrors/mirror-status.tsx diff --git a/ui/app/api/mirrors/alerts/route.ts b/ui/app/api/mirrors/alerts/route.ts new file mode 100644 index 0000000000..ecb9891cbd --- /dev/null +++ b/ui/app/api/mirrors/alerts/route.ts @@ -0,0 +1,21 @@ +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)); +} diff --git a/ui/app/dto/MirrorsDTO.ts b/ui/app/dto/MirrorsDTO.ts index 4a76200fd4..977f7f353c 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/mirrors/errors/[mirrorName]/page.tsx b/ui/app/mirrors/errors/[mirrorName]/page.tsx new file mode 100644 index 0000000000..75e8c91d2b --- /dev/null +++ b/ui/app/mirrors/errors/[mirrorName]/page.tsx @@ -0,0 +1,75 @@ +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'; + +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', + }, + distinct: ['error_message'], + }); + + 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 0000000000..2a68b7b3d2 --- /dev/null +++ b/ui/app/mirrors/mirror-status.tsx @@ -0,0 +1,83 @@ +'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/tables.tsx b/ui/app/mirrors/tables.tsx index 743bbc28a4..6c1289befc 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 }) { + + + Date: Fri, 22 Dec 2023 23:39:00 +0000 Subject: [PATCH 30/52] Update go dependencies, fixes GHSA-mhpq-9638-x6pw (#882) https://github.com/dvsekhvalnov/jose2go/pull/32 --- flow/go.mod | 16 ++++++++-------- flow/go.sum | 40 ++++++++++++++++++++-------------------- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/flow/go.mod b/flow/go.mod index dd59212b71..5cd582a51c 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -9,7 +9,7 @@ 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.5 + github.com/aws/aws-sdk-go v1.49.8 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cockroachdb/pebble v0.0.0-20231210175914-b4d301aeb46a github.com/google/uuid v1.5.0 @@ -39,7 +39,7 @@ require ( google.golang.org/api v0.154.0 google.golang.org/genproto/googleapis/api v0.0.0-20231212172506-995d672761c0 google.golang.org/grpc v1.60.1 - google.golang.org/protobuf v1.31.0 + google.golang.org/protobuf v1.32.0 ) require ( @@ -52,7 +52,7 @@ 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 @@ -90,8 +90,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.8 // 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 @@ -99,12 +99,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.6 // 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 @@ -150,7 +150,7 @@ require ( github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect golang.org/x/crypto v0.17.0 - golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611 + 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 e9a2d2b5eb..714e5dfd3f 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.5 h1:y2yfBlwjPDi3/sBVKeznYEdDy6wIhjA2L5NCBMLUIYA= -github.com/aws/aws-sdk-go v1.49.5/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.49.8 h1:gKgEiyJ8CPnr4r6pS06WfNXvp6z34JER1pBIwuocvVA= +github.com/aws/aws-sdk-go v1.49.8/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.8 h1:7wCngExMTAW2Bjf0Y92uWap6ZUcenLLWI5T3VJiQneU= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.8/go.mod h1:XVrAWYYM4ZRwOCOuLoUiao5hbLqNutEdqwCR3ZvkXgc= +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.6 h1:bkmlzokzTJyrFNA0J+EPlsF8x4/wp+9D45HTHO/ZUiY= -github.com/aws/aws-sdk-go-v2/service/s3 v1.47.6/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,8 +164,8 @@ 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= @@ -434,8 +434,8 @@ golang.org/x/crypto v0.0.0-20220829220503-c86fa9a7ed90/go.mod h1:IxCIyHEi3zRg3s0 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= @@ -569,8 +569,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= From be6b5c238560d3d36e3b5647b0cebb97b134a44e Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Fri, 22 Dec 2023 19:48:13 -0500 Subject: [PATCH 31/52] Replace CAST with TRY_CAST (#885) Co-authored-by: Sai Srirampur --- flow/activities/flowable.go | 5 +++++ flow/connectors/snowflake/snowflake.go | 13 ++++++++++--- flow/peerdbenv/config.go | 5 +++++ flow/peerdbenv/env.go | 17 +++++++++++++++++ 4 files changed, 37 insertions(+), 3 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 1c71c26b21..9591cab251 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -741,6 +741,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() diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index e61830db88..e47c27a5f4 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -824,7 +824,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) @@ -849,8 +850,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, ""), ",") diff --git a/flow/peerdbenv/config.go b/flow/peerdbenv/config.go index 33b1058066..cdefa6a376 100644 --- a/flow/peerdbenv/config.go +++ b/flow/peerdbenv/config.go @@ -80,3 +80,8 @@ func PeerDBAlertingGapMinutesAsDuration() time.Duration { 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 11e363d1eb..3bba77c46d 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 { From e20a2e575d54509928ca157b6c83cec52ca9db96 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Sat, 23 Dec 2023 18:16:41 -0500 Subject: [PATCH 32/52] Make it a non-fatal error if we are unable to get slot information (#891) --- flow/activities/slot.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flow/activities/slot.go b/flow/activities/slot.go index cf1375b4e4..117dbecea3 100644 --- a/flow/activities/slot.go +++ b/flow/activities/slot.go @@ -23,6 +23,11 @@ func (a *FlowableActivity) handleSlotInfo( return err } + if slotInfo == nil || 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()) From 7e1813a68bc3749dcc83e82b7a9d16062f8d0833 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Sat, 23 Dec 2023 23:47:23 +0000 Subject: [PATCH 33/52] be compatible with podman-compose (#892) podman complains if multiple arguments are passed to `CMD-SHELL`. Turns out we don't need it. podman doesn't support `--no-attach`. Otherwise everything works. Adjust `dev-peerdb.sh` to support podman-compose fallback or being invoked with `USE_PODMAN=1 ./dev-peerdb.sh` --- dev-peerdb.sh | 29 ++++++++++++++++++++++------- docker-compose-dev.yml | 2 +- docker-compose.yml | 2 +- 3 files changed, 24 insertions(+), 9 deletions(-) diff --git a/dev-peerdb.sh b/dev-peerdb.sh index 2305884213..796425f3dd 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 158483a7eb..cc6c50b77d 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 diff --git a/docker-compose.yml b/docker-compose.yml index add4bb2db1..b5d37dbdc1 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 From b16075360554c2ad2772fff084d5d653362e56d9 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Sun, 24 Dec 2023 11:47:22 +0530 Subject: [PATCH 34/52] making BQ merge a single statement without temp tables (#889) --- flow/connectors/bigquery/bigquery.go | 14 +++---- .../bigquery/merge_statement_generator.go | 41 +++++-------------- flow/connectors/bigquery/qrep_avro_sync.go | 3 +- 3 files changed, 17 insertions(+), 41 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index b8831be301..64d12057f6 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -233,11 +233,11 @@ func (c *BigQueryConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } -func (c *BigQueryConnector) WaitForTableReady(tblName string) error { +func (c *BigQueryConnector) waitForTableReady(tblName string) error { table := c.client.Dataset(c.datasetID).Table(tblName) maxDuration := 5 * time.Minute deadline := time.Now().Add(maxDuration) - sleepInterval := 15 * time.Second + sleepInterval := 5 * time.Second attempt := 0 for { @@ -816,20 +816,16 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) }, } // 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';", + "UPDATE %s.%s SET normalize_batch_id=%d WHERE mirror_job_name='%s';", c.datasetID, MirrorJobsTable, 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) if err != nil { return nil, fmt.Errorf("failed to execute statements %s in a transaction: %v", strings.Join(stmts, "\n"), err) diff --git a/flow/connectors/bigquery/merge_statement_generator.go b/flow/connectors/bigquery/merge_statement_generator.go index 22161c434b..22f876b8c3 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -8,7 +8,6 @@ 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 { @@ -30,24 +29,6 @@ type mergeStmtGenerator struct { peerdbCols *protos.PeerDBColumns } -// 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, m.peerdbCols) - - dropTempTableStmt := fmt.Sprintf("DROP TABLE %s;", tempTable) - - return []string{createTempTableStmt, mergeStmt, dropTempTableStmt} -} - // generateFlattenedCTE generates a flattened CTE. func (m *mergeStmtGenerator) generateFlattenedCTE() string { // for each column in the normalized table, generate CAST + JSON_EXTRACT_SCALAR @@ -129,7 +110,7 @@ func (m *mergeStmtGenerator) generateDeDupedCTE() string { } // generateMergeStmt generates a merge statement. -func (m *mergeStmtGenerator) generateMergeStmt(tempTable string, peerdbCols *protos.PeerDBColumns) 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)) @@ -138,13 +119,13 @@ func (m *mergeStmtGenerator) generateMergeStmt(tempTable string, peerdbCols *pro pureColNames = append(pureColNames, colName) } csep := strings.Join(backtickColNames, ", ") - insertColumnsSQL := csep + fmt.Sprintf(", `%s`", peerdbCols.SyncedAtColName) + insertColumnsSQL := csep + fmt.Sprintf(", `%s`", m.peerdbCols.SyncedAtColName) insertValuesSQL := csep + ",CURRENT_TIMESTAMP" updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, - m.UnchangedToastColumns, peerdbCols) + m.UnchangedToastColumns, m.peerdbCols) if m.peerdbCols.SoftDelete { - softDeleteInsertColumnsSQL := insertColumnsSQL + fmt.Sprintf(", `%s`", peerdbCols.SoftDeleteColName) + softDeleteInsertColumnsSQL := insertColumnsSQL + fmt.Sprintf(", `%s`", m.peerdbCols.SoftDeleteColName) softDeleteInsertValuesSQL := insertValuesSQL + ", TRUE" updateStatementsforToastCols = append(updateStatementsforToastCols, @@ -162,25 +143,25 @@ func (m *mergeStmtGenerator) generateMergeStmt(tempTable string, peerdbCols *pro pkeySelectSQL := strings.Join(pkeySelectSQLArray, " AND ") deletePart := "DELETE" - if peerdbCols.SoftDelete { - colName := peerdbCols.SoftDeleteColName + if m.peerdbCols.SoftDelete { + colName := m.peerdbCols.SoftDeleteColName deletePart = fmt.Sprintf("UPDATE SET %s = TRUE", colName) - if peerdbCols.SyncedAtColName != "" { + if m.peerdbCols.SyncedAtColName != "" { deletePart = fmt.Sprintf("%s, %s = CURRENT_TIMESTAMP", - deletePart, peerdbCols.SyncedAtColName) + deletePart, m.peerdbCols.SyncedAtColName) } } return fmt.Sprintf(` - MERGE %s.%s _peerdb_target USING %s _peerdb_deduped + MERGE %s.%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 %s; - `, m.Dataset, m.NormalizedTable, tempTable, pkeySelectSQL, insertColumnsSQL, insertValuesSQL, - updateStringToastCols, deletePart) + `, m.Dataset, m.NormalizedTable, m.generateFlattenedCTE(), m.generateDeDupedCTE(), + pkeySelectSQL, insertColumnsSQL, insertValuesSQL, updateStringToastCols, deletePart) } /* diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 6a83d23ae8..d52e7c42e3 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -343,7 +343,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) @@ -415,7 +414,7 @@ func (s *QRepAvroSyncMethod) writeToStage( } slog.Info(fmt.Sprintf("Pushed into %s/%s", avroFile.FilePath, syncID)) - 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) } From d407f9e8ba3b5ea2a8b6e2319dcc2cef49b60385 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 25 Dec 2023 13:09:21 +0000 Subject: [PATCH 35/52] BQ/SF: always use GREATEST when setting offset (#896) While adding SetLastOffset, missed updating previous metadata update sql to use GREATEST too --- flow/connectors/bigquery/bigquery.go | 2 +- flow/connectors/snowflake/snowflake.go | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 64d12057f6..3c0787527e 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -927,7 +927,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) } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index e47c27a5f4..db13e188b8 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -39,11 +39,10 @@ const ( 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 + 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 @@ -66,7 +65,8 @@ 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 From ac5bbb2b851ffa033c9ddf3815f706fa2cc7b254 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Mon, 25 Dec 2023 18:39:56 +0530 Subject: [PATCH 36/52] UI: Create Mirror And Mirror Overview Improvements (#890) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## UI Improvement Features These are some features which I thought would be nice to get in. At the end of the day, these are just proposals from my end. ### Mirror Create Pages: Error Toasts And Floating Button We now show errors as toasts and have the Create Mirror button with a fixed position on the bottom-right. Users now don't have to do a lot of scrolling up and down to look at the error message, come back, and click create. Screenshot 2023-12-23 at 10 26 14 PM ### QRep Mirror: Upsert Columns Selection of Unique Key Columns for QRep Upsert mode now looks like this, saving users from having to type out columns. Also added validation for the columns being an empty array. Screenshot 2023-12-23 at 9 49 35 PM ### Better Tabs UI for Mirror Overview I thought the tabs we have there look unpolished so used Tremor to come up with this. This also achieves significant code reduction in that file. Screenshot 2023-12-23 at 11 37 58 PM ### Wiring Status in Mirror Overview Page Wires in the Status we show in the mirror overview page. This is a follow-up to #883 Screenshot 2023-12-23 at 10 28 23 PM ### Others - Removes 'Authentication failed' message in login landing page. - Makes the source-destination table list in Mirror Overview page have scrollable height and sticky headers - Error table now has time column before message column and the rows are sorted by timestamp (latest first) --------- Co-authored-by: Kaushik Iska --- ui/app/login/page.tsx | 4 +- ui/app/mirrors/create/handlers.ts | 75 +++++-------- ui/app/mirrors/create/helpers/qrep.ts | 3 +- ui/app/mirrors/create/page.tsx | 93 ++++++++-------- ui/app/mirrors/create/qrep/qrep.tsx | 13 ++- ui/app/mirrors/create/qrep/upsertcols.tsx | 95 +++++++++++++++++ ui/app/mirrors/edit/[mirrorId]/cdc.tsx | 87 ++++----------- ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx | 13 +-- ui/app/mirrors/edit/[mirrorId]/tablePairs.tsx | 100 ++++++++++-------- ui/app/mirrors/errors/[mirrorName]/page.tsx | 19 ++-- ui/app/mirrors/mirror-status.tsx | 15 ++- ui/app/mirrors/tables.tsx | 2 +- ui/package-lock.json | 21 ++++ ui/package.json | 1 + 14 files changed, 308 insertions(+), 233 deletions(-) create mode 100644 ui/app/mirrors/create/qrep/upsertcols.tsx diff --git a/ui/app/login/page.tsx b/ui/app/login/page.tsx index 41c00c0dd8..1172b0e811 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/handlers.ts b/ui/app/mirrors/create/handlers.ts index 039766171d..81f36bd739 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; @@ -172,10 +158,7 @@ export const handleCreateCDC = async ( } if (config.doInitialCopy == false && config.initialCopyOnly == true) { - setMsg({ - ok: false, - msg: 'Initial Copy Only cannot be true if Initial Copy is false.', - }); + notify('Initial Copy Only cannot be true if Initial Copy is false.'); return; } @@ -187,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); }; @@ -209,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 @@ -222,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; } @@ -237,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; @@ -267,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/qrep.ts b/ui/app/mirrors/create/helpers/qrep.ts index 654d5c7ff3..fca1f9fcb2 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 a075432bb3..497d3aea4b 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 6b5b7b4b35..9dd7943b80 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 7f54d227ab..8c33fa7c9b 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 32992f871d..e7729c487d 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,12 +32,10 @@ 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 5289e77a04..516b8dd3ce 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]/page.tsx b/ui/app/mirrors/errors/[mirrorName]/page.tsx index 75e8c91d2b..97efca644a 100644 --- a/ui/app/mirrors/errors/[mirrorName]/page.tsx +++ b/ui/app/mirrors/errors/[mirrorName]/page.tsx @@ -15,6 +15,9 @@ const MirrorError = async ({ params: { mirrorName } }: MirrorErrorProps) => { error_type: 'error', }, distinct: ['error_message'], + orderBy: { + error_timestamp: 'desc', + }, }); return ( @@ -40,12 +43,10 @@ const MirrorError = async ({ params: { mirrorName } }: MirrorErrorProps) => { header={ Type + Message - - - } > @@ -54,15 +55,15 @@ const MirrorError = async ({ params: { mirrorName } }: MirrorErrorProps) => { {mirrorError.error_type.toUpperCase()} - - {mirrorError.error_message} - - + + + {mirrorError.error_message} + ))} diff --git a/ui/app/mirrors/mirror-status.tsx b/ui/app/mirrors/mirror-status.tsx index 2a68b7b3d2..27d797e389 100644 --- a/ui/app/mirrors/mirror-status.tsx +++ b/ui/app/mirrors/mirror-status.tsx @@ -26,7 +26,13 @@ export const ErrorModal = ({ flowName }: { flowName: string }) => { ); }; -export const MirrorError = ({ flowName }: { flowName: string }) => { +export const MirrorError = ({ + flowName, + detailed, +}: { + flowName: string; + detailed: boolean; +}) => { const [flowStatus, setFlowStatus] = useState(); const [isLoading, setIsLoading] = useState(true); const [error, setError] = useState(null); @@ -76,6 +82,13 @@ export const MirrorError = ({ flowName }: { flowName: string }) => { } if (flowStatus == 'healthy') { + if (detailed) + return ( +
+ + +
+ ); return ; } diff --git a/ui/app/mirrors/tables.tsx b/ui/app/mirrors/tables.tsx index 6c1289befc..106c7cd22d 100644 --- a/ui/app/mirrors/tables.tsx +++ b/ui/app/mirrors/tables.tsx @@ -90,7 +90,7 @@ export function CDCFlows({ cdcFlows }: { cdcFlows: any }) {
- + =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", diff --git a/ui/package.json b/ui/package.json index 35fdd5f17d..1ff4f487ca 100644 --- a/ui/package.json +++ b/ui/package.json @@ -49,6 +49,7 @@ "react-dom": "18.2.0", "react-select": "^5.8.0", "react-spinners": "^0.13.8", + "react-toastify": "^9.1.3", "styled-components": "^6.1.1", "swr": "^2.2.4", "zod": "^3.22.4", From 0806d33a47cd7375f98e3d17ce3487e92256d308 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 25 Dec 2023 14:19:07 +0000 Subject: [PATCH 37/52] reqwest: use rustls (#899) Consolidates dependency tree flow-rs wasn't using its reqwest dependency, nor was it using serde, serde_yaml, or tokio --- nexus/Cargo.lock | 77 --------------------------------- nexus/flow-rs/Cargo.toml | 10 ----- nexus/peer-snowflake/Cargo.toml | 2 +- 3 files changed, 1 insertion(+), 88 deletions(-) diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 652a9ebd11..59914b0952 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -897,11 +897,7 @@ dependencies = [ "anyhow", "catalog", "pt", - "reqwest", - "serde", "serde_json", - "serde_yaml", - "tokio", "tonic", "tonic-health", "tracing", @@ -1286,19 +1282,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" @@ -1574,24 +1557,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" @@ -1737,15 +1702,6 @@ 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" @@ -1754,7 +1710,6 @@ checksum = "c3eaad34cdd97d81de97964fc7f29e2d104f483840d906ef56daa1912338460b" dependencies = [ "cc", "libc", - "openssl-src", "pkg-config", "vcpkg", ] @@ -2603,12 +2558,10 @@ dependencies = [ "http-body", "hyper", "hyper-rustls", - "hyper-tls", "ipnet", "js-sys", "log", "mime", - "native-tls", "once_cell", "percent-encoding", "pin-project-lite", @@ -2619,7 +2572,6 @@ dependencies = [ "serde_urlencoded", "system-configuration", "tokio", - "tokio-native-tls", "tokio-rustls 0.24.1", "tokio-util", "tower-service", @@ -2970,19 +2922,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" @@ -3400,16 +3339,6 @@ dependencies = [ "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", -] - [[package]] name = "tokio-openssl" version = "0.6.4" @@ -3756,12 +3685,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" diff --git a/nexus/flow-rs/Cargo.toml b/nexus/flow-rs/Cargo.toml index fc989a378f..2fd4bb1cf9 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/peer-snowflake/Cargo.toml b/nexus/peer-snowflake/Cargo.toml index 6e16ff63cf..45b3fe413d 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -25,7 +25,7 @@ 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" From 79e9ce381f0a3adab40c8789e151c35e436a6e2c Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Mon, 25 Dec 2023 10:12:09 -0500 Subject: [PATCH 38/52] Never update consumed xlogpos even when num records is zero (#900) This could potentially cause the WAL to build up but based on reports in the field this seems like the safer alternative for now. --- flow/connectors/postgres/cdc.go | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index f2eda2e5f4..b3686f4d09 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -197,7 +197,6 @@ func (p *PostgresCDCSource) consumeStream( return fmt.Errorf("[initial-flush] SendStandbyStatusUpdate failed: %w", err) } } - proposedConsumedXLogPos := consumedXLogPos var standByLastLogged time.Time cdcRecordsStorage := cdc_records.NewCDCRecordsStore(p.flowJobName) @@ -254,17 +253,6 @@ 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. - if proposedConsumedXLogPos > consumedXLogPos { - p.logger.Info(fmt.Sprintf("Heartbeat adjusting lsn from %d to %d", consumedXLogPos, proposedConsumedXLogPos)) - consumedXLogPos = proposedConsumedXLogPos - err := p.SetLastOffset(int64(consumedXLogPos)) - if err != nil { - return fmt.Errorf("storing updated LSN failed: %w", err) - } - } - err := pglogrepl.SendStandbyStatusUpdate(p.ctx, conn, pglogrepl.StandbyStatusUpdate{WALWritePosition: consumedXLogPos}) if err != nil { @@ -477,13 +465,6 @@ 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. - proposedConsumedXLogPos = clientXLogPos - records.UpdateLatestCheckpoint(int64(clientXLogPos)) - } } } } From ed67a63ec9c0bfee98dc8501ac6194acce781837 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 25 Dec 2023 15:28:19 +0000 Subject: [PATCH 39/52] BQ merge: don't be transactional (#895) Discussed with Kevin BQ lock contention, he made #889 to remove temp table Merge is idempotent, so no need to have transaction, which removes need to have advisory lock on catalog --- flow/connectors/bigquery/bigquery.go | 63 ++-------------------- flow/connectors/bigquery/qrep_avro_sync.go | 26 ++++----- 2 files changed, 18 insertions(+), 71 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 3c0787527e..0a220ef424 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -781,25 +781,11 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) 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 { mergeGen := &mergeStmtGenerator{ Dataset: c.datasetID, @@ -824,11 +810,11 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) "UPDATE %s.%s SET normalize_batch_id=%d WHERE mirror_job_name='%s';", c.datasetID, MirrorJobsTable, syncBatchID, req.FlowJobName) stmts = append(stmts, updateMetadataStmt) - stmts = append(stmts, "COMMIT TRANSACTION;") - _, 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{ @@ -1023,21 +1009,9 @@ 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) } @@ -1069,33 +1043,6 @@ func (c *BigQueryConnector) getStagingTableName(flowJobName string) string { 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 diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index d52e7c42e3..7ed87b0c06 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -74,12 +74,12 @@ func (s *QRepAvroSyncMethod) SyncRecords( flowJobName, dstTableName, 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) @@ -136,8 +136,6 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( ) bqClient := s.connector.client datasetID := s.connector.datasetID - // Start a transaction - stmts := []string{"BEGIN TRANSACTION;"} selector := "*" if softDeleteCol != "" { // PeerDB column @@ -150,16 +148,18 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT %s FROM `%s.%s`;", datasetID, dstTableName, selector, datasetID, stagingTable) - stmts = append(stmts, insertStmt) - 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) From f1038ba4613edc66850e8b14e5610dc9c00595e2 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Tue, 26 Dec 2023 01:09:10 +0530 Subject: [PATCH 40/52] Errors UI: Acknowledge functionality (#901) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Users can now mark errors as 'Acknowledged'. If there are no unacknowledged errors for a mirror, it's status is said to be 'Active' Screenshot 2023-12-25 at 10 51 03 PM Users can click on the status and be taken to the page above. Screenshot 2023-12-25 at 10 22 34 PM - Loading indicator for acknowledge button - Error toast incase acknowledge operation fails --- ui/app/api/mirrors/alerts/route.ts | 19 +++ ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx | 7 +- .../mirrors/errors/[mirrorName]/ackbutton.tsx | 57 +++++++++ ui/app/mirrors/errors/[mirrorName]/page.tsx | 119 +++++++++++------- ui/app/mirrors/mirror-status.tsx | 41 +++--- ui/app/mirrors/tables.tsx | 2 +- 6 files changed, 174 insertions(+), 71 deletions(-) create mode 100644 ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx diff --git a/ui/app/api/mirrors/alerts/route.ts b/ui/app/api/mirrors/alerts/route.ts index ecb9891cbd..13cc612503 100644 --- a/ui/app/api/mirrors/alerts/route.ts +++ b/ui/app/api/mirrors/alerts/route.ts @@ -19,3 +19,22 @@ export async function POST(request: Request) { } 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/mirrors/edit/[mirrorId]/cdcDetails.tsx b/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx index e7729c487d..738218c64f 100644 --- a/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx +++ b/ui/app/mirrors/edit/[mirrorId]/cdcDetails.tsx @@ -32,10 +32,9 @@ function CdcDetails({ syncs, createdAt, mirrorConfig }: props) {
- +
diff --git a/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx b/ui/app/mirrors/errors/[mirrorName]/ackbutton.tsx new file mode 100644 index 0000000000..b6d14eed89 --- /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 index 97efca644a..899b25a496 100644 --- a/ui/app/mirrors/errors/[mirrorName]/page.tsx +++ b/ui/app/mirrors/errors/[mirrorName]/page.tsx @@ -3,6 +3,9 @@ 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 }; @@ -14,62 +17,84 @@ const MirrorError = async ({ params: { mirrorName } }: MirrorErrorProps) => { flow_name: mirrorName, error_type: 'error', }, - distinct: ['error_message'], orderBy: { error_timestamp: 'desc', }, }); return ( -
- -
-
- - -
- - Type - - Message - - } + <> +
+ +
+
+ + + +
+ +
+ +
- {mirrorErrors.map((mirrorError) => ( - - - {mirrorError.error_type.toUpperCase()} - - - - - - {mirrorError.error_message} - - - ))} -
+ + Type + + + + Message + + + } + > + {mirrorErrors.map((mirrorError) => ( + + + {mirrorError.error_type.toUpperCase()} + + + + + + {mirrorError.error_message} + + + + + + ))} +
+
-
+ + ); }; diff --git a/ui/app/mirrors/mirror-status.tsx b/ui/app/mirrors/mirror-status.tsx index 27d797e389..b64d87ff23 100644 --- a/ui/app/mirrors/mirror-status.tsx +++ b/ui/app/mirrors/mirror-status.tsx @@ -13,26 +13,21 @@ export const ErrorModal = ({ flowName }: { flowName: string }) => { ); }; -export const MirrorError = ({ - flowName, - detailed, -}: { - flowName: string; - detailed: boolean; -}) => { +export const MirrorError = ({ flowName }: { flowName: string }) => { const [flowStatus, setFlowStatus] = useState(); const [isLoading, setIsLoading] = useState(true); const [error, setError] = useState(null); @@ -81,15 +76,23 @@ export const MirrorError = ({ ); } - if (flowStatus == 'healthy') { - if (detailed) - return ( -
+ if (flowStatus === 'healthy') { + return ( + +
- ); - return ; + + + + ); } return ; diff --git a/ui/app/mirrors/tables.tsx b/ui/app/mirrors/tables.tsx index 106c7cd22d..6c1289befc 100644 --- a/ui/app/mirrors/tables.tsx +++ b/ui/app/mirrors/tables.tsx @@ -90,7 +90,7 @@ export function CDCFlows({ cdcFlows }: { cdcFlows: any }) {
- + Date: Tue, 26 Dec 2023 01:37:18 +0530 Subject: [PATCH 41/52] Fix flow err log and cleanup flowable.go (#887) A few places we were logging peer names instead of flow name for the `peerdb_stats.flow_errors` table. Also some long lines have been split in this PR Co-authored-by: Kaushik Iska --- flow/activities/flowable.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 9591cab251..66a4a2033f 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -72,8 +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, config.Name, err) + a.Alerter.LogFlowError(ctx, flowName, err) return fmt.Errorf("failed to setup metadata tables: %w", err) } @@ -112,7 +113,7 @@ func (a *FlowableActivity) EnsurePullability( output, err := srcConn.EnsurePullability(config) if err != nil { - a.Alerter.LogFlowError(ctx, config.PeerConnectionConfig.Name, err) + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return nil, fmt.Errorf("failed to ensure pullability: %w", err) } @@ -169,7 +170,8 @@ func (a *FlowableActivity) CreateNormalizedTable( setupNormalizedTablesOutput, err := conn.SetupNormalizedTables(config) if err != nil { - a.Alerter.LogFlowError(ctx, config.PeerConnectionConfig.Name, err) + flowName, _ := ctx.Value(shared.FlowNameKey).(string) + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to setup normalized tables: %w", err) } @@ -580,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 @@ -935,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 } @@ -945,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 From 93d754a405cd672baa2ca3ea3c9388cb575fbb15 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Mon, 25 Dec 2023 20:45:46 +0000 Subject: [PATCH 42/52] Fix golangci-lint failures (#902) --- flow/.golangci.yml | 8 ++++++-- flow/activities/slot.go | 2 +- flow/cmd/api.go | 5 ++++- flow/cmd/handler.go | 4 +++- flow/cmd/peer_data.go | 16 ++++++++-------- flow/cmd/snapshot_worker.go | 5 ++++- flow/cmd/worker.go | 5 ++++- flow/connectors/postgres/cdc.go | 4 ++-- flow/connectors/postgres/client.go | 6 +++--- .../snowflake/avro_file_writer_test.go | 1 + flow/connectors/utils/ssh.go | 5 +++-- flow/e2e/snowflake/qrep_flow_sf_test.go | 1 + 12 files changed, 40 insertions(+), 22 deletions(-) diff --git a/flow/.golangci.yml b/flow/.golangci.yml index 2c8032f91d..ab612fb893 100644 --- a/flow/.golangci.yml +++ b/flow/.golangci.yml @@ -4,7 +4,6 @@ run: linters: enable: - dogsled - - dupl - gofumpt - gosec - gosimple @@ -18,9 +17,14 @@ linters: - prealloc - staticcheck - ineffassign + - unparam - unused - lll linters-settings: + stylecheck: + checks: + - all + - '-ST1003' lll: - line-length: 120 + line-length: 144 tab-width: 4 diff --git a/flow/activities/slot.go b/flow/activities/slot.go index 117dbecea3..baa0fbc0fa 100644 --- a/flow/activities/slot.go +++ b/flow/activities/slot.go @@ -23,7 +23,7 @@ func (a *FlowableActivity) handleSlotInfo( return err } - if slotInfo == nil || len(slotInfo) == 0 { + if len(slotInfo) == 0 { slog.WarnContext(ctx, "warning: unable to get slot info", slog.Any("slotName", slotName)) return nil } diff --git a/flow/cmd/api.go b/flow/cmd/api.go index 09185a0fc1..b16034bf20 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 7b03c9de67..dd922ba1f5 100644 --- a/flow/cmd/handler.go +++ b/flow/cmd/handler.go @@ -261,7 +261,9 @@ 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 diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 34f31219ed..f9383d8c5e 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 } diff --git a/flow/cmd/snapshot_worker.go b/flow/cmd/snapshot_worker.go index 16008cc6a5..c68d44d925 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/worker.go b/flow/cmd/worker.go index eea0e9184f..f060230b63 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -100,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 } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index b3686f4d09..2be3fcb2a5 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -360,7 +360,6 @@ 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, clientXLogPos) - if err != nil { return fmt.Errorf("error processing message: %w", err) } @@ -470,7 +469,8 @@ func (p *PostgresCDCSource) consumeStream( } func (p *PostgresCDCSource) processMessage(batch *model.CDCRecordStream, xld pglogrepl.XLogData, - currentClientXlogPos pglogrepl.LSN) (model.Record, error) { + 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) diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index dc604d5631..e48c71b29d 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -84,9 +84,9 @@ type ReplicaIdentityType rune const ( ReplicaIdentityDefault ReplicaIdentityType = 'd' - ReplicaIdentityFull = 'f' - ReplicaIdentityIndex = 'i' - ReplicaIdentityNothing = 'n' + ReplicaIdentityFull ReplicaIdentityType = 'f' + ReplicaIdentityIndex ReplicaIdentityType = 'i' + ReplicaIdentityNothing ReplicaIdentityType = 'n' ) // getRelIDForTable returns the relation ID for a table. diff --git a/flow/connectors/snowflake/avro_file_writer_test.go b/flow/connectors/snowflake/avro_file_writer_test.go index 46f18aaa3f..f08b66a6c8 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/utils/ssh.go b/flow/connectors/utils/ssh.go index 7bd8ed141f..511eea672a 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/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index 3ac7fee713..b3cd9b9c2a 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -12,6 +12,7 @@ import ( "github.com/stretchr/testify/require" ) +// nolint:unparam func (s PeerFlowE2ETestSuiteSF) setupSourceTable(tableName string, numRows int) { err := e2e.CreateTableForQRep(s.pool, s.pgSuffix, tableName) require.NoError(s.t, err) From eb63a7685f85f4162430b9b33bbeffd3ed0de5f5 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Tue, 26 Dec 2023 18:44:21 +0530 Subject: [PATCH 43/52] added capability for BQ CDC across datasets (#904) 1) Just like Snowflake and Postgres, now BigQuery takes tables in the form of `.`. If dataset is omitted then it defaults to using the dataset specified at the time of peer creation. 2) If the dataset doesn't exist at the time of mirror creation, it is created during `SetupNormalizedTables` before the tables in the dataset. 3) A check has also been added so that two source tables cannot point to the same destination table specified in 2 different formats. --- flow/connectors/bigquery/bigquery.go | 175 ++++++++++-------- .../bigquery/merge_statement_generator.go | 46 ++--- flow/connectors/bigquery/qrep.go | 10 +- flow/connectors/bigquery/qrep_avro_sync.go | 53 +++--- flow/connectors/eventhub/eventhub.go | 2 +- flow/connectors/postgres/cdc.go | 4 +- flow/e2e/bigquery/bigquery_helper.go | 23 ++- flow/e2e/bigquery/peer_flow_bq_test.go | 70 ++++++- flow/model/model.go | 12 +- 9 files changed, 248 insertions(+), 147 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 0a220ef424..3da34f99d7 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -29,9 +29,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. */ @@ -233,8 +231,8 @@ 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 := 5 * time.Second @@ -242,7 +240,7 @@ func (c *BigQueryConnector) waitForTableReady(tblName string) error { 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 +248,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 +266,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, @@ -593,16 +593,11 @@ func (c *BigQueryConnector) syncRecordsViaAvro( 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, @@ -626,16 +621,11 @@ func (c *BigQueryConnector) syncRecordsViaAvro( 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, @@ -660,10 +650,6 @@ func (c *BigQueryConnector) syncRecordsViaAvro( 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, @@ -698,6 +684,10 @@ func (c *BigQueryConnector) syncRecordsViaAvro( Kind: qvalue.QValueKindInt64, Value: time.Now().UnixNano(), } + entries[3] = qvalue.QValue{ + Kind: qvalue.QValueKindString, + Value: record.GetDestinationTableName(), + } entries[7] = qvalue.QValue{ Kind: qvalue.QValueKindInt64, Value: syncBatchID, @@ -787,14 +777,18 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) c.datasetID, rawTableName, distinctTableNames)) 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: syncBatchID, + normalizeBatchID: normalizeBatchID, + unchangedToastColumns: tableNametoUnchangedToastCols[tableName], peerdbCols: &protos.PeerDBColumns{ SoftDeleteColName: req.SoftDeleteColName, SyncedAtColName: req.SyncedAtColName, @@ -846,19 +840,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) @@ -883,16 +864,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 @@ -952,14 +923,41 @@ 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 } @@ -999,6 +997,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)) } @@ -1015,10 +1014,6 @@ func (c *BigQueryConnector) SyncFlowCleanup(jobName string) error { 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) @@ -1036,35 +1031,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) -} - 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{ @@ -1076,13 +1069,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) } @@ -1094,3 +1089,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 22f876b8c3..e9a71b06cd 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -11,20 +11,20 @@ import ( ) 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 + unchangedToastColumns []string // _PEERDB_IS_DELETED and _SYNCED_AT columns peerdbCols *protos.PeerDBColumns } @@ -34,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 { @@ -87,10 +87,10 @@ func (m *mergeStmtGenerator) generateFlattenedCTE() string { // 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. @@ -104,7 +104,7 @@ func (m *mergeStmtGenerator) generateDeDupedCTE() string { ) _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) } @@ -112,9 +112,9 @@ func (m *mergeStmtGenerator) generateDeDupedCTE() string { // generateMergeStmt generates a merge statement. 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) } @@ -123,7 +123,7 @@ func (m *mergeStmtGenerator) generateMergeStmt() string { insertValuesSQL := csep + ",CURRENT_TIMESTAMP" updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, - m.UnchangedToastColumns, m.peerdbCols) + m.unchangedToastColumns, m.peerdbCols) if m.peerdbCols.SoftDelete { softDeleteInsertColumnsSQL := insertColumnsSQL + fmt.Sprintf(", `%s`", m.peerdbCols.SoftDeleteColName) softDeleteInsertValuesSQL := insertValuesSQL + ", TRUE" @@ -134,8 +134,8 @@ func (m *mergeStmtGenerator) generateMergeStmt() string { } 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)) } @@ -153,14 +153,14 @@ func (m *mergeStmtGenerator) generateMergeStmt() string { } return fmt.Sprintf(` - MERGE %s.%s _peerdb_target USING (%s,%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 %s; - `, m.Dataset, m.NormalizedTable, m.generateFlattenedCTE(), m.generateDeDupedCTE(), + `, m.dstDatasetTable.string(), m.generateFlattenedCTE(), m.generateDeDupedCTE(), pkeySelectSQL, insertColumnsSQL, insertValuesSQL, updateStringToastCols, deletePart) } diff --git a/flow/connectors/bigquery/qrep.go b/flow/connectors/bigquery/qrep.go index df771e50a2..305bab01eb 100644 --- a/flow/connectors/bigquery/qrep.go +++ b/flow/connectors/bigquery/qrep.go @@ -45,7 +45,7 @@ func (c *BigQueryConnector) SyncQRepRecords( " partition %s of destination table %s", partition.PartitionId, destTable)) - avroSync := &QRepAvroSyncMethod{connector: c, gcsBucket: config.StagingPath} + avroSync := NewQRepAvroSyncMethod(c, config.StagingPath, config.FlowJobName) return avroSync.SyncQRepRecords(config.FlowJobName, destTable, partition, tblMetadata, stream, config.SyncedAtColName, config.SoftDeleteColName) } @@ -53,11 +53,11 @@ func (c *BigQueryConnector) SyncQRepRecords( 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{ @@ -92,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 7ed87b0c06..8e600d5279 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -35,7 +35,7 @@ func NewQRepAvroSyncMethod(connector *BigQueryConnector, gcsBucket string, } func (s *QRepAvroSyncMethod) SyncRecords( - dstTableName string, + rawTableName string, flowJobName string, lastCP int64, dstTableMetadata *bigquery.TableMetadata, @@ -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,7 @@ 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) updateMetadataStmt, err := s.connector.getUpdateMetadataStmt(flowJobName, lastCP, syncBatchID) if err != nil { return -1, fmt.Errorf("failed to update metadata: %v", err) @@ -71,7 +75,7 @@ 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), ) stmts := []string{ @@ -91,12 +95,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 } @@ -124,8 +128,14 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( 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) } @@ -135,7 +145,6 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( flowJobName, dstTableName, partition.PartitionId), ) bqClient := s.connector.client - datasetID := s.connector.datasetID selector := "*" if softDeleteCol != "" { // PeerDB column @@ -145,8 +154,8 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( selector += ", CURRENT_TIMESTAMP" } // Insert the records from the staging table into the destination table - insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT %s FROM `%s.%s`;", - datasetID, dstTableName, selector, datasetID, stagingTable) + 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 { @@ -166,14 +175,15 @@ func (s *QRepAvroSyncMethod) SyncQRepRecords( } // 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 } @@ -323,7 +333,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, @@ -379,7 +389,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)) @@ -396,7 +405,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) @@ -412,7 +421,7 @@ 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 into %s", avroFile.FilePath)) err = s.connector.waitForTableReady(stagingTable) if err != nil { diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 05347a4263..c8ba3dad41 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -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 diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 2be3fcb2a5..4c5693f292 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -365,7 +365,7 @@ func (p *PostgresCDCSource) consumeStream( } if rec != nil { - tableName := rec.GetTableName() + tableName := rec.GetDestinationTableName() switch r := rec.(type) { case *model.UpdateRecord: // tableName here is destination tableName. @@ -843,7 +843,7 @@ func (p *PostgresCDCSource) processRelationMessage( 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/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index fb9dadb9ba..21bd3b5c75 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) diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index b28577f4d3..c76688f79b 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -150,7 +150,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() @@ -1203,3 +1203,71 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Columns_BQ() { 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) +} diff --git a/flow/model/model.go b/flow/model/model.go index 581b57178b..fc2c12d849 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -58,7 +58,7 @@ 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 } @@ -244,7 +244,7 @@ func (r *InsertRecord) GetCheckPointID() int64 { return r.CheckPointID } -func (r *InsertRecord) GetTableName() string { +func (r *InsertRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -273,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 } @@ -299,7 +299,7 @@ func (r *DeleteRecord) GetCheckPointID() int64 { return r.CheckPointID } -func (r *DeleteRecord) GetTableName() string { +func (r *DeleteRecord) GetDestinationTableName() string { return r.DestinationTableName } @@ -470,8 +470,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 { From 7ffce7c2780fe063ad309d0861ad526196a05ea1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 26 Dec 2023 14:34:38 +0000 Subject: [PATCH 44/52] chore: dependencies (#897) pgwire 0.18: https://github.com/sunng87/pgwire/releases/tag/v0.18.0 --- flow/go.mod | 2 +- flow/go.sum | 4 +- nexus/Cargo.lock | 154 ++-- nexus/parser/Cargo.toml | 2 +- nexus/peer-bigquery/Cargo.toml | 2 +- nexus/peer-cursor/Cargo.toml | 2 +- nexus/peer-postgres/Cargo.toml | 2 +- nexus/peer-snowflake/Cargo.toml | 2 +- nexus/server/Cargo.toml | 2 +- nexus/server/src/main.rs | 8 - nexus/value/Cargo.toml | 2 +- ui/package-lock.json | 1169 +++++++++++++++++-------------- ui/package.json | 24 +- 13 files changed, 751 insertions(+), 624 deletions(-) diff --git a/flow/go.mod b/flow/go.mod index 5cd582a51c..cd1b4f0349 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -9,7 +9,7 @@ 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.8 + 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 diff --git a/flow/go.sum b/flow/go.sum index 714e5dfd3f..f3b3ff8180 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -62,8 +62,8 @@ 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.8 h1:gKgEiyJ8CPnr4r6pS06WfNXvp6z34JER1pBIwuocvVA= -github.com/aws/aws-sdk-go v1.49.8/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= diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 59914b0952..59e1bbf8b9 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]] @@ -941,9 +940,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", @@ -956,9 +955,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", @@ -966,15 +965,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", @@ -983,38 +982,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", @@ -1510,15 +1509,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" @@ -1657,9 +1647,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", ] @@ -1672,9 +1662,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", @@ -1693,7 +1683,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -1704,9 +1694,9 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[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", @@ -2003,9 +1993,9 @@ dependencies = [ [[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", @@ -2064,7 +2054,7 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2119,9 +2109,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" @@ -2227,7 +2217,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]] @@ -2266,9 +2256,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", ] @@ -2300,7 +2290,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.41", + "syn 2.0.43", "tempfile", "which", ] @@ -2315,7 +2305,7 @@ dependencies = [ "itertools 0.11.0", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2485,7 +2475,7 @@ dependencies = [ "quote", "refinery-core", "regex", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2732,9 +2722,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" @@ -2887,7 +2877,7 @@ checksum = "43576ca501357b9b071ac53cdc7da8ef0cbd9493d8df094cd821777ea6e894d3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -2903,9 +2893,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", ] @@ -3096,7 +3086,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]] @@ -3135,9 +3125,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", @@ -3153,7 +3143,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3239,7 +3229,7 @@ checksum = "01742297787513b79cf8e29d1056ede1313e2420b7b3b15d0a768b4921f549df" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3300,9 +3290,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", @@ -3336,7 +3326,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3598,7 +3588,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] @@ -3825,7 +3815,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", "wasm-bindgen-shared", ] @@ -3859,7 +3849,7 @@ checksum = "f0eb82fcb7930ae6219a7ecfd55b217f5f0893484b7a13022ebb2b2bf20b5283" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4149,9 +4139,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", @@ -4191,7 +4181,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.41", + "syn 2.0.43", ] [[package]] diff --git a/nexus/parser/Cargo.toml b/nexus/parser/Cargo.toml index 2f4c1698de..0b1eae1678 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 ff6a6c4e90..9f565388fb 100644 --- a/nexus/peer-bigquery/Cargo.toml +++ b/nexus/peer-bigquery/Cargo.toml @@ -14,7 +14,7 @@ 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-cursor/Cargo.toml b/nexus/peer-cursor/Cargo.toml index 921d0d52af..cc644064b1 100644 --- a/nexus/peer-cursor/Cargo.toml +++ b/nexus/peer-cursor/Cargo.toml @@ -10,7 +10,7 @@ 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-postgres/Cargo.toml b/nexus/peer-postgres/Cargo.toml index 130e8830f8..8f8ef39ead 100644 --- a/nexus/peer-postgres/Cargo.toml +++ b/nexus/peer-postgres/Cargo.toml @@ -15,7 +15,7 @@ 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-snowflake/Cargo.toml b/nexus/peer-snowflake/Cargo.toml index 45b3fe413d..913824b31d 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -17,7 +17,7 @@ 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" } rsa = { version = "0.9.2", features = ["pem", "pkcs5"] } diff --git a/nexus/server/Cargo.toml b/nexus/server/Cargo.toml index 36f7c65b4f..1bf4542c22 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"] } diff --git a/nexus/server/src/main.rs b/nexus/server/src/main.rs index c5011bd4f3..a3f1048bdc 100644 --- a/nexus/server/src/main.rs +++ b/nexus/server/src/main.rs @@ -30,7 +30,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 +81,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 +99,6 @@ impl NexusBackend { Self { catalog, peer_connections, - portal_store: Arc::new(MemPortalStore::new()), query_parser, peer_cursors: Mutex::new(PeerCursors::new()), executors: DashMap::new(), @@ -1008,13 +1005,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()) } diff --git a/nexus/value/Cargo.toml b/nexus/value/Cargo.toml index 921865f636..d4fdf39cbf 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/ui/package-lock.json b/ui/package-lock.json index 3dced606dc..bde9fe3a38 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", @@ -44,18 +44,18 @@ "react-select": "^5.8.0", "react-spinners": "^0.13.8", "react-toastify": "^9.1.3", - "styled-components": "^6.1.1", + "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", @@ -64,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" @@ -2355,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", @@ -2378,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", @@ -3028,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", @@ -3045,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" }, @@ -3057,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", @@ -3074,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" }, @@ -3581,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" @@ -3652,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" @@ -3669,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": { @@ -4927,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", @@ -4945,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", @@ -4960,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" }, @@ -4975,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", @@ -5010,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": { @@ -5040,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" @@ -5053,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" @@ -5070,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", @@ -5093,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", @@ -5107,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", @@ -5189,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", @@ -5199,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" @@ -5212,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", @@ -5226,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", @@ -5265,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", @@ -5293,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", @@ -5389,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", @@ -5407,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", @@ -5541,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" @@ -5554,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", @@ -5615,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" @@ -5641,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", @@ -5655,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", @@ -5699,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" @@ -5712,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", @@ -5827,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" }, @@ -5862,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": { @@ -5876,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", @@ -5886,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" @@ -5903,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", @@ -5928,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", @@ -5938,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", @@ -6004,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", @@ -6022,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", @@ -6157,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", @@ -6167,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", @@ -6177,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", @@ -6264,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", @@ -6274,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", @@ -6300,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", @@ -6365,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", @@ -6388,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" }, @@ -6403,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", @@ -6434,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" }, @@ -6454,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" @@ -6898,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": "*", @@ -7075,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": { @@ -7151,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", @@ -7183,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": { @@ -7211,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" @@ -7228,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" @@ -7241,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", @@ -7496,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": { @@ -7821,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": { @@ -8755,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" @@ -9043,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", @@ -9166,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", @@ -9496,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", @@ -9549,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", @@ -9784,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", @@ -10612,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", @@ -10637,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": { @@ -10672,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", @@ -11816,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" } @@ -12115,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" @@ -12136,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" @@ -12327,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", @@ -12500,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", @@ -12570,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" }, @@ -12603,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", @@ -12641,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" } @@ -12650,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" }, @@ -13017,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", @@ -13036,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": { @@ -13081,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": { @@ -13215,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" @@ -13223,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", @@ -13759,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", @@ -13813,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" }, @@ -14429,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" } @@ -14712,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" }, @@ -14732,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" } @@ -14811,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", @@ -15083,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": { @@ -15201,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", @@ -15348,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", @@ -15373,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" } @@ -15613,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" } @@ -15621,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" } @@ -15635,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" @@ -15651,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" } @@ -16038,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" @@ -16266,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" @@ -16463,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", @@ -16492,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", @@ -16695,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" } }, @@ -17075,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", @@ -17866,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" }, @@ -17878,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" } @@ -17901,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" }, @@ -18152,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", @@ -18273,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", @@ -18286,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", @@ -18403,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" }, @@ -18494,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" @@ -18520,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", @@ -18553,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", @@ -18570,7 +18736,7 @@ "sucrase-node": "bin/sucrase-node" }, "engines": { - "node": ">=8" + "node": ">=16 || 14 >=14.17" } }, "node_modules/sucrase/node_modules/commander": { @@ -18581,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", @@ -18666,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", @@ -19125,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": { @@ -19399,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", @@ -20055,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" }, @@ -20152,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", @@ -20170,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", @@ -20186,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", @@ -20207,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" }, @@ -20219,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" }, @@ -20231,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", @@ -20248,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" }, @@ -20262,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 1ff4f487ca..bfd883e694 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", @@ -50,18 +50,18 @@ "react-select": "^5.8.0", "react-spinners": "^0.13.8", "react-toastify": "^9.1.3", - "styled-components": "^6.1.1", + "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", @@ -70,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" From a8e07d90eb805d385d49ef6b69c8aab8d8ee547c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 26 Dec 2023 15:04:08 +0000 Subject: [PATCH 45/52] Remove nexus/pgerror (#898) There's no need to implement a string error since `Box` implements `From`/`From<&'static str>` In a few places I opted to box original error rather than box original error's error string --- nexus/Cargo.lock | 41 ---- nexus/Cargo.toml | 1 - nexus/peer-bigquery/Cargo.toml | 1 - nexus/peer-bigquery/src/lib.rs | 31 +-- nexus/peer-bigquery/src/stream.rs | 21 +- nexus/peer-cursor/Cargo.toml | 1 - nexus/peer-cursor/src/util.rs | 16 +- nexus/peer-postgres/Cargo.toml | 1 - nexus/peer-postgres/src/lib.rs | 21 +- nexus/peer-postgres/src/stream.rs | 6 +- nexus/peer-snowflake/Cargo.toml | 1 - nexus/peer-snowflake/src/auth.rs | 7 +- nexus/peer-snowflake/src/lib.rs | 28 +-- nexus/peer-snowflake/src/stream.rs | 21 +- nexus/pgerror/Cargo.toml | 9 - nexus/pgerror/src/lib.rs | 8 - nexus/postgres-connection/src/lib.rs | 9 +- nexus/server/Cargo.toml | 1 - nexus/server/src/main.rs | 285 ++++++++++++--------------- 19 files changed, 171 insertions(+), 338 deletions(-) delete mode 100644 nexus/pgerror/Cargo.toml delete mode 100644 nexus/pgerror/src/lib.rs diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 59e1bbf8b9..09159e4d29 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -781,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" @@ -1792,7 +1786,6 @@ dependencies = [ "gcp-bigquery-client", "peer-connections", "peer-cursor", - "pgerror", "pgwire", "pt", "rust_decimal", @@ -1827,7 +1820,6 @@ dependencies = [ "anyhow", "async-trait", "futures", - "pgerror", "pgwire", "sqlparser", "tokio", @@ -1845,7 +1837,6 @@ dependencies = [ "futures", "peer-connections", "peer-cursor", - "pgerror", "pgwire", "postgres-connection", "postgres-inet", @@ -1877,7 +1868,6 @@ dependencies = [ "hex", "jsonwebtoken", "peer-cursor", - "pgerror", "pgwire", "pt", "reqwest", @@ -1932,7 +1922,6 @@ dependencies = [ "peer-postgres", "peer-snowflake", "peerdb-parser", - "pgerror", "pgwire", "postgres", "prost", @@ -1984,13 +1973,6 @@ dependencies = [ "indexmap 2.1.0", ] -[[package]] -name = "pgerror" -version = "0.1.0" -dependencies = [ - "snafu", -] - [[package]] name = "pgwire" version = "0.18.0" @@ -3015,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" diff --git a/nexus/Cargo.toml b/nexus/Cargo.toml index a07c40ea46..2aaa406ac5 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/peer-bigquery/Cargo.toml b/nexus/peer-bigquery/Cargo.toml index 9f565388fb..a7b570f9d3 100644 --- a/nexus/peer-bigquery/Cargo.toml +++ b/nexus/peer-bigquery/Cargo.toml @@ -13,7 +13,6 @@ 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.18" pt = { path = "../pt" } rust_decimal = { version = "1.30.0", features = [ "tokio-pg" ] } diff --git a/nexus/peer-bigquery/src/lib.rs b/nexus/peer-bigquery/src/lib.rs index c338518f45..29d58fb24f 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 53e85c021a..a831f6818f 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 cc644064b1..7623f8f007 100644 --- a/nexus/peer-cursor/Cargo.toml +++ b/nexus/peer-cursor/Cargo.toml @@ -9,7 +9,6 @@ edition = "2021" anyhow = "1.0" async-trait = "0.1" futures = "0.3" -pgerror = { path = "../pgerror" } pgwire = "0.18" sqlparser = { git = "https://github.com/peerdb-io/sqlparser-rs.git" } tokio = { version = "1.0", features = ["full"] } diff --git a/nexus/peer-cursor/src/util.rs b/nexus/peer-cursor/src/util.rs index e87478d67b..e9b9d55b00 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 8f8ef39ead..9cc37b8fd5 100644 --- a/nexus/peer-postgres/Cargo.toml +++ b/nexus/peer-postgres/Cargo.toml @@ -14,7 +14,6 @@ chrono = { version = "0.4", features = ["serde"] } futures = "0.3" peer-cursor = { path = "../peer-cursor" } peer-connections = { path = "../peer-connections" } -pgerror = { path = "../pgerror" } pgwire = "0.18" postgres-connection = { path = "../postgres-connection" } pt = { path = "../pt" } diff --git a/nexus/peer-postgres/src/lib.rs b/nexus/peer-postgres/src/lib.rs index 6617eeefd3..7df2da1914 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 e7195a2aeb..21905c1cc6 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 913824b31d..7017ced427 100644 --- a/nexus/peer-snowflake/Cargo.toml +++ b/nexus/peer-snowflake/Cargo.toml @@ -11,7 +11,6 @@ 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"] } diff --git a/nexus/peer-snowflake/src/auth.rs b/nexus/peer-snowflake/src/auth.rs index 64bb0f0458..4ec1a90823 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 rsa::RsaPrivateKey; use rsa::pkcs1::EncodeRsaPrivateKey; use rsa::pkcs8::{DecodePrivateKey, EncodePublicKey}; +use rsa::RsaPrivateKey; use secrecy::{Secret, SecretString}; use serde::Serialize; use sha2::{Digest, Sha256}; @@ -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( - self.private_key.to_pkcs1_der()?.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 9fe3a8536c..ac4d0154d9 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 3434b70dfa..4740270d12 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 2dca877fea..0000000000 --- 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 bf842f7780..0000000000 --- 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 3eb6558a2d..dfa165b7dc 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/server/Cargo.toml b/nexus/server/Cargo.toml index 1bf4542c22..33d6d14920 100644 --- a/nexus/server/Cargo.toml +++ b/nexus/server/Cargo.toml @@ -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 a3f1048bdc..9aa4dcb9a4 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::{ @@ -169,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) } @@ -181,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) } @@ -224,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( @@ -251,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; @@ -266,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: {:?}", @@ -283,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( @@ -331,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; { @@ -348,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(), + ) })?; } @@ -421,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?; @@ -435,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(), + )); } } } @@ -461,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 @@ -481,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); @@ -509,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) = { @@ -520,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?; @@ -545,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; @@ -557,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( @@ -598,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 = { @@ -610,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(), + ) })? }; @@ -644,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; @@ -660,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); @@ -688,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; @@ -703,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: {:?}", @@ -721,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( @@ -749,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; @@ -764,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: {:?}", @@ -782,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( @@ -813,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 => { @@ -849,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(), + ) })?, } }; @@ -872,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. @@ -892,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) @@ -1073,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? } From 42d71e072c6d1e58caf5905609d78cc4f742aeea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 26 Dec 2023 15:20:00 +0000 Subject: [PATCH 46/52] Cleanup normalize (#894) Clean up code, fetch `syncBatchID`/`normalizeBatchID` together Reduce redundantly setting FlowName on context in cdc_flow --- flow/cmd/handler.go | 7 +-- flow/connectors/bigquery/bigquery.go | 60 +++++++++++--------- flow/connectors/postgres/postgres.go | 2 +- flow/connectors/snowflake/snowflake.go | 64 +++++++++++---------- flow/model/model.go | 5 ++ flow/workflows/cdc_flow.go | 77 +++++++++++--------------- flow/workflows/normalize_flow.go | 1 - 7 files changed, 108 insertions(+), 108 deletions(-) diff --git a/flow/cmd/handler.go b/flow/cmd/handler.go index dd922ba1f5..04a9ccd708 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 == "" { diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 3da34f99d7..11a88e8382 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -382,29 +382,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, @@ -736,13 +742,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) } @@ -753,20 +753,28 @@ 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) } @@ -786,8 +794,8 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) dstTableName: tableName, dstDatasetTable: dstDatasetTable, normalizedTableSchema: c.tableNameSchemaMapping[tableName], - syncBatchID: syncBatchID, - normalizeBatchID: normalizeBatchID, + syncBatchID: batchIDs.SyncBatchID, + normalizeBatchID: batchIDs.NormalizeBatchID, unchangedToastColumns: tableNametoUnchangedToastCols[tableName], peerdbCols: &protos.PeerDBColumns{ SoftDeleteColName: req.SoftDeleteColName, @@ -802,7 +810,7 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) // 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) + c.datasetID, MirrorJobsTable, batchIDs.SyncBatchID, req.FlowJobName) stmts = append(stmts, updateMetadataStmt) query := strings.Join(stmts, "\n") @@ -813,8 +821,8 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) return &model.NormalizeResponse{ Done: true, - StartBatchID: normalizeBatchID + 1, - EndBatchID: syncBatchID, + StartBatchID: batchIDs.NormalizeBatchID + 1, + EndBatchID: batchIDs.SyncBatchID, }, nil } diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 1152493b01..59a1b835bd 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -423,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{ diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index db13e188b8..5d2c7e03b1 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -71,14 +71,14 @@ const ( 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=?" - 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=?" - 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 { @@ -345,23 +345,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, @@ -590,20 +594,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 } @@ -617,12 +617,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) } @@ -640,7 +644,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)) @@ -657,15 +661,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 } diff --git a/flow/model/model.go b/flow/model/model.go index fc2c12d849..b579ccfb56 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -416,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. diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index e9091a6c3f..55751bd096 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) @@ -351,9 +345,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, @@ -363,18 +354,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 { @@ -387,20 +377,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 @@ -419,7 +395,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, @@ -436,24 +411,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 index af14e11b8f..39256eac1a 100644 --- a/flow/workflows/normalize_flow.go +++ b/flow/workflows/normalize_flow.go @@ -51,7 +51,6 @@ func (s *NormalizeFlowExecution) executeNormalizeFlow( HeartbeatTimeout: 5 * time.Minute, }) - // execute StartFlow on the peers to start the flow startNormalizeInput := &protos.StartNormalizeInput{ FlowConnectionConfigs: config, } From 67ab274820d3d422ad58e02def24b69e0bd7f255 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 27 Dec 2023 00:09:35 +0530 Subject: [PATCH 47/52] soft delete logic fixing tests for BigQuery (#857) --- flow/connectors/bigquery/bigquery.go | 6 +- flow/e2e/bigquery/bigquery_helper.go | 12 + flow/e2e/bigquery/peer_flow_bq_test.go | 338 +++++++++++++++++++++++++ 3 files changed, 355 insertions(+), 1 deletion(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 11a88e8382..22977ed4d9 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -456,9 +456,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) diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 21bd3b5c75..40a42ea64f 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -455,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 c76688f79b..6cbac4c915 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -5,10 +5,12 @@ 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" @@ -1271,3 +1273,339 @@ func (s PeerFlowE2ETestSuiteBQ) Test_Multi_Table_Multi_Dataset_BQ() { 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) +} From 2f18f76362e4c283d7634c5ca190decfb7658673 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Philip=20Dub=C3=A9?= Date: Tue, 26 Dec 2023 19:25:01 +0000 Subject: [PATCH 48/52] Ignore unique constraint violation during CREATE IF NOT EXISTS (#905) https://stackoverflow.com/questions/29900845/create-schema-if-not-exists-raises-duplicate-key-error Ran into this while looking to make all tests run in parallel --- flow/connectors/external_metadata/store.go | 12 ++++++++++-- flow/go.mod | 1 + flow/go.sum | 2 ++ flow/workflows/setup_flow.go | 2 +- 4 files changed, 14 insertions(+), 3 deletions(-) diff --git a/flow/connectors/external_metadata/store.go b/flow/connectors/external_metadata/store.go index eee1d4ef66..9fe72828ad 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 } diff --git a/flow/go.mod b/flow/go.mod index cd1b4f0349..69307cd2ae 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -15,6 +15,7 @@ require ( 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 diff --git a/flow/go.sum b/flow/go.sum index f3b3ff8180..360572bb2c 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -255,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= diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index ab8168191c..03ed6dd3c0 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 } From e16a37163cba4b5c9b3b14825f036f476f81c685 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 27 Dec 2023 01:08:48 +0530 Subject: [PATCH 49/52] replacing SF tableNameComponents with PG's version (#906) keepin' our codebase DRY --- flow/connectors/snowflake/client.go | 3 ++- flow/connectors/snowflake/qrep.go | 8 ++------ flow/connectors/snowflake/snowflake.go | 25 ++++--------------------- flow/connectors/utils/identifiers.go | 25 ++++++++++++++++++++++++- flow/connectors/utils/postgres.go | 24 ------------------------ 5 files changed, 32 insertions(+), 53 deletions(-) diff --git a/flow/connectors/snowflake/client.go b/flow/connectors/snowflake/client.go index ae2da1a891..beb38a4a04 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 def870c183..1d81392d71 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -278,21 +278,17 @@ func (c *SnowflakeConnector) CleanupQRepFlow(config *protos.QRepConfig) error { 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.QueryContext(c.ctx, queryString) if err != nil { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 5d2c7e03b1..55dd4444df 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -81,11 +81,6 @@ const ( 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 @@ -245,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) } @@ -423,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) } @@ -940,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, diff --git a/flow/connectors/utils/identifiers.go b/flow/connectors/utils/identifiers.go index 0b91b9e4f3..2ae919488d 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/postgres.go b/flow/connectors/utils/postgres.go index 72aaf86f5e..58cd02f205 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 -} From a3b280046d0c50d27867264d96b5e92922bd3f39 Mon Sep 17 00:00:00 2001 From: Kevin Biju <52661649+heavycrystal@users.noreply.github.com> Date: Wed, 27 Dec 2023 18:39:21 +0530 Subject: [PATCH 50/52] [breaking] BQ SyncRecords now streams properly, code cleanup (#909) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### ⚠️ This change can break existing CDC mirrors from Postgres to BigQuery! Fixes a bug in BigQuery CDC where a batch with number of records greater than `2 ** 20` causes the Avro file generation part to hang. This is because all records were being written to a bounded channel first and then consumed by the Avro writer instead of the 2 operations happening in parallel. With a large number of records, the channel would fill up and block before the records finished writing, leading to the loop deadlocking itself. Fixed by switching BigQuery record generation to the mechanism used by Snowflake, where the record generation happens in another goroutine and therefore the channel consumption happens in parallel. As part of this change, some code was cleaned up and the BigQuery raw table schema was changed in a breaking manner to be similar to the SF/PG equivalent. Specifically, the column `_peerdb_timestamp` of type `TIMESTAMP` was removed and the column `_peerdb_timestamp_nanos` of type `INTEGER` was renamed to the former. Existing raw tables will need to be fixed up to match this new, simpler schema. ``` ALTER TABLE <...> DROP COLUMN _peerdb_timestamp; ALTER TABLE <...> RENAME COLUMN _peerdb_timestamp_nanos TO _peerdb_timestamp; ``` Closes #908 --- flow/connectors/bigquery/bigquery.go | 221 +----------------- .../bigquery/merge_statement_generator.go | 3 +- flow/connectors/bigquery/qrep_avro_sync.go | 9 +- flow/connectors/utils/stream.go | 20 +- 4 files changed, 23 insertions(+), 230 deletions(-) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 22977ed4d9..7905fc05bc 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" @@ -69,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 @@ -493,22 +479,6 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync return resultMap, nil } -// ValueSaver interface for bqRecord -func (r StagingBQRecord) Save() (map[string]bigquery.Value, string, error) { - return map[string]bigquery.Value{ - "_peerdb_uid": r.uid, - "_peerdb_timestamp": r.timestamp, - "_peerdb_timestamp_nanos": r.timestampNanos, - "_peerdb_destination_table_name": r.destinationTableName, - "_peerdb_data": r.data, - "_peerdb_record_type": r.recordType, - "_peerdb_match_data": r.matchData, - "_peerdb_batch_id": r.batchID, - "_peerdb_staging_batch_id": r.stagingBatchID, - "_peerdb_unchanged_toast_columns": r.unchangedToastColumns, - }, bigquery.NoDedupeID, nil -} - // SyncRecords pushes records to the destination. // Currently only supports inserts, updates, and deletes. // More record types will be added in the future. @@ -539,201 +509,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[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[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[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[3] = qvalue.QValue{ - Kind: qvalue.QValueKindString, - Value: record.GetDestinationTableName(), - } - 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), @@ -842,8 +642,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}, diff --git a/flow/connectors/bigquery/merge_statement_generator.go b/flow/connectors/bigquery/merge_statement_generator.go index e9a71b06cd..569ff611e9 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -80,7 +80,6 @@ func (m *mergeStmtGenerator) generateFlattenedCTE() string { flattenedProjs = append( flattenedProjs, "_peerdb_timestamp", - "_peerdb_timestamp_nanos", "_peerdb_record_type", "_peerdb_unchanged_toast_columns", ) @@ -99,7 +98,7 @@ 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 diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index 8e600d5279..9f36ceb3ae 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -37,7 +37,7 @@ func NewQRepAvroSyncMethod(connector *BigQueryConnector, gcsBucket string, func (s *QRepAvroSyncMethod) SyncRecords( rawTableName string, flowJobName string, - lastCP int64, + records *model.CDCRecordStream, dstTableMetadata *bigquery.TableMetadata, syncBatchID int64, stream *model.QRecordStream, @@ -67,6 +67,11 @@ func (s *QRepAvroSyncMethod) SyncRecords( datasetID := s.connector.datasetID insertStmt := fmt.Sprintf("INSERT INTO `%s.%s` SELECT * FROM `%s.%s`;", 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) @@ -421,7 +426,7 @@ 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", avroFile.FilePath)) + slog.Info(fmt.Sprintf("Pushed from %s to BigQuery", avroFile.FilePath)) err = s.connector.waitForTableReady(stagingTable) if err != nil { diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index 7cd8ae8391..bbfe0a2fa6 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, From f3d1d8e93de7143951d2ee94b46b69642f7048c6 Mon Sep 17 00:00:00 2001 From: Kaushik Iska Date: Wed, 27 Dec 2023 09:27:53 -0500 Subject: [PATCH 51/52] update max batch size on signal (#910) --- flow/workflows/cdc_flow.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 55751bd096..b73f6945fe 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -291,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) From 4425e205f1e10c11045011cfdf7abff876bdfab9 Mon Sep 17 00:00:00 2001 From: Amogh Bharadwaj Date: Wed, 27 Dec 2023 20:35:12 +0530 Subject: [PATCH 52/52] Show version, make ui dependent on api in docker (#912) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ![Screenshot 2023-12-27 at 7 44 07 PM](https://github.com/PeerDB-io/peerdb/assets/65964360/e0ddd66d-c733-4c02-bc75-45acd02169ee) Docker compose changes: Figured it would be good to have ui start after the flow-api container --- docker-compose-dev.yml | 2 ++ docker-compose.yml | 2 ++ ui/app/api/version/route.ts | 23 +++++++++++++++++ ui/app/dto/VersionDTO.ts | 3 +++ ui/components/SidebarComponent.tsx | 40 ++++++++++++++++++++++-------- 5 files changed, 60 insertions(+), 10 deletions(-) create mode 100644 ui/app/api/version/route.ts create mode 100644 ui/app/dto/VersionDTO.ts diff --git a/docker-compose-dev.yml b/docker-compose-dev.yml index cc6c50b77d..4f5a8ca8d9 100644 --- a/docker-compose-dev.yml +++ b/docker-compose-dev.yml @@ -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 b5d37dbdc1..d6a4fbb127 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -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/ui/app/api/version/route.ts b/ui/app/api/version/route.ts new file mode 100644 index 0000000000..1197ca2049 --- /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/VersionDTO.ts b/ui/app/dto/VersionDTO.ts new file mode 100644 index 0000000000..cb0e4b1896 --- /dev/null +++ b/ui/app/dto/VersionDTO.ts @@ -0,0 +1,3 @@ +export type UVersionResponse = { + version: string; +}; diff --git a/ui/components/SidebarComponent.tsx b/ui/components/SidebarComponent.tsx index c9afc54ca6..7914fad61a 100644 --- a/ui/components/SidebarComponent.tsx +++ b/ui/components/SidebarComponent.tsx @@ -1,5 +1,6 @@ 'use client'; +import { UVersionResponse } from '@/app/dto/VersionDTO'; import useTZStore from '@/app/globalstate/time'; import Logout from '@/components/Logout'; import { BrandLogo } from '@/lib/BrandLogo'; @@ -8,11 +9,30 @@ import { Label } from '@/lib/Label'; import { RowWithSelect } from '@/lib/Layout'; import { Sidebar, SidebarItem } from '@/lib/Sidebar'; import Link from 'next/link'; +import useSWR from 'swr'; +const centerFlexStyle = { + display: 'flex', + justifyContent: 'center', + alignItems: 'center', + width: '100%', + marginBottom: '0.5rem', +}; +const fetcher = (...args: [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={ +
+ +
+ } >