diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 25b73e2ef9..559774a192 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -9,6 +9,7 @@ import ( "github.com/PeerDB-io/peer-flow/connectors" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" + connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/connectors/utils/metrics" "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" @@ -18,12 +19,14 @@ import ( "github.com/jackc/pglogrepl" log "github.com/sirupsen/logrus" "go.temporal.io/sdk/activity" + "golang.org/x/exp/maps" ) // CheckConnectionResult is the result of a CheckConnection call. type CheckConnectionResult struct { - // True of metadata tables need to be set up. + // True if metadata tables need to be set up. NeedsSetupMetadataTables bool + SupportsSchemaMapping bool } type SlotSnapshotSignal struct { @@ -37,8 +40,17 @@ type FlowableActivity struct { CatalogMirrorMonitor *monitoring.CatalogMirrorMonitor } -// CheckConnection implements CheckConnection. -func (a *FlowableActivity) CheckConnection( +func (a *FlowableActivity) CheckPullConnection(ctx context.Context, config *protos.Peer) (bool, error) { + srcConn, err := connectors.GetCDCPullConnector(ctx, config) + if err != nil { + return false, fmt.Errorf("failed to get connector: %w", err) + } + defer connectors.CloseConnector(srcConn) + + return srcConn.ConnectionActive(), nil +} + +func (a *FlowableActivity) CheckSyncConnection( ctx context.Context, config *protos.Peer, ) (*CheckConnectionResult, error) { @@ -50,8 +62,17 @@ func (a *FlowableActivity) CheckConnection( needsSetup := dstConn.NeedsSetupMetadataTables() + supportsSchemaMapping := false + switch dstConn.(type) { + case *connpostgres.PostgresConnector: + supportsSchemaMapping = true + case *connsnowflake.SnowflakeConnector: + supportsSchemaMapping = true + } + return &CheckConnectionResult{ NeedsSetupMetadataTables: needsSetup, + SupportsSchemaMapping: supportsSchemaMapping, }, nil } @@ -146,18 +167,18 @@ func (a *FlowableActivity) CreateNormalizedTable( ctx context.Context, config *protos.SetupNormalizedTableBatchInput, ) (*protos.SetupNormalizedTableBatchOutput, error) { - conn, err := connectors.GetCDCSyncConnector(ctx, config.PeerConnectionConfig) + dstConn, err := connectors.GetCDCSyncConnector(ctx, config.PeerConnectionConfig) if err != nil { return nil, fmt.Errorf("failed to get connector: %w", err) } - defer connectors.CloseConnector(conn) + defer connectors.CloseConnector(dstConn) - return conn.SetupNormalizedTables(config) + return dstConn.SetupNormalizedTables(config) } // StartFlow implements StartFlow. func (a *FlowableActivity) StartFlow(ctx context.Context, - input *protos.StartFlowInput) (*model.SyncResponse, error) { + input *protos.StartFlowInput) (*protos.SyncResponse, error) { activity.RecordHeartbeat(ctx, "starting flow...") conn := input.FlowConnectionConfigs @@ -188,8 +209,12 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, "flowName": input.FlowConnectionConfigs.FlowJobName, }).Info("pulling records...") + var schemas []string + if input.FlowConnectionConfigs.MappingType == protos.MappingType_SCHEMA { + schemas = maps.Keys(input.FlowConnectionConfigs.SchemaMapping) + } startTime := time.Now() - recordsWithTableSchemaDelta, err := srcConn.PullRecords(&model.PullRecordsRequest{ + recordsWithDeltaInfo, err := srcConn.PullRecords(&model.PullRecordsRequest{ FlowJobName: input.FlowConnectionConfigs.FlowJobName, SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, TableNameMapping: input.FlowConnectionConfigs.TableNameMapping, @@ -200,11 +225,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, OverridePublicationName: input.FlowConnectionConfigs.PublicationName, OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, RelationMessageMapping: input.RelationMessageMapping, + Schemas: schemas, + AllowTableAdditions: input.FlowConnectionConfigs.AllowTableAdditions, }) if err != nil { return nil, fmt.Errorf("failed to pull records: %w", err) } - recordBatch := recordsWithTableSchemaDelta.RecordBatch + recordBatch := recordsWithDeltaInfo.RecordBatch pullRecordWithCount := fmt.Sprintf("pulled %d records", len(recordBatch.Records)) activity.RecordHeartbeat(ctx, pullRecordWithCount) @@ -242,9 +269,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, metrics.LogSyncMetrics(ctx, input.FlowConnectionConfigs.FlowJobName, 0, 1) metrics.LogNormalizeMetrics(ctx, input.FlowConnectionConfigs.FlowJobName, 0, 1, 0) metrics.LogCDCRawThroughputMetrics(ctx, input.FlowConnectionConfigs.FlowJobName, 0) - return &model.SyncResponse{ - RelationMessageMapping: recordsWithTableSchemaDelta.RelationMessageMapping, - TableSchemaDelta: recordsWithTableSchemaDelta.TableSchemaDelta, + return &protos.SyncResponse{ + RelationMessageMapping: recordsWithDeltaInfo.RelationMessageMapping, + MirrorDelta: recordsWithDeltaInfo.MirrorDelta, }, nil } @@ -284,7 +311,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, } if res.TableNameRowsMapping != nil { err = a.CatalogMirrorMonitor.AddCDCBatchTablesForFlow(ctx, input.FlowConnectionConfigs.FlowJobName, - res.CurrentSyncBatchID, res.TableNameRowsMapping) + res.CurrentSyncBatchId, res.TableNameRowsMapping) if err != nil { return nil, err } @@ -292,8 +319,8 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, if err != nil { return nil, err } - res.TableSchemaDelta = recordsWithTableSchemaDelta.TableSchemaDelta - res.RelationMessageMapping = recordsWithTableSchemaDelta.RelationMessageMapping + res.MirrorDelta = recordsWithDeltaInfo.MirrorDelta + res.RelationMessageMapping = recordsWithDeltaInfo.RelationMessageMapping pushedRecordsWithCount := fmt.Sprintf("pushed %d records", numRecords) activity.RecordHeartbeat(ctx, pushedRecordsWithCount) @@ -627,3 +654,86 @@ func (a *FlowableActivity) DropFlow(ctx context.Context, config *protos.Shutdown } return nil } + +// PopulateTableMappingFromSchemas sets up the TableNameMapping from SchemaMapping for MappingType SCHEMA +func (a *FlowableActivity) PopulateTableMappingFromSchemas( + ctx context.Context, + config *protos.ListTablesInSchemasInput, +) (*protos.ListTablesInSchemasOutput, error) { + srcConn, err := connectors.GetCDCPullConnector(ctx, config.PeerConnectionConfig) + if err != nil { + return nil, fmt.Errorf("failed to get source connector: %w", err) + } + defer connectors.CloseConnector(srcConn) + + srcSchemas := maps.Keys(config.SchemaMapping) + + schemaTablesMapping, err := srcConn.ListTablesInSchemas(srcSchemas) + if err != nil { + return nil, fmt.Errorf("failed to get schemaTablesMapping: %w", err) + } + + schemaTablesMappingProto := make(map[string]*protos.TablesList) + for schema, tables := range schemaTablesMapping { + schemaTablesMappingProto[schema] = &protos.TablesList{ + Tables: tables, + } + } + + return &protos.ListTablesInSchemasOutput{ + SchemaToTables: schemaTablesMappingProto, + }, nil +} + +func (a *FlowableActivity) CreateAdditionalTable( + ctx context.Context, + input *protos.CreateAdditionalTableInput) (*protos.AdditionalTableDelta, error) { + srcConn, err := connectors.GetCDCPullConnector(ctx, input.FlowConnectionConfigs.Source) + if err != nil { + return nil, fmt.Errorf("failed to get source connector: %w", err) + } + defer connectors.CloseConnector(srcConn) + + dstConn, err := connectors.GetCDCSyncConnector(ctx, input.FlowConnectionConfigs.Destination) + if err != nil { + return nil, fmt.Errorf("failed to get destination connector: %w", err) + } + defer connectors.CloseConnector(dstConn) + + srcTableIdentifier := fmt.Sprintf("%s.%s", input.AdditionalTableInfo.SrcSchema, + input.AdditionalTableInfo.TableName) + dstTableIdentifier := fmt.Sprintf("%s.%s", input.AdditionalTableInfo.DstSchema, + input.AdditionalTableInfo.TableName) + + tableRelIDMapping, err := srcConn.EnsurePullability(&protos.EnsurePullabilityBatchInput{ + SourceTableIdentifiers: []string{srcTableIdentifier}, + }) + if err != nil { + return nil, fmt.Errorf("failed to ensure pullability for additional table: %w", err) + } + + tableNameSchemaMapping, err := srcConn.GetTableSchema(&protos.GetTableSchemaBatchInput{ + TableIdentifiers: []string{srcTableIdentifier}, + }) + if err != nil { + return nil, fmt.Errorf("failed to get schema for additional table: %w", err) + } + + _, err = dstConn.SetupNormalizedTables(&protos.SetupNormalizedTableBatchInput{ + TableNameSchemaMapping: map[string]*protos.TableSchema{ + dstTableIdentifier: tableNameSchemaMapping.TableNameSchemaMapping[srcTableIdentifier]}, + }) + if err != nil { + return nil, fmt.Errorf("failed to create additional table at destination: %w", err) + } + + input.AdditionalTableInfo.RelId = tableRelIDMapping. + TableIdentifierMapping[srcTableIdentifier].GetPostgresTableIdentifier().RelId + input.AdditionalTableInfo.TableSchema = tableNameSchemaMapping.TableNameSchemaMapping[srcTableIdentifier] + + log.WithFields(log.Fields{ + "flowName": input.FlowConnectionConfigs.FlowJobName, + }).Infof("finished creating additional table %s\n", dstTableIdentifier) + + return input.AdditionalTableInfo, nil +} diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 8f6df0f5bb..e069759d27 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -438,11 +438,11 @@ 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. -func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*protos.SyncResponse, error) { if len(req.Records.Records) == 0 { - return &model.SyncResponse{ - FirstSyncedCheckPointID: 0, - LastSyncedCheckPointID: 0, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: 0, + LastSyncedCheckpointId: 0, NumRecordsSynced: 0, }, nil } @@ -460,7 +460,7 @@ func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } syncBatchID = syncBatchID + 1 - var res *model.SyncResponse + var res *protos.SyncResponse if req.SyncMode == protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO { res, err = c.syncRecordsViaAvro(req, rawTableName, syncBatchID) if err != nil { @@ -478,7 +478,7 @@ func (c *BigQueryConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } func (c *BigQueryConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, - rawTableName string, syncBatchID int64) (*model.SyncResponse, error) { + rawTableName string, syncBatchID int64) (*protos.SyncResponse, error) { stagingTableName := c.getStagingTableName(req.FlowJobName) stagingTable := c.client.Dataset(c.datasetID).Table(stagingTableName) err := c.truncateTable(stagingTableName) @@ -594,9 +594,9 @@ func (c *BigQueryConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, numRecords := len(records) if numRecords == 0 { - return &model.SyncResponse{ - FirstSyncedCheckPointID: 0, - LastSyncedCheckPointID: 0, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: 0, + LastSyncedCheckpointId: 0, NumRecordsSynced: 0, }, nil } @@ -645,17 +645,17 @@ func (c *BigQueryConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, metrics.LogSyncMetrics(c.ctx, req.FlowJobName, int64(numRecords), time.Since(startTime)) log.Printf("pushed %d records to %s.%s", numRecords, c.datasetID, rawTableName) - return &model.SyncResponse{ - FirstSyncedCheckPointID: firstCP, - LastSyncedCheckPointID: lastCP, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: firstCP, + LastSyncedCheckpointId: lastCP, NumRecordsSynced: int64(numRecords), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchId: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, }, nil } func (c *BigQueryConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, - rawTableName string, syncBatchID int64) (*model.SyncResponse, error) { + rawTableName string, syncBatchID int64) (*protos.SyncResponse, error) { tableNameRowsMapping := make(map[string]uint32) first := true var firstCP int64 = 0 @@ -867,11 +867,11 @@ func (c *BigQueryConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, metrics.LogSyncMetrics(c.ctx, req.FlowJobName, int64(numRecords), time.Since(startTime)) log.Printf("pushed %d records to %s.%s", numRecords, c.datasetID, rawTableName) - return &model.SyncResponse{ - FirstSyncedCheckPointID: firstCP, - LastSyncedCheckPointID: lastCP, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: firstCP, + LastSyncedCheckpointId: lastCP, NumRecordsSynced: int64(numRecords), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchId: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, }, nil } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 48c3e7e612..f7a9c2b3d5 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -32,14 +32,15 @@ type CDCPullConnector interface { EnsurePullability(req *protos.EnsurePullabilityBatchInput) ( *protos.EnsurePullabilityBatchOutput, error) - // Methods related to retrieving and pusing records for this connector as a source and destination. - // PullRecords pulls records from the source, and returns a RecordBatch. // This method should be idempotent, and should be able to be called multiple times with the same request. - PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) + PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithDeltaInfo, error) // PullFlowCleanup drops both the Postgres publication and replication slot, as a part of DROP MIRROR PullFlowCleanup(jobName string) error + + // ListTablesInSchemas... gets all the tables in multiple schemas + ListTablesInSchemas(schemas []string) (map[string][]string, error) } type CDCSyncConnector interface { @@ -69,7 +70,7 @@ type CDCSyncConnector interface { // SyncRecords pushes records to the destination peer and stores it in PeerDB specific tables. // This method should be idempotent, and should be able to be called multiple times with the same request. - SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) + SyncRecords(req *model.SyncRecordsRequest) (*protos.SyncResponse, error) // SyncFlowCleanup drops metadata tables on the destination, as a part of DROP MIRROR. SyncFlowCleanup(jobName string) error diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 35c1e6ee38..7302dfa308 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -85,7 +85,7 @@ func (c *EventHubConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } -func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*protos.SyncResponse, error) { shutdown := utils.HeartbeatRoutine(c.ctx, 10*time.Second, func() string { return fmt.Sprintf("syncing records to eventhub with"+ " push parallelism %d and push batch size %d", @@ -219,9 +219,9 @@ func (c *EventHubConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S metrics.LogSyncMetrics(c.ctx, req.FlowJobName, int64(rowsSynced), time.Since(startTime)) metrics.LogNormalizeMetrics(c.ctx, req.FlowJobName, int64(rowsSynced), time.Since(startTime), int64(rowsSynced)) - return &model.SyncResponse{ - FirstSyncedCheckPointID: batch.FirstCheckPointID, - LastSyncedCheckPointID: batch.LastCheckPointID, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: batch.FirstCheckPointID, + LastSyncedCheckpointId: batch.LastCheckPointID, NumRecordsSynced: int64(len(batch.Records)), TableNameRowsMapping: tableNameRowsMapping.Items(), }, nil diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index 65e75acfc0..368c37f8f0 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -17,47 +17,54 @@ import ( "github.com/jackc/pgx/v5/pgxpool" "github.com/lib/pq/oid" log "github.com/sirupsen/logrus" + "golang.org/x/exp/slices" ) -type PostgresCDCSource struct { +type postgresCDCSource struct { ctx context.Context replPool *pgxpool.Pool - SrcTableIDNameMapping map[uint32]string - TableNameMapping map[string]string + srcTableIDNameMapping map[uint32]string + tableNameMapping map[string]string slot string publication string relationMessageMapping model.RelationMessageMapping typeMap *pgtype.Map startLSN pglogrepl.LSN + schemas []string + allowTableAdditions bool } -type PostgresCDCConfig struct { - AppContext context.Context - Connection *pgxpool.Pool - Slot string - Publication string - SrcTableIDNameMapping map[uint32]string - TableNameMapping map[string]string - RelationMessageMapping model.RelationMessageMapping +type postgresCDCConfig struct { + appContext context.Context + connection *pgxpool.Pool + slot string + publication string + srcTableIDNameMapping map[uint32]string + tableNameMapping map[string]string + relationMessageMapping model.RelationMessageMapping + schemas []string + allowTableAdditions bool } // Create a new PostgresCDCSource -func NewPostgresCDCSource(cdcConfig *PostgresCDCConfig) (*PostgresCDCSource, error) { - return &PostgresCDCSource{ - ctx: cdcConfig.AppContext, - replPool: cdcConfig.Connection, - SrcTableIDNameMapping: cdcConfig.SrcTableIDNameMapping, - TableNameMapping: cdcConfig.TableNameMapping, - slot: cdcConfig.Slot, - publication: cdcConfig.Publication, - relationMessageMapping: cdcConfig.RelationMessageMapping, +func NewPostgresCDCSource(cdcConfig *postgresCDCConfig) (*postgresCDCSource, error) { + return &postgresCDCSource{ + ctx: cdcConfig.appContext, + replPool: cdcConfig.connection, + srcTableIDNameMapping: cdcConfig.srcTableIDNameMapping, + tableNameMapping: cdcConfig.tableNameMapping, + slot: cdcConfig.slot, + publication: cdcConfig.publication, + relationMessageMapping: cdcConfig.relationMessageMapping, typeMap: pgtype.NewMap(), + schemas: cdcConfig.schemas, + allowTableAdditions: cdcConfig.allowTableAdditions, }, nil } // PullRecords pulls records from the cdc stream -func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) ( - *model.RecordsWithTableSchemaDelta, error) { +func (p *postgresCDCSource) PullRecords(req *model.PullRecordsRequest) ( + *model.RecordsWithDeltaInfo, error) { // setup options pluginArguments := []string{ "proto_version '1'", @@ -110,20 +117,19 @@ func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) ( } // start consuming the cdc stream -func (p *PostgresCDCSource) consumeStream( +func (p *postgresCDCSource) consumeStream( conn *pgconn.PgConn, req *model.PullRecordsRequest, clientXLogPos pglogrepl.LSN, -) (*model.RecordsWithTableSchemaDelta, error) { +) (*model.RecordsWithDeltaInfo, error) { // TODO (kaushik): take into consideration the MaxBatchSize // parameters in the original request. records := &model.RecordBatch{ Records: make([]model.Record, 0), TablePKeyLastSeen: make(map[model.TableWithPkey]int), } - result := &model.RecordsWithTableSchemaDelta{ + result := &model.RecordsWithDeltaInfo{ RecordBatch: records, - TableSchemaDelta: nil, RelationMessageMapping: p.relationMessageMapping, } @@ -276,11 +282,27 @@ func (p *PostgresCDCSource) consumeStream( case *model.RelationRecord: tableSchemaDelta := rec.(*model.RelationRecord).TableSchemaDelta if len(tableSchemaDelta.AddedColumns) > 0 || len(tableSchemaDelta.DroppedColumns) > 0 { - result.TableSchemaDelta = tableSchemaDelta + result.MirrorDelta = &protos.MirrorDelta{ + Delta: &protos.MirrorDelta_TableSchemaDelta{ + TableSchemaDelta: tableSchemaDelta, + }, + } log.Infof("Detected schema change for table %s, returning currently accumulated records", - result.TableSchemaDelta.SrcTableName) + result.MirrorDelta.GetTableSchemaDelta().SrcTableName) earlyReturn = true } + case *model.AddedTableRecord: + log.Infof("Detected additional table %s, returning currently accumulated records", + rec.GetTableName()) + result.MirrorDelta = &protos.MirrorDelta{ + Delta: &protos.MirrorDelta_AdditionalTableDelta{ + AdditionalTableDelta: &protos.AdditionalTableDelta{ + TableName: rec.(*model.AddedTableRecord).TableName, + SrcSchema: rec.(*model.AddedTableRecord).SrcSchema, + }, + }, + } + earlyReturn = true } } @@ -295,7 +317,7 @@ func (p *PostgresCDCSource) consumeStream( } } -func (p *PostgresCDCSource) processMessage(batch *model.RecordBatch, xld pglogrepl.XLogData) (model.Record, error) { +func (p *postgresCDCSource) processMessage(batch *model.RecordBatch, xld pglogrepl.XLogData) (model.Record, error) { logicalMsg, err := pglogrepl.Parse(xld.WALData) if err != nil { return nil, fmt.Errorf("error parsing logical message: %w", err) @@ -314,13 +336,28 @@ func (p *PostgresCDCSource) processMessage(batch *model.RecordBatch, xld pglogre // for a commit message, update the last checkpoint id for the record batch. batch.LastCheckPointID = int64(xld.WALStart) case *pglogrepl.RelationMessage: - // TODO (kaushik): consider persistent state for a mirror job - // to be stored somewhere in temporal state. We might need to persist - // the state of the relation message somewhere - log.Debugf("RelationMessage => RelationID: %d, Namespace: %s, RelationName: %s, Columns: %v", + log.Infof("RelationMessage => RelationID: %d, Namespace: %s, RelationName: %s, Columns: %v", msg.RelationID, msg.Namespace, msg.RelationName, msg.Columns) if p.relationMessageMapping[msg.RelationID] == nil { - p.relationMessageMapping[msg.RelationID] = convertRelationMessageToProto(msg) + _, ok := p.tableNameMapping[fmt.Sprintf("%s.%s", msg.Namespace, msg.RelationName)] + // either it's a table we are aware of from SetupFlow, so it is in the map + // or it is a table we are not aware of, in a schema we are aware of. + if ok || slices.Contains(p.schemas, msg.Namespace) { + p.relationMessageMapping[msg.RelationID] = convertRelationMessageToProto(msg) + if !ok { + if p.allowTableAdditions { + // stop processing, return this to ensure new table is created + return &model.AddedTableRecord{ + CheckPointID: int64(xld.WALStart), + TableName: msg.RelationName, + SrcSchema: msg.Namespace, + }, nil + } else { + // the table is new, but we aren't going to add it to the destination. + delete(p.relationMessageMapping, msg.RelationID) + } + } + } } else { return p.processRelationMessage(xld.WALStart, convertRelationMessageToProto(msg)) } @@ -335,11 +372,11 @@ func (p *PostgresCDCSource) processMessage(batch *model.RecordBatch, xld pglogre return nil, nil } -func (p *PostgresCDCSource) processInsertMessage( +func (p *postgresCDCSource) processInsertMessage( lsn pglogrepl.LSN, msg *pglogrepl.InsertMessage, ) (model.Record, error) { - tableName, exists := p.SrcTableIDNameMapping[msg.RelationID] + tableName, exists := p.srcTableIDNameMapping[msg.RelationID] if !exists { return nil, nil } @@ -349,7 +386,8 @@ func (p *PostgresCDCSource) processInsertMessage( rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { - return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) + log.Warnf("unknown relation id: %d", msg.RelationID) + return nil, nil } // create empty map of string to interface{} @@ -361,18 +399,18 @@ func (p *PostgresCDCSource) processInsertMessage( return &model.InsertRecord{ CheckPointID: int64(lsn), Items: items, - DestinationTableName: p.TableNameMapping[tableName], + DestinationTableName: p.tableNameMapping[tableName], SourceTableName: tableName, UnchangedToastColumns: unchangedToastColumns, }, nil } // processUpdateMessage processes an update message and returns an UpdateRecord -func (p *PostgresCDCSource) processUpdateMessage( +func (p *postgresCDCSource) processUpdateMessage( lsn pglogrepl.LSN, msg *pglogrepl.UpdateMessage, ) (model.Record, error) { - tableName, exists := p.SrcTableIDNameMapping[msg.RelationID] + tableName, exists := p.srcTableIDNameMapping[msg.RelationID] if !exists { return nil, nil } @@ -382,7 +420,8 @@ func (p *PostgresCDCSource) processUpdateMessage( rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { - return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) + log.Warnf("unknown relation id: %d", msg.RelationID) + return nil, nil } // create empty map of string to interface{} @@ -400,18 +439,18 @@ func (p *PostgresCDCSource) processUpdateMessage( CheckPointID: int64(lsn), OldItems: oldItems, NewItems: newItems, - DestinationTableName: p.TableNameMapping[tableName], + DestinationTableName: p.tableNameMapping[tableName], SourceTableName: tableName, UnchangedToastColumns: unchangedToastColumns, }, nil } // processDeleteMessage processes a delete message and returns a DeleteRecord -func (p *PostgresCDCSource) processDeleteMessage( +func (p *postgresCDCSource) processDeleteMessage( lsn pglogrepl.LSN, msg *pglogrepl.DeleteMessage, ) (model.Record, error) { - tableName, exists := p.SrcTableIDNameMapping[msg.RelationID] + tableName, exists := p.srcTableIDNameMapping[msg.RelationID] if !exists { return nil, nil } @@ -421,7 +460,8 @@ func (p *PostgresCDCSource) processDeleteMessage( rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { - return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) + log.Warnf("unknown relation id: %d", msg.RelationID) + return nil, nil } // create empty map of string to interface{} @@ -433,7 +473,7 @@ func (p *PostgresCDCSource) processDeleteMessage( return &model.DeleteRecord{ CheckPointID: int64(lsn), Items: items, - DestinationTableName: p.TableNameMapping[tableName], + DestinationTableName: p.tableNameMapping[tableName], SourceTableName: tableName, UnchangedToastColumns: unchangedToastColumns, }, nil @@ -446,7 +486,7 @@ It takes a tuple and a relation message as input and returns 1. a map of column names to values and 2. a string slice of unchanged TOAST column names */ -func (p *PostgresCDCSource) convertTupleToMap( +func (p *postgresCDCSource) convertTupleToMap( tuple *pglogrepl.TupleData, rel *protos.RelationMessage, ) (*model.RecordItems, map[string]bool, error) { @@ -487,7 +527,7 @@ func (p *PostgresCDCSource) convertTupleToMap( return items, unchangedToastColumns, nil } -func (p *PostgresCDCSource) decodeColumnData(data []byte, dataType uint32, formatCode int16) (*qvalue.QValue, error) { +func (p *postgresCDCSource) decodeColumnData(data []byte, dataType uint32, formatCode int16) (*qvalue.QValue, error) { var parsedData any var err error if dt, ok := p.typeMap.TypeForOID(dataType); ok { @@ -532,7 +572,7 @@ func convertRelationMessageToProto(msg *pglogrepl.RelationMessage) *protos.Relat } // processRelationMessage processes a delete message and returns a TableSchemaDelta -func (p *PostgresCDCSource) processRelationMessage( +func (p *postgresCDCSource) processRelationMessage( lsn pglogrepl.LSN, currRel *protos.RelationMessage, ) (model.Record, error) { @@ -549,10 +589,8 @@ func (p *PostgresCDCSource) processRelationMessage( } schemaDelta := &protos.TableSchemaDelta{ - // set it to the source table for now, so we can update the schema on the source side - // then at the Workflow level we set it t - SrcTableName: p.SrcTableIDNameMapping[currRel.RelationId], - DstTableName: p.TableNameMapping[p.SrcTableIDNameMapping[currRel.RelationId]], + SrcTableName: p.srcTableIDNameMapping[currRel.RelationId], + DstTableName: p.tableNameMapping[p.srcTableIDNameMapping[currRel.RelationId]], AddedColumns: make([]*protos.DeltaAddedColumn, 0), DroppedColumns: make([]string, 0), } diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index f50687ff98..b09ae6d30b 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -70,6 +70,9 @@ const ( dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=$1" + + getAllTablesInSchemaSQL = `SELECT table_name FROM information_schema.tables WHERE + table_schema=$1 AND table_type='BASE TABLE'` ) // getRelIDForTable returns the relation ID for a table. @@ -203,23 +206,25 @@ func (c *PostgresConnector) createSlotAndPublication( publication string, tableNameMapping map[string]string, doInitialCopy bool, + schemas []string, ) error { - /* - iterating through source tables and creating a publication. - expecting tablenames to be schema qualified - */ - srcTableNames := make([]string, 0, len(tableNameMapping)) - for srcTableName := range tableNameMapping { - if len(strings.Split(srcTableName, ".")) != 2 { - return fmt.Errorf("source tables identifier is invalid: %v", srcTableName) - } - srcTableNames = append(srcTableNames, srcTableName) - } - tableNameString := strings.Join(srcTableNames, ", ") if !s.PublicationExists { - // Create the publication to help filter changes only for the given tables - stmt := fmt.Sprintf("CREATE PUBLICATION %s FOR TABLE %s", publication, tableNameString) + var stmt string + if len(schemas) > 0 { + // Create the publication to filter changes for all tables in the given schemas + stmt = fmt.Sprintf("CREATE PUBLICATION %s FOR TABLES IN SCHEMA %s", + publication, strings.Join(schemas, ",")) + } else { + // Create the publication to filter changes only for the given tables + /* + iterating through source tables and creating a publication. + expecting tablenames to be schema qualified + */ + tableNameString := strings.Join(maps.Keys(tableNameMapping), ", ") + + stmt = fmt.Sprintf("CREATE PUBLICATION %s FOR TABLE %s", publication, tableNameString) + } _, err := c.pool.Exec(c.ctx, stmt) if err != nil { log.Warnf("Error creating publication '%s': %v", publication, err) diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index f37fc88bdf..f19a3aec79 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -20,6 +20,7 @@ import ( "github.com/jackc/pgx/v5/pgconn" "github.com/jackc/pgx/v5/pgxpool" log "github.com/sirupsen/logrus" + "go.temporal.io/sdk/activity" "golang.org/x/exp/maps" ) @@ -172,7 +173,7 @@ func (c *PostgresConnector) GetLastOffset(jobName string) (*protos.LastSyncState } // PullRecords pulls records from the source. -func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { +func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithDeltaInfo, error) { // Slotname would be the job name prefixed with "peerflow_slot_" slotName := fmt.Sprintf("peerflow_slot_%s", req.FlowJobName) if req.OverrideReplicationSlotName != "" { @@ -209,21 +210,23 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R "flowName": req.FlowJobName, }).Infof("PullRecords: performed checks for slot and publication") - cdc, err := NewPostgresCDCSource(&PostgresCDCConfig{ - AppContext: c.ctx, - Connection: c.replPool, - SrcTableIDNameMapping: req.SrcTableIDNameMapping, - Slot: slotName, - Publication: publicationName, - TableNameMapping: req.TableNameMapping, - RelationMessageMapping: req.RelationMessageMapping, + cdc, err := NewPostgresCDCSource(&postgresCDCConfig{ + appContext: c.ctx, + connection: c.replPool, + srcTableIDNameMapping: req.SrcTableIDNameMapping, + slot: slotName, + publication: publicationName, + tableNameMapping: req.TableNameMapping, + relationMessageMapping: req.RelationMessageMapping, + schemas: req.Schemas, + allowTableAdditions: req.AllowTableAdditions, }) if err != nil { return nil, fmt.Errorf("failed to create cdc source: %w", err) } startTime := time.Now() - recordsWithSchemaDelta, err := cdc.PullRecords(req) + recordsWithDelta, err := cdc.PullRecords(req) if err != nil { return nil, err } @@ -232,9 +235,9 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R if err != nil { return nil, err } - metrics.LogPullMetrics(c.ctx, req.FlowJobName, recordsWithSchemaDelta.RecordBatch, + metrics.LogPullMetrics(c.ctx, req.FlowJobName, recordsWithDelta.RecordBatch, totalRecordsAtSource, time.Since(startTime)) - if len(recordsWithSchemaDelta.RecordBatch.Records) > 0 { + if len(recordsWithDelta.RecordBatch.Records) > 0 { cdcMirrorMonitor, ok := c.ctx.Value(shared.CDCMirrorMonitorKey).(*monitoring.CatalogMirrorMonitor) if ok { latestLSN, err := c.getCurrentLSN() @@ -248,11 +251,11 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R } } - return recordsWithSchemaDelta, nil + return recordsWithDelta, nil } // SyncRecords pushes records to the destination. -func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*protos.SyncResponse, error) { rawTableIdentifier := getRawTableIdentifier(req.FlowJobName) log.WithFields(log.Fields{ "flowName": req.FlowJobName, @@ -338,9 +341,9 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S } if len(records) == 0 { - return &model.SyncResponse{ - FirstSyncedCheckPointID: 0, - LastSyncedCheckPointID: 0, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: 0, + LastSyncedCheckpointId: 0, NumRecordsSynced: 0, }, nil } @@ -387,11 +390,11 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S return nil, err } - return &model.SyncResponse{ - FirstSyncedCheckPointID: firstCP, - LastSyncedCheckPointID: lastCP, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: firstCP, + LastSyncedCheckpointId: lastCP, NumRecordsSynced: int64(len(records)), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchId: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, }, nil } @@ -749,7 +752,9 @@ func (c *PostgresConnector) EnsurePullability(req *protos.EnsurePullabilityBatch utils.RecordHeartbeatWithRecover(c.ctx, fmt.Sprintf("ensured pullability table %s", tableName)) } - return &protos.EnsurePullabilityBatchOutput{TableIdentifierMapping: tableIdentifierMapping}, nil + return &protos.EnsurePullabilityBatchOutput{ + TableIdentifierMapping: tableIdentifierMapping, + }, nil } // SetupReplication sets up replication for the source connector. @@ -780,7 +785,7 @@ func (c *PostgresConnector) SetupReplication(signal *SlotSignal, req *protos.Set // Create the replication slot and publication err = c.createSlotAndPublication(signal, exists, - slotName, publicationName, req.TableNameMapping, req.DoInitialCopy) + slotName, publicationName, req.TableNameMapping, req.DoInitialCopy, req.Schemas) if err != nil { return fmt.Errorf("error creating replication slot and publication: %w", err) } @@ -870,3 +875,41 @@ func parseSchemaTable(tableName string) (*SchemaTable, error) { Table: parts[1], }, nil } + +// if the functions are being called outside the context of a Temporal workflow, +// activity.RecordHeartbeat panics, this is a bandaid for that. +func (c *PostgresConnector) recordHeartbeatWithRecover(details ...interface{}) { + defer func() { + if r := recover(); r != nil { + log.Warnln("ignoring panic from activity.RecordHeartbeat") + log.Warnln("this can happen when function is invoked outside of a Temporal workflow") + } + }() + activity.RecordHeartbeat(c.ctx, details...) +} + +func (c *PostgresConnector) ListTablesInSchemas(schemas []string) (map[string][]string, error) { + schemaTablesMap := make(map[string][]string) + + for _, schema := range schemas { + tables := make([]string, 0) + rows, err := c.pool.Query(c.ctx, getAllTablesInSchemaSQL, schema) + if err != nil { + return nil, fmt.Errorf("error while retrieving table names in schema %s: %w", schema, err) + } + defer rows.Close() + + var tableName string + // Process the rows and populate the array + for rows.Next() { + err := rows.Scan(&tableName) + if err != nil { + log.Fatalf("Failed to scan row: %v", err) + return nil, err + } + tables = append(tables, tableName) + } + schemaTablesMap[schema] = tables + } + return schemaTablesMap, nil +} diff --git a/flow/connectors/postgres/postgres_cdc_test.go b/flow/connectors/postgres/postgres_cdc_test.go index 2735c5ff49..9b8edf786f 100644 --- a/flow/connectors/postgres/postgres_cdc_test.go +++ b/flow/connectors/postgres/postgres_cdc_test.go @@ -438,7 +438,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { }) suite.failTestError(err) suite.dropTable(nonExistentFlowSrcTableName) - recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: nonExistentFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -448,7 +448,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { TableNameSchemaMapping: tableNameSchemaMapping, RelationMessageMapping: relationMessageMapping, }) - suite.Nil(recordsWithSchemaDelta) + suite.Nil(recordsWithDelta) suite.Errorf( err, "error while closing statement batch: ERROR: relation \"%s\" does not exist (SQLSTATE 42P01)", @@ -513,8 +513,8 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { tableNameSchemaMapping[simpleHappyFlowDstTableName] = tableNameSchema.TableNameSchemaMapping[simpleHappyFlowSrcTableName] - // pulling with no recordsWithSchemaDelta. - recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + // pulling with no recordsWithDelta. + recordsWithDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -525,15 +525,15 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Equal(0, len(recordsWithSchemaDelta.RecordBatch.Records)) - suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) - suite.Equal(int64(0), recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) - suite.Equal(int64(0), recordsWithSchemaDelta.RecordBatch.LastCheckPointID) - relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping + suite.Equal(0, len(recordsWithDelta.RecordBatch.Records)) + suite.Nil(recordsWithDelta.MirrorDelta) + suite.Equal(int64(0), recordsWithDelta.RecordBatch.FirstCheckPointID) + suite.Equal(int64(0), recordsWithDelta.RecordBatch.LastCheckPointID) + relationMessageMapping = recordsWithDelta.RelationMessageMapping // pulling after inserting records. suite.insertSimpleRecords(simpleHappyFlowSrcTableName) - recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -544,21 +544,22 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) - suite.validateInsertedSimpleRecords(recordsWithSchemaDelta.RecordBatch.Records, + suite.Len(recordsWithDelta.RelationMessageMapping, 1) + suite.Nil(recordsWithDelta.MirrorDelta) + suite.validateInsertedSimpleRecords(recordsWithDelta.RecordBatch.Records, simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) - suite.Greater(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, int64(0)) - suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, - recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) - currentCheckPointID := recordsWithSchemaDelta.RecordBatch.LastCheckPointID - relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping + suite.Greater(recordsWithDelta.RecordBatch.FirstCheckPointID, int64(0)) + suite.GreaterOrEqual(recordsWithDelta.RecordBatch.LastCheckPointID, + recordsWithDelta.RecordBatch.FirstCheckPointID) + currentCheckPointID := recordsWithDelta.RecordBatch.LastCheckPointID + relationMessageMapping = recordsWithDelta.RelationMessageMapping // pulling after mutating records. suite.mutateSimpleRecords(simpleHappyFlowSrcTableName) - recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: &protos.LastSyncState{ - Checkpoint: recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + Checkpoint: recordsWithDelta.RecordBatch.LastCheckPointID, LastSyncedAt: nil, }, IdleTimeout: 5 * time.Second, @@ -569,12 +570,13 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) - suite.validateSimpleMutatedRecords(recordsWithSchemaDelta.RecordBatch.Records, + suite.Len(recordsWithDelta.RelationMessageMapping, 1) + suite.Nil(recordsWithDelta.MirrorDelta) + suite.validateSimpleMutatedRecords(recordsWithDelta.RecordBatch.Records, simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) - suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, currentCheckPointID) - suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, - recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) + suite.GreaterOrEqual(recordsWithDelta.RecordBatch.FirstCheckPointID, currentCheckPointID) + suite.GreaterOrEqual(recordsWithDelta.RecordBatch.LastCheckPointID, + recordsWithDelta.RecordBatch.FirstCheckPointID) err = suite.connector.PullFlowCleanup(simpleHappyFlowName) suite.failTestError(err) @@ -698,6 +700,8 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) + suite.Len(records.RelationMessageMapping, 1) + suite.Nil(records.MirrorDelta) require.Equal(suite.T(), 1, len(records.RecordBatch.Records)) items := records.RecordBatch.Records[0].GetItems() @@ -774,7 +778,7 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { tableNameSchema.TableNameSchemaMapping[toastHappyFlowSrcTableName] suite.insertToastRecords(toastHappyFlowSrcTableName) - recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: nil, IdleTimeout: 10 * time.Second, @@ -785,30 +789,20 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ - FlowJobName: toastHappyFlowName, - LastSyncState: nil, - IdleTimeout: 10 * time.Second, - MaxBatchSize: 100, - SrcTableIDNameMapping: relIDTableNameMapping, - TableNameMapping: tableNameMapping, - TableNameSchemaMapping: tableNameSchemaMapping, - RelationMessageMapping: relationMessageMapping, - }) - suite.failTestError(err) - suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) - suite.validateInsertedToastRecords(recordsWithSchemaDelta.RecordBatch.Records, + suite.Len(recordsWithDelta.RelationMessageMapping, 1) + suite.Nil(recordsWithDelta.MirrorDelta) + suite.validateInsertedToastRecords(recordsWithDelta.RecordBatch.Records, toastHappyFlowSrcTableName, toastHappyFlowDstTableName) - suite.Greater(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, int64(0)) - suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, - recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) - relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping + suite.Greater(recordsWithDelta.RecordBatch.FirstCheckPointID, int64(0)) + suite.GreaterOrEqual(recordsWithDelta.RecordBatch.LastCheckPointID, + recordsWithDelta.RecordBatch.FirstCheckPointID) + relationMessageMapping = recordsWithDelta.RelationMessageMapping suite.mutateToastRecords(toastHappyFlowSrcTableName) - recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: &protos.LastSyncState{ - Checkpoint: recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + Checkpoint: recordsWithDelta.RecordBatch.LastCheckPointID, LastSyncedAt: nil, }, IdleTimeout: 10 * time.Second, @@ -819,7 +813,7 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateMutatedToastRecords(recordsWithSchemaDelta.RecordBatch.Records, toastHappyFlowSrcTableName, + suite.validateMutatedToastRecords(recordsWithDelta.RecordBatch.Records, toastHappyFlowSrcTableName, toastHappyFlowDstTableName) err = suite.connector.PullFlowCleanup(toastHappyFlowName) diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 4310764333..30dc8b7acc 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -416,11 +416,11 @@ func (c *SnowflakeConnector) ReplayTableSchemaDelta(flowJobName string, schemaDe return nil } -func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.SyncResponse, error) { +func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*protos.SyncResponse, error) { if len(req.Records.Records) == 0 { - return &model.SyncResponse{ - FirstSyncedCheckPointID: 0, - LastSyncedCheckPointID: 0, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: 0, + LastSyncedCheckpointId: 0, NumRecordsSynced: 0, }, nil } @@ -450,7 +450,7 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. } }() - var res *model.SyncResponse + var res *protos.SyncResponse if req.SyncMode == protos.QRepSyncMode_QREP_SYNC_MODE_STORAGE_AVRO { res, err = c.syncRecordsViaAvro(req, rawTableIdentifier, syncBatchID) if err != nil { @@ -464,7 +464,7 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. } // updating metadata with new offset and syncBatchID - err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckPointID, syncBatchID, syncRecordsTx) + err = c.updateSyncMetadata(req.FlowJobName, res.LastSyncedCheckpointId, syncBatchID, syncRecordsTx) if err != nil { return nil, err } @@ -478,7 +478,7 @@ func (c *SnowflakeConnector) SyncRecords(req *model.SyncRecordsRequest) (*model. } func (c *SnowflakeConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, rawTableIdentifier string, - syncBatchID int64, syncRecordsTx *sql.Tx) (*model.SyncResponse, error) { + syncBatchID int64, syncRecordsTx *sql.Tx) (*protos.SyncResponse, error) { records := make([]snowflakeRawRecord, 0) tableNameRowsMapping := make(map[string]uint32) @@ -574,17 +574,17 @@ func (c *SnowflakeConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, ra } metrics.LogSyncMetrics(c.ctx, req.FlowJobName, int64(numRecords), time.Since(startTime)) - return &model.SyncResponse{ - FirstSyncedCheckPointID: firstCP, - LastSyncedCheckPointID: lastCP, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: firstCP, + LastSyncedCheckpointId: lastCP, NumRecordsSynced: int64(len(records)), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchId: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, }, nil } func (c *SnowflakeConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, rawTableIdentifier string, - syncBatchID int64) (*model.SyncResponse, error) { + syncBatchID int64) (*protos.SyncResponse, error) { recordStream := model.NewQRecordStream(len(req.Records.Records)) err := recordStream.SetSchema(&model.QRecordSchema{ @@ -782,11 +782,11 @@ func (c *SnowflakeConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, r } metrics.LogSyncMetrics(c.ctx, req.FlowJobName, int64(numRecords), time.Since(startTime)) - return &model.SyncResponse{ - FirstSyncedCheckPointID: firstCP, - LastSyncedCheckPointID: lastCP, + return &protos.SyncResponse{ + FirstSyncedCheckpointId: firstCP, + LastSyncedCheckpointId: lastCP, NumRecordsSynced: int64(len(req.Records.Records)), - CurrentSyncBatchID: syncBatchID, + CurrentSyncBatchId: syncBatchID, TableNameRowsMapping: tableNameRowsMapping, }, nil } diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index dd6de4ccaf..69a5b8b006 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -21,6 +21,55 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +type MappingType int32 + +const ( + MappingType_UNKNOWN MappingType = 0 + MappingType_TABLE MappingType = 1 + MappingType_SCHEMA MappingType = 2 +) + +// Enum value maps for MappingType. +var ( + MappingType_name = map[int32]string{ + 0: "UNKNOWN", + 1: "TABLE", + 2: "SCHEMA", + } + MappingType_value = map[string]int32{ + "UNKNOWN": 0, + "TABLE": 1, + "SCHEMA": 2, + } +) + +func (x MappingType) Enum() *MappingType { + p := new(MappingType) + *p = x + return p +} + +func (x MappingType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (MappingType) Descriptor() protoreflect.EnumDescriptor { + return file_flow_proto_enumTypes[0].Descriptor() +} + +func (MappingType) Type() protoreflect.EnumType { + return &file_flow_proto_enumTypes[0] +} + +func (x MappingType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use MappingType.Descriptor instead. +func (MappingType) EnumDescriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{0} +} + // protos for qrep type QRepSyncMode int32 @@ -52,11 +101,11 @@ func (x QRepSyncMode) String() string { } func (QRepSyncMode) Descriptor() protoreflect.EnumDescriptor { - return file_flow_proto_enumTypes[0].Descriptor() + return file_flow_proto_enumTypes[1].Descriptor() } func (QRepSyncMode) Type() protoreflect.EnumType { - return &file_flow_proto_enumTypes[0] + return &file_flow_proto_enumTypes[1] } func (x QRepSyncMode) Number() protoreflect.EnumNumber { @@ -65,7 +114,7 @@ func (x QRepSyncMode) Number() protoreflect.EnumNumber { // Deprecated: Use QRepSyncMode.Descriptor instead. func (QRepSyncMode) EnumDescriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{0} + return file_flow_proto_rawDescGZIP(), []int{1} } type QRepWriteType int32 @@ -102,11 +151,11 @@ func (x QRepWriteType) String() string { } func (QRepWriteType) Descriptor() protoreflect.EnumDescriptor { - return file_flow_proto_enumTypes[1].Descriptor() + return file_flow_proto_enumTypes[2].Descriptor() } func (QRepWriteType) Type() protoreflect.EnumType { - return &file_flow_proto_enumTypes[1] + return &file_flow_proto_enumTypes[2] } func (x QRepWriteType) Number() protoreflect.EnumNumber { @@ -115,62 +164,7 @@ func (x QRepWriteType) Number() protoreflect.EnumNumber { // Deprecated: Use QRepWriteType.Descriptor instead. func (QRepWriteType) EnumDescriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{1} -} - -type TableNameMapping struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - SourceTableName string `protobuf:"bytes,1,opt,name=source_table_name,json=sourceTableName,proto3" json:"source_table_name,omitempty"` - DestinationTableName string `protobuf:"bytes,2,opt,name=destination_table_name,json=destinationTableName,proto3" json:"destination_table_name,omitempty"` -} - -func (x *TableNameMapping) Reset() { - *x = TableNameMapping{} - if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *TableNameMapping) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*TableNameMapping) ProtoMessage() {} - -func (x *TableNameMapping) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use TableNameMapping.ProtoReflect.Descriptor instead. -func (*TableNameMapping) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{0} -} - -func (x *TableNameMapping) GetSourceTableName() string { - if x != nil { - return x.SourceTableName - } - return "" -} - -func (x *TableNameMapping) GetDestinationTableName() string { - if x != nil { - return x.DestinationTableName - } - return "" + return file_flow_proto_rawDescGZIP(), []int{2} } type RelationMessageColumn struct { @@ -186,7 +180,7 @@ type RelationMessageColumn struct { func (x *RelationMessageColumn) Reset() { *x = RelationMessageColumn{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[1] + mi := &file_flow_proto_msgTypes[0] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -199,7 +193,7 @@ func (x *RelationMessageColumn) String() string { func (*RelationMessageColumn) ProtoMessage() {} func (x *RelationMessageColumn) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[1] + mi := &file_flow_proto_msgTypes[0] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -212,7 +206,7 @@ func (x *RelationMessageColumn) ProtoReflect() protoreflect.Message { // Deprecated: Use RelationMessageColumn.ProtoReflect.Descriptor instead. func (*RelationMessageColumn) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{1} + return file_flow_proto_rawDescGZIP(), []int{0} } func (x *RelationMessageColumn) GetFlags() uint32 { @@ -249,7 +243,7 @@ type RelationMessage struct { func (x *RelationMessage) Reset() { *x = RelationMessage{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[2] + mi := &file_flow_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -262,7 +256,7 @@ func (x *RelationMessage) String() string { func (*RelationMessage) ProtoMessage() {} func (x *RelationMessage) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[2] + mi := &file_flow_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -275,7 +269,7 @@ func (x *RelationMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use RelationMessage.ProtoReflect.Descriptor instead. func (*RelationMessage) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{2} + return file_flow_proto_rawDescGZIP(), []int{1} } func (x *RelationMessage) GetRelationId() uint32 { @@ -304,10 +298,11 @@ type FlowConnectionConfigs struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Source *Peer `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"` - Destination *Peer `protobuf:"bytes,2,opt,name=destination,proto3" json:"destination,omitempty"` - FlowJobName string `protobuf:"bytes,3,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` - TableSchema *TableSchema `protobuf:"bytes,4,opt,name=table_schema,json=tableSchema,proto3" json:"table_schema,omitempty"` + Source *Peer `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"` + Destination *Peer `protobuf:"bytes,2,opt,name=destination,proto3" json:"destination,omitempty"` + FlowJobName string `protobuf:"bytes,3,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` + TableSchema *TableSchema `protobuf:"bytes,4,opt,name=table_schema,json=tableSchema,proto3" json:"table_schema,omitempty"` + // if MappingType is TABLE, this contains the table level mappings. TableNameMapping map[string]string `protobuf:"bytes,5,rep,name=table_name_mapping,json=tableNameMapping,proto3" json:"table_name_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` SrcTableIdNameMapping map[uint32]string `protobuf:"bytes,6,rep,name=src_table_id_name_mapping,json=srcTableIdNameMapping,proto3" json:"src_table_id_name_mapping,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` TableNameSchemaMapping map[string]*TableSchema `protobuf:"bytes,7,rep,name=table_name_schema_mapping,json=tableNameSchemaMapping,proto3" json:"table_name_schema_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` @@ -329,14 +324,18 @@ type FlowConnectionConfigs struct { SoftDelete bool `protobuf:"varint,19,opt,name=soft_delete,json=softDelete,proto3" json:"soft_delete,omitempty"` ReplicationSlotName string `protobuf:"bytes,20,opt,name=replication_slot_name,json=replicationSlotName,proto3" json:"replication_slot_name,omitempty"` // the below two are for eventhub only - PushBatchSize int64 `protobuf:"varint,21,opt,name=push_batch_size,json=pushBatchSize,proto3" json:"push_batch_size,omitempty"` - PushParallelism int64 `protobuf:"varint,22,opt,name=push_parallelism,json=pushParallelism,proto3" json:"push_parallelism,omitempty"` + PushBatchSize int64 `protobuf:"varint,21,opt,name=push_batch_size,json=pushBatchSize,proto3" json:"push_batch_size,omitempty"` + PushParallelism int64 `protobuf:"varint,22,opt,name=push_parallelism,json=pushParallelism,proto3" json:"push_parallelism,omitempty"` + MappingType MappingType `protobuf:"varint,23,opt,name=mapping_type,json=mappingType,proto3,enum=peerdb_flow.MappingType" json:"mapping_type,omitempty"` + // if MappingType is SCHEMA, this contains the schema level mappings. + SchemaMapping map[string]string `protobuf:"bytes,24,rep,name=schema_mapping,json=schemaMapping,proto3" json:"schema_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + AllowTableAdditions bool `protobuf:"varint,25,opt,name=allow_table_additions,json=allowTableAdditions,proto3" json:"allow_table_additions,omitempty"` } func (x *FlowConnectionConfigs) Reset() { *x = FlowConnectionConfigs{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[3] + mi := &file_flow_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -349,7 +348,7 @@ func (x *FlowConnectionConfigs) String() string { func (*FlowConnectionConfigs) ProtoMessage() {} func (x *FlowConnectionConfigs) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[3] + mi := &file_flow_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -362,7 +361,7 @@ func (x *FlowConnectionConfigs) ProtoReflect() protoreflect.Message { // Deprecated: Use FlowConnectionConfigs.ProtoReflect.Descriptor instead. func (*FlowConnectionConfigs) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{3} + return file_flow_proto_rawDescGZIP(), []int{2} } func (x *FlowConnectionConfigs) GetSource() *Peer { @@ -519,6 +518,27 @@ func (x *FlowConnectionConfigs) GetPushParallelism() int64 { return 0 } +func (x *FlowConnectionConfigs) GetMappingType() MappingType { + if x != nil { + return x.MappingType + } + return MappingType_UNKNOWN +} + +func (x *FlowConnectionConfigs) GetSchemaMapping() map[string]string { + if x != nil { + return x.SchemaMapping + } + return nil +} + +func (x *FlowConnectionConfigs) GetAllowTableAdditions() bool { + if x != nil { + return x.AllowTableAdditions + } + return false +} + type SyncFlowOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -531,7 +551,7 @@ type SyncFlowOptions struct { func (x *SyncFlowOptions) Reset() { *x = SyncFlowOptions{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[4] + mi := &file_flow_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -544,7 +564,7 @@ func (x *SyncFlowOptions) String() string { func (*SyncFlowOptions) ProtoMessage() {} func (x *SyncFlowOptions) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[4] + mi := &file_flow_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -557,7 +577,7 @@ func (x *SyncFlowOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SyncFlowOptions.ProtoReflect.Descriptor instead. func (*SyncFlowOptions) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{4} + return file_flow_proto_rawDescGZIP(), []int{3} } func (x *SyncFlowOptions) GetBatchSize() int32 { @@ -585,7 +605,7 @@ type NormalizeFlowOptions struct { func (x *NormalizeFlowOptions) Reset() { *x = NormalizeFlowOptions{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[5] + mi := &file_flow_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -598,7 +618,7 @@ func (x *NormalizeFlowOptions) String() string { func (*NormalizeFlowOptions) ProtoMessage() {} func (x *NormalizeFlowOptions) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[5] + mi := &file_flow_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -611,7 +631,7 @@ func (x *NormalizeFlowOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use NormalizeFlowOptions.ProtoReflect.Descriptor instead. func (*NormalizeFlowOptions) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{5} + return file_flow_proto_rawDescGZIP(), []int{4} } func (x *NormalizeFlowOptions) GetBatchSize() int32 { @@ -633,7 +653,7 @@ type LastSyncState struct { func (x *LastSyncState) Reset() { *x = LastSyncState{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[6] + mi := &file_flow_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -646,7 +666,7 @@ func (x *LastSyncState) String() string { func (*LastSyncState) ProtoMessage() {} func (x *LastSyncState) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[6] + mi := &file_flow_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -659,7 +679,7 @@ func (x *LastSyncState) ProtoReflect() protoreflect.Message { // Deprecated: Use LastSyncState.ProtoReflect.Descriptor instead. func (*LastSyncState) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{6} + return file_flow_proto_rawDescGZIP(), []int{5} } func (x *LastSyncState) GetCheckpoint() int64 { @@ -690,7 +710,7 @@ type StartFlowInput struct { func (x *StartFlowInput) Reset() { *x = StartFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[7] + mi := &file_flow_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -703,7 +723,7 @@ func (x *StartFlowInput) String() string { func (*StartFlowInput) ProtoMessage() {} func (x *StartFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[7] + mi := &file_flow_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -716,7 +736,7 @@ func (x *StartFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use StartFlowInput.ProtoReflect.Descriptor instead. func (*StartFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{7} + return file_flow_proto_rawDescGZIP(), []int{6} } func (x *StartFlowInput) GetLastSyncState() *LastSyncState { @@ -758,7 +778,7 @@ type StartNormalizeInput struct { func (x *StartNormalizeInput) Reset() { *x = StartNormalizeInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[8] + mi := &file_flow_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -771,7 +791,7 @@ func (x *StartNormalizeInput) String() string { func (*StartNormalizeInput) ProtoMessage() {} func (x *StartNormalizeInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[8] + mi := &file_flow_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -784,7 +804,7 @@ func (x *StartNormalizeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use StartNormalizeInput.ProtoReflect.Descriptor instead. func (*StartNormalizeInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{8} + return file_flow_proto_rawDescGZIP(), []int{7} } func (x *StartNormalizeInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { @@ -806,7 +826,7 @@ type GetLastSyncedIDInput struct { func (x *GetLastSyncedIDInput) Reset() { *x = GetLastSyncedIDInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[9] + mi := &file_flow_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -819,7 +839,7 @@ func (x *GetLastSyncedIDInput) String() string { func (*GetLastSyncedIDInput) ProtoMessage() {} func (x *GetLastSyncedIDInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[9] + mi := &file_flow_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -832,7 +852,7 @@ func (x *GetLastSyncedIDInput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetLastSyncedIDInput.ProtoReflect.Descriptor instead. func (*GetLastSyncedIDInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{9} + return file_flow_proto_rawDescGZIP(), []int{8} } func (x *GetLastSyncedIDInput) GetPeerConnectionConfig() *Peer { @@ -849,69 +869,6 @@ func (x *GetLastSyncedIDInput) GetFlowJobName() string { return "" } -type EnsurePullabilityInput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - PeerConnectionConfig *Peer `protobuf:"bytes,1,opt,name=peer_connection_config,json=peerConnectionConfig,proto3" json:"peer_connection_config,omitempty"` - FlowJobName string `protobuf:"bytes,2,opt,name=flow_job_name,json=flowJobName,proto3" json:"flow_job_name,omitempty"` - SourceTableIdentifier string `protobuf:"bytes,3,opt,name=source_table_identifier,json=sourceTableIdentifier,proto3" json:"source_table_identifier,omitempty"` -} - -func (x *EnsurePullabilityInput) Reset() { - *x = EnsurePullabilityInput{} - if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *EnsurePullabilityInput) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*EnsurePullabilityInput) ProtoMessage() {} - -func (x *EnsurePullabilityInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[10] - 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 EnsurePullabilityInput.ProtoReflect.Descriptor instead. -func (*EnsurePullabilityInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{10} -} - -func (x *EnsurePullabilityInput) GetPeerConnectionConfig() *Peer { - if x != nil { - return x.PeerConnectionConfig - } - return nil -} - -func (x *EnsurePullabilityInput) GetFlowJobName() string { - if x != nil { - return x.FlowJobName - } - return "" -} - -func (x *EnsurePullabilityInput) GetSourceTableIdentifier() string { - if x != nil { - return x.SourceTableIdentifier - } - return "" -} - type EnsurePullabilityBatchInput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -925,7 +882,7 @@ type EnsurePullabilityBatchInput struct { func (x *EnsurePullabilityBatchInput) Reset() { *x = EnsurePullabilityBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[11] + mi := &file_flow_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -938,7 +895,7 @@ func (x *EnsurePullabilityBatchInput) String() string { func (*EnsurePullabilityBatchInput) ProtoMessage() {} func (x *EnsurePullabilityBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[11] + mi := &file_flow_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -951,7 +908,7 @@ func (x *EnsurePullabilityBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityBatchInput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{11} + return file_flow_proto_rawDescGZIP(), []int{9} } func (x *EnsurePullabilityBatchInput) GetPeerConnectionConfig() *Peer { @@ -986,7 +943,7 @@ type PostgresTableIdentifier struct { func (x *PostgresTableIdentifier) Reset() { *x = PostgresTableIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[12] + mi := &file_flow_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -999,7 +956,7 @@ func (x *PostgresTableIdentifier) String() string { func (*PostgresTableIdentifier) ProtoMessage() {} func (x *PostgresTableIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[12] + mi := &file_flow_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1012,7 +969,7 @@ func (x *PostgresTableIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use PostgresTableIdentifier.ProtoReflect.Descriptor instead. func (*PostgresTableIdentifier) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{12} + return file_flow_proto_rawDescGZIP(), []int{10} } func (x *PostgresTableIdentifier) GetRelId() uint32 { @@ -1036,7 +993,7 @@ type TableIdentifier struct { func (x *TableIdentifier) Reset() { *x = TableIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[13] + mi := &file_flow_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1049,7 +1006,7 @@ func (x *TableIdentifier) String() string { func (*TableIdentifier) ProtoMessage() {} func (x *TableIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[13] + mi := &file_flow_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1062,7 +1019,7 @@ func (x *TableIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use TableIdentifier.ProtoReflect.Descriptor instead. func (*TableIdentifier) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{13} + return file_flow_proto_rawDescGZIP(), []int{11} } func (m *TableIdentifier) GetTableIdentifier() isTableIdentifier_TableIdentifier { @@ -1089,53 +1046,6 @@ type TableIdentifier_PostgresTableIdentifier struct { func (*TableIdentifier_PostgresTableIdentifier) isTableIdentifier_TableIdentifier() {} -type EnsurePullabilityOutput struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - TableIdentifier *TableIdentifier `protobuf:"bytes,1,opt,name=table_identifier,json=tableIdentifier,proto3" json:"table_identifier,omitempty"` -} - -func (x *EnsurePullabilityOutput) Reset() { - *x = EnsurePullabilityOutput{} - if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *EnsurePullabilityOutput) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*EnsurePullabilityOutput) ProtoMessage() {} - -func (x *EnsurePullabilityOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[14] - 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 EnsurePullabilityOutput.ProtoReflect.Descriptor instead. -func (*EnsurePullabilityOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{14} -} - -func (x *EnsurePullabilityOutput) GetTableIdentifier() *TableIdentifier { - if x != nil { - return x.TableIdentifier - } - return nil -} - type EnsurePullabilityBatchOutput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1147,7 +1057,7 @@ type EnsurePullabilityBatchOutput struct { func (x *EnsurePullabilityBatchOutput) Reset() { *x = EnsurePullabilityBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[15] + mi := &file_flow_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1160,7 +1070,7 @@ func (x *EnsurePullabilityBatchOutput) String() string { func (*EnsurePullabilityBatchOutput) ProtoMessage() {} func (x *EnsurePullabilityBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[15] + mi := &file_flow_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1173,7 +1083,7 @@ func (x *EnsurePullabilityBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityBatchOutput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{15} + return file_flow_proto_rawDescGZIP(), []int{12} } func (x *EnsurePullabilityBatchOutput) GetTableIdentifierMapping() map[string]*TableIdentifier { @@ -1196,12 +1106,14 @@ type SetupReplicationInput struct { DoInitialCopy bool `protobuf:"varint,5,opt,name=do_initial_copy,json=doInitialCopy,proto3" json:"do_initial_copy,omitempty"` ExistingPublicationName string `protobuf:"bytes,6,opt,name=existing_publication_name,json=existingPublicationName,proto3" json:"existing_publication_name,omitempty"` ExistingReplicationSlotName string `protobuf:"bytes,7,opt,name=existing_replication_slot_name,json=existingReplicationSlotName,proto3" json:"existing_replication_slot_name,omitempty"` + // if length > 0, ignore table name mapping and use schemas as basis of publication creation + Schemas []string `protobuf:"bytes,8,rep,name=schemas,proto3" json:"schemas,omitempty"` } func (x *SetupReplicationInput) Reset() { *x = SetupReplicationInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[16] + mi := &file_flow_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1214,7 +1126,7 @@ func (x *SetupReplicationInput) String() string { func (*SetupReplicationInput) ProtoMessage() {} func (x *SetupReplicationInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[16] + mi := &file_flow_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1227,7 +1139,7 @@ func (x *SetupReplicationInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupReplicationInput.ProtoReflect.Descriptor instead. func (*SetupReplicationInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{16} + return file_flow_proto_rawDescGZIP(), []int{13} } func (x *SetupReplicationInput) GetPeerConnectionConfig() *Peer { @@ -1279,6 +1191,13 @@ func (x *SetupReplicationInput) GetExistingReplicationSlotName() string { return "" } +func (x *SetupReplicationInput) GetSchemas() []string { + if x != nil { + return x.Schemas + } + return nil +} + type SetupReplicationOutput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1291,7 +1210,7 @@ type SetupReplicationOutput struct { func (x *SetupReplicationOutput) Reset() { *x = SetupReplicationOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[17] + mi := &file_flow_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1304,7 +1223,7 @@ func (x *SetupReplicationOutput) String() string { func (*SetupReplicationOutput) ProtoMessage() {} func (x *SetupReplicationOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[17] + mi := &file_flow_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1317,7 +1236,7 @@ func (x *SetupReplicationOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupReplicationOutput.ProtoReflect.Descriptor instead. func (*SetupReplicationOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{17} + return file_flow_proto_rawDescGZIP(), []int{14} } func (x *SetupReplicationOutput) GetSlotName() string { @@ -1348,7 +1267,7 @@ type CreateRawTableInput struct { func (x *CreateRawTableInput) Reset() { *x = CreateRawTableInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[18] + mi := &file_flow_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1361,7 +1280,7 @@ func (x *CreateRawTableInput) String() string { func (*CreateRawTableInput) ProtoMessage() {} func (x *CreateRawTableInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[18] + mi := &file_flow_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1374,7 +1293,7 @@ func (x *CreateRawTableInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateRawTableInput.ProtoReflect.Descriptor instead. func (*CreateRawTableInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{18} + return file_flow_proto_rawDescGZIP(), []int{15} } func (x *CreateRawTableInput) GetPeerConnectionConfig() *Peer { @@ -1416,7 +1335,7 @@ type CreateRawTableOutput struct { func (x *CreateRawTableOutput) Reset() { *x = CreateRawTableOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[19] + mi := &file_flow_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1429,7 +1348,7 @@ func (x *CreateRawTableOutput) String() string { func (*CreateRawTableOutput) ProtoMessage() {} func (x *CreateRawTableOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[19] + mi := &file_flow_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1442,7 +1361,7 @@ func (x *CreateRawTableOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateRawTableOutput.ProtoReflect.Descriptor instead. func (*CreateRawTableOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{19} + return file_flow_proto_rawDescGZIP(), []int{16} } func (x *CreateRawTableOutput) GetTableIdentifier() string { @@ -1468,7 +1387,7 @@ type TableSchema struct { func (x *TableSchema) Reset() { *x = TableSchema{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[20] + mi := &file_flow_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1481,7 +1400,7 @@ func (x *TableSchema) String() string { func (*TableSchema) ProtoMessage() {} func (x *TableSchema) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[20] + mi := &file_flow_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1494,7 +1413,7 @@ func (x *TableSchema) ProtoReflect() protoreflect.Message { // Deprecated: Use TableSchema.ProtoReflect.Descriptor instead. func (*TableSchema) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{20} + return file_flow_proto_rawDescGZIP(), []int{17} } func (x *TableSchema) GetTableIdentifier() string { @@ -1537,7 +1456,7 @@ type GetTableSchemaBatchInput struct { func (x *GetTableSchemaBatchInput) Reset() { *x = GetTableSchemaBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[21] + mi := &file_flow_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1550,7 +1469,7 @@ func (x *GetTableSchemaBatchInput) String() string { func (*GetTableSchemaBatchInput) ProtoMessage() {} func (x *GetTableSchemaBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[21] + mi := &file_flow_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1563,7 +1482,7 @@ func (x *GetTableSchemaBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTableSchemaBatchInput.ProtoReflect.Descriptor instead. func (*GetTableSchemaBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{21} + return file_flow_proto_rawDescGZIP(), []int{18} } func (x *GetTableSchemaBatchInput) GetPeerConnectionConfig() *Peer { @@ -1591,7 +1510,7 @@ type GetTableSchemaBatchOutput struct { func (x *GetTableSchemaBatchOutput) Reset() { *x = GetTableSchemaBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[22] + mi := &file_flow_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1604,7 +1523,7 @@ func (x *GetTableSchemaBatchOutput) String() string { func (*GetTableSchemaBatchOutput) ProtoMessage() {} func (x *GetTableSchemaBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[22] + mi := &file_flow_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1617,7 +1536,7 @@ func (x *GetTableSchemaBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTableSchemaBatchOutput.ProtoReflect.Descriptor instead. func (*GetTableSchemaBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{22} + return file_flow_proto_rawDescGZIP(), []int{19} } func (x *GetTableSchemaBatchOutput) GetTableNameSchemaMapping() map[string]*TableSchema { @@ -1640,7 +1559,7 @@ type SetupNormalizedTableInput struct { func (x *SetupNormalizedTableInput) Reset() { *x = SetupNormalizedTableInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[23] + mi := &file_flow_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1653,7 +1572,7 @@ func (x *SetupNormalizedTableInput) String() string { func (*SetupNormalizedTableInput) ProtoMessage() {} func (x *SetupNormalizedTableInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[23] + mi := &file_flow_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1666,7 +1585,7 @@ func (x *SetupNormalizedTableInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableInput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{23} + return file_flow_proto_rawDescGZIP(), []int{20} } func (x *SetupNormalizedTableInput) GetPeerConnectionConfig() *Peer { @@ -1702,7 +1621,7 @@ type SetupNormalizedTableBatchInput struct { func (x *SetupNormalizedTableBatchInput) Reset() { *x = SetupNormalizedTableBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[24] + mi := &file_flow_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1715,7 +1634,7 @@ func (x *SetupNormalizedTableBatchInput) String() string { func (*SetupNormalizedTableBatchInput) ProtoMessage() {} func (x *SetupNormalizedTableBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[24] + mi := &file_flow_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1728,7 +1647,7 @@ func (x *SetupNormalizedTableBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableBatchInput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{24} + return file_flow_proto_rawDescGZIP(), []int{21} } func (x *SetupNormalizedTableBatchInput) GetPeerConnectionConfig() *Peer { @@ -1757,7 +1676,7 @@ type SetupNormalizedTableOutput struct { func (x *SetupNormalizedTableOutput) Reset() { *x = SetupNormalizedTableOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[25] + mi := &file_flow_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1770,7 +1689,7 @@ func (x *SetupNormalizedTableOutput) String() string { func (*SetupNormalizedTableOutput) ProtoMessage() {} func (x *SetupNormalizedTableOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[25] + mi := &file_flow_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1783,7 +1702,7 @@ func (x *SetupNormalizedTableOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableOutput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{25} + return file_flow_proto_rawDescGZIP(), []int{22} } func (x *SetupNormalizedTableOutput) GetTableIdentifier() string { @@ -1811,7 +1730,7 @@ type SetupNormalizedTableBatchOutput struct { func (x *SetupNormalizedTableBatchOutput) Reset() { *x = SetupNormalizedTableBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[26] + mi := &file_flow_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1824,7 +1743,7 @@ func (x *SetupNormalizedTableBatchOutput) String() string { func (*SetupNormalizedTableBatchOutput) ProtoMessage() {} func (x *SetupNormalizedTableBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[26] + mi := &file_flow_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1837,7 +1756,7 @@ func (x *SetupNormalizedTableBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableBatchOutput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{26} + return file_flow_proto_rawDescGZIP(), []int{23} } func (x *SetupNormalizedTableBatchOutput) GetTableExistsMapping() map[string]bool { @@ -1860,7 +1779,7 @@ type IntPartitionRange struct { func (x *IntPartitionRange) Reset() { *x = IntPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[27] + mi := &file_flow_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1873,7 +1792,7 @@ func (x *IntPartitionRange) String() string { func (*IntPartitionRange) ProtoMessage() {} func (x *IntPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[27] + mi := &file_flow_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1886,7 +1805,7 @@ func (x *IntPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use IntPartitionRange.ProtoReflect.Descriptor instead. func (*IntPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{27} + return file_flow_proto_rawDescGZIP(), []int{24} } func (x *IntPartitionRange) GetStart() int64 { @@ -1915,7 +1834,7 @@ type TimestampPartitionRange struct { func (x *TimestampPartitionRange) Reset() { *x = TimestampPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[28] + mi := &file_flow_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1928,7 +1847,7 @@ func (x *TimestampPartitionRange) String() string { func (*TimestampPartitionRange) ProtoMessage() {} func (x *TimestampPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[28] + mi := &file_flow_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1941,7 +1860,7 @@ func (x *TimestampPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use TimestampPartitionRange.ProtoReflect.Descriptor instead. func (*TimestampPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{28} + return file_flow_proto_rawDescGZIP(), []int{25} } func (x *TimestampPartitionRange) GetStart() *timestamppb.Timestamp { @@ -1970,7 +1889,7 @@ type TID struct { func (x *TID) Reset() { *x = TID{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[29] + mi := &file_flow_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1983,7 +1902,7 @@ func (x *TID) String() string { func (*TID) ProtoMessage() {} func (x *TID) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[29] + mi := &file_flow_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1996,7 +1915,7 @@ func (x *TID) ProtoReflect() protoreflect.Message { // Deprecated: Use TID.ProtoReflect.Descriptor instead. func (*TID) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{29} + return file_flow_proto_rawDescGZIP(), []int{26} } func (x *TID) GetBlockNumber() uint32 { @@ -2025,7 +1944,7 @@ type TIDPartitionRange struct { func (x *TIDPartitionRange) Reset() { *x = TIDPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[30] + mi := &file_flow_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2038,7 +1957,7 @@ func (x *TIDPartitionRange) String() string { func (*TIDPartitionRange) ProtoMessage() {} func (x *TIDPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[30] + mi := &file_flow_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2051,7 +1970,7 @@ func (x *TIDPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use TIDPartitionRange.ProtoReflect.Descriptor instead. func (*TIDPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{30} + return file_flow_proto_rawDescGZIP(), []int{27} } func (x *TIDPartitionRange) GetStart() *TID { @@ -2080,7 +1999,7 @@ type XMINPartitionRange struct { func (x *XMINPartitionRange) Reset() { *x = XMINPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[31] + mi := &file_flow_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2093,7 +2012,7 @@ func (x *XMINPartitionRange) String() string { func (*XMINPartitionRange) ProtoMessage() {} func (x *XMINPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[31] + mi := &file_flow_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2106,7 +2025,7 @@ func (x *XMINPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use XMINPartitionRange.ProtoReflect.Descriptor instead. func (*XMINPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{31} + return file_flow_proto_rawDescGZIP(), []int{28} } func (x *XMINPartitionRange) GetStart() uint32 { @@ -2142,7 +2061,7 @@ type PartitionRange struct { func (x *PartitionRange) Reset() { *x = PartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[32] + mi := &file_flow_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2155,7 +2074,7 @@ func (x *PartitionRange) String() string { func (*PartitionRange) ProtoMessage() {} func (x *PartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[32] + mi := &file_flow_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2168,7 +2087,7 @@ func (x *PartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use PartitionRange.ProtoReflect.Descriptor instead. func (*PartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{32} + return file_flow_proto_rawDescGZIP(), []int{29} } func (m *PartitionRange) GetRange() isPartitionRange_Range { @@ -2246,7 +2165,7 @@ type QRepWriteMode struct { func (x *QRepWriteMode) Reset() { *x = QRepWriteMode{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2259,7 +2178,7 @@ func (x *QRepWriteMode) String() string { func (*QRepWriteMode) ProtoMessage() {} func (x *QRepWriteMode) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2272,7 +2191,7 @@ func (x *QRepWriteMode) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepWriteMode.ProtoReflect.Descriptor instead. func (*QRepWriteMode) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{33} + return file_flow_proto_rawDescGZIP(), []int{30} } func (x *QRepWriteMode) GetWriteType() QRepWriteType { @@ -2325,7 +2244,7 @@ type QRepConfig struct { func (x *QRepConfig) Reset() { *x = QRepConfig{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2338,7 +2257,7 @@ func (x *QRepConfig) String() string { func (*QRepConfig) ProtoMessage() {} func (x *QRepConfig) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2351,7 +2270,7 @@ func (x *QRepConfig) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepConfig.ProtoReflect.Descriptor instead. func (*QRepConfig) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{34} + return file_flow_proto_rawDescGZIP(), []int{31} } func (x *QRepConfig) GetFlowJobName() string { @@ -2479,7 +2398,7 @@ type QRepPartition struct { func (x *QRepPartition) Reset() { *x = QRepPartition{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[35] + mi := &file_flow_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2492,7 +2411,7 @@ func (x *QRepPartition) String() string { func (*QRepPartition) ProtoMessage() {} func (x *QRepPartition) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[35] + mi := &file_flow_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2505,7 +2424,7 @@ func (x *QRepPartition) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepPartition.ProtoReflect.Descriptor instead. func (*QRepPartition) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{35} + return file_flow_proto_rawDescGZIP(), []int{32} } func (x *QRepPartition) GetPartitionId() string { @@ -2541,7 +2460,7 @@ type QRepPartitionBatch struct { func (x *QRepPartitionBatch) Reset() { *x = QRepPartitionBatch{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[36] + mi := &file_flow_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2554,7 +2473,7 @@ func (x *QRepPartitionBatch) String() string { func (*QRepPartitionBatch) ProtoMessage() {} func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[36] + mi := &file_flow_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2567,7 +2486,7 @@ func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepPartitionBatch.ProtoReflect.Descriptor instead. func (*QRepPartitionBatch) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{36} + return file_flow_proto_rawDescGZIP(), []int{33} } func (x *QRepPartitionBatch) GetBatchId() int32 { @@ -2595,7 +2514,7 @@ type QRepParitionResult struct { func (x *QRepParitionResult) Reset() { *x = QRepParitionResult{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[37] + mi := &file_flow_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2608,7 +2527,7 @@ func (x *QRepParitionResult) String() string { func (*QRepParitionResult) ProtoMessage() {} func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[37] + mi := &file_flow_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2621,7 +2540,7 @@ func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { // Deprecated: Use QRepParitionResult.ProtoReflect.Descriptor instead. func (*QRepParitionResult) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{37} + return file_flow_proto_rawDescGZIP(), []int{34} } func (x *QRepParitionResult) GetPartitions() []*QRepPartition { @@ -2642,7 +2561,7 @@ type DropFlowInput struct { func (x *DropFlowInput) Reset() { *x = DropFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[38] + mi := &file_flow_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2655,7 +2574,7 @@ func (x *DropFlowInput) String() string { func (*DropFlowInput) ProtoMessage() {} func (x *DropFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[38] + mi := &file_flow_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2668,7 +2587,7 @@ func (x *DropFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use DropFlowInput.ProtoReflect.Descriptor instead. func (*DropFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{38} + return file_flow_proto_rawDescGZIP(), []int{35} } func (x *DropFlowInput) GetFlowName() string { @@ -2690,7 +2609,7 @@ type DeltaAddedColumn struct { func (x *DeltaAddedColumn) Reset() { *x = DeltaAddedColumn{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[39] + mi := &file_flow_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2703,7 +2622,7 @@ func (x *DeltaAddedColumn) String() string { func (*DeltaAddedColumn) ProtoMessage() {} func (x *DeltaAddedColumn) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[39] + mi := &file_flow_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2716,7 +2635,7 @@ func (x *DeltaAddedColumn) ProtoReflect() protoreflect.Message { // Deprecated: Use DeltaAddedColumn.ProtoReflect.Descriptor instead. func (*DeltaAddedColumn) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{39} + return file_flow_proto_rawDescGZIP(), []int{36} } func (x *DeltaAddedColumn) GetColumnName() string { @@ -2747,7 +2666,7 @@ type TableSchemaDelta struct { func (x *TableSchemaDelta) Reset() { *x = TableSchemaDelta{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[40] + mi := &file_flow_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2760,7 +2679,7 @@ func (x *TableSchemaDelta) String() string { func (*TableSchemaDelta) ProtoMessage() {} func (x *TableSchemaDelta) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[40] + mi := &file_flow_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2773,7 +2692,7 @@ func (x *TableSchemaDelta) ProtoReflect() protoreflect.Message { // Deprecated: Use TableSchemaDelta.ProtoReflect.Descriptor instead. func (*TableSchemaDelta) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{40} + return file_flow_proto_rawDescGZIP(), []int{37} } func (x *TableSchemaDelta) GetSrcTableName() string { @@ -2816,7 +2735,7 @@ type ReplayTableSchemaDeltaInput struct { func (x *ReplayTableSchemaDeltaInput) Reset() { *x = ReplayTableSchemaDeltaInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[41] + mi := &file_flow_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2829,7 +2748,7 @@ func (x *ReplayTableSchemaDeltaInput) String() string { func (*ReplayTableSchemaDeltaInput) ProtoMessage() {} func (x *ReplayTableSchemaDeltaInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[41] + mi := &file_flow_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2842,7 +2761,7 @@ func (x *ReplayTableSchemaDeltaInput) ProtoReflect() protoreflect.Message { // Deprecated: Use ReplayTableSchemaDeltaInput.ProtoReflect.Descriptor instead. func (*ReplayTableSchemaDeltaInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{41} + return file_flow_proto_rawDescGZIP(), []int{38} } func (x *ReplayTableSchemaDeltaInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { @@ -2859,357 +2778,818 @@ func (x *ReplayTableSchemaDeltaInput) GetTableSchemaDelta() *TableSchemaDelta { return nil } -var File_flow_proto protoreflect.FileDescriptor +type ListTablesInSchemasInput struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -var file_flow_proto_rawDesc = []byte{ - 0x0a, 0x0a, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0b, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0b, 0x70, 0x65, 0x65, 0x72, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x74, 0x0a, 0x10, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2a, 0x0a, 0x11, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x64, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x14, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x5e, 0x0a, - 0x15, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x22, 0x95, 0x01, - 0x0a, 0x0f, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 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, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xb9, 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, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x64, - 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, - 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 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, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, - 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, 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, 0x77, 0x0a, 0x19, 0x73, 0x72, - 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, - 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 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, 0x2e, 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, 0x52, 0x15, 0x73, 0x72, - 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x12, 0x79, 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, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 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, 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, 0x37, - 0x0a, 0x0d, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, - 0x08, 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, 0x0c, 0x6d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0x50, 0x65, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x0c, 0x6d, 0x61, 0x78, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x26, 0x0a, - 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, - 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, - 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, 0x6d, - 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, 0x73, - 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x1d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, - 0x6f, 0x74, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, - 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1a, 0x73, - 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, - 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, - 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, - 0x5f, 0x69, 0x6e, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x18, 0x0e, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x12, - 0x47, 0x0a, 0x12, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, - 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0f, 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, 0x10, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, - 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, - 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x10, 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, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x6e, 0x61, 0x70, 0x73, - 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, - 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, - 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x28, 0x0a, 0x10, 0x63, - 0x64, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, - 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x64, 0x63, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x32, 0x0a, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x70, 0x75, - 0x73, 0x68, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x15, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, 0x75, 0x73, 0x68, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, - 0x7a, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x70, 0x75, 0x73, 0x68, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, - 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x70, 0x75, - 0x73, 0x68, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 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, 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, 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, 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, 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, 0x9c, 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, 0x2c, 0x0a, 0x12, - 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 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, + PeerConnectionConfig *Peer `protobuf:"bytes,1,opt,name=peer_connection_config,json=peerConnectionConfig,proto3" json:"peer_connection_config,omitempty"` + SchemaMapping map[string]string `protobuf:"bytes,2,rep,name=schema_mapping,json=schemaMapping,proto3" json:"schema_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *ListTablesInSchemasInput) Reset() { + *x = ListTablesInSchemasInput{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListTablesInSchemasInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListTablesInSchemasInput) ProtoMessage() {} + +func (x *ListTablesInSchemasInput) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[39] + 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 ListTablesInSchemasInput.ProtoReflect.Descriptor instead. +func (*ListTablesInSchemasInput) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{39} +} + +func (x *ListTablesInSchemasInput) GetPeerConnectionConfig() *Peer { + if x != nil { + return x.PeerConnectionConfig + } + return nil +} + +func (x *ListTablesInSchemasInput) GetSchemaMapping() map[string]string { + if x != nil { + return x.SchemaMapping + } + return nil +} + +type TablesList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Tables []string `protobuf:"bytes,1,rep,name=tables,proto3" json:"tables,omitempty"` +} + +func (x *TablesList) Reset() { + *x = TablesList{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TablesList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TablesList) ProtoMessage() {} + +func (x *TablesList) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[40] + 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 TablesList.ProtoReflect.Descriptor instead. +func (*TablesList) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{40} +} + +func (x *TablesList) GetTables() []string { + if x != nil { + return x.Tables + } + return nil +} + +type ListTablesInSchemasOutput struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SchemaToTables map[string]*TablesList `protobuf:"bytes,1,rep,name=schema_to_tables,json=schemaToTables,proto3" json:"schema_to_tables,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *ListTablesInSchemasOutput) Reset() { + *x = ListTablesInSchemasOutput{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ListTablesInSchemasOutput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ListTablesInSchemasOutput) ProtoMessage() {} + +func (x *ListTablesInSchemasOutput) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[41] + 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 ListTablesInSchemasOutput.ProtoReflect.Descriptor instead. +func (*ListTablesInSchemasOutput) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{41} +} + +func (x *ListTablesInSchemasOutput) GetSchemaToTables() map[string]*TablesList { + if x != nil { + return x.SchemaToTables + } + return nil +} + +type AdditionalTableDelta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TableName string `protobuf:"bytes,1,opt,name=table_name,json=tableName,proto3" json:"table_name,omitempty"` + SrcSchema string `protobuf:"bytes,2,opt,name=src_schema,json=srcSchema,proto3" json:"src_schema,omitempty"` + DstSchema string `protobuf:"bytes,3,opt,name=dst_schema,json=dstSchema,proto3" json:"dst_schema,omitempty"` + RelId uint32 `protobuf:"varint,4,opt,name=rel_id,json=relId,proto3" json:"rel_id,omitempty"` + TableSchema *TableSchema `protobuf:"bytes,5,opt,name=table_schema,json=tableSchema,proto3" json:"table_schema,omitempty"` +} + +func (x *AdditionalTableDelta) Reset() { + *x = AdditionalTableDelta{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AdditionalTableDelta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AdditionalTableDelta) ProtoMessage() {} + +func (x *AdditionalTableDelta) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[42] + 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 AdditionalTableDelta.ProtoReflect.Descriptor instead. +func (*AdditionalTableDelta) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{42} +} + +func (x *AdditionalTableDelta) GetTableName() string { + if x != nil { + return x.TableName + } + return "" +} + +func (x *AdditionalTableDelta) GetSrcSchema() string { + if x != nil { + return x.SrcSchema + } + return "" +} + +func (x *AdditionalTableDelta) GetDstSchema() string { + if x != nil { + return x.DstSchema + } + return "" +} + +func (x *AdditionalTableDelta) GetRelId() uint32 { + if x != nil { + return x.RelId + } + return 0 +} + +func (x *AdditionalTableDelta) GetTableSchema() *TableSchema { + if x != nil { + return x.TableSchema + } + return nil +} + +type MirrorDelta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Delta: + // + // *MirrorDelta_TableSchemaDelta + // *MirrorDelta_AdditionalTableDelta + Delta isMirrorDelta_Delta `protobuf_oneof:"delta"` +} + +func (x *MirrorDelta) Reset() { + *x = MirrorDelta{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *MirrorDelta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MirrorDelta) ProtoMessage() {} + +func (x *MirrorDelta) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[43] + 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 MirrorDelta.ProtoReflect.Descriptor instead. +func (*MirrorDelta) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{43} +} + +func (m *MirrorDelta) GetDelta() isMirrorDelta_Delta { + if m != nil { + return m.Delta + } + return nil +} + +func (x *MirrorDelta) GetTableSchemaDelta() *TableSchemaDelta { + if x, ok := x.GetDelta().(*MirrorDelta_TableSchemaDelta); ok { + return x.TableSchemaDelta + } + return nil +} + +func (x *MirrorDelta) GetAdditionalTableDelta() *AdditionalTableDelta { + if x, ok := x.GetDelta().(*MirrorDelta_AdditionalTableDelta); ok { + return x.AdditionalTableDelta + } + return nil +} + +type isMirrorDelta_Delta interface { + isMirrorDelta_Delta() +} + +type MirrorDelta_TableSchemaDelta struct { + // adding or dropping columns to an existing table + TableSchemaDelta *TableSchemaDelta `protobuf:"bytes,1,opt,name=table_schema_delta,json=tableSchemaDelta,proto3,oneof"` +} + +type MirrorDelta_AdditionalTableDelta struct { + // creating a new table, for MappingType SCHEMA only + AdditionalTableDelta *AdditionalTableDelta `protobuf:"bytes,2,opt,name=additional_table_delta,json=additionalTableDelta,proto3,oneof"` +} + +func (*MirrorDelta_TableSchemaDelta) isMirrorDelta_Delta() {} + +func (*MirrorDelta_AdditionalTableDelta) isMirrorDelta_Delta() {} + +type CreateAdditionalTableInput struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FlowConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,1,opt,name=flow_connection_configs,json=flowConnectionConfigs,proto3" json:"flow_connection_configs,omitempty"` + AdditionalTableInfo *AdditionalTableDelta `protobuf:"bytes,2,opt,name=additional_table_info,json=additionalTableInfo,proto3" json:"additional_table_info,omitempty"` +} + +func (x *CreateAdditionalTableInput) Reset() { + *x = CreateAdditionalTableInput{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CreateAdditionalTableInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CreateAdditionalTableInput) ProtoMessage() {} + +func (x *CreateAdditionalTableInput) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[44] + 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 CreateAdditionalTableInput.ProtoReflect.Descriptor instead. +func (*CreateAdditionalTableInput) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{44} +} + +func (x *CreateAdditionalTableInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { + if x != nil { + return x.FlowConnectionConfigs + } + return nil +} + +func (x *CreateAdditionalTableInput) GetAdditionalTableInfo() *AdditionalTableDelta { + if x != nil { + return x.AdditionalTableInfo + } + return nil +} + +type SyncResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // FirstSyncedCheckPointID is the first ID that was synced. + FirstSyncedCheckpointId int64 `protobuf:"varint,1,opt,name=first_synced_checkpoint_id,json=firstSyncedCheckpointId,proto3" json:"first_synced_checkpoint_id,omitempty"` + // LastSyncedCheckPointID is the last ID that was synced. + LastSyncedCheckpointId int64 `protobuf:"varint,2,opt,name=last_synced_checkpoint_id,json=lastSyncedCheckpointId,proto3" json:"last_synced_checkpoint_id,omitempty"` + // NumRecordsSynced is the number of records that were synced. + NumRecordsSynced int64 `protobuf:"varint,3,opt,name=num_records_synced,json=numRecordsSynced,proto3" json:"num_records_synced,omitempty"` + // CurrentSyncBatchID is the ID of the currently synced batch. + CurrentSyncBatchId int64 `protobuf:"varint,4,opt,name=current_sync_batch_id,json=currentSyncBatchId,proto3" json:"current_sync_batch_id,omitempty"` + // TableNameRowsMapping tells how many records need to be synced to each destination table. + TableNameRowsMapping map[string]uint32 `protobuf:"bytes,5,rep,name=table_name_rows_mapping,json=tableNameRowsMapping,proto3" json:"table_name_rows_mapping,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + // to be stored in state for future PullFlows + RelationMessageMapping map[uint32]*RelationMessage `protobuf:"bytes,6,rep,name=relation_message_mapping,json=relationMessageMapping,proto3" json:"relation_message_mapping,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // if not-nil, needs to be applied before next StartFlow + MirrorDelta *MirrorDelta `protobuf:"bytes,7,opt,name=mirror_delta,json=mirrorDelta,proto3" json:"mirror_delta,omitempty"` +} + +func (x *SyncResponse) Reset() { + *x = SyncResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncResponse) ProtoMessage() {} + +func (x *SyncResponse) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[45] + 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 SyncResponse.ProtoReflect.Descriptor instead. +func (*SyncResponse) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{45} +} + +func (x *SyncResponse) GetFirstSyncedCheckpointId() int64 { + if x != nil { + return x.FirstSyncedCheckpointId + } + return 0 +} + +func (x *SyncResponse) GetLastSyncedCheckpointId() int64 { + if x != nil { + return x.LastSyncedCheckpointId + } + return 0 +} + +func (x *SyncResponse) GetNumRecordsSynced() int64 { + if x != nil { + return x.NumRecordsSynced + } + return 0 +} + +func (x *SyncResponse) GetCurrentSyncBatchId() int64 { + if x != nil { + return x.CurrentSyncBatchId + } + return 0 +} + +func (x *SyncResponse) GetTableNameRowsMapping() map[string]uint32 { + if x != nil { + return x.TableNameRowsMapping + } + return nil +} + +func (x *SyncResponse) GetRelationMessageMapping() map[uint32]*RelationMessage { + if x != nil { + return x.RelationMessageMapping + } + return nil +} + +func (x *SyncResponse) GetMirrorDelta() *MirrorDelta { + if x != nil { + return x.MirrorDelta + } + return nil +} + +var File_flow_proto protoreflect.FileDescriptor + +var file_flow_proto_rawDesc = []byte{ + 0x0a, 0x0a, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0b, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0b, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x5e, 0x0a, 0x15, 0x52, 0x65, 0x6c, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x61, + 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x64, + 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x22, 0x95, 0x01, 0x0a, 0x0f, 0x52, 0x65, 0x6c, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x0a, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, + 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x3c, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x22, 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, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, + 0xca, 0x0e, 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, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 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, 0x0b, + 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0d, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x04, 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, 0x12, 0x66, 0x0a, 0x12, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 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, 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, 0x77, 0x0a, 0x19, 0x73, 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 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, 0x2e, 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, 0x52, 0x15, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x79, 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, 0x07, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x3e, 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, 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, 0x37, 0x0a, 0x0d, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x08, 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, 0x0c, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x50, 0x65, 0x65, + 0x72, 0x12, 0x24, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6d, 0x61, 0x78, 0x42, 0x61, + 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, + 0x29, 0x0a, 0x10, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x70, 0x75, 0x62, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, + 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, + 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, + 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x41, 0x0a, 0x1d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6d, 0x61, 0x78, + 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1a, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, + 0x74, 0x4d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, + 0x65, 0x72, 0x73, 0x12, 0x44, 0x0a, 0x1f, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, + 0x6e, 0x75, 0x6d, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x5f, 0x69, 0x6e, 0x5f, 0x70, 0x61, + 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x1b, 0x73, 0x6e, + 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x75, 0x6d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, + 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x12, 0x47, 0x0a, 0x12, 0x73, 0x6e, 0x61, + 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, + 0x0f, 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, 0x10, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, + 0x64, 0x65, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, + 0x6f, 0x64, 0x65, 0x18, 0x10, 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, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x73, 0x74, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x13, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x28, 0x0a, 0x10, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x74, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x12, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x63, 0x64, 0x63, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, + 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x13, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x12, 0x32, 0x0a, 0x15, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x13, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x70, 0x75, 0x73, 0x68, 0x5f, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x70, + 0x75, 0x73, 0x68, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x29, 0x0a, 0x10, + 0x70, 0x75, 0x73, 0x68, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, + 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x70, 0x75, 0x73, 0x68, 0x50, 0x61, 0x72, 0x61, + 0x6c, 0x6c, 0x65, 0x6c, 0x69, 0x73, 0x6d, 0x12, 0x3b, 0x0a, 0x0c, 0x6d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x5c, 0x0a, 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x18, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 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, + 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x12, 0x32, 0x0a, 0x15, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x19, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x13, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x64, 0x64, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 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, 0x1a, 0x40, 0x0a, 0x12, 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, 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, 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, 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, + 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, 0xb4, 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, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x07, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 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, 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, 0x9c, 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, 0x2c, 0x0a, 0x12, + 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 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, @@ -3445,26 +3825,144 @@ var file_flow_proto_rawDesc = []byte{ 0x02, 0x20, 0x01, 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, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 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, + 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x22, 0x87, 0x02, 0x0a, 0x18, 0x4c, 0x69, 0x73, 0x74, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 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, 0x5f, 0x0a, + 0x0e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 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, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0d, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x40, + 0x0a, 0x12, 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, 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, 0x24, 0x0a, 0x0a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x16, + 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0xdd, 0x01, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, 0x4f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x12, 0x64, 0x0a, 0x10, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x74, + 0x6f, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x73, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x54, 0x6f, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x54, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x1a, 0x5a, 0x0a, 0x13, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x54, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 0x2d, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc7, 0x01, 0x0a, 0x14, 0x41, 0x64, 0x64, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, + 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, + 0x0a, 0x0a, 0x73, 0x72, 0x63, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x73, 0x72, 0x63, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1d, 0x0a, + 0x0a, 0x64, 0x73, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x64, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x15, 0x0a, 0x06, + 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x72, 0x65, + 0x6c, 0x49, 0x64, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x18, 0x05, 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, 0xc0, 0x01, 0x0a, 0x0b, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x6c, 0x74, 0x61, + 0x12, 0x4d, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 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, 0x48, 0x00, 0x52, 0x10, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, + 0x59, 0x0a, 0x16, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x21, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x41, 0x64, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x6c, + 0x74, 0x61, 0x48, 0x00, 0x52, 0x14, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x42, 0x07, 0x0a, 0x05, 0x64, 0x65, + 0x6c, 0x74, 0x61, 0x22, 0xcf, 0x01, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x64, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 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, 0x12, 0x55, + 0x0a, 0x15, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x41, 0x64, 0x64, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, + 0x52, 0x13, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0xb3, 0x05, 0x0a, 0x0c, 0x53, 0x79, 0x6e, 0x63, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3b, 0x0a, 0x1a, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, + 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x66, 0x69, 0x72, 0x73, + 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x49, 0x64, 0x12, 0x39, 0x0a, 0x19, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, + 0x65, 0x64, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, + 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x2c, + 0x0a, 0x12, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x5f, 0x73, 0x79, + 0x6e, 0x63, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x52, + 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x53, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x12, 0x31, 0x0a, 0x15, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x63, 0x75, 0x72, + 0x72, 0x65, 0x6e, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, + 0x6a, 0x0a, 0x17, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x33, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, + 0x79, 0x6e, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x52, 0x6f, 0x77, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x6f, 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, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x12, 0x3b, 0x0a, 0x0c, + 0x6d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x4d, 0x69, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x52, 0x0b, 0x6d, 0x69, + 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x1a, 0x47, 0x0a, 0x19, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x6f, 0x77, 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, 0x0d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 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, 0x2a, 0x31, 0x0a, 0x0b, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, + 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x41, 0x42, 0x4c, 0x45, + 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x43, 0x48, 0x45, 0x4d, 0x41, 0x10, 0x02, 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 ( @@ -3479,12 +3977,12 @@ func file_flow_proto_rawDescGZIP() []byte { return file_flow_proto_rawDescData } -var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 54) +var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 63) var file_flow_proto_goTypes = []interface{}{ - (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode - (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType - (*TableNameMapping)(nil), // 2: peerdb_flow.TableNameMapping + (MappingType)(0), // 0: peerdb_flow.MappingType + (QRepSyncMode)(0), // 1: peerdb_flow.QRepSyncMode + (QRepWriteType)(0), // 2: peerdb_flow.QRepWriteType (*RelationMessageColumn)(nil), // 3: peerdb_flow.RelationMessageColumn (*RelationMessage)(nil), // 4: peerdb_flow.RelationMessage (*FlowConnectionConfigs)(nil), // 5: peerdb_flow.FlowConnectionConfigs @@ -3494,121 +3992,144 @@ var file_flow_proto_goTypes = []interface{}{ (*StartFlowInput)(nil), // 9: peerdb_flow.StartFlowInput (*StartNormalizeInput)(nil), // 10: peerdb_flow.StartNormalizeInput (*GetLastSyncedIDInput)(nil), // 11: peerdb_flow.GetLastSyncedIDInput - (*EnsurePullabilityInput)(nil), // 12: peerdb_flow.EnsurePullabilityInput - (*EnsurePullabilityBatchInput)(nil), // 13: peerdb_flow.EnsurePullabilityBatchInput - (*PostgresTableIdentifier)(nil), // 14: peerdb_flow.PostgresTableIdentifier - (*TableIdentifier)(nil), // 15: peerdb_flow.TableIdentifier - (*EnsurePullabilityOutput)(nil), // 16: peerdb_flow.EnsurePullabilityOutput - (*EnsurePullabilityBatchOutput)(nil), // 17: peerdb_flow.EnsurePullabilityBatchOutput - (*SetupReplicationInput)(nil), // 18: peerdb_flow.SetupReplicationInput - (*SetupReplicationOutput)(nil), // 19: peerdb_flow.SetupReplicationOutput - (*CreateRawTableInput)(nil), // 20: peerdb_flow.CreateRawTableInput - (*CreateRawTableOutput)(nil), // 21: peerdb_flow.CreateRawTableOutput - (*TableSchema)(nil), // 22: peerdb_flow.TableSchema - (*GetTableSchemaBatchInput)(nil), // 23: peerdb_flow.GetTableSchemaBatchInput - (*GetTableSchemaBatchOutput)(nil), // 24: peerdb_flow.GetTableSchemaBatchOutput - (*SetupNormalizedTableInput)(nil), // 25: peerdb_flow.SetupNormalizedTableInput - (*SetupNormalizedTableBatchInput)(nil), // 26: peerdb_flow.SetupNormalizedTableBatchInput - (*SetupNormalizedTableOutput)(nil), // 27: peerdb_flow.SetupNormalizedTableOutput - (*SetupNormalizedTableBatchOutput)(nil), // 28: peerdb_flow.SetupNormalizedTableBatchOutput - (*IntPartitionRange)(nil), // 29: peerdb_flow.IntPartitionRange - (*TimestampPartitionRange)(nil), // 30: peerdb_flow.TimestampPartitionRange - (*TID)(nil), // 31: peerdb_flow.TID - (*TIDPartitionRange)(nil), // 32: peerdb_flow.TIDPartitionRange - (*XMINPartitionRange)(nil), // 33: peerdb_flow.XMINPartitionRange - (*PartitionRange)(nil), // 34: peerdb_flow.PartitionRange - (*QRepWriteMode)(nil), // 35: peerdb_flow.QRepWriteMode - (*QRepConfig)(nil), // 36: peerdb_flow.QRepConfig - (*QRepPartition)(nil), // 37: peerdb_flow.QRepPartition - (*QRepPartitionBatch)(nil), // 38: peerdb_flow.QRepPartitionBatch - (*QRepParitionResult)(nil), // 39: peerdb_flow.QRepParitionResult - (*DropFlowInput)(nil), // 40: peerdb_flow.DropFlowInput - (*DeltaAddedColumn)(nil), // 41: peerdb_flow.DeltaAddedColumn - (*TableSchemaDelta)(nil), // 42: peerdb_flow.TableSchemaDelta - (*ReplayTableSchemaDeltaInput)(nil), // 43: peerdb_flow.ReplayTableSchemaDeltaInput - nil, // 44: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - nil, // 45: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 46: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 47: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry - nil, // 48: peerdb_flow.StartFlowInput.RelationMessageMappingEntry - nil, // 49: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 50: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 51: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 52: peerdb_flow.TableSchema.ColumnsEntry - nil, // 53: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 54: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 55: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 56: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 57: google.protobuf.Timestamp + (*EnsurePullabilityBatchInput)(nil), // 12: peerdb_flow.EnsurePullabilityBatchInput + (*PostgresTableIdentifier)(nil), // 13: peerdb_flow.PostgresTableIdentifier + (*TableIdentifier)(nil), // 14: peerdb_flow.TableIdentifier + (*EnsurePullabilityBatchOutput)(nil), // 15: peerdb_flow.EnsurePullabilityBatchOutput + (*SetupReplicationInput)(nil), // 16: peerdb_flow.SetupReplicationInput + (*SetupReplicationOutput)(nil), // 17: peerdb_flow.SetupReplicationOutput + (*CreateRawTableInput)(nil), // 18: peerdb_flow.CreateRawTableInput + (*CreateRawTableOutput)(nil), // 19: peerdb_flow.CreateRawTableOutput + (*TableSchema)(nil), // 20: peerdb_flow.TableSchema + (*GetTableSchemaBatchInput)(nil), // 21: peerdb_flow.GetTableSchemaBatchInput + (*GetTableSchemaBatchOutput)(nil), // 22: peerdb_flow.GetTableSchemaBatchOutput + (*SetupNormalizedTableInput)(nil), // 23: peerdb_flow.SetupNormalizedTableInput + (*SetupNormalizedTableBatchInput)(nil), // 24: peerdb_flow.SetupNormalizedTableBatchInput + (*SetupNormalizedTableOutput)(nil), // 25: peerdb_flow.SetupNormalizedTableOutput + (*SetupNormalizedTableBatchOutput)(nil), // 26: peerdb_flow.SetupNormalizedTableBatchOutput + (*IntPartitionRange)(nil), // 27: peerdb_flow.IntPartitionRange + (*TimestampPartitionRange)(nil), // 28: peerdb_flow.TimestampPartitionRange + (*TID)(nil), // 29: peerdb_flow.TID + (*TIDPartitionRange)(nil), // 30: peerdb_flow.TIDPartitionRange + (*XMINPartitionRange)(nil), // 31: peerdb_flow.XMINPartitionRange + (*PartitionRange)(nil), // 32: peerdb_flow.PartitionRange + (*QRepWriteMode)(nil), // 33: peerdb_flow.QRepWriteMode + (*QRepConfig)(nil), // 34: peerdb_flow.QRepConfig + (*QRepPartition)(nil), // 35: peerdb_flow.QRepPartition + (*QRepPartitionBatch)(nil), // 36: peerdb_flow.QRepPartitionBatch + (*QRepParitionResult)(nil), // 37: peerdb_flow.QRepParitionResult + (*DropFlowInput)(nil), // 38: peerdb_flow.DropFlowInput + (*DeltaAddedColumn)(nil), // 39: peerdb_flow.DeltaAddedColumn + (*TableSchemaDelta)(nil), // 40: peerdb_flow.TableSchemaDelta + (*ReplayTableSchemaDeltaInput)(nil), // 41: peerdb_flow.ReplayTableSchemaDeltaInput + (*ListTablesInSchemasInput)(nil), // 42: peerdb_flow.ListTablesInSchemasInput + (*TablesList)(nil), // 43: peerdb_flow.TablesList + (*ListTablesInSchemasOutput)(nil), // 44: peerdb_flow.ListTablesInSchemasOutput + (*AdditionalTableDelta)(nil), // 45: peerdb_flow.AdditionalTableDelta + (*MirrorDelta)(nil), // 46: peerdb_flow.MirrorDelta + (*CreateAdditionalTableInput)(nil), // 47: peerdb_flow.CreateAdditionalTableInput + (*SyncResponse)(nil), // 48: peerdb_flow.SyncResponse + nil, // 49: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + nil, // 50: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 51: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 52: peerdb_flow.FlowConnectionConfigs.SchemaMappingEntry + 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 + nil, // 62: peerdb_flow.ListTablesInSchemasInput.SchemaMappingEntry + nil, // 63: peerdb_flow.ListTablesInSchemasOutput.SchemaToTablesEntry + nil, // 64: peerdb_flow.SyncResponse.TableNameRowsMappingEntry + nil, // 65: peerdb_flow.SyncResponse.RelationMessageMappingEntry + (*Peer)(nil), // 66: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 67: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ 3, // 0: peerdb_flow.RelationMessage.columns:type_name -> peerdb_flow.RelationMessageColumn - 56, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 56, // 2: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer - 22, // 3: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema - 44, // 4: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - 45, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 46, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 56, // 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 - 47, // 10: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry - 57, // 11: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp - 8, // 12: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState - 5, // 13: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 6, // 14: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions - 48, // 15: peerdb_flow.StartFlowInput.relation_message_mapping:type_name -> peerdb_flow.StartFlowInput.RelationMessageMappingEntry - 5, // 16: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 56, // 17: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 56, // 18: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 56, // 19: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 14, // 20: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier - 15, // 21: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier - 49, // 22: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 56, // 23: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 50, // 24: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 56, // 25: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 56, // 26: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 51, // 27: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry - 0, // 28: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 52, // 29: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 56, // 30: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 53, // 31: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 56, // 32: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 22, // 33: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema - 56, // 34: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 54, // 35: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 55, // 36: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 57, // 37: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 57, // 38: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp - 31, // 39: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID - 31, // 40: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID - 29, // 41: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange - 30, // 42: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange - 32, // 43: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange - 33, // 44: peerdb_flow.PartitionRange.xmin_range:type_name -> peerdb_flow.XMINPartitionRange - 1, // 45: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType - 56, // 46: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 56, // 47: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer - 0, // 48: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode - 35, // 49: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode - 34, // 50: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange - 37, // 51: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition - 37, // 52: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition - 41, // 53: peerdb_flow.TableSchemaDelta.added_columns:type_name -> peerdb_flow.DeltaAddedColumn + 66, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 66, // 2: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer + 20, // 3: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema + 49, // 4: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + 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 + 66, // 7: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer + 1, // 8: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode + 1, // 9: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode + 0, // 10: peerdb_flow.FlowConnectionConfigs.mapping_type:type_name -> peerdb_flow.MappingType + 52, // 11: peerdb_flow.FlowConnectionConfigs.schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SchemaMappingEntry + 53, // 12: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + 67, // 13: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp + 8, // 14: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState + 5, // 15: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 6, // 16: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions + 54, // 17: peerdb_flow.StartFlowInput.relation_message_mapping:type_name -> peerdb_flow.StartFlowInput.RelationMessageMappingEntry + 5, // 18: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 66, // 19: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 66, // 20: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 13, // 21: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier + 55, // 22: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 66, // 23: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 56, // 24: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 66, // 25: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 66, // 26: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 57, // 27: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry + 1, // 28: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode + 58, // 29: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 66, // 30: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 59, // 31: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 66, // 32: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 20, // 33: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema + 66, // 34: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 60, // 35: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 61, // 36: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 67, // 37: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 67, // 38: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp + 29, // 39: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID + 29, // 40: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID + 27, // 41: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange + 28, // 42: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange + 30, // 43: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange + 31, // 44: peerdb_flow.PartitionRange.xmin_range:type_name -> peerdb_flow.XMINPartitionRange + 2, // 45: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType + 66, // 46: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 66, // 47: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer + 1, // 48: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode + 33, // 49: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode + 32, // 50: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange + 35, // 51: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition + 35, // 52: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition + 39, // 53: peerdb_flow.TableSchemaDelta.added_columns:type_name -> peerdb_flow.DeltaAddedColumn 5, // 54: peerdb_flow.ReplayTableSchemaDeltaInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 42, // 55: peerdb_flow.ReplayTableSchemaDeltaInput.table_schema_delta:type_name -> peerdb_flow.TableSchemaDelta - 22, // 56: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 4, // 57: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage - 4, // 58: peerdb_flow.StartFlowInput.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage - 15, // 59: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier - 22, // 60: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 22, // 61: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 62, // [62:62] is the sub-list for method output_type - 62, // [62:62] is the sub-list for method input_type - 62, // [62:62] is the sub-list for extension type_name - 62, // [62:62] is the sub-list for extension extendee - 0, // [0:62] is the sub-list for field type_name + 40, // 55: peerdb_flow.ReplayTableSchemaDeltaInput.table_schema_delta:type_name -> peerdb_flow.TableSchemaDelta + 66, // 56: peerdb_flow.ListTablesInSchemasInput.peer_connection_config:type_name -> peerdb_peers.Peer + 62, // 57: peerdb_flow.ListTablesInSchemasInput.schema_mapping:type_name -> peerdb_flow.ListTablesInSchemasInput.SchemaMappingEntry + 63, // 58: peerdb_flow.ListTablesInSchemasOutput.schema_to_tables:type_name -> peerdb_flow.ListTablesInSchemasOutput.SchemaToTablesEntry + 20, // 59: peerdb_flow.AdditionalTableDelta.table_schema:type_name -> peerdb_flow.TableSchema + 40, // 60: peerdb_flow.MirrorDelta.table_schema_delta:type_name -> peerdb_flow.TableSchemaDelta + 45, // 61: peerdb_flow.MirrorDelta.additional_table_delta:type_name -> peerdb_flow.AdditionalTableDelta + 5, // 62: peerdb_flow.CreateAdditionalTableInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 45, // 63: peerdb_flow.CreateAdditionalTableInput.additional_table_info:type_name -> peerdb_flow.AdditionalTableDelta + 64, // 64: peerdb_flow.SyncResponse.table_name_rows_mapping:type_name -> peerdb_flow.SyncResponse.TableNameRowsMappingEntry + 65, // 65: peerdb_flow.SyncResponse.relation_message_mapping:type_name -> peerdb_flow.SyncResponse.RelationMessageMappingEntry + 46, // 66: peerdb_flow.SyncResponse.mirror_delta:type_name -> peerdb_flow.MirrorDelta + 20, // 67: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 4, // 68: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 4, // 69: peerdb_flow.StartFlowInput.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 14, // 70: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier + 20, // 71: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 20, // 72: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 43, // 73: peerdb_flow.ListTablesInSchemasOutput.SchemaToTablesEntry.value:type_name -> peerdb_flow.TablesList + 4, // 74: peerdb_flow.SyncResponse.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 75, // [75:75] is the sub-list for method output_type + 75, // [75:75] is the sub-list for method input_type + 75, // [75:75] is the sub-list for extension type_name + 75, // [75:75] is the sub-list for extension extendee + 0, // [0:75] is the sub-list for field type_name } func init() { file_flow_proto_init() } @@ -3619,7 +4140,7 @@ func file_flow_proto_init() { file_peers_proto_init() if !protoimpl.UnsafeEnabled { file_flow_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableNameMapping); i { + switch v := v.(*RelationMessageColumn); i { case 0: return &v.state case 1: @@ -3631,7 +4152,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RelationMessageColumn); i { + switch v := v.(*RelationMessage); i { case 0: return &v.state case 1: @@ -3643,7 +4164,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RelationMessage); i { + switch v := v.(*FlowConnectionConfigs); i { case 0: return &v.state case 1: @@ -3655,7 +4176,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FlowConnectionConfigs); i { + switch v := v.(*SyncFlowOptions); i { case 0: return &v.state case 1: @@ -3667,7 +4188,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SyncFlowOptions); i { + switch v := v.(*NormalizeFlowOptions); i { case 0: return &v.state case 1: @@ -3679,7 +4200,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NormalizeFlowOptions); i { + switch v := v.(*LastSyncState); i { case 0: return &v.state case 1: @@ -3691,7 +4212,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LastSyncState); i { + switch v := v.(*StartFlowInput); i { case 0: return &v.state case 1: @@ -3703,7 +4224,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartFlowInput); i { + switch v := v.(*StartNormalizeInput); i { case 0: return &v.state case 1: @@ -3715,7 +4236,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartNormalizeInput); i { + switch v := v.(*GetLastSyncedIDInput); i { case 0: return &v.state case 1: @@ -3727,7 +4248,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetLastSyncedIDInput); i { + switch v := v.(*EnsurePullabilityBatchInput); i { case 0: return &v.state case 1: @@ -3739,7 +4260,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityInput); i { + switch v := v.(*PostgresTableIdentifier); i { case 0: return &v.state case 1: @@ -3751,7 +4272,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityBatchInput); i { + switch v := v.(*TableIdentifier); i { case 0: return &v.state case 1: @@ -3763,7 +4284,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PostgresTableIdentifier); i { + switch v := v.(*EnsurePullabilityBatchOutput); i { case 0: return &v.state case 1: @@ -3775,7 +4296,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableIdentifier); i { + switch v := v.(*SetupReplicationInput); i { case 0: return &v.state case 1: @@ -3787,7 +4308,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityOutput); i { + switch v := v.(*SetupReplicationOutput); i { case 0: return &v.state case 1: @@ -3799,7 +4320,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityBatchOutput); i { + switch v := v.(*CreateRawTableInput); i { case 0: return &v.state case 1: @@ -3811,7 +4332,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupReplicationInput); i { + switch v := v.(*CreateRawTableOutput); i { case 0: return &v.state case 1: @@ -3823,7 +4344,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupReplicationOutput); i { + switch v := v.(*TableSchema); i { case 0: return &v.state case 1: @@ -3835,7 +4356,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateRawTableInput); i { + switch v := v.(*GetTableSchemaBatchInput); i { case 0: return &v.state case 1: @@ -3847,7 +4368,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateRawTableOutput); i { + switch v := v.(*GetTableSchemaBatchOutput); i { case 0: return &v.state case 1: @@ -3859,7 +4380,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableSchema); i { + switch v := v.(*SetupNormalizedTableInput); i { case 0: return &v.state case 1: @@ -3871,7 +4392,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetTableSchemaBatchInput); i { + switch v := v.(*SetupNormalizedTableBatchInput); i { case 0: return &v.state case 1: @@ -3883,7 +4404,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetTableSchemaBatchOutput); i { + switch v := v.(*SetupNormalizedTableOutput); i { case 0: return &v.state case 1: @@ -3895,7 +4416,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableInput); i { + switch v := v.(*SetupNormalizedTableBatchOutput); i { case 0: return &v.state case 1: @@ -3907,7 +4428,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableBatchInput); i { + switch v := v.(*IntPartitionRange); i { case 0: return &v.state case 1: @@ -3919,7 +4440,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableOutput); i { + switch v := v.(*TimestampPartitionRange); i { case 0: return &v.state case 1: @@ -3931,7 +4452,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableBatchOutput); i { + switch v := v.(*TID); i { case 0: return &v.state case 1: @@ -3943,7 +4464,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IntPartitionRange); i { + switch v := v.(*TIDPartitionRange); i { case 0: return &v.state case 1: @@ -3955,7 +4476,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimestampPartitionRange); i { + switch v := v.(*XMINPartitionRange); i { case 0: return &v.state case 1: @@ -3967,7 +4488,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TID); i { + switch v := v.(*PartitionRange); i { case 0: return &v.state case 1: @@ -3979,7 +4500,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TIDPartitionRange); i { + switch v := v.(*QRepWriteMode); i { case 0: return &v.state case 1: @@ -3991,7 +4512,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*XMINPartitionRange); i { + switch v := v.(*QRepConfig); i { case 0: return &v.state case 1: @@ -4003,7 +4524,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartitionRange); i { + switch v := v.(*QRepPartition); i { case 0: return &v.state case 1: @@ -4015,7 +4536,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepWriteMode); i { + switch v := v.(*QRepPartitionBatch); i { case 0: return &v.state case 1: @@ -4027,7 +4548,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepConfig); i { + switch v := v.(*QRepParitionResult); i { case 0: return &v.state case 1: @@ -4039,7 +4560,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepPartition); i { + switch v := v.(*DropFlowInput); i { case 0: return &v.state case 1: @@ -4051,7 +4572,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepPartitionBatch); i { + switch v := v.(*DeltaAddedColumn); i { case 0: return &v.state case 1: @@ -4063,7 +4584,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepParitionResult); i { + switch v := v.(*TableSchemaDelta); i { case 0: return &v.state case 1: @@ -4075,7 +4596,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DropFlowInput); i { + switch v := v.(*ReplayTableSchemaDeltaInput); i { case 0: return &v.state case 1: @@ -4087,7 +4608,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeltaAddedColumn); i { + switch v := v.(*ListTablesInSchemasInput); i { case 0: return &v.state case 1: @@ -4099,7 +4620,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableSchemaDelta); i { + switch v := v.(*TablesList); i { case 0: return &v.state case 1: @@ -4111,7 +4632,55 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplayTableSchemaDeltaInput); i { + switch v := v.(*ListTablesInSchemasOutput); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AdditionalTableDelta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*MirrorDelta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateAdditionalTableInput); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncResponse); i { case 0: return &v.state case 1: @@ -4123,22 +4692,26 @@ func file_flow_proto_init() { } } } - file_flow_proto_msgTypes[13].OneofWrappers = []interface{}{ + file_flow_proto_msgTypes[11].OneofWrappers = []interface{}{ (*TableIdentifier_PostgresTableIdentifier)(nil), } - file_flow_proto_msgTypes[32].OneofWrappers = []interface{}{ + file_flow_proto_msgTypes[29].OneofWrappers = []interface{}{ (*PartitionRange_IntRange)(nil), (*PartitionRange_TimestampRange)(nil), (*PartitionRange_TidRange)(nil), (*PartitionRange_XminRange)(nil), } + file_flow_proto_msgTypes[43].OneofWrappers = []interface{}{ + (*MirrorDelta_TableSchemaDelta)(nil), + (*MirrorDelta_AdditionalTableDelta)(nil), + } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, - NumEnums: 2, - NumMessages: 54, + NumEnums: 3, + NumMessages: 63, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/model/model.go b/flow/model/model.go index 64582127a9..4b8549bf7c 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -32,6 +32,10 @@ type PullRecordsRequest struct { OverrideReplicationSlotName string // for supporting schema changes RelationMessageMapping RelationMessageMapping + // schemas to allow adding new tables for, empty if MappingType != SCHEMA + Schemas []string + // should we add new tables detected in the source schemas? + AllowTableAdditions bool } type Record interface { @@ -267,23 +271,6 @@ type NormalizeRecordsRequest struct { SoftDelete bool } -type SyncResponse struct { - // FirstSyncedCheckPointID is the first ID that was synced. - FirstSyncedCheckPointID int64 - // LastSyncedCheckPointID is the last ID that was synced. - LastSyncedCheckPointID int64 - // NumRecordsSynced is the number of records that were synced. - NumRecordsSynced int64 - // CurrentSyncBatchID is the ID of the currently synced batch. - CurrentSyncBatchID int64 - // TableNameRowsMapping tells how many records need to be synced to each destination table. - TableNameRowsMapping map[string]uint32 - // to be carried to NormalizeFlow - TableSchemaDelta *protos.TableSchemaDelta - // to be stored in state for future PullFlows - RelationMessageMapping RelationMessageMapping -} - type NormalizeResponse struct { // Flag to depict if normalization is done Done bool @@ -291,11 +278,12 @@ type NormalizeResponse struct { EndBatchID int64 } -// sync all the records normally, then apply the schema delta after NormalizeFlow. -type RecordsWithTableSchemaDelta struct { +// sync all the records normally, then apply any schema delta after NormalizeFlow. +// add any new tables at the end of SyncFlow. +type RecordsWithDeltaInfo struct { RecordBatch *RecordBatch - TableSchemaDelta *protos.TableSchemaDelta RelationMessageMapping RelationMessageMapping + MirrorDelta *protos.MirrorDelta } // being clever and passing the delta back as a regular record instead of heavy CDC refactoring. @@ -318,3 +306,23 @@ func (r *RelationRecord) GetItems() *RecordItems { } type RelationMessageMapping map[uint32]*protos.RelationMessage + +// being clever and passing the new table back as a regular record instead of heavy refactoring in processMessage. +type AddedTableRecord struct { + CheckPointID int64 + TableName string + SrcSchema string +} + +// Implement Record interface for RelationRecord. +func (r *AddedTableRecord) GetCheckPointID() int64 { + return r.CheckPointID +} + +func (r *AddedTableRecord) GetTableName() string { + return r.TableName +} + +func (r *AddedTableRecord) GetItems() *RecordItems { + return nil +} diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 46d44ad51b..43f254aca9 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -55,7 +55,7 @@ type CDCFlowState struct { // Progress events for the peer flow. Progress []string // Accumulates status for sync flows spawned. - SyncFlowStatuses []*model.SyncResponse + SyncFlowStatuses []*protos.SyncResponse // Accumulates status for sync flows spawned. NormalizeFlowStatuses []*model.NormalizeResponse // Current signalled state of the peer flow. @@ -266,7 +266,7 @@ func CDCFlowWorkflowWithConfig( syncFlowOptions, ) - var childSyncFlowRes *model.SyncResponse + var childSyncFlowRes *protos.SyncResponse if err := childSyncFlowFuture.Get(ctx, &childSyncFlowRes); err != nil { w.logger.Error("failed to execute sync flow: ", err) state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) @@ -274,6 +274,20 @@ func CDCFlowWorkflowWithConfig( state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) if childSyncFlowRes != nil { state.RelationMessageMapping = childSyncFlowRes.RelationMessageMapping + + // additional table was created, update all flow configuration to reflect the same. + if childSyncFlowRes.MirrorDelta.GetAdditionalTableDelta() != nil { + additionalTableDelta := childSyncFlowRes.MirrorDelta.GetAdditionalTableDelta() + srcTableIdentifier := fmt.Sprintf("%s.%s", additionalTableDelta.SrcSchema, + additionalTableDelta.TableName) + dstTableIdentifier := fmt.Sprintf("%s.%s", additionalTableDelta.DstSchema, + additionalTableDelta.TableName) + + cfg.TableNameMapping[srcTableIdentifier] = dstTableIdentifier + cfg.SrcTableIdNameMapping[additionalTableDelta.RelId] = srcTableIdentifier + // because + cfg.TableNameSchemaMapping[dstTableIdentifier] = additionalTableDelta.TableSchema + } } } @@ -293,7 +307,7 @@ func CDCFlowWorkflowWithConfig( var tableSchemaDelta *protos.TableSchemaDelta = nil if childSyncFlowRes != nil { - tableSchemaDelta = childSyncFlowRes.TableSchemaDelta + tableSchemaDelta = childSyncFlowRes.MirrorDelta.GetTableSchemaDelta() } childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index cf5d395698..66a62273ad 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -8,6 +8,7 @@ import ( "github.com/PeerDB-io/peer-flow/activities" "github.com/PeerDB-io/peer-flow/generated/protos" "golang.org/x/exp/maps" + "golang.org/x/exp/slices" "go.temporal.io/sdk/log" "go.temporal.io/sdk/workflow" @@ -62,19 +63,26 @@ func (s *SetupFlowExecution) checkConnectionsAndSetupMetadataTables( }) // first check the source peer connection - srcConnStatusFuture := workflow.ExecuteActivity(ctx, flowable.CheckConnection, config.Source) - var srcConnStatus activities.CheckConnectionResult + srcConnStatusFuture := workflow.ExecuteActivity(ctx, flowable.CheckPullConnection, config.Source) + var srcConnStatus bool if err := srcConnStatusFuture.Get(ctx, &srcConnStatus); err != nil { return fmt.Errorf("failed to check source peer connection: %w", err) } + if !srcConnStatus { + return fmt.Errorf("source peer connection is not active") + } // then check the destination peer connection - destConnStatusFuture := workflow.ExecuteActivity(ctx, flowable.CheckConnection, config.Destination) + destConnStatusFuture := workflow.ExecuteActivity(ctx, flowable.CheckSyncConnection, config.Destination) var destConnStatus activities.CheckConnectionResult if err := destConnStatusFuture.Get(ctx, &destConnStatus); err != nil { return fmt.Errorf("failed to check destination peer connection: %w", err) } + if config.MappingType == protos.MappingType_SCHEMA && !destConnStatus.SupportsSchemaMapping { + return fmt.Errorf("SCHEMA mapping is not supported by the destination connector") + } + s.logger.Info("ensuring metadata table exists - ", s.CDCFlowName) // then setup the destination peer metadata tables @@ -95,7 +103,7 @@ func (s *SetupFlowExecution) ensurePullability( ctx workflow.Context, config *protos.FlowConnectionConfigs, ) error { - s.logger.Info("ensuring pullability for peer flow - ", s.CDCFlowName) + s.logger.Info("ensuring pullability for CDC flow - ", s.CDCFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 15 * time.Minute, @@ -103,7 +111,7 @@ func (s *SetupFlowExecution) ensurePullability( tmpMap := make(map[uint32]string) srcTblIdentifiers := maps.Keys(config.TableNameMapping) - sort.Strings(srcTblIdentifiers) + slices.Sort(srcTblIdentifiers) // create EnsurePullabilityInput for the srcTableName ensurePullabilityInput := &protos.EnsurePullabilityBatchInput{ @@ -164,7 +172,7 @@ func (s *SetupFlowExecution) createRawTable( // sets up the normalized tables on the destination peer. func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( ctx workflow.Context, flowConnectionConfigs *protos.FlowConnectionConfigs) (map[string]*protos.TableSchema, error) { - s.logger.Info("fetching table schema for peer flow - ", s.CDCFlowName) + s.logger.Info("fetching table schema for CDC flow - ", s.CDCFlowName) ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 1 * time.Hour, @@ -191,7 +199,7 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( sortedSourceTables := maps.Keys(tableNameSchemaMapping) sort.Strings(sortedSourceTables) - s.logger.Info("setting up normalized tables for peer flow - ", s.CDCFlowName) + s.logger.Info("setting up normalized tables for CDC flow - ", s.CDCFlowName) normalizedTableMapping := make(map[string]*protos.TableSchema) for _, srcTableName := range sortedSourceTables { tableSchema := tableNameSchemaMapping[srcTableName] @@ -213,10 +221,45 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( return nil, fmt.Errorf("failed to create normalized tables: %w", err) } - s.logger.Info("finished setting up normalized tables for peer flow - ", s.CDCFlowName) + s.logger.Info("finished setting up normalized tables for CDC flow - ", s.CDCFlowName) return normalizedTableMapping, nil } +func (s *SetupFlowExecution) populateTableMappingFromSchemas(ctx workflow.Context, + flowConnectionConfigs *protos.FlowConnectionConfigs) error { + if flowConnectionConfigs.MappingType != protos.MappingType_SCHEMA { + return nil + } + + ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 10 * time.Minute, + }) + + populateTableMappingFromSchemasFuture := workflow.ExecuteActivity(ctx, + flowable.PopulateTableMappingFromSchemas, + &protos.ListTablesInSchemasInput{ + PeerConnectionConfig: flowConnectionConfigs.Source, + SchemaMapping: flowConnectionConfigs.SchemaMapping, + }) + var populateTableMappingFromSchemasOutput *protos.ListTablesInSchemasOutput + if err := populateTableMappingFromSchemasFuture.Get(ctx, &populateTableMappingFromSchemasOutput); err != nil { + s.logger.Error("failed to populate table mapping from schemas: ", err) + return fmt.Errorf("failed to populate table mapping from schemas: %w", err) + } + + flowConnectionConfigs.TableNameMapping = make(map[string]string) + for srcSchema, dstSchema := range flowConnectionConfigs.SchemaMapping { + for _, table := range populateTableMappingFromSchemasOutput.SchemaToTables[srcSchema].Tables { + flowConnectionConfigs.TableNameMapping[fmt.Sprintf("%s.%s", srcSchema, table)] = + fmt.Sprintf("%s.%s", dstSchema, table) + + } + } + s.logger.Info("finished populating table mapping from schemas for CDC flow - ", s.CDCFlowName) + + return nil +} + // executeSetupFlow executes the setup flow. func (s *SetupFlowExecution) executeSetupFlow( ctx workflow.Context, @@ -224,11 +267,16 @@ func (s *SetupFlowExecution) executeSetupFlow( ) (map[string]*protos.TableSchema, error) { s.logger.Info("executing setup flow - ", s.CDCFlowName) - // first check the connectionsAndSetupMetadataTables + // first check the connectionsAndSetupMetadataTables, fail here if MappingType is SCHEMA if err := s.checkConnectionsAndSetupMetadataTables(ctx, config); err != nil { return nil, fmt.Errorf("failed to check connections and setup metadata tables: %w", err) } + // then populate the initial table mapping from the schemas, if MappingType is SCHEMA + if err := s.populateTableMappingFromSchemas(ctx, config); err != nil { + return nil, err + } + // then ensure pullability if err := s.ensurePullability(ctx, config); err != nil { return nil, fmt.Errorf("failed to ensure pullability: %w", err) diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 585ae0bc25..81a129e84b 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -44,6 +44,9 @@ func (s *SnapshotFlowExecution) setupReplication( ExistingPublicationName: s.config.PublicationName, ExistingReplicationSlotName: s.config.ReplicationSlotName, } + if s.config.MappingType == protos.MappingType_SCHEMA { + setupReplicationInput.Schemas = maps.Keys(s.config.SchemaMapping) + } res := &protos.SetupReplicationOutput{} setupReplicationFuture := workflow.ExecuteActivity(ctx, snapshot.SetupReplication, setupReplicationInput) diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index fe7a285838..503b593cb6 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -55,7 +55,7 @@ func (s *SyncFlowExecution) executeSyncFlow( config *protos.FlowConnectionConfigs, opts *protos.SyncFlowOptions, relationMessageMapping model.RelationMessageMapping, -) (*model.SyncResponse, error) { +) (*protos.SyncResponse, error) { s.logger.Info("executing sync flow - ", s.CDCFlowName) syncMetaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ @@ -83,9 +83,7 @@ func (s *SyncFlowExecution) executeSyncFlow( startFlowCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ StartToCloseTimeout: 24 * time.Hour, - // TODO: activity needs to call heartbeat. - // see https://github.com/PeerDB-io/nexus/issues/216 - HeartbeatTimeout: 30 * time.Second, + HeartbeatTimeout: 30 * time.Second, }) // execute StartFlow on the peers to start the flow @@ -97,11 +95,35 @@ func (s *SyncFlowExecution) executeSyncFlow( } fStartFlow := workflow.ExecuteActivity(startFlowCtx, flowable.StartFlow, startFlowInput) - var syncRes *model.SyncResponse + var syncRes *protos.SyncResponse if err := fStartFlow.Get(startFlowCtx, &syncRes); err != nil { return nil, fmt.Errorf("failed to flow: %w", err) } + if syncRes.MirrorDelta.GetAdditionalTableDelta() != nil { + additionalTableDelta := syncRes.MirrorDelta.GetAdditionalTableDelta() + createAdditionalTableCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 10 * time.Minute, + }) + additionalTableDelta.DstSchema = config.SchemaMapping[additionalTableDelta.SrcSchema] + fCreateAdditionalTable := workflow.ExecuteActivity(createAdditionalTableCtx, + flowable.CreateAdditionalTable, &protos.CreateAdditionalTableInput{ + FlowConnectionConfigs: config, + AdditionalTableInfo: additionalTableDelta, + }) + + var createAdditionalTableOutput *protos.AdditionalTableDelta + if err := fCreateAdditionalTable.Get(createAdditionalTableCtx, + &createAdditionalTableOutput); err != nil { + return nil, fmt.Errorf("failed to create additional table: %w", err) + } + syncRes.MirrorDelta = &protos.MirrorDelta{ + Delta: &protos.MirrorDelta_AdditionalTableDelta{ + AdditionalTableDelta: createAdditionalTableOutput, + }, + } + } + return syncRes, nil } @@ -111,7 +133,7 @@ func (s *SyncFlowExecution) executeSyncFlow( func SyncFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionConfigs, options *protos.SyncFlowOptions, -) (*model.SyncResponse, error) { +) (*protos.SyncResponse, error) { s := NewSyncFlowExecution(ctx, &SyncFlowState{ CDCFlowName: config.FlowJobName, Progress: []string{}, diff --git a/nexus/Cargo.lock b/nexus/Cargo.lock index 3876b4be1f..c787207a38 100644 --- a/nexus/Cargo.lock +++ b/nexus/Cargo.lock @@ -962,6 +962,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", + "sqlparser", "tokio", "tonic", "tonic-health", diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index 13172f511a..10806bee18 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -8,7 +8,7 @@ use std::{ use anyhow::Context; use pt::{ - flow_model::{FlowJob, FlowJobTableMapping, FlowSyncMode, QRepFlowJob}, + flow_model::{FlowJob, FlowJobMapping, FlowSyncMode, QRepFlowJob}, peerdb_peers::{ peer::Config, BigqueryConfig, DbType, EventHubConfig, MongoConfig, Peer, PostgresConfig, S3Config, SnowflakeConfig, SqlServerConfig, @@ -151,15 +151,15 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { } => { match create_mirror { CDC(cdc) => { - let mut flow_job_table_mappings = vec![]; - for table_mapping in &cdc.table_mappings { - flow_job_table_mappings.push(FlowJobTableMapping { - source_table_identifier: table_mapping + let mut flow_job_mappings = vec![]; + for mapping in &cdc.mappings { + flow_job_mappings.push(FlowJobMapping { + source_identifier: mapping .source_table_identifier .to_string() .to_lowercase(), - target_table_identifier: table_mapping - .target_table_identifier + target_identifier: mapping + .target_identifier .to_string() .to_lowercase(), }); @@ -262,11 +262,16 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { _ => None, }; + let allow_table_additions = match raw_options.remove("allow_table_additions") { + 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(), target_peer: cdc.target_peer.to_string().to_lowercase(), - table_mappings: flow_job_table_mappings, + table_mappings: flow_job_mappings, description: "".to_string(), // TODO: add description do_initial_copy, publication_name, @@ -282,6 +287,8 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { push_batch_size, push_parallelism, max_batch_size, + mapping_type: cdc.mapping_type, + allow_table_additions }; // Error reporting diff --git a/nexus/catalog/src/lib.rs b/nexus/catalog/src/lib.rs index dbce51dfaf..7d80f95500 100644 --- a/nexus/catalog/src/lib.rs +++ b/nexus/catalog/src/lib.rs @@ -397,13 +397,13 @@ impl Catalog { &job.description, &self .normalize_schema_for_table_identifier( - &table_mapping.source_table_identifier, + &table_mapping.source_identifier, source_peer_id, ) .await?, &self .normalize_schema_for_table_identifier( - &table_mapping.target_table_identifier, + &table_mapping.target_identifier, destination_peer_id, ) .await?, diff --git a/nexus/flow-rs/Cargo.toml b/nexus/flow-rs/Cargo.toml index 86d2b871d2..50e0a6e3da 100644 --- a/nexus/flow-rs/Cargo.toml +++ b/nexus/flow-rs/Cargo.toml @@ -21,3 +21,4 @@ tonic = "0.9" tonic-health = "0.9" pt = { path = "../pt" } catalog = { path = "../catalog" } +sqlparser = { path = "../sqlparser-rs", features = ["visitor"] } diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index f15b431066..17e1b83242 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -4,7 +4,7 @@ use anyhow::Context; use catalog::WorkflowDetails; use pt::{ flow_model::{FlowJob, QRepFlowJob}, - peerdb_flow::{QRepWriteMode, QRepWriteType}, + peerdb_flow::{QRepWriteMode, QRepWriteType, MappingType}, peerdb_route, }; use serde_json::Value; @@ -123,11 +123,11 @@ impl FlowGrpcClient { src: pt::peerdb_peers::Peer, dst: pt::peerdb_peers::Peer, ) -> anyhow::Result { - let mut src_dst_name_map: HashMap = HashMap::new(); + let mut src_dst_name_mapping: HashMap = HashMap::new(); job.table_mappings.iter().for_each(|mapping| { - src_dst_name_map.insert( - mapping.source_table_identifier.clone(), - mapping.target_table_identifier.clone(), + src_dst_name_mapping.insert( + mapping.source_identifier.clone(), + mapping.target_identifier.clone(), ); }); @@ -137,12 +137,15 @@ impl FlowGrpcClient { let snapshot_num_rows_per_partition = job.snapshot_num_rows_per_partition; let snapshot_max_parallel_workers = job.snapshot_max_parallel_workers; let snapshot_num_tables_in_parallel = job.snapshot_num_tables_in_parallel; + let mapping_type = match job.mapping_type { + sqlparser::ast::MappingType::Table => pt::peerdb_flow::MappingType::Table, + sqlparser::ast::MappingType::Schema => pt::peerdb_flow::MappingType::Schema + }; - let flow_conn_cfg = pt::peerdb_flow::FlowConnectionConfigs { + let mut flow_conn_cfg = pt::peerdb_flow::FlowConnectionConfigs { source: Some(src), destination: Some(dst), flow_job_name: job.name.clone(), - table_name_mapping: src_dst_name_map, do_initial_copy, publication_name: publication_name.unwrap_or_default(), snapshot_num_rows_per_partition: snapshot_num_rows_per_partition.unwrap_or(0), @@ -165,8 +168,19 @@ impl FlowGrpcClient { push_batch_size: job.push_batch_size.unwrap_or_default(), push_parallelism: job.push_parallelism.unwrap_or_default(), max_batch_size: job.max_batch_size.unwrap_or_default(), + mapping_type: mapping_type as i32, + allow_table_additions: job.allow_table_additions, ..Default::default() }; + match mapping_type { + MappingType::Table => { + flow_conn_cfg.table_name_mapping = src_dst_name_mapping; + }, + MappingType::Schema => { + flow_conn_cfg.schema_mapping = src_dst_name_mapping; + }, + _ => unreachable!() + } self.start_peer_flow(flow_conn_cfg).await } diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index cad02e6dfc..d518ecc956 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -2,13 +2,14 @@ use std::collections::HashMap; use serde::{Deserialize, Serialize}; use serde_json::Value; +use sqlparser::ast::MappingType; use crate::peerdb_flow; #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] -pub struct FlowJobTableMapping { - pub source_table_identifier: String, - pub target_table_identifier: String, +pub struct FlowJobMapping { + pub source_identifier: String, + pub target_identifier: String, } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] @@ -55,12 +56,12 @@ impl ToString for FlowSyncMode { } } -#[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] +#[derive(Debug, PartialEq, Eq, Clone)] pub struct FlowJob { pub name: String, pub source_peer: String, pub target_peer: String, - pub table_mappings: Vec, + pub table_mappings: Vec, pub description: String, pub do_initial_copy: bool, pub publication_name: Option, @@ -76,6 +77,8 @@ pub struct FlowJob { pub push_parallelism: Option, pub push_batch_size: Option, pub max_batch_size: Option, + pub mapping_type: MappingType, + pub allow_table_additions: 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 8e996e6446..0a330ebca6 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -1,14 +1,6 @@ // @generated #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct TableNameMapping { - #[prost(string, tag="1")] - pub source_table_name: ::prost::alloc::string::String, - #[prost(string, tag="2")] - pub destination_table_name: ::prost::alloc::string::String, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct RelationMessageColumn { #[prost(uint32, tag="1")] pub flags: u32, @@ -38,6 +30,7 @@ pub struct FlowConnectionConfigs { pub flow_job_name: ::prost::alloc::string::String, #[prost(message, optional, tag="4")] pub table_schema: ::core::option::Option, + /// if MappingType is TABLE, this contains the table level mappings. #[prost(map="string, string", tag="5")] pub table_name_mapping: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, #[prost(map="uint32, string", tag="6")] @@ -79,6 +72,13 @@ pub struct FlowConnectionConfigs { pub push_batch_size: i64, #[prost(int64, tag="22")] pub push_parallelism: i64, + #[prost(enumeration="MappingType", tag="23")] + pub mapping_type: i32, + /// if MappingType is SCHEMA, this contains the schema level mappings. + #[prost(map="string, string", tag="24")] + pub schema_mapping: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, + #[prost(bool, tag="25")] + pub allow_table_additions: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -130,16 +130,6 @@ pub struct GetLastSyncedIdInput { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct EnsurePullabilityInput { - #[prost(message, optional, tag="1")] - pub peer_connection_config: ::core::option::Option, - #[prost(string, tag="2")] - pub flow_job_name: ::prost::alloc::string::String, - #[prost(string, tag="3")] - pub source_table_identifier: ::prost::alloc::string::String, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct EnsurePullabilityBatchInput { #[prost(message, optional, tag="1")] pub peer_connection_config: ::core::option::Option, @@ -171,12 +161,6 @@ pub mod table_identifier { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct EnsurePullabilityOutput { - #[prost(message, optional, tag="1")] - pub table_identifier: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct EnsurePullabilityBatchOutput { #[prost(map="string, message", tag="1")] pub table_identifier_mapping: ::std::collections::HashMap<::prost::alloc::string::String, TableIdentifier>, @@ -199,6 +183,9 @@ pub struct SetupReplicationInput { pub existing_publication_name: ::prost::alloc::string::String, #[prost(string, tag="7")] pub existing_replication_slot_name: ::prost::alloc::string::String, + /// if length > 0, ignore table name mapping and use schemas as basis of publication creation + #[prost(string, repeated, tag="8")] + pub schemas: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -462,6 +449,121 @@ pub struct ReplayTableSchemaDeltaInput { #[prost(message, optional, tag="2")] pub table_schema_delta: ::core::option::Option, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListTablesInSchemasInput { + #[prost(message, optional, tag="1")] + pub peer_connection_config: ::core::option::Option, + #[prost(map="string, string", tag="2")] + pub schema_mapping: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TablesList { + #[prost(string, repeated, tag="1")] + pub tables: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListTablesInSchemasOutput { + #[prost(map="string, message", tag="1")] + pub schema_to_tables: ::std::collections::HashMap<::prost::alloc::string::String, TablesList>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AdditionalTableDelta { + #[prost(string, tag="1")] + pub table_name: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub src_schema: ::prost::alloc::string::String, + #[prost(string, tag="3")] + pub dst_schema: ::prost::alloc::string::String, + #[prost(uint32, tag="4")] + pub rel_id: u32, + #[prost(message, optional, tag="5")] + pub table_schema: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MirrorDelta { + #[prost(oneof="mirror_delta::Delta", tags="1, 2")] + pub delta: ::core::option::Option, +} +/// Nested message and enum types in `MirrorDelta`. +pub mod mirror_delta { + #[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Delta { + /// adding or dropping columns to an existing table + #[prost(message, tag="1")] + TableSchemaDelta(super::TableSchemaDelta), + /// creating a new table, for MappingType SCHEMA only + #[prost(message, tag="2")] + AdditionalTableDelta(super::AdditionalTableDelta), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct CreateAdditionalTableInput { + #[prost(message, optional, tag="1")] + pub flow_connection_configs: ::core::option::Option, + #[prost(message, optional, tag="2")] + pub additional_table_info: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SyncResponse { + /// FirstSyncedCheckPointID is the first ID that was synced. + #[prost(int64, tag="1")] + pub first_synced_checkpoint_id: i64, + /// LastSyncedCheckPointID is the last ID that was synced. + #[prost(int64, tag="2")] + pub last_synced_checkpoint_id: i64, + /// NumRecordsSynced is the number of records that were synced. + #[prost(int64, tag="3")] + pub num_records_synced: i64, + /// CurrentSyncBatchID is the ID of the currently synced batch. + #[prost(int64, tag="4")] + pub current_sync_batch_id: i64, + /// TableNameRowsMapping tells how many records need to be synced to each destination table. + #[prost(map="string, uint32", tag="5")] + pub table_name_rows_mapping: ::std::collections::HashMap<::prost::alloc::string::String, u32>, + /// to be stored in state for future PullFlows + #[prost(map="uint32, message", tag="6")] + pub relation_message_mapping: ::std::collections::HashMap, + /// if not-nil, needs to be applied before next StartFlow + #[prost(message, optional, tag="7")] + pub mirror_delta: ::core::option::Option, +} +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum MappingType { + Unknown = 0, + Table = 1, + Schema = 2, +} +impl MappingType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + MappingType::Unknown => "UNKNOWN", + MappingType::Table => "TABLE", + MappingType::Schema => "SCHEMA", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "UNKNOWN" => Some(Self::Unknown), + "TABLE" => Some(Self::Table), + "SCHEMA" => Some(Self::Schema), + _ => None, + } + } +} /// 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 c8ad147bdc..1a698687d0 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -1,4 +1,288 @@ // @generated +impl serde::Serialize for AdditionalTableDelta { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.table_name.is_empty() { + len += 1; + } + if !self.src_schema.is_empty() { + len += 1; + } + if !self.dst_schema.is_empty() { + len += 1; + } + if self.rel_id != 0 { + len += 1; + } + if self.table_schema.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.AdditionalTableDelta", len)?; + if !self.table_name.is_empty() { + struct_ser.serialize_field("tableName", &self.table_name)?; + } + if !self.src_schema.is_empty() { + struct_ser.serialize_field("srcSchema", &self.src_schema)?; + } + if !self.dst_schema.is_empty() { + struct_ser.serialize_field("dstSchema", &self.dst_schema)?; + } + if self.rel_id != 0 { + struct_ser.serialize_field("relId", &self.rel_id)?; + } + if let Some(v) = self.table_schema.as_ref() { + struct_ser.serialize_field("tableSchema", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for AdditionalTableDelta { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "table_name", + "tableName", + "src_schema", + "srcSchema", + "dst_schema", + "dstSchema", + "rel_id", + "relId", + "table_schema", + "tableSchema", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + TableName, + SrcSchema, + DstSchema, + RelId, + TableSchema, + __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 { + "tableName" | "table_name" => Ok(GeneratedField::TableName), + "srcSchema" | "src_schema" => Ok(GeneratedField::SrcSchema), + "dstSchema" | "dst_schema" => Ok(GeneratedField::DstSchema), + "relId" | "rel_id" => Ok(GeneratedField::RelId), + "tableSchema" | "table_schema" => Ok(GeneratedField::TableSchema), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = AdditionalTableDelta; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.AdditionalTableDelta") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut table_name__ = None; + let mut src_schema__ = None; + let mut dst_schema__ = None; + let mut rel_id__ = None; + let mut table_schema__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::TableName => { + if table_name__.is_some() { + return Err(serde::de::Error::duplicate_field("tableName")); + } + table_name__ = Some(map.next_value()?); + } + GeneratedField::SrcSchema => { + if src_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("srcSchema")); + } + src_schema__ = Some(map.next_value()?); + } + GeneratedField::DstSchema => { + if dst_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("dstSchema")); + } + dst_schema__ = Some(map.next_value()?); + } + GeneratedField::RelId => { + if rel_id__.is_some() { + return Err(serde::de::Error::duplicate_field("relId")); + } + rel_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::TableSchema => { + if table_schema__.is_some() { + return Err(serde::de::Error::duplicate_field("tableSchema")); + } + table_schema__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(AdditionalTableDelta { + table_name: table_name__.unwrap_or_default(), + src_schema: src_schema__.unwrap_or_default(), + dst_schema: dst_schema__.unwrap_or_default(), + rel_id: rel_id__.unwrap_or_default(), + table_schema: table_schema__, + }) + } + } + deserializer.deserialize_struct("peerdb_flow.AdditionalTableDelta", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for CreateAdditionalTableInput { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.flow_connection_configs.is_some() { + len += 1; + } + if self.additional_table_info.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.CreateAdditionalTableInput", len)?; + if let Some(v) = self.flow_connection_configs.as_ref() { + struct_ser.serialize_field("flowConnectionConfigs", v)?; + } + if let Some(v) = self.additional_table_info.as_ref() { + struct_ser.serialize_field("additionalTableInfo", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for CreateAdditionalTableInput { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "flow_connection_configs", + "flowConnectionConfigs", + "additional_table_info", + "additionalTableInfo", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + FlowConnectionConfigs, + AdditionalTableInfo, + __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 { + "flowConnectionConfigs" | "flow_connection_configs" => Ok(GeneratedField::FlowConnectionConfigs), + "additionalTableInfo" | "additional_table_info" => Ok(GeneratedField::AdditionalTableInfo), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = CreateAdditionalTableInput; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.CreateAdditionalTableInput") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flow_connection_configs__ = None; + let mut additional_table_info__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::FlowConnectionConfigs => { + if flow_connection_configs__.is_some() { + return Err(serde::de::Error::duplicate_field("flowConnectionConfigs")); + } + flow_connection_configs__ = map.next_value()?; + } + GeneratedField::AdditionalTableInfo => { + if additional_table_info__.is_some() { + return Err(serde::de::Error::duplicate_field("additionalTableInfo")); + } + additional_table_info__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(CreateAdditionalTableInput { + flow_connection_configs: flow_connection_configs__, + additional_table_info: additional_table_info__, + }) + } + } + deserializer.deserialize_struct("peerdb_flow.CreateAdditionalTableInput", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for CreateRawTableInput { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -689,7 +973,7 @@ impl<'de> serde::Deserialize<'de> for EnsurePullabilityBatchOutput { deserializer.deserialize_struct("peerdb_flow.EnsurePullabilityBatchOutput", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for EnsurePullabilityInput { +impl serde::Serialize for FlowConnectionConfigs { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -697,257 +981,29 @@ impl serde::Serialize for EnsurePullabilityInput { { use serde::ser::SerializeStruct; let mut len = 0; - if self.peer_connection_config.is_some() { + if self.source.is_some() { + len += 1; + } + if self.destination.is_some() { len += 1; } if !self.flow_job_name.is_empty() { len += 1; } - if !self.source_table_identifier.is_empty() { + if self.table_schema.is_some() { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_flow.EnsurePullabilityInput", len)?; - if let Some(v) = self.peer_connection_config.as_ref() { - struct_ser.serialize_field("peerConnectionConfig", v)?; + if !self.table_name_mapping.is_empty() { + len += 1; } - if !self.flow_job_name.is_empty() { - struct_ser.serialize_field("flowJobName", &self.flow_job_name)?; + if !self.src_table_id_name_mapping.is_empty() { + len += 1; } - if !self.source_table_identifier.is_empty() { - struct_ser.serialize_field("sourceTableIdentifier", &self.source_table_identifier)?; + if !self.table_name_schema_mapping.is_empty() { + len += 1; } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for EnsurePullabilityInput { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "peer_connection_config", - "peerConnectionConfig", - "flow_job_name", - "flowJobName", - "source_table_identifier", - "sourceTableIdentifier", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - PeerConnectionConfig, - FlowJobName, - SourceTableIdentifier, - __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 { - "peerConnectionConfig" | "peer_connection_config" => Ok(GeneratedField::PeerConnectionConfig), - "flowJobName" | "flow_job_name" => Ok(GeneratedField::FlowJobName), - "sourceTableIdentifier" | "source_table_identifier" => Ok(GeneratedField::SourceTableIdentifier), - _ => Ok(GeneratedField::__SkipField__), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = EnsurePullabilityInput; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.EnsurePullabilityInput") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut peer_connection_config__ = None; - let mut flow_job_name__ = None; - let mut source_table_identifier__ = None; - while let Some(k) = map.next_key()? { - match k { - GeneratedField::PeerConnectionConfig => { - if peer_connection_config__.is_some() { - return Err(serde::de::Error::duplicate_field("peerConnectionConfig")); - } - peer_connection_config__ = map.next_value()?; - } - GeneratedField::FlowJobName => { - if flow_job_name__.is_some() { - return Err(serde::de::Error::duplicate_field("flowJobName")); - } - flow_job_name__ = Some(map.next_value()?); - } - GeneratedField::SourceTableIdentifier => { - if source_table_identifier__.is_some() { - return Err(serde::de::Error::duplicate_field("sourceTableIdentifier")); - } - source_table_identifier__ = Some(map.next_value()?); - } - GeneratedField::__SkipField__ => { - let _ = map.next_value::()?; - } - } - } - Ok(EnsurePullabilityInput { - peer_connection_config: peer_connection_config__, - flow_job_name: flow_job_name__.unwrap_or_default(), - source_table_identifier: source_table_identifier__.unwrap_or_default(), - }) - } - } - deserializer.deserialize_struct("peerdb_flow.EnsurePullabilityInput", FIELDS, GeneratedVisitor) - } -} -impl serde::Serialize for EnsurePullabilityOutput { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if self.table_identifier.is_some() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("peerdb_flow.EnsurePullabilityOutput", len)?; - if let Some(v) = self.table_identifier.as_ref() { - struct_ser.serialize_field("tableIdentifier", v)?; - } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for EnsurePullabilityOutput { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "table_identifier", - "tableIdentifier", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - TableIdentifier, - __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 { - "tableIdentifier" | "table_identifier" => Ok(GeneratedField::TableIdentifier), - _ => Ok(GeneratedField::__SkipField__), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = EnsurePullabilityOutput; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.EnsurePullabilityOutput") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut table_identifier__ = None; - while let Some(k) = map.next_key()? { - match k { - GeneratedField::TableIdentifier => { - if table_identifier__.is_some() { - return Err(serde::de::Error::duplicate_field("tableIdentifier")); - } - table_identifier__ = map.next_value()?; - } - GeneratedField::__SkipField__ => { - let _ = map.next_value::()?; - } - } - } - Ok(EnsurePullabilityOutput { - table_identifier: table_identifier__, - }) - } - } - deserializer.deserialize_struct("peerdb_flow.EnsurePullabilityOutput", FIELDS, GeneratedVisitor) - } -} -impl serde::Serialize for FlowConnectionConfigs { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if self.source.is_some() { - len += 1; - } - if self.destination.is_some() { - len += 1; - } - if !self.flow_job_name.is_empty() { - len += 1; - } - if self.table_schema.is_some() { - len += 1; - } - if !self.table_name_mapping.is_empty() { - len += 1; - } - if !self.src_table_id_name_mapping.is_empty() { - len += 1; - } - if !self.table_name_schema_mapping.is_empty() { - len += 1; - } - if self.metadata_peer.is_some() { - len += 1; + if self.metadata_peer.is_some() { + len += 1; } if self.max_batch_size != 0 { len += 1; @@ -991,6 +1047,15 @@ impl serde::Serialize for FlowConnectionConfigs { if self.push_parallelism != 0 { len += 1; } + if self.mapping_type != 0 { + len += 1; + } + if !self.schema_mapping.is_empty() { + len += 1; + } + if self.allow_table_additions { + 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)?; @@ -1062,6 +1127,17 @@ impl serde::Serialize for FlowConnectionConfigs { if self.push_parallelism != 0 { struct_ser.serialize_field("pushParallelism", ToString::to_string(&self.push_parallelism).as_str())?; } + if self.mapping_type != 0 { + let v = MappingType::from_i32(self.mapping_type) + .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.mapping_type)))?; + struct_ser.serialize_field("mappingType", &v)?; + } + if !self.schema_mapping.is_empty() { + struct_ser.serialize_field("schemaMapping", &self.schema_mapping)?; + } + if self.allow_table_additions { + struct_ser.serialize_field("allowTableAdditions", &self.allow_table_additions)?; + } struct_ser.end() } } @@ -1114,6 +1190,12 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "pushBatchSize", "push_parallelism", "pushParallelism", + "mapping_type", + "mappingType", + "schema_mapping", + "schemaMapping", + "allow_table_additions", + "allowTableAdditions", ]; #[allow(clippy::enum_variant_names)] @@ -1140,6 +1222,9 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { ReplicationSlotName, PushBatchSize, PushParallelism, + MappingType, + SchemaMapping, + AllowTableAdditions, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1184,6 +1269,9 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "replicationSlotName" | "replication_slot_name" => Ok(GeneratedField::ReplicationSlotName), "pushBatchSize" | "push_batch_size" => Ok(GeneratedField::PushBatchSize), "pushParallelism" | "push_parallelism" => Ok(GeneratedField::PushParallelism), + "mappingType" | "mapping_type" => Ok(GeneratedField::MappingType), + "schemaMapping" | "schema_mapping" => Ok(GeneratedField::SchemaMapping), + "allowTableAdditions" | "allow_table_additions" => Ok(GeneratedField::AllowTableAdditions), _ => Ok(GeneratedField::__SkipField__), } } @@ -1225,6 +1313,9 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { let mut replication_slot_name__ = None; let mut push_batch_size__ = None; let mut push_parallelism__ = None; + let mut mapping_type__ = None; + let mut schema_mapping__ = None; + let mut allow_table_additions__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Source => { @@ -1378,6 +1469,26 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } + GeneratedField::MappingType => { + if mapping_type__.is_some() { + return Err(serde::de::Error::duplicate_field("mappingType")); + } + mapping_type__ = Some(map.next_value::()? as i32); + } + GeneratedField::SchemaMapping => { + if schema_mapping__.is_some() { + return Err(serde::de::Error::duplicate_field("schemaMapping")); + } + schema_mapping__ = Some( + map.next_value::>()? + ); + } + GeneratedField::AllowTableAdditions => { + if allow_table_additions__.is_some() { + return Err(serde::de::Error::duplicate_field("allowTableAdditions")); + } + allow_table_additions__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1406,6 +1517,9 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { replication_slot_name: replication_slot_name__.unwrap_or_default(), push_batch_size: push_batch_size__.unwrap_or_default(), push_parallelism: push_parallelism__.unwrap_or_default(), + mapping_type: mapping_type__.unwrap_or_default(), + schema_mapping: schema_mapping__.unwrap_or_default(), + allow_table_additions: allow_table_additions__.unwrap_or_default(), }) } } @@ -1752,31 +1866,371 @@ impl serde::Serialize for IntPartitionRange { if self.end != 0 { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_flow.IntPartitionRange", len)?; - if self.start != 0 { - struct_ser.serialize_field("start", ToString::to_string(&self.start).as_str())?; - } - if self.end != 0 { - struct_ser.serialize_field("end", ToString::to_string(&self.end).as_str())?; + let mut struct_ser = serializer.serialize_struct("peerdb_flow.IntPartitionRange", len)?; + if self.start != 0 { + struct_ser.serialize_field("start", ToString::to_string(&self.start).as_str())?; + } + if self.end != 0 { + struct_ser.serialize_field("end", ToString::to_string(&self.end).as_str())?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for IntPartitionRange { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "start", + "end", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Start, + End, + __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 { + "start" => Ok(GeneratedField::Start), + "end" => Ok(GeneratedField::End), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = IntPartitionRange; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.IntPartitionRange") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut start__ = None; + let mut end__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Start => { + if start__.is_some() { + return Err(serde::de::Error::duplicate_field("start")); + } + start__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::End => { + if end__.is_some() { + return Err(serde::de::Error::duplicate_field("end")); + } + end__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(IntPartitionRange { + start: start__.unwrap_or_default(), + end: end__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.IntPartitionRange", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for LastSyncState { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.checkpoint != 0 { + len += 1; + } + if self.last_synced_at.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.LastSyncState", len)?; + if self.checkpoint != 0 { + struct_ser.serialize_field("checkpoint", ToString::to_string(&self.checkpoint).as_str())?; + } + if let Some(v) = self.last_synced_at.as_ref() { + struct_ser.serialize_field("lastSyncedAt", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for LastSyncState { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "checkpoint", + "last_synced_at", + "lastSyncedAt", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Checkpoint, + LastSyncedAt, + __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 { + "checkpoint" => Ok(GeneratedField::Checkpoint), + "lastSyncedAt" | "last_synced_at" => Ok(GeneratedField::LastSyncedAt), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = LastSyncState; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.LastSyncState") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut checkpoint__ = None; + let mut last_synced_at__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Checkpoint => { + if checkpoint__.is_some() { + return Err(serde::de::Error::duplicate_field("checkpoint")); + } + checkpoint__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::LastSyncedAt => { + if last_synced_at__.is_some() { + return Err(serde::de::Error::duplicate_field("lastSyncedAt")); + } + last_synced_at__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(LastSyncState { + checkpoint: checkpoint__.unwrap_or_default(), + last_synced_at: last_synced_at__, + }) + } + } + deserializer.deserialize_struct("peerdb_flow.LastSyncState", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ListTablesInSchemasInput { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.peer_connection_config.is_some() { + len += 1; + } + if !self.schema_mapping.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.ListTablesInSchemasInput", len)?; + if let Some(v) = self.peer_connection_config.as_ref() { + struct_ser.serialize_field("peerConnectionConfig", v)?; + } + if !self.schema_mapping.is_empty() { + struct_ser.serialize_field("schemaMapping", &self.schema_mapping)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ListTablesInSchemasInput { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "peer_connection_config", + "peerConnectionConfig", + "schema_mapping", + "schemaMapping", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + PeerConnectionConfig, + SchemaMapping, + __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 { + "peerConnectionConfig" | "peer_connection_config" => Ok(GeneratedField::PeerConnectionConfig), + "schemaMapping" | "schema_mapping" => Ok(GeneratedField::SchemaMapping), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ListTablesInSchemasInput; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.ListTablesInSchemasInput") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut peer_connection_config__ = None; + let mut schema_mapping__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::PeerConnectionConfig => { + if peer_connection_config__.is_some() { + return Err(serde::de::Error::duplicate_field("peerConnectionConfig")); + } + peer_connection_config__ = map.next_value()?; + } + GeneratedField::SchemaMapping => { + if schema_mapping__.is_some() { + return Err(serde::de::Error::duplicate_field("schemaMapping")); + } + schema_mapping__ = Some( + map.next_value::>()? + ); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(ListTablesInSchemasInput { + peer_connection_config: peer_connection_config__, + schema_mapping: schema_mapping__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.ListTablesInSchemasInput", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ListTablesInSchemasOutput { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.schema_to_tables.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.ListTablesInSchemasOutput", len)?; + if !self.schema_to_tables.is_empty() { + struct_ser.serialize_field("schemaToTables", &self.schema_to_tables)?; } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for IntPartitionRange { +impl<'de> serde::Deserialize<'de> for ListTablesInSchemasOutput { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "start", - "end", + "schema_to_tables", + "schemaToTables", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Start, - End, + SchemaToTables, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1799,8 +2253,7 @@ impl<'de> serde::Deserialize<'de> for IntPartitionRange { E: serde::de::Error, { match value { - "start" => Ok(GeneratedField::Start), - "end" => Ok(GeneratedField::End), + "schemaToTables" | "schema_to_tables" => Ok(GeneratedField::SchemaToTables), _ => Ok(GeneratedField::__SkipField__), } } @@ -1810,51 +2263,117 @@ impl<'de> serde::Deserialize<'de> for IntPartitionRange { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = IntPartitionRange; + type Value = ListTablesInSchemasOutput; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.IntPartitionRange") + formatter.write_str("struct peerdb_flow.ListTablesInSchemasOutput") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut start__ = None; - let mut end__ = None; + let mut schema_to_tables__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::Start => { - if start__.is_some() { - return Err(serde::de::Error::duplicate_field("start")); - } - start__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; - } - GeneratedField::End => { - if end__.is_some() { - return Err(serde::de::Error::duplicate_field("end")); + GeneratedField::SchemaToTables => { + if schema_to_tables__.is_some() { + return Err(serde::de::Error::duplicate_field("schemaToTables")); } - end__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + schema_to_tables__ = Some( + map.next_value::>()? + ); } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } } } - Ok(IntPartitionRange { - start: start__.unwrap_or_default(), - end: end__.unwrap_or_default(), + Ok(ListTablesInSchemasOutput { + schema_to_tables: schema_to_tables__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("peerdb_flow.IntPartitionRange", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_flow.ListTablesInSchemasOutput", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for LastSyncState { +impl serde::Serialize for MappingType { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Unknown => "UNKNOWN", + Self::Table => "TABLE", + Self::Schema => "SCHEMA", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for MappingType { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "UNKNOWN", + "TABLE", + "SCHEMA", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MappingType; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + use std::convert::TryFrom; + i32::try_from(v) + .ok() + .and_then(MappingType::from_i32) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + use std::convert::TryFrom; + i32::try_from(v) + .ok() + .and_then(MappingType::from_i32) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "UNKNOWN" => Ok(MappingType::Unknown), + "TABLE" => Ok(MappingType::Table), + "SCHEMA" => Ok(MappingType::Schema), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for MirrorDelta { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -1862,38 +2381,40 @@ impl serde::Serialize for LastSyncState { { use serde::ser::SerializeStruct; let mut len = 0; - if self.checkpoint != 0 { - len += 1; - } - if self.last_synced_at.is_some() { + if self.delta.is_some() { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_flow.LastSyncState", len)?; - if self.checkpoint != 0 { - struct_ser.serialize_field("checkpoint", ToString::to_string(&self.checkpoint).as_str())?; - } - if let Some(v) = self.last_synced_at.as_ref() { - struct_ser.serialize_field("lastSyncedAt", v)?; + let mut struct_ser = serializer.serialize_struct("peerdb_flow.MirrorDelta", len)?; + if let Some(v) = self.delta.as_ref() { + match v { + mirror_delta::Delta::TableSchemaDelta(v) => { + struct_ser.serialize_field("tableSchemaDelta", v)?; + } + mirror_delta::Delta::AdditionalTableDelta(v) => { + struct_ser.serialize_field("additionalTableDelta", v)?; + } + } } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for LastSyncState { +impl<'de> serde::Deserialize<'de> for MirrorDelta { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "checkpoint", - "last_synced_at", - "lastSyncedAt", + "table_schema_delta", + "tableSchemaDelta", + "additional_table_delta", + "additionalTableDelta", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Checkpoint, - LastSyncedAt, + TableSchemaDelta, + AdditionalTableDelta, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1916,8 +2437,8 @@ impl<'de> serde::Deserialize<'de> for LastSyncState { E: serde::de::Error, { match value { - "checkpoint" => Ok(GeneratedField::Checkpoint), - "lastSyncedAt" | "last_synced_at" => Ok(GeneratedField::LastSyncedAt), + "tableSchemaDelta" | "table_schema_delta" => Ok(GeneratedField::TableSchemaDelta), + "additionalTableDelta" | "additional_table_delta" => Ok(GeneratedField::AdditionalTableDelta), _ => Ok(GeneratedField::__SkipField__), } } @@ -1927,46 +2448,44 @@ impl<'de> serde::Deserialize<'de> for LastSyncState { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = LastSyncState; + type Value = MirrorDelta; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.LastSyncState") + formatter.write_str("struct peerdb_flow.MirrorDelta") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut checkpoint__ = None; - let mut last_synced_at__ = None; + let mut delta__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::Checkpoint => { - if checkpoint__.is_some() { - return Err(serde::de::Error::duplicate_field("checkpoint")); + GeneratedField::TableSchemaDelta => { + if delta__.is_some() { + return Err(serde::de::Error::duplicate_field("tableSchemaDelta")); } - checkpoint__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + delta__ = map.next_value::<::std::option::Option<_>>()?.map(mirror_delta::Delta::TableSchemaDelta) +; } - GeneratedField::LastSyncedAt => { - if last_synced_at__.is_some() { - return Err(serde::de::Error::duplicate_field("lastSyncedAt")); + GeneratedField::AdditionalTableDelta => { + if delta__.is_some() { + return Err(serde::de::Error::duplicate_field("additionalTableDelta")); } - last_synced_at__ = map.next_value()?; + delta__ = map.next_value::<::std::option::Option<_>>()?.map(mirror_delta::Delta::AdditionalTableDelta) +; } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } } } - Ok(LastSyncState { - checkpoint: checkpoint__.unwrap_or_default(), - last_synced_at: last_synced_at__, + Ok(MirrorDelta { + delta: delta__, }) } } - deserializer.deserialize_struct("peerdb_flow.LastSyncState", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_flow.MirrorDelta", FIELDS, GeneratedVisitor) } } impl serde::Serialize for NormalizeFlowOptions { @@ -4161,6 +4680,9 @@ impl serde::Serialize for SetupReplicationInput { if !self.existing_replication_slot_name.is_empty() { len += 1; } + if !self.schemas.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.SetupReplicationInput", len)?; if let Some(v) = self.peer_connection_config.as_ref() { struct_ser.serialize_field("peerConnectionConfig", v)?; @@ -4183,6 +4705,9 @@ impl serde::Serialize for SetupReplicationInput { if !self.existing_replication_slot_name.is_empty() { struct_ser.serialize_field("existingReplicationSlotName", &self.existing_replication_slot_name)?; } + if !self.schemas.is_empty() { + struct_ser.serialize_field("schemas", &self.schemas)?; + } struct_ser.end() } } @@ -4207,6 +4732,7 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { "existingPublicationName", "existing_replication_slot_name", "existingReplicationSlotName", + "schemas", ]; #[allow(clippy::enum_variant_names)] @@ -4218,6 +4744,7 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { DoInitialCopy, ExistingPublicationName, ExistingReplicationSlotName, + Schemas, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4247,6 +4774,7 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { "doInitialCopy" | "do_initial_copy" => Ok(GeneratedField::DoInitialCopy), "existingPublicationName" | "existing_publication_name" => Ok(GeneratedField::ExistingPublicationName), "existingReplicationSlotName" | "existing_replication_slot_name" => Ok(GeneratedField::ExistingReplicationSlotName), + "schemas" => Ok(GeneratedField::Schemas), _ => Ok(GeneratedField::__SkipField__), } } @@ -4273,6 +4801,7 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { let mut do_initial_copy__ = None; let mut existing_publication_name__ = None; let mut existing_replication_slot_name__ = None; + let mut schemas__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::PeerConnectionConfig => { @@ -4319,6 +4848,12 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { } existing_replication_slot_name__ = Some(map.next_value()?); } + GeneratedField::Schemas => { + if schemas__.is_some() { + return Err(serde::de::Error::duplicate_field("schemas")); + } + schemas__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -4332,6 +4867,7 @@ impl<'de> serde::Deserialize<'de> for SetupReplicationInput { do_initial_copy: do_initial_copy__.unwrap_or_default(), existing_publication_name: existing_publication_name__.unwrap_or_default(), existing_replication_slot_name: existing_replication_slot_name__.unwrap_or_default(), + schemas: schemas__.unwrap_or_default(), }) } } @@ -4630,13 +5166,118 @@ impl<'de> serde::Deserialize<'de> for StartNormalizeInput { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "flow_connection_configs", - "flowConnectionConfigs", + "flow_connection_configs", + "flowConnectionConfigs", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + FlowConnectionConfigs, + __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 { + "flowConnectionConfigs" | "flow_connection_configs" => Ok(GeneratedField::FlowConnectionConfigs), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = StartNormalizeInput; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.StartNormalizeInput") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flow_connection_configs__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::FlowConnectionConfigs => { + if flow_connection_configs__.is_some() { + return Err(serde::de::Error::duplicate_field("flowConnectionConfigs")); + } + flow_connection_configs__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(StartNormalizeInput { + flow_connection_configs: flow_connection_configs__, + }) + } + } + deserializer.deserialize_struct("peerdb_flow.StartNormalizeInput", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for SyncFlowOptions { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.batch_size != 0 { + len += 1; + } + if !self.relation_message_mapping.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.SyncFlowOptions", len)?; + if self.batch_size != 0 { + struct_ser.serialize_field("batchSize", &self.batch_size)?; + } + if !self.relation_message_mapping.is_empty() { + struct_ser.serialize_field("relationMessageMapping", &self.relation_message_mapping)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SyncFlowOptions { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "batch_size", + "batchSize", + "relation_message_mapping", + "relationMessageMapping", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - FlowConnectionConfigs, + BatchSize, + RelationMessageMapping, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4659,7 +5300,8 @@ impl<'de> serde::Deserialize<'de> for StartNormalizeInput { E: serde::de::Error, { match value { - "flowConnectionConfigs" | "flow_connection_configs" => Ok(GeneratedField::FlowConnectionConfigs), + "batchSize" | "batch_size" => Ok(GeneratedField::BatchSize), + "relationMessageMapping" | "relation_message_mapping" => Ok(GeneratedField::RelationMessageMapping), _ => Ok(GeneratedField::__SkipField__), } } @@ -4669,39 +5311,52 @@ impl<'de> serde::Deserialize<'de> for StartNormalizeInput { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = StartNormalizeInput; + type Value = SyncFlowOptions; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.StartNormalizeInput") + formatter.write_str("struct peerdb_flow.SyncFlowOptions") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut flow_connection_configs__ = None; + let mut batch_size__ = None; + let mut relation_message_mapping__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::FlowConnectionConfigs => { - if flow_connection_configs__.is_some() { - return Err(serde::de::Error::duplicate_field("flowConnectionConfigs")); + GeneratedField::BatchSize => { + if batch_size__.is_some() { + return Err(serde::de::Error::duplicate_field("batchSize")); } - flow_connection_configs__ = map.next_value()?; + batch_size__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::RelationMessageMapping => { + if relation_message_mapping__.is_some() { + return Err(serde::de::Error::duplicate_field("relationMessageMapping")); + } + relation_message_mapping__ = Some( + map.next_value::, _>>()? + .into_iter().map(|(k,v)| (k.0, v)).collect() + ); } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } } } - Ok(StartNormalizeInput { - flow_connection_configs: flow_connection_configs__, + Ok(SyncFlowOptions { + batch_size: batch_size__.unwrap_or_default(), + relation_message_mapping: relation_message_mapping__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("peerdb_flow.StartNormalizeInput", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_flow.SyncFlowOptions", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for SyncFlowOptions { +impl serde::Serialize for SyncResponse { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -4709,39 +5364,84 @@ impl serde::Serialize for SyncFlowOptions { { use serde::ser::SerializeStruct; let mut len = 0; - if self.batch_size != 0 { + if self.first_synced_checkpoint_id != 0 { + len += 1; + } + if self.last_synced_checkpoint_id != 0 { + len += 1; + } + if self.num_records_synced != 0 { + len += 1; + } + if self.current_sync_batch_id != 0 { + len += 1; + } + if !self.table_name_rows_mapping.is_empty() { len += 1; } if !self.relation_message_mapping.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("peerdb_flow.SyncFlowOptions", len)?; - if self.batch_size != 0 { - struct_ser.serialize_field("batchSize", &self.batch_size)?; + if self.mirror_delta.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.SyncResponse", len)?; + if self.first_synced_checkpoint_id != 0 { + struct_ser.serialize_field("firstSyncedCheckpointId", ToString::to_string(&self.first_synced_checkpoint_id).as_str())?; + } + if self.last_synced_checkpoint_id != 0 { + struct_ser.serialize_field("lastSyncedCheckpointId", ToString::to_string(&self.last_synced_checkpoint_id).as_str())?; + } + if self.num_records_synced != 0 { + struct_ser.serialize_field("numRecordsSynced", ToString::to_string(&self.num_records_synced).as_str())?; + } + if self.current_sync_batch_id != 0 { + struct_ser.serialize_field("currentSyncBatchId", ToString::to_string(&self.current_sync_batch_id).as_str())?; + } + if !self.table_name_rows_mapping.is_empty() { + struct_ser.serialize_field("tableNameRowsMapping", &self.table_name_rows_mapping)?; } if !self.relation_message_mapping.is_empty() { struct_ser.serialize_field("relationMessageMapping", &self.relation_message_mapping)?; } + if let Some(v) = self.mirror_delta.as_ref() { + struct_ser.serialize_field("mirrorDelta", v)?; + } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for SyncFlowOptions { +impl<'de> serde::Deserialize<'de> for SyncResponse { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "batch_size", - "batchSize", + "first_synced_checkpoint_id", + "firstSyncedCheckpointId", + "last_synced_checkpoint_id", + "lastSyncedCheckpointId", + "num_records_synced", + "numRecordsSynced", + "current_sync_batch_id", + "currentSyncBatchId", + "table_name_rows_mapping", + "tableNameRowsMapping", "relation_message_mapping", "relationMessageMapping", + "mirror_delta", + "mirrorDelta", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - BatchSize, + FirstSyncedCheckpointId, + LastSyncedCheckpointId, + NumRecordsSynced, + CurrentSyncBatchId, + TableNameRowsMapping, RelationMessageMapping, + MirrorDelta, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4764,8 +5464,13 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { E: serde::de::Error, { match value { - "batchSize" | "batch_size" => Ok(GeneratedField::BatchSize), + "firstSyncedCheckpointId" | "first_synced_checkpoint_id" => Ok(GeneratedField::FirstSyncedCheckpointId), + "lastSyncedCheckpointId" | "last_synced_checkpoint_id" => Ok(GeneratedField::LastSyncedCheckpointId), + "numRecordsSynced" | "num_records_synced" => Ok(GeneratedField::NumRecordsSynced), + "currentSyncBatchId" | "current_sync_batch_id" => Ok(GeneratedField::CurrentSyncBatchId), + "tableNameRowsMapping" | "table_name_rows_mapping" => Ok(GeneratedField::TableNameRowsMapping), "relationMessageMapping" | "relation_message_mapping" => Ok(GeneratedField::RelationMessageMapping), + "mirrorDelta" | "mirror_delta" => Ok(GeneratedField::MirrorDelta), _ => Ok(GeneratedField::__SkipField__), } } @@ -4775,28 +5480,66 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = SyncFlowOptions; + type Value = SyncResponse; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.SyncFlowOptions") + formatter.write_str("struct peerdb_flow.SyncResponse") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut batch_size__ = None; + let mut first_synced_checkpoint_id__ = None; + let mut last_synced_checkpoint_id__ = None; + let mut num_records_synced__ = None; + let mut current_sync_batch_id__ = None; + let mut table_name_rows_mapping__ = None; let mut relation_message_mapping__ = None; + let mut mirror_delta__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::BatchSize => { - if batch_size__.is_some() { - return Err(serde::de::Error::duplicate_field("batchSize")); + GeneratedField::FirstSyncedCheckpointId => { + if first_synced_checkpoint_id__.is_some() { + return Err(serde::de::Error::duplicate_field("firstSyncedCheckpointId")); } - batch_size__ = + first_synced_checkpoint_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::LastSyncedCheckpointId => { + if last_synced_checkpoint_id__.is_some() { + return Err(serde::de::Error::duplicate_field("lastSyncedCheckpointId")); + } + last_synced_checkpoint_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::NumRecordsSynced => { + if num_records_synced__.is_some() { + return Err(serde::de::Error::duplicate_field("numRecordsSynced")); + } + num_records_synced__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::CurrentSyncBatchId => { + if current_sync_batch_id__.is_some() { + return Err(serde::de::Error::duplicate_field("currentSyncBatchId")); + } + current_sync_batch_id__ = Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } + GeneratedField::TableNameRowsMapping => { + if table_name_rows_mapping__.is_some() { + return Err(serde::de::Error::duplicate_field("tableNameRowsMapping")); + } + table_name_rows_mapping__ = Some( + map.next_value::>>()? + .into_iter().map(|(k,v)| (k, v.0)).collect() + ); + } GeneratedField::RelationMessageMapping => { if relation_message_mapping__.is_some() { return Err(serde::de::Error::duplicate_field("relationMessageMapping")); @@ -4806,18 +5549,29 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { .into_iter().map(|(k,v)| (k.0, v)).collect() ); } + GeneratedField::MirrorDelta => { + if mirror_delta__.is_some() { + return Err(serde::de::Error::duplicate_field("mirrorDelta")); + } + mirror_delta__ = map.next_value()?; + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } } } - Ok(SyncFlowOptions { - batch_size: batch_size__.unwrap_or_default(), + Ok(SyncResponse { + first_synced_checkpoint_id: first_synced_checkpoint_id__.unwrap_or_default(), + last_synced_checkpoint_id: last_synced_checkpoint_id__.unwrap_or_default(), + num_records_synced: num_records_synced__.unwrap_or_default(), + current_sync_batch_id: current_sync_batch_id__.unwrap_or_default(), + table_name_rows_mapping: table_name_rows_mapping__.unwrap_or_default(), relation_message_mapping: relation_message_mapping__.unwrap_or_default(), + mirror_delta: mirror_delta__, }) } } - deserializer.deserialize_struct("peerdb_flow.SyncFlowOptions", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("peerdb_flow.SyncResponse", FIELDS, GeneratedVisitor) } } impl serde::Serialize for Tid { @@ -5151,120 +5905,6 @@ impl<'de> serde::Deserialize<'de> for TableIdentifier { deserializer.deserialize_struct("peerdb_flow.TableIdentifier", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for TableNameMapping { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if !self.source_table_name.is_empty() { - len += 1; - } - if !self.destination_table_name.is_empty() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("peerdb_flow.TableNameMapping", len)?; - if !self.source_table_name.is_empty() { - struct_ser.serialize_field("sourceTableName", &self.source_table_name)?; - } - if !self.destination_table_name.is_empty() { - struct_ser.serialize_field("destinationTableName", &self.destination_table_name)?; - } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for TableNameMapping { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "source_table_name", - "sourceTableName", - "destination_table_name", - "destinationTableName", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - SourceTableName, - DestinationTableName, - __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 { - "sourceTableName" | "source_table_name" => Ok(GeneratedField::SourceTableName), - "destinationTableName" | "destination_table_name" => Ok(GeneratedField::DestinationTableName), - _ => Ok(GeneratedField::__SkipField__), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = TableNameMapping; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct peerdb_flow.TableNameMapping") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut source_table_name__ = None; - let mut destination_table_name__ = None; - while let Some(k) = map.next_key()? { - match k { - GeneratedField::SourceTableName => { - if source_table_name__.is_some() { - return Err(serde::de::Error::duplicate_field("sourceTableName")); - } - source_table_name__ = Some(map.next_value()?); - } - GeneratedField::DestinationTableName => { - if destination_table_name__.is_some() { - return Err(serde::de::Error::duplicate_field("destinationTableName")); - } - destination_table_name__ = Some(map.next_value()?); - } - GeneratedField::__SkipField__ => { - let _ = map.next_value::()?; - } - } - } - Ok(TableNameMapping { - source_table_name: source_table_name__.unwrap_or_default(), - destination_table_name: destination_table_name__.unwrap_or_default(), - }) - } - } - deserializer.deserialize_struct("peerdb_flow.TableNameMapping", FIELDS, GeneratedVisitor) - } -} impl serde::Serialize for TableSchema { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -5566,6 +6206,101 @@ impl<'de> serde::Deserialize<'de> for TableSchemaDelta { deserializer.deserialize_struct("peerdb_flow.TableSchemaDelta", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for TablesList { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.tables.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.TablesList", len)?; + if !self.tables.is_empty() { + struct_ser.serialize_field("tables", &self.tables)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for TablesList { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "tables", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Tables, + __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 { + "tables" => Ok(GeneratedField::Tables), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = TablesList; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.TablesList") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut tables__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Tables => { + if tables__.is_some() { + return Err(serde::de::Error::duplicate_field("tables")); + } + tables__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(TablesList { + tables: tables__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.TablesList", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for TimestampPartitionRange { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/nexus/sqlparser-rs b/nexus/sqlparser-rs index 49b806a49c..fc553c5efe 160000 --- a/nexus/sqlparser-rs +++ b/nexus/sqlparser-rs @@ -1 +1 @@ -Subproject commit 49b806a49c325cff7203f9b71d0c68cf9e237f30 +Subproject commit fc553c5efe69a627bb4820d62fd06c6381f27f9d diff --git a/protos/flow.proto b/protos/flow.proto index 428b8ae253..44ead1fcc3 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -5,11 +5,6 @@ import "peers.proto"; package peerdb_flow; -message TableNameMapping { - string source_table_name = 1; - string destination_table_name = 2; -} - message RelationMessageColumn { uint32 flags = 1; string name = 2; @@ -22,11 +17,18 @@ message RelationMessage { repeated RelationMessageColumn columns = 3; } +enum MappingType { + UNKNOWN = 0; + TABLE = 1; + SCHEMA = 2; +} + message FlowConnectionConfigs { peerdb_peers.Peer source = 1; peerdb_peers.Peer destination = 2; string flow_job_name = 3; TableSchema table_schema = 4; + // if MappingType is TABLE, this contains the table level mappings. map table_name_mapping = 5; map src_table_id_name_mapping = 6; map table_name_schema_mapping = 7; @@ -56,6 +58,11 @@ message FlowConnectionConfigs { // the below two are for eventhub only int64 push_batch_size = 21; int64 push_parallelism = 22; + + MappingType mapping_type = 23; + // if MappingType is SCHEMA, this contains the schema level mappings. + map schema_mapping = 24; + bool allow_table_additions = 25; } message SyncFlowOptions { @@ -88,12 +95,6 @@ message GetLastSyncedIDInput { string flow_job_name = 2; } -message EnsurePullabilityInput { - peerdb_peers.Peer peer_connection_config = 1; - string flow_job_name = 2; - string source_table_identifier = 3; -} - message EnsurePullabilityBatchInput { peerdb_peers.Peer peer_connection_config = 1; string flow_job_name = 2; @@ -110,10 +111,6 @@ message TableIdentifier { } } -message EnsurePullabilityOutput { - TableIdentifier table_identifier = 1; -} - message EnsurePullabilityBatchOutput { map table_identifier_mapping = 1; } @@ -127,6 +124,9 @@ message SetupReplicationInput { bool do_initial_copy = 5; string existing_publication_name = 6; string existing_replication_slot_name = 7; + + // if length > 0, ignore table name mapping and use schemas as basis of publication creation + repeated string schemas = 8; } message SetupReplicationOutput { @@ -310,3 +310,55 @@ message ReplayTableSchemaDeltaInput { FlowConnectionConfigs flow_connection_configs = 1; TableSchemaDelta table_schema_delta = 2; } + +message ListTablesInSchemasInput { + peerdb_peers.Peer peer_connection_config = 1; + map schema_mapping = 2; +} + +message TablesList { + repeated string tables = 1; +} + +message ListTablesInSchemasOutput { + map schema_to_tables = 1; +} + +message AdditionalTableDelta { + string table_name = 1; + string src_schema = 2; + string dst_schema = 3; + uint32 rel_id = 4; + TableSchema table_schema = 5; +} + +message MirrorDelta { + oneof delta { + // adding or dropping columns to an existing table + TableSchemaDelta table_schema_delta = 1; + // creating a new table, for MappingType SCHEMA only + AdditionalTableDelta additional_table_delta = 2; + } +} + +message CreateAdditionalTableInput { + FlowConnectionConfigs flow_connection_configs = 1; + AdditionalTableDelta additional_table_info = 2; +} + +message SyncResponse { + // FirstSyncedCheckPointID is the first ID that was synced. + int64 first_synced_checkpoint_id = 1; + // LastSyncedCheckPointID is the last ID that was synced. + int64 last_synced_checkpoint_id = 2; + // NumRecordsSynced is the number of records that were synced. + int64 num_records_synced = 3; + // CurrentSyncBatchID is the ID of the currently synced batch. + int64 current_sync_batch_id = 4; + // TableNameRowsMapping tells how many records need to be synced to each destination table. + map table_name_rows_mapping = 5; + // to be stored in state for future PullFlows + map relation_message_mapping = 6; + // if not-nil, needs to be applied before next StartFlow + MirrorDelta mirror_delta = 7; +} diff --git a/ui/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index 417a473da0..d8e41468a5 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -6,6 +6,45 @@ import { Peer } from "./peers"; export const protobufPackage = "peerdb_flow"; +export enum MappingType { + UNKNOWN = 0, + TABLE = 1, + SCHEMA = 2, + UNRECOGNIZED = -1, +} + +export function mappingTypeFromJSON(object: any): MappingType { + switch (object) { + case 0: + case "UNKNOWN": + return MappingType.UNKNOWN; + case 1: + case "TABLE": + return MappingType.TABLE; + case 2: + case "SCHEMA": + return MappingType.SCHEMA; + case -1: + case "UNRECOGNIZED": + default: + return MappingType.UNRECOGNIZED; + } +} + +export function mappingTypeToJSON(object: MappingType): string { + switch (object) { + case MappingType.UNKNOWN: + return "UNKNOWN"; + case MappingType.TABLE: + return "TABLE"; + case MappingType.SCHEMA: + return "SCHEMA"; + case MappingType.UNRECOGNIZED: + default: + return "UNRECOGNIZED"; + } +} + /** protos for qrep */ export enum QRepSyncMode { QREP_SYNC_MODE_MULTI_INSERT = 0, @@ -80,11 +119,6 @@ export function qRepWriteTypeToJSON(object: QRepWriteType): string { } } -export interface TableNameMapping { - sourceTableName: string; - destinationTableName: string; -} - export interface RelationMessageColumn { flags: number; name: string; @@ -101,7 +135,10 @@ export interface FlowConnectionConfigs { source: Peer | undefined; destination: Peer | undefined; flowJobName: string; - tableSchema: TableSchema | undefined; + tableSchema: + | TableSchema + | undefined; + /** if MappingType is TABLE, this contains the table level mappings. */ tableNameMapping: { [key: string]: string }; srcTableIdNameMapping: { [key: number]: string }; tableNameSchemaMapping: { [key: string]: TableSchema }; @@ -127,6 +164,10 @@ export interface FlowConnectionConfigs { /** the below two are for eventhub only */ pushBatchSize: number; pushParallelism: number; + mappingType: MappingType; + /** if MappingType is SCHEMA, this contains the schema level mappings. */ + schemaMapping: { [key: string]: string }; + allowTableAdditions: boolean; } export interface FlowConnectionConfigs_TableNameMappingEntry { @@ -144,6 +185,11 @@ export interface FlowConnectionConfigs_TableNameSchemaMappingEntry { value: TableSchema | undefined; } +export interface FlowConnectionConfigs_SchemaMappingEntry { + key: string; + value: string; +} + export interface SyncFlowOptions { batchSize: number; relationMessageMapping: { [key: number]: RelationMessage }; @@ -184,12 +230,6 @@ export interface GetLastSyncedIDInput { flowJobName: string; } -export interface EnsurePullabilityInput { - peerConnectionConfig: Peer | undefined; - flowJobName: string; - sourceTableIdentifier: string; -} - export interface EnsurePullabilityBatchInput { peerConnectionConfig: Peer | undefined; flowJobName: string; @@ -204,10 +244,6 @@ export interface TableIdentifier { postgresTableIdentifier?: PostgresTableIdentifier | undefined; } -export interface EnsurePullabilityOutput { - tableIdentifier: TableIdentifier | undefined; -} - export interface EnsurePullabilityBatchOutput { tableIdentifierMapping: { [key: string]: TableIdentifier }; } @@ -226,6 +262,8 @@ export interface SetupReplicationInput { doInitialCopy: boolean; existingPublicationName: string; existingReplicationSlotName: string; + /** if length > 0, ignore table name mapping and use schemas as basis of publication creation */ + schemas: string[]; } export interface SetupReplicationInput_TableNameMappingEntry { @@ -423,79 +461,77 @@ export interface ReplayTableSchemaDeltaInput { tableSchemaDelta: TableSchemaDelta | undefined; } -function createBaseTableNameMapping(): TableNameMapping { - return { sourceTableName: "", destinationTableName: "" }; +export interface ListTablesInSchemasInput { + peerConnectionConfig: Peer | undefined; + schemaMapping: { [key: string]: string }; } -export const TableNameMapping = { - encode(message: TableNameMapping, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.sourceTableName !== "") { - writer.uint32(10).string(message.sourceTableName); - } - if (message.destinationTableName !== "") { - writer.uint32(18).string(message.destinationTableName); - } - return writer; - }, +export interface ListTablesInSchemasInput_SchemaMappingEntry { + key: string; + value: string; +} - decode(input: _m0.Reader | Uint8Array, length?: number): TableNameMapping { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseTableNameMapping(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 10) { - break; - } +export interface TablesList { + tables: string[]; +} - message.sourceTableName = reader.string(); - continue; - case 2: - if (tag !== 18) { - break; - } +export interface ListTablesInSchemasOutput { + schemaToTables: { [key: string]: TablesList }; +} - message.destinationTableName = reader.string(); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, +export interface ListTablesInSchemasOutput_SchemaToTablesEntry { + key: string; + value: TablesList | undefined; +} - fromJSON(object: any): TableNameMapping { - return { - sourceTableName: isSet(object.sourceTableName) ? String(object.sourceTableName) : "", - destinationTableName: isSet(object.destinationTableName) ? String(object.destinationTableName) : "", - }; - }, +export interface AdditionalTableDelta { + tableName: string; + srcSchema: string; + dstSchema: string; + relId: number; + tableSchema: TableSchema | undefined; +} - toJSON(message: TableNameMapping): unknown { - const obj: any = {}; - if (message.sourceTableName !== "") { - obj.sourceTableName = message.sourceTableName; - } - if (message.destinationTableName !== "") { - obj.destinationTableName = message.destinationTableName; - } - return obj; - }, +export interface MirrorDelta { + /** adding or dropping columns to an existing table */ + tableSchemaDelta?: + | TableSchemaDelta + | undefined; + /** creating a new table, for MappingType SCHEMA only */ + additionalTableDelta?: AdditionalTableDelta | undefined; +} - create, I>>(base?: I): TableNameMapping { - return TableNameMapping.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): TableNameMapping { - const message = createBaseTableNameMapping(); - message.sourceTableName = object.sourceTableName ?? ""; - message.destinationTableName = object.destinationTableName ?? ""; - return message; - }, -}; +export interface CreateAdditionalTableInput { + flowConnectionConfigs: FlowConnectionConfigs | undefined; + additionalTableInfo: AdditionalTableDelta | undefined; +} + +export interface SyncResponse { + /** FirstSyncedCheckPointID is the first ID that was synced. */ + firstSyncedCheckpointId: number; + /** LastSyncedCheckPointID is the last ID that was synced. */ + lastSyncedCheckpointId: number; + /** NumRecordsSynced is the number of records that were synced. */ + numRecordsSynced: number; + /** CurrentSyncBatchID is the ID of the currently synced batch. */ + currentSyncBatchId: number; + /** TableNameRowsMapping tells how many records need to be synced to each destination table. */ + tableNameRowsMapping: { [key: string]: number }; + /** to be stored in state for future PullFlows */ + relationMessageMapping: { [key: number]: RelationMessage }; + /** if not-nil, needs to be applied before next StartFlow */ + mirrorDelta: MirrorDelta | undefined; +} + +export interface SyncResponse_TableNameRowsMappingEntry { + key: string; + value: number; +} + +export interface SyncResponse_RelationMessageMappingEntry { + key: number; + value: RelationMessage | undefined; +} function createBaseRelationMessageColumn(): RelationMessageColumn { return { flags: 0, name: "", dataType: 0 }; @@ -699,6 +735,9 @@ function createBaseFlowConnectionConfigs(): FlowConnectionConfigs { replicationSlotName: "", pushBatchSize: 0, pushParallelism: 0, + mappingType: 0, + schemaMapping: {}, + allowTableAdditions: false, }; } @@ -772,6 +811,15 @@ export const FlowConnectionConfigs = { if (message.pushParallelism !== 0) { writer.uint32(176).int64(message.pushParallelism); } + if (message.mappingType !== 0) { + writer.uint32(184).int32(message.mappingType); + } + Object.entries(message.schemaMapping).forEach(([key, value]) => { + FlowConnectionConfigs_SchemaMappingEntry.encode({ key: key as any, value }, writer.uint32(194).fork()).ldelim(); + }); + if (message.allowTableAdditions === true) { + writer.uint32(200).bool(message.allowTableAdditions); + } return writer; }, @@ -945,6 +993,30 @@ export const FlowConnectionConfigs = { message.pushParallelism = longToNumber(reader.int64() as Long); continue; + case 23: + if (tag !== 184) { + break; + } + + message.mappingType = reader.int32() as any; + continue; + case 24: + if (tag !== 194) { + break; + } + + const entry24 = FlowConnectionConfigs_SchemaMappingEntry.decode(reader, reader.uint32()); + if (entry24.value !== undefined) { + message.schemaMapping[entry24.key] = entry24.value; + } + continue; + case 25: + if (tag !== 200) { + break; + } + + message.allowTableAdditions = reader.bool(); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -999,6 +1071,14 @@ export const FlowConnectionConfigs = { replicationSlotName: isSet(object.replicationSlotName) ? String(object.replicationSlotName) : "", pushBatchSize: isSet(object.pushBatchSize) ? Number(object.pushBatchSize) : 0, pushParallelism: isSet(object.pushParallelism) ? Number(object.pushParallelism) : 0, + mappingType: isSet(object.mappingType) ? mappingTypeFromJSON(object.mappingType) : 0, + schemaMapping: isObject(object.schemaMapping) + ? Object.entries(object.schemaMapping).reduce<{ [key: string]: string }>((acc, [key, value]) => { + acc[key] = String(value); + return acc; + }, {}) + : {}, + allowTableAdditions: isSet(object.allowTableAdditions) ? Boolean(object.allowTableAdditions) : false, }; }, @@ -1088,6 +1168,21 @@ export const FlowConnectionConfigs = { if (message.pushParallelism !== 0) { obj.pushParallelism = Math.round(message.pushParallelism); } + if (message.mappingType !== 0) { + obj.mappingType = mappingTypeToJSON(message.mappingType); + } + if (message.schemaMapping) { + const entries = Object.entries(message.schemaMapping); + if (entries.length > 0) { + obj.schemaMapping = {}; + entries.forEach(([k, v]) => { + obj.schemaMapping[k] = v; + }); + } + } + if (message.allowTableAdditions === true) { + obj.allowTableAdditions = message.allowTableAdditions; + } return obj; }, @@ -1148,6 +1243,17 @@ export const FlowConnectionConfigs = { message.replicationSlotName = object.replicationSlotName ?? ""; message.pushBatchSize = object.pushBatchSize ?? 0; message.pushParallelism = object.pushParallelism ?? 0; + message.mappingType = object.mappingType ?? 0; + message.schemaMapping = Object.entries(object.schemaMapping ?? {}).reduce<{ [key: string]: string }>( + (acc, [key, value]) => { + if (value !== undefined) { + acc[key] = String(value); + } + return acc; + }, + {}, + ); + message.allowTableAdditions = object.allowTableAdditions ?? false; return message; }, }; @@ -1388,6 +1494,81 @@ export const FlowConnectionConfigs_TableNameSchemaMappingEntry = { }, }; +function createBaseFlowConnectionConfigs_SchemaMappingEntry(): FlowConnectionConfigs_SchemaMappingEntry { + return { key: "", value: "" }; +} + +export const FlowConnectionConfigs_SchemaMappingEntry = { + encode(message: FlowConnectionConfigs_SchemaMappingEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== "") { + writer.uint32(18).string(message.value); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): FlowConnectionConfigs_SchemaMappingEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseFlowConnectionConfigs_SchemaMappingEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): FlowConnectionConfigs_SchemaMappingEntry { + return { key: isSet(object.key) ? String(object.key) : "", value: isSet(object.value) ? String(object.value) : "" }; + }, + + toJSON(message: FlowConnectionConfigs_SchemaMappingEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== "") { + obj.value = message.value; + } + return obj; + }, + + create, I>>( + base?: I, + ): FlowConnectionConfigs_SchemaMappingEntry { + return FlowConnectionConfigs_SchemaMappingEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): FlowConnectionConfigs_SchemaMappingEntry { + const message = createBaseFlowConnectionConfigs_SchemaMappingEntry(); + message.key = object.key ?? ""; + message.value = object.value ?? ""; + return message; + }, +}; + function createBaseSyncFlowOptions(): SyncFlowOptions { return { batchSize: 0, relationMessageMapping: {} }; } @@ -2059,28 +2240,28 @@ export const GetLastSyncedIDInput = { }, }; -function createBaseEnsurePullabilityInput(): EnsurePullabilityInput { - return { peerConnectionConfig: undefined, flowJobName: "", sourceTableIdentifier: "" }; +function createBaseEnsurePullabilityBatchInput(): EnsurePullabilityBatchInput { + return { peerConnectionConfig: undefined, flowJobName: "", sourceTableIdentifiers: [] }; } -export const EnsurePullabilityInput = { - encode(message: EnsurePullabilityInput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { +export const EnsurePullabilityBatchInput = { + encode(message: EnsurePullabilityBatchInput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { if (message.peerConnectionConfig !== undefined) { Peer.encode(message.peerConnectionConfig, writer.uint32(10).fork()).ldelim(); } if (message.flowJobName !== "") { writer.uint32(18).string(message.flowJobName); } - if (message.sourceTableIdentifier !== "") { - writer.uint32(26).string(message.sourceTableIdentifier); + for (const v of message.sourceTableIdentifiers) { + writer.uint32(26).string(v!); } return writer; }, - decode(input: _m0.Reader | Uint8Array, length?: number): EnsurePullabilityInput { + decode(input: _m0.Reader | Uint8Array, length?: number): EnsurePullabilityBatchInput { const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseEnsurePullabilityInput(); + const message = createBaseEnsurePullabilityBatchInput(); while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { @@ -2103,7 +2284,7 @@ export const EnsurePullabilityInput = { break; } - message.sourceTableIdentifier = reader.string(); + message.sourceTableIdentifiers.push(reader.string()); continue; } if ((tag & 7) === 4 || tag === 0) { @@ -2114,15 +2295,17 @@ export const EnsurePullabilityInput = { return message; }, - fromJSON(object: any): EnsurePullabilityInput { + fromJSON(object: any): EnsurePullabilityBatchInput { return { peerConnectionConfig: isSet(object.peerConnectionConfig) ? Peer.fromJSON(object.peerConnectionConfig) : undefined, flowJobName: isSet(object.flowJobName) ? String(object.flowJobName) : "", - sourceTableIdentifier: isSet(object.sourceTableIdentifier) ? String(object.sourceTableIdentifier) : "", + sourceTableIdentifiers: Array.isArray(object?.sourceTableIdentifiers) + ? object.sourceTableIdentifiers.map((e: any) => String(e)) + : [], }; }, - toJSON(message: EnsurePullabilityInput): unknown { + toJSON(message: EnsurePullabilityBatchInput): unknown { const obj: any = {}; if (message.peerConnectionConfig !== undefined) { obj.peerConnectionConfig = Peer.toJSON(message.peerConnectionConfig); @@ -2130,144 +2313,51 @@ export const EnsurePullabilityInput = { if (message.flowJobName !== "") { obj.flowJobName = message.flowJobName; } - if (message.sourceTableIdentifier !== "") { - obj.sourceTableIdentifier = message.sourceTableIdentifier; + if (message.sourceTableIdentifiers?.length) { + obj.sourceTableIdentifiers = message.sourceTableIdentifiers; } return obj; }, - create, I>>(base?: I): EnsurePullabilityInput { - return EnsurePullabilityInput.fromPartial(base ?? ({} as any)); + create, I>>(base?: I): EnsurePullabilityBatchInput { + return EnsurePullabilityBatchInput.fromPartial(base ?? ({} as any)); }, - fromPartial, I>>(object: I): EnsurePullabilityInput { - const message = createBaseEnsurePullabilityInput(); + fromPartial, I>>(object: I): EnsurePullabilityBatchInput { + const message = createBaseEnsurePullabilityBatchInput(); message.peerConnectionConfig = (object.peerConnectionConfig !== undefined && object.peerConnectionConfig !== null) ? Peer.fromPartial(object.peerConnectionConfig) : undefined; message.flowJobName = object.flowJobName ?? ""; - message.sourceTableIdentifier = object.sourceTableIdentifier ?? ""; + message.sourceTableIdentifiers = object.sourceTableIdentifiers?.map((e) => e) || []; return message; }, }; -function createBaseEnsurePullabilityBatchInput(): EnsurePullabilityBatchInput { - return { peerConnectionConfig: undefined, flowJobName: "", sourceTableIdentifiers: [] }; +function createBasePostgresTableIdentifier(): PostgresTableIdentifier { + return { relId: 0 }; } -export const EnsurePullabilityBatchInput = { - encode(message: EnsurePullabilityBatchInput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.peerConnectionConfig !== undefined) { - Peer.encode(message.peerConnectionConfig, writer.uint32(10).fork()).ldelim(); - } - if (message.flowJobName !== "") { - writer.uint32(18).string(message.flowJobName); - } - for (const v of message.sourceTableIdentifiers) { - writer.uint32(26).string(v!); +export const PostgresTableIdentifier = { + encode(message: PostgresTableIdentifier, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.relId !== 0) { + writer.uint32(8).uint32(message.relId); } return writer; }, - decode(input: _m0.Reader | Uint8Array, length?: number): EnsurePullabilityBatchInput { + decode(input: _m0.Reader | Uint8Array, length?: number): PostgresTableIdentifier { const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseEnsurePullabilityBatchInput(); + const message = createBasePostgresTableIdentifier(); while (reader.pos < end) { const tag = reader.uint32(); switch (tag >>> 3) { case 1: - if (tag !== 10) { - break; - } - - message.peerConnectionConfig = Peer.decode(reader, reader.uint32()); - continue; - case 2: - if (tag !== 18) { - break; - } - - message.flowJobName = reader.string(); - continue; - case 3: - if (tag !== 26) { + if (tag !== 8) { break; } - message.sourceTableIdentifiers.push(reader.string()); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): EnsurePullabilityBatchInput { - return { - peerConnectionConfig: isSet(object.peerConnectionConfig) ? Peer.fromJSON(object.peerConnectionConfig) : undefined, - flowJobName: isSet(object.flowJobName) ? String(object.flowJobName) : "", - sourceTableIdentifiers: Array.isArray(object?.sourceTableIdentifiers) - ? object.sourceTableIdentifiers.map((e: any) => String(e)) - : [], - }; - }, - - toJSON(message: EnsurePullabilityBatchInput): unknown { - const obj: any = {}; - if (message.peerConnectionConfig !== undefined) { - obj.peerConnectionConfig = Peer.toJSON(message.peerConnectionConfig); - } - if (message.flowJobName !== "") { - obj.flowJobName = message.flowJobName; - } - if (message.sourceTableIdentifiers?.length) { - obj.sourceTableIdentifiers = message.sourceTableIdentifiers; - } - return obj; - }, - - create, I>>(base?: I): EnsurePullabilityBatchInput { - return EnsurePullabilityBatchInput.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): EnsurePullabilityBatchInput { - const message = createBaseEnsurePullabilityBatchInput(); - message.peerConnectionConfig = (object.peerConnectionConfig !== undefined && object.peerConnectionConfig !== null) - ? Peer.fromPartial(object.peerConnectionConfig) - : undefined; - message.flowJobName = object.flowJobName ?? ""; - message.sourceTableIdentifiers = object.sourceTableIdentifiers?.map((e) => e) || []; - return message; - }, -}; - -function createBasePostgresTableIdentifier(): PostgresTableIdentifier { - return { relId: 0 }; -} - -export const PostgresTableIdentifier = { - encode(message: PostgresTableIdentifier, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.relId !== 0) { - writer.uint32(8).uint32(message.relId); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): PostgresTableIdentifier { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBasePostgresTableIdentifier(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 8) { - break; - } - - message.relId = reader.uint32(); + message.relId = reader.uint32(); continue; } if ((tag & 7) === 4 || tag === 0) { @@ -2364,67 +2454,6 @@ export const TableIdentifier = { }, }; -function createBaseEnsurePullabilityOutput(): EnsurePullabilityOutput { - return { tableIdentifier: undefined }; -} - -export const EnsurePullabilityOutput = { - encode(message: EnsurePullabilityOutput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.tableIdentifier !== undefined) { - TableIdentifier.encode(message.tableIdentifier, writer.uint32(10).fork()).ldelim(); - } - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): EnsurePullabilityOutput { - const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseEnsurePullabilityOutput(); - while (reader.pos < end) { - const tag = reader.uint32(); - switch (tag >>> 3) { - case 1: - if (tag !== 10) { - break; - } - - message.tableIdentifier = TableIdentifier.decode(reader, reader.uint32()); - continue; - } - if ((tag & 7) === 4 || tag === 0) { - break; - } - reader.skipType(tag & 7); - } - return message; - }, - - fromJSON(object: any): EnsurePullabilityOutput { - return { - tableIdentifier: isSet(object.tableIdentifier) ? TableIdentifier.fromJSON(object.tableIdentifier) : undefined, - }; - }, - - toJSON(message: EnsurePullabilityOutput): unknown { - const obj: any = {}; - if (message.tableIdentifier !== undefined) { - obj.tableIdentifier = TableIdentifier.toJSON(message.tableIdentifier); - } - return obj; - }, - - create, I>>(base?: I): EnsurePullabilityOutput { - return EnsurePullabilityOutput.fromPartial(base ?? ({} as any)); - }, - fromPartial, I>>(object: I): EnsurePullabilityOutput { - const message = createBaseEnsurePullabilityOutput(); - message.tableIdentifier = (object.tableIdentifier !== undefined && object.tableIdentifier !== null) - ? TableIdentifier.fromPartial(object.tableIdentifier) - : undefined; - return message; - }, -}; - function createBaseEnsurePullabilityBatchOutput(): EnsurePullabilityBatchOutput { return { tableIdentifierMapping: {} }; } @@ -2603,6 +2632,7 @@ function createBaseSetupReplicationInput(): SetupReplicationInput { doInitialCopy: false, existingPublicationName: "", existingReplicationSlotName: "", + schemas: [], }; } @@ -2629,6 +2659,9 @@ export const SetupReplicationInput = { if (message.existingReplicationSlotName !== "") { writer.uint32(58).string(message.existingReplicationSlotName); } + for (const v of message.schemas) { + writer.uint32(66).string(v!); + } return writer; }, @@ -2691,6 +2724,13 @@ export const SetupReplicationInput = { message.existingReplicationSlotName = reader.string(); continue; + case 8: + if (tag !== 66) { + break; + } + + message.schemas.push(reader.string()); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -2716,6 +2756,7 @@ export const SetupReplicationInput = { existingReplicationSlotName: isSet(object.existingReplicationSlotName) ? String(object.existingReplicationSlotName) : "", + schemas: Array.isArray(object?.schemas) ? object.schemas.map((e: any) => String(e)) : [], }; }, @@ -2748,6 +2789,9 @@ export const SetupReplicationInput = { if (message.existingReplicationSlotName !== "") { obj.existingReplicationSlotName = message.existingReplicationSlotName; } + if (message.schemas?.length) { + obj.schemas = message.schemas; + } return obj; }, @@ -2775,6 +2819,7 @@ export const SetupReplicationInput = { message.doInitialCopy = object.doInitialCopy ?? false; message.existingPublicationName = object.existingPublicationName ?? ""; message.existingReplicationSlotName = object.existingReplicationSlotName ?? ""; + message.schemas = object.schemas?.map((e) => e) || []; return message; }, }; @@ -5548,6 +5593,1042 @@ export const ReplayTableSchemaDeltaInput = { }, }; +function createBaseListTablesInSchemasInput(): ListTablesInSchemasInput { + return { peerConnectionConfig: undefined, schemaMapping: {} }; +} + +export const ListTablesInSchemasInput = { + encode(message: ListTablesInSchemasInput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.peerConnectionConfig !== undefined) { + Peer.encode(message.peerConnectionConfig, writer.uint32(10).fork()).ldelim(); + } + Object.entries(message.schemaMapping).forEach(([key, value]) => { + ListTablesInSchemasInput_SchemaMappingEntry.encode({ key: key as any, value }, writer.uint32(18).fork()).ldelim(); + }); + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): ListTablesInSchemasInput { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseListTablesInSchemasInput(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.peerConnectionConfig = Peer.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + const entry2 = ListTablesInSchemasInput_SchemaMappingEntry.decode(reader, reader.uint32()); + if (entry2.value !== undefined) { + message.schemaMapping[entry2.key] = entry2.value; + } + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): ListTablesInSchemasInput { + return { + peerConnectionConfig: isSet(object.peerConnectionConfig) ? Peer.fromJSON(object.peerConnectionConfig) : undefined, + schemaMapping: isObject(object.schemaMapping) + ? Object.entries(object.schemaMapping).reduce<{ [key: string]: string }>((acc, [key, value]) => { + acc[key] = String(value); + return acc; + }, {}) + : {}, + }; + }, + + toJSON(message: ListTablesInSchemasInput): unknown { + const obj: any = {}; + if (message.peerConnectionConfig !== undefined) { + obj.peerConnectionConfig = Peer.toJSON(message.peerConnectionConfig); + } + if (message.schemaMapping) { + const entries = Object.entries(message.schemaMapping); + if (entries.length > 0) { + obj.schemaMapping = {}; + entries.forEach(([k, v]) => { + obj.schemaMapping[k] = v; + }); + } + } + return obj; + }, + + create, I>>(base?: I): ListTablesInSchemasInput { + return ListTablesInSchemasInput.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ListTablesInSchemasInput { + const message = createBaseListTablesInSchemasInput(); + message.peerConnectionConfig = (object.peerConnectionConfig !== undefined && object.peerConnectionConfig !== null) + ? Peer.fromPartial(object.peerConnectionConfig) + : undefined; + message.schemaMapping = Object.entries(object.schemaMapping ?? {}).reduce<{ [key: string]: string }>( + (acc, [key, value]) => { + if (value !== undefined) { + acc[key] = String(value); + } + return acc; + }, + {}, + ); + return message; + }, +}; + +function createBaseListTablesInSchemasInput_SchemaMappingEntry(): ListTablesInSchemasInput_SchemaMappingEntry { + return { key: "", value: "" }; +} + +export const ListTablesInSchemasInput_SchemaMappingEntry = { + encode(message: ListTablesInSchemasInput_SchemaMappingEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== "") { + writer.uint32(18).string(message.value); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): ListTablesInSchemasInput_SchemaMappingEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseListTablesInSchemasInput_SchemaMappingEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = reader.string(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): ListTablesInSchemasInput_SchemaMappingEntry { + return { key: isSet(object.key) ? String(object.key) : "", value: isSet(object.value) ? String(object.value) : "" }; + }, + + toJSON(message: ListTablesInSchemasInput_SchemaMappingEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== "") { + obj.value = message.value; + } + return obj; + }, + + create, I>>( + base?: I, + ): ListTablesInSchemasInput_SchemaMappingEntry { + return ListTablesInSchemasInput_SchemaMappingEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): ListTablesInSchemasInput_SchemaMappingEntry { + const message = createBaseListTablesInSchemasInput_SchemaMappingEntry(); + message.key = object.key ?? ""; + message.value = object.value ?? ""; + return message; + }, +}; + +function createBaseTablesList(): TablesList { + return { tables: [] }; +} + +export const TablesList = { + encode(message: TablesList, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + for (const v of message.tables) { + writer.uint32(10).string(v!); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): TablesList { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseTablesList(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.tables.push(reader.string()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): TablesList { + return { tables: Array.isArray(object?.tables) ? object.tables.map((e: any) => String(e)) : [] }; + }, + + toJSON(message: TablesList): unknown { + const obj: any = {}; + if (message.tables?.length) { + obj.tables = message.tables; + } + return obj; + }, + + create, I>>(base?: I): TablesList { + return TablesList.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): TablesList { + const message = createBaseTablesList(); + message.tables = object.tables?.map((e) => e) || []; + return message; + }, +}; + +function createBaseListTablesInSchemasOutput(): ListTablesInSchemasOutput { + return { schemaToTables: {} }; +} + +export const ListTablesInSchemasOutput = { + encode(message: ListTablesInSchemasOutput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + Object.entries(message.schemaToTables).forEach(([key, value]) => { + ListTablesInSchemasOutput_SchemaToTablesEntry.encode({ key: key as any, value }, writer.uint32(10).fork()) + .ldelim(); + }); + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): ListTablesInSchemasOutput { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseListTablesInSchemasOutput(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + const entry1 = ListTablesInSchemasOutput_SchemaToTablesEntry.decode(reader, reader.uint32()); + if (entry1.value !== undefined) { + message.schemaToTables[entry1.key] = entry1.value; + } + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): ListTablesInSchemasOutput { + return { + schemaToTables: isObject(object.schemaToTables) + ? Object.entries(object.schemaToTables).reduce<{ [key: string]: TablesList }>((acc, [key, value]) => { + acc[key] = TablesList.fromJSON(value); + return acc; + }, {}) + : {}, + }; + }, + + toJSON(message: ListTablesInSchemasOutput): unknown { + const obj: any = {}; + if (message.schemaToTables) { + const entries = Object.entries(message.schemaToTables); + if (entries.length > 0) { + obj.schemaToTables = {}; + entries.forEach(([k, v]) => { + obj.schemaToTables[k] = TablesList.toJSON(v); + }); + } + } + return obj; + }, + + create, I>>(base?: I): ListTablesInSchemasOutput { + return ListTablesInSchemasOutput.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): ListTablesInSchemasOutput { + const message = createBaseListTablesInSchemasOutput(); + message.schemaToTables = Object.entries(object.schemaToTables ?? {}).reduce<{ [key: string]: TablesList }>( + (acc, [key, value]) => { + if (value !== undefined) { + acc[key] = TablesList.fromPartial(value); + } + return acc; + }, + {}, + ); + return message; + }, +}; + +function createBaseListTablesInSchemasOutput_SchemaToTablesEntry(): ListTablesInSchemasOutput_SchemaToTablesEntry { + return { key: "", value: undefined }; +} + +export const ListTablesInSchemasOutput_SchemaToTablesEntry = { + encode(message: ListTablesInSchemasOutput_SchemaToTablesEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== undefined) { + TablesList.encode(message.value, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): ListTablesInSchemasOutput_SchemaToTablesEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseListTablesInSchemasOutput_SchemaToTablesEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = TablesList.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): ListTablesInSchemasOutput_SchemaToTablesEntry { + return { + key: isSet(object.key) ? String(object.key) : "", + value: isSet(object.value) ? TablesList.fromJSON(object.value) : undefined, + }; + }, + + toJSON(message: ListTablesInSchemasOutput_SchemaToTablesEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== undefined) { + obj.value = TablesList.toJSON(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): ListTablesInSchemasOutput_SchemaToTablesEntry { + return ListTablesInSchemasOutput_SchemaToTablesEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): ListTablesInSchemasOutput_SchemaToTablesEntry { + const message = createBaseListTablesInSchemasOutput_SchemaToTablesEntry(); + message.key = object.key ?? ""; + message.value = (object.value !== undefined && object.value !== null) + ? TablesList.fromPartial(object.value) + : undefined; + return message; + }, +}; + +function createBaseAdditionalTableDelta(): AdditionalTableDelta { + return { tableName: "", srcSchema: "", dstSchema: "", relId: 0, tableSchema: undefined }; +} + +export const AdditionalTableDelta = { + encode(message: AdditionalTableDelta, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.tableName !== "") { + writer.uint32(10).string(message.tableName); + } + if (message.srcSchema !== "") { + writer.uint32(18).string(message.srcSchema); + } + if (message.dstSchema !== "") { + writer.uint32(26).string(message.dstSchema); + } + if (message.relId !== 0) { + writer.uint32(32).uint32(message.relId); + } + if (message.tableSchema !== undefined) { + TableSchema.encode(message.tableSchema, writer.uint32(42).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): AdditionalTableDelta { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseAdditionalTableDelta(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.tableName = reader.string(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.srcSchema = reader.string(); + continue; + case 3: + if (tag !== 26) { + break; + } + + message.dstSchema = reader.string(); + continue; + case 4: + if (tag !== 32) { + break; + } + + message.relId = reader.uint32(); + continue; + case 5: + if (tag !== 42) { + break; + } + + message.tableSchema = TableSchema.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): AdditionalTableDelta { + return { + tableName: isSet(object.tableName) ? String(object.tableName) : "", + srcSchema: isSet(object.srcSchema) ? String(object.srcSchema) : "", + dstSchema: isSet(object.dstSchema) ? String(object.dstSchema) : "", + relId: isSet(object.relId) ? Number(object.relId) : 0, + tableSchema: isSet(object.tableSchema) ? TableSchema.fromJSON(object.tableSchema) : undefined, + }; + }, + + toJSON(message: AdditionalTableDelta): unknown { + const obj: any = {}; + if (message.tableName !== "") { + obj.tableName = message.tableName; + } + if (message.srcSchema !== "") { + obj.srcSchema = message.srcSchema; + } + if (message.dstSchema !== "") { + obj.dstSchema = message.dstSchema; + } + if (message.relId !== 0) { + obj.relId = Math.round(message.relId); + } + if (message.tableSchema !== undefined) { + obj.tableSchema = TableSchema.toJSON(message.tableSchema); + } + return obj; + }, + + create, I>>(base?: I): AdditionalTableDelta { + return AdditionalTableDelta.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): AdditionalTableDelta { + const message = createBaseAdditionalTableDelta(); + message.tableName = object.tableName ?? ""; + message.srcSchema = object.srcSchema ?? ""; + message.dstSchema = object.dstSchema ?? ""; + message.relId = object.relId ?? 0; + message.tableSchema = (object.tableSchema !== undefined && object.tableSchema !== null) + ? TableSchema.fromPartial(object.tableSchema) + : undefined; + return message; + }, +}; + +function createBaseMirrorDelta(): MirrorDelta { + return { tableSchemaDelta: undefined, additionalTableDelta: undefined }; +} + +export const MirrorDelta = { + encode(message: MirrorDelta, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.tableSchemaDelta !== undefined) { + TableSchemaDelta.encode(message.tableSchemaDelta, writer.uint32(10).fork()).ldelim(); + } + if (message.additionalTableDelta !== undefined) { + AdditionalTableDelta.encode(message.additionalTableDelta, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): MirrorDelta { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseMirrorDelta(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.tableSchemaDelta = TableSchemaDelta.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.additionalTableDelta = AdditionalTableDelta.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): MirrorDelta { + return { + tableSchemaDelta: isSet(object.tableSchemaDelta) ? TableSchemaDelta.fromJSON(object.tableSchemaDelta) : undefined, + additionalTableDelta: isSet(object.additionalTableDelta) + ? AdditionalTableDelta.fromJSON(object.additionalTableDelta) + : undefined, + }; + }, + + toJSON(message: MirrorDelta): unknown { + const obj: any = {}; + if (message.tableSchemaDelta !== undefined) { + obj.tableSchemaDelta = TableSchemaDelta.toJSON(message.tableSchemaDelta); + } + if (message.additionalTableDelta !== undefined) { + obj.additionalTableDelta = AdditionalTableDelta.toJSON(message.additionalTableDelta); + } + return obj; + }, + + create, I>>(base?: I): MirrorDelta { + return MirrorDelta.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): MirrorDelta { + const message = createBaseMirrorDelta(); + message.tableSchemaDelta = (object.tableSchemaDelta !== undefined && object.tableSchemaDelta !== null) + ? TableSchemaDelta.fromPartial(object.tableSchemaDelta) + : undefined; + message.additionalTableDelta = (object.additionalTableDelta !== undefined && object.additionalTableDelta !== null) + ? AdditionalTableDelta.fromPartial(object.additionalTableDelta) + : undefined; + return message; + }, +}; + +function createBaseCreateAdditionalTableInput(): CreateAdditionalTableInput { + return { flowConnectionConfigs: undefined, additionalTableInfo: undefined }; +} + +export const CreateAdditionalTableInput = { + encode(message: CreateAdditionalTableInput, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.flowConnectionConfigs !== undefined) { + FlowConnectionConfigs.encode(message.flowConnectionConfigs, writer.uint32(10).fork()).ldelim(); + } + if (message.additionalTableInfo !== undefined) { + AdditionalTableDelta.encode(message.additionalTableInfo, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): CreateAdditionalTableInput { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseCreateAdditionalTableInput(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.flowConnectionConfigs = FlowConnectionConfigs.decode(reader, reader.uint32()); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.additionalTableInfo = AdditionalTableDelta.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): CreateAdditionalTableInput { + return { + flowConnectionConfigs: isSet(object.flowConnectionConfigs) + ? FlowConnectionConfigs.fromJSON(object.flowConnectionConfigs) + : undefined, + additionalTableInfo: isSet(object.additionalTableInfo) + ? AdditionalTableDelta.fromJSON(object.additionalTableInfo) + : undefined, + }; + }, + + toJSON(message: CreateAdditionalTableInput): unknown { + const obj: any = {}; + if (message.flowConnectionConfigs !== undefined) { + obj.flowConnectionConfigs = FlowConnectionConfigs.toJSON(message.flowConnectionConfigs); + } + if (message.additionalTableInfo !== undefined) { + obj.additionalTableInfo = AdditionalTableDelta.toJSON(message.additionalTableInfo); + } + return obj; + }, + + create, I>>(base?: I): CreateAdditionalTableInput { + return CreateAdditionalTableInput.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): CreateAdditionalTableInput { + const message = createBaseCreateAdditionalTableInput(); + message.flowConnectionConfigs = + (object.flowConnectionConfigs !== undefined && object.flowConnectionConfigs !== null) + ? FlowConnectionConfigs.fromPartial(object.flowConnectionConfigs) + : undefined; + message.additionalTableInfo = (object.additionalTableInfo !== undefined && object.additionalTableInfo !== null) + ? AdditionalTableDelta.fromPartial(object.additionalTableInfo) + : undefined; + return message; + }, +}; + +function createBaseSyncResponse(): SyncResponse { + return { + firstSyncedCheckpointId: 0, + lastSyncedCheckpointId: 0, + numRecordsSynced: 0, + currentSyncBatchId: 0, + tableNameRowsMapping: {}, + relationMessageMapping: {}, + mirrorDelta: undefined, + }; +} + +export const SyncResponse = { + encode(message: SyncResponse, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.firstSyncedCheckpointId !== 0) { + writer.uint32(8).int64(message.firstSyncedCheckpointId); + } + if (message.lastSyncedCheckpointId !== 0) { + writer.uint32(16).int64(message.lastSyncedCheckpointId); + } + if (message.numRecordsSynced !== 0) { + writer.uint32(24).int64(message.numRecordsSynced); + } + if (message.currentSyncBatchId !== 0) { + writer.uint32(32).int64(message.currentSyncBatchId); + } + Object.entries(message.tableNameRowsMapping).forEach(([key, value]) => { + SyncResponse_TableNameRowsMappingEntry.encode({ key: key as any, value }, writer.uint32(42).fork()).ldelim(); + }); + Object.entries(message.relationMessageMapping).forEach(([key, value]) => { + SyncResponse_RelationMessageMappingEntry.encode({ key: key as any, value }, writer.uint32(50).fork()).ldelim(); + }); + if (message.mirrorDelta !== undefined) { + MirrorDelta.encode(message.mirrorDelta, writer.uint32(58).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SyncResponse { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSyncResponse(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.firstSyncedCheckpointId = longToNumber(reader.int64() as Long); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.lastSyncedCheckpointId = longToNumber(reader.int64() as Long); + continue; + case 3: + if (tag !== 24) { + break; + } + + message.numRecordsSynced = longToNumber(reader.int64() as Long); + continue; + case 4: + if (tag !== 32) { + break; + } + + message.currentSyncBatchId = longToNumber(reader.int64() as Long); + continue; + case 5: + if (tag !== 42) { + break; + } + + const entry5 = SyncResponse_TableNameRowsMappingEntry.decode(reader, reader.uint32()); + if (entry5.value !== undefined) { + message.tableNameRowsMapping[entry5.key] = entry5.value; + } + continue; + case 6: + if (tag !== 50) { + break; + } + + const entry6 = SyncResponse_RelationMessageMappingEntry.decode(reader, reader.uint32()); + if (entry6.value !== undefined) { + message.relationMessageMapping[entry6.key] = entry6.value; + } + continue; + case 7: + if (tag !== 58) { + break; + } + + message.mirrorDelta = MirrorDelta.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SyncResponse { + return { + firstSyncedCheckpointId: isSet(object.firstSyncedCheckpointId) ? Number(object.firstSyncedCheckpointId) : 0, + lastSyncedCheckpointId: isSet(object.lastSyncedCheckpointId) ? Number(object.lastSyncedCheckpointId) : 0, + numRecordsSynced: isSet(object.numRecordsSynced) ? Number(object.numRecordsSynced) : 0, + currentSyncBatchId: isSet(object.currentSyncBatchId) ? Number(object.currentSyncBatchId) : 0, + tableNameRowsMapping: isObject(object.tableNameRowsMapping) + ? Object.entries(object.tableNameRowsMapping).reduce<{ [key: string]: number }>((acc, [key, value]) => { + acc[key] = Number(value); + return acc; + }, {}) + : {}, + relationMessageMapping: isObject(object.relationMessageMapping) + ? Object.entries(object.relationMessageMapping).reduce<{ [key: number]: RelationMessage }>( + (acc, [key, value]) => { + acc[Number(key)] = RelationMessage.fromJSON(value); + return acc; + }, + {}, + ) + : {}, + mirrorDelta: isSet(object.mirrorDelta) ? MirrorDelta.fromJSON(object.mirrorDelta) : undefined, + }; + }, + + toJSON(message: SyncResponse): unknown { + const obj: any = {}; + if (message.firstSyncedCheckpointId !== 0) { + obj.firstSyncedCheckpointId = Math.round(message.firstSyncedCheckpointId); + } + if (message.lastSyncedCheckpointId !== 0) { + obj.lastSyncedCheckpointId = Math.round(message.lastSyncedCheckpointId); + } + if (message.numRecordsSynced !== 0) { + obj.numRecordsSynced = Math.round(message.numRecordsSynced); + } + if (message.currentSyncBatchId !== 0) { + obj.currentSyncBatchId = Math.round(message.currentSyncBatchId); + } + if (message.tableNameRowsMapping) { + const entries = Object.entries(message.tableNameRowsMapping); + if (entries.length > 0) { + obj.tableNameRowsMapping = {}; + entries.forEach(([k, v]) => { + obj.tableNameRowsMapping[k] = Math.round(v); + }); + } + } + if (message.relationMessageMapping) { + const entries = Object.entries(message.relationMessageMapping); + if (entries.length > 0) { + obj.relationMessageMapping = {}; + entries.forEach(([k, v]) => { + obj.relationMessageMapping[k] = RelationMessage.toJSON(v); + }); + } + } + if (message.mirrorDelta !== undefined) { + obj.mirrorDelta = MirrorDelta.toJSON(message.mirrorDelta); + } + return obj; + }, + + create, I>>(base?: I): SyncResponse { + return SyncResponse.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>(object: I): SyncResponse { + const message = createBaseSyncResponse(); + message.firstSyncedCheckpointId = object.firstSyncedCheckpointId ?? 0; + message.lastSyncedCheckpointId = object.lastSyncedCheckpointId ?? 0; + message.numRecordsSynced = object.numRecordsSynced ?? 0; + message.currentSyncBatchId = object.currentSyncBatchId ?? 0; + message.tableNameRowsMapping = Object.entries(object.tableNameRowsMapping ?? {}).reduce<{ [key: string]: number }>( + (acc, [key, value]) => { + if (value !== undefined) { + acc[key] = Number(value); + } + return acc; + }, + {}, + ); + message.relationMessageMapping = Object.entries(object.relationMessageMapping ?? {}).reduce< + { [key: number]: RelationMessage } + >((acc, [key, value]) => { + if (value !== undefined) { + acc[Number(key)] = RelationMessage.fromPartial(value); + } + return acc; + }, {}); + message.mirrorDelta = (object.mirrorDelta !== undefined && object.mirrorDelta !== null) + ? MirrorDelta.fromPartial(object.mirrorDelta) + : undefined; + return message; + }, +}; + +function createBaseSyncResponse_TableNameRowsMappingEntry(): SyncResponse_TableNameRowsMappingEntry { + return { key: "", value: 0 }; +} + +export const SyncResponse_TableNameRowsMappingEntry = { + encode(message: SyncResponse_TableNameRowsMappingEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== "") { + writer.uint32(10).string(message.key); + } + if (message.value !== 0) { + writer.uint32(16).uint32(message.value); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SyncResponse_TableNameRowsMappingEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSyncResponse_TableNameRowsMappingEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 10) { + break; + } + + message.key = reader.string(); + continue; + case 2: + if (tag !== 16) { + break; + } + + message.value = reader.uint32(); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SyncResponse_TableNameRowsMappingEntry { + return { key: isSet(object.key) ? String(object.key) : "", value: isSet(object.value) ? Number(object.value) : 0 }; + }, + + toJSON(message: SyncResponse_TableNameRowsMappingEntry): unknown { + const obj: any = {}; + if (message.key !== "") { + obj.key = message.key; + } + if (message.value !== 0) { + obj.value = Math.round(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): SyncResponse_TableNameRowsMappingEntry { + return SyncResponse_TableNameRowsMappingEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): SyncResponse_TableNameRowsMappingEntry { + const message = createBaseSyncResponse_TableNameRowsMappingEntry(); + message.key = object.key ?? ""; + message.value = object.value ?? 0; + return message; + }, +}; + +function createBaseSyncResponse_RelationMessageMappingEntry(): SyncResponse_RelationMessageMappingEntry { + return { key: 0, value: undefined }; +} + +export const SyncResponse_RelationMessageMappingEntry = { + encode(message: SyncResponse_RelationMessageMappingEntry, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.key !== 0) { + writer.uint32(8).uint32(message.key); + } + if (message.value !== undefined) { + RelationMessage.encode(message.value, writer.uint32(18).fork()).ldelim(); + } + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): SyncResponse_RelationMessageMappingEntry { + const reader = input instanceof _m0.Reader ? input : _m0.Reader.create(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseSyncResponse_RelationMessageMappingEntry(); + while (reader.pos < end) { + const tag = reader.uint32(); + switch (tag >>> 3) { + case 1: + if (tag !== 8) { + break; + } + + message.key = reader.uint32(); + continue; + case 2: + if (tag !== 18) { + break; + } + + message.value = RelationMessage.decode(reader, reader.uint32()); + continue; + } + if ((tag & 7) === 4 || tag === 0) { + break; + } + reader.skipType(tag & 7); + } + return message; + }, + + fromJSON(object: any): SyncResponse_RelationMessageMappingEntry { + return { + key: isSet(object.key) ? Number(object.key) : 0, + value: isSet(object.value) ? RelationMessage.fromJSON(object.value) : undefined, + }; + }, + + toJSON(message: SyncResponse_RelationMessageMappingEntry): unknown { + const obj: any = {}; + if (message.key !== 0) { + obj.key = Math.round(message.key); + } + if (message.value !== undefined) { + obj.value = RelationMessage.toJSON(message.value); + } + return obj; + }, + + create, I>>( + base?: I, + ): SyncResponse_RelationMessageMappingEntry { + return SyncResponse_RelationMessageMappingEntry.fromPartial(base ?? ({} as any)); + }, + fromPartial, I>>( + object: I, + ): SyncResponse_RelationMessageMappingEntry { + const message = createBaseSyncResponse_RelationMessageMappingEntry(); + message.key = object.key ?? 0; + message.value = (object.value !== undefined && object.value !== null) + ? RelationMessage.fromPartial(object.value) + : undefined; + return message; + }, +}; + declare const self: any | undefined; declare const window: any | undefined; declare const global: any | undefined;