diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index a71ac05c36..4837eb9d2e 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -159,7 +159,8 @@ func (a *FlowableActivity) CreateNormalizedTable( } // StartFlow implements StartFlow. -func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlowInput) (*model.SyncResponse, error) { +func (a *FlowableActivity) StartFlow(ctx context.Context, + input *protos.StartFlowInput) (*model.SyncResponse, error) { conn := input.FlowConnectionConfigs ctx = context.WithValue(ctx, shared.EnableMetricsKey, a.EnableMetrics) @@ -189,7 +190,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo }).Info("pulling records...") startTime := time.Now() - records, err := src.PullRecords(&model.PullRecordsRequest{ + recordsWithTableSchemaDelta, err := src.PullRecords(&model.PullRecordsRequest{ FlowJobName: input.FlowConnectionConfigs.FlowJobName, SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, TableNameMapping: input.FlowConnectionConfigs.TableNameMapping, @@ -199,11 +200,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo TableNameSchemaMapping: input.FlowConnectionConfigs.TableNameSchemaMapping, OverridePublicationName: input.FlowConnectionConfigs.PublicationName, OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, + RelationMessageMapping: input.RelationMessageMapping, }) if err != nil { return nil, fmt.Errorf("failed to pull records: %w", err) } - if a.CatalogMirrorMonitor.IsActive() && len(records.Records) > 0 { + recordBatch := recordsWithTableSchemaDelta.RecordBatch + if a.CatalogMirrorMonitor.IsActive() && len(recordBatch.Records) > 0 { syncBatchID, err := dest.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err @@ -212,9 +215,9 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo err = a.CatalogMirrorMonitor.AddCDCBatchForFlow(ctx, input.FlowConnectionConfigs.FlowJobName, monitoring.CDCBatchInfo{ BatchID: syncBatchID + 1, - RowsInBatch: uint32(len(records.Records)), - BatchStartLSN: pglogrepl.LSN(records.FirstCheckPointID), - BatchEndlSN: pglogrepl.LSN(records.LastCheckPointID), + RowsInBatch: uint32(len(recordBatch.Records)), + BatchStartLSN: pglogrepl.LSN(recordBatch.FirstCheckPointID), + BatchEndlSN: pglogrepl.LSN(recordBatch.LastCheckPointID), StartTime: startTime, }) if err != nil { @@ -223,7 +226,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo } // log the number of records - numRecords := len(records.Records) + numRecords := len(recordBatch.Records) log.WithFields(log.Fields{ "flowName": input.FlowConnectionConfigs.FlowJobName, }).Printf("pulled %d records", numRecords) @@ -233,11 +236,14 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo log.WithFields(log.Fields{ "flowName": input.FlowConnectionConfigs.FlowJobName, }).Info("no records to push") - return nil, nil + return &model.SyncResponse{ + RelationMessageMapping: recordsWithTableSchemaDelta.RelationMessageMapping, + TableSchemaDelta: recordsWithTableSchemaDelta.TableSchemaDelta, + }, nil } res, err := dest.SyncRecords(&model.SyncRecordsRequest{ - Records: records, + Records: recordBatch, FlowJobName: input.FlowConnectionConfigs.FlowJobName, SyncMode: input.FlowConnectionConfigs.CdcSyncMode, StagingPath: input.FlowConnectionConfigs.CdcStagingPath, @@ -254,7 +260,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo err = a.CatalogMirrorMonitor. UpdateLatestLSNAtTargetForCDCFlow(ctx, input.FlowConnectionConfigs.FlowJobName, - pglogrepl.LSN(records.LastCheckPointID)) + pglogrepl.LSN(recordBatch.LastCheckPointID)) if err != nil { return nil, err } @@ -265,7 +271,8 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, input *protos.StartFlo return nil, err } } - + res.TableSchemaDelta = recordsWithTableSchemaDelta.TableSchemaDelta + res.RelationMessageMapping = recordsWithTableSchemaDelta.RelationMessageMapping activity.RecordHeartbeat(ctx, "pushed records") return res, nil @@ -325,6 +332,19 @@ func (a *FlowableActivity) StartNormalize( return res, nil } +func (a *FlowableActivity) ReplayTableSchemaDelta( + ctx context.Context, + input *protos.ReplayTableSchemaDeltaInput, +) error { + dest, err := connectors.GetConnector(ctx, input.FlowConnectionConfigs.Destination) + defer connectors.CloseConnector(dest) + if err != nil { + return fmt.Errorf("failed to get destination connector: %w", err) + } + + return dest.ReplayTableSchemaDelta(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDelta) +} + // SetupQRepMetadataTables sets up the metadata tables for QReplication. func (a *FlowableActivity) SetupQRepMetadataTables(ctx context.Context, config *protos.QRepConfig) error { conn, err := connectors.GetConnector(ctx, config.DestinationPeer) diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index f3e0d84863..2074f2b62e 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -205,6 +205,43 @@ func (c *BigQueryConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } +// ReplayTableSchemaDelta changes a destination table to match the schema at source +// This could involve adding or dropping multiple columns. +func (c *BigQueryConnector) ReplayTableSchemaDelta(flowJobName string, + schemaDelta *protos.TableSchemaDelta) error { + if (schemaDelta == nil) || (len(schemaDelta.AddedColumns) == 0 && len(schemaDelta.DroppedColumns) == 0) { + return nil + } + + for _, droppedColumn := range schemaDelta.DroppedColumns { + _, err := c.client.Query(fmt.Sprintf("ALTER TABLE %s.%s DROP COLUMN %s", c.datasetID, + schemaDelta.DstTableName, droppedColumn)).Read(c.ctx) + if err != nil { + return fmt.Errorf("failed to drop column %s for table %s: %w", droppedColumn, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] dropped column %s", droppedColumn) + } + for _, addedColumn := range schemaDelta.AddedColumns { + _, err := c.client.Query(fmt.Sprintf("ALTER TABLE %s.%s ADD COLUMN %s %s", c.datasetID, + schemaDelta.DstTableName, addedColumn.ColumnName, addedColumn.ColumnType)).Read(c.ctx) + if err != nil { + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] added column %s with data type %s", addedColumn.ColumnName, + addedColumn.ColumnType) + } + + return nil +} + // SetupMetadataTables sets up the metadata tables. func (c *BigQueryConnector) SetupMetadataTables() error { // check if the dataset exists @@ -383,7 +420,8 @@ func (c *BigQueryConnector) getTableNametoUnchangedCols(flowJobName string, sync } // PullRecords pulls records from the source. -func (c *BigQueryConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *BigQueryConnector) PullRecords(req *model.PullRecordsRequest) ( + *model.RecordsWithTableSchemaDelta, error) { panic("not implemented") } diff --git a/flow/connectors/core.go b/flow/connectors/core.go index 18ef1ada1c..37d6a0aec2 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -35,12 +35,15 @@ type Connector interface { // InitializeTableSchema initializes the table schema of all the destination tables for the connector. InitializeTableSchema(req map[string]*protos.TableSchema) error + // ReplayTableSchemaDelta changes a destination table to match the schema at source + // This could involve adding or dropping multiple columns. + ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) 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.RecordBatch, error) + PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) // 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. diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index e6e1e050ff..70c1adafa8 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -99,7 +99,12 @@ func (c *EventHubConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } -func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *EventHubConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + log.Warnf("ReplayTableSchemaDelta is a no-op for EventHub flow connector") + return nil +} + +func (c *EventHubConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { panic("pull records not implemented for event hub") } diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index a3e45b6ed4..34d91dea45 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -6,6 +6,7 @@ import ( "reflect" "time" + "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pglogrepl" @@ -18,42 +19,44 @@ import ( ) type PostgresCDCSource struct { - ctx context.Context - replPool *pgxpool.Pool - SrcTableIDNameMapping map[uint32]string - TableNameMapping map[string]string - slot string - publication string - relations map[uint32]*pglogrepl.RelationMessage - typeMap *pgtype.Map - startLSN pglogrepl.LSN + ctx context.Context + replPool *pgxpool.Pool + SrcTableIDNameMapping map[uint32]string + TableNameMapping map[string]string + slot string + publication string + relationMessageMapping model.RelationMessageMapping + typeMap *pgtype.Map + startLSN pglogrepl.LSN } type PostgresCDCConfig struct { - AppContext context.Context - Connection *pgxpool.Pool - Slot string - Publication string - SrcTableIDNameMapping map[uint32]string - TableNameMapping map[string]string + AppContext context.Context + Connection *pgxpool.Pool + Slot string + Publication string + SrcTableIDNameMapping map[uint32]string + TableNameMapping map[string]string + RelationMessageMapping model.RelationMessageMapping } // 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, - relations: make(map[uint32]*pglogrepl.RelationMessage), - typeMap: pgtype.NewMap(), + ctx: cdcConfig.AppContext, + replPool: cdcConfig.Connection, + SrcTableIDNameMapping: cdcConfig.SrcTableIDNameMapping, + TableNameMapping: cdcConfig.TableNameMapping, + slot: cdcConfig.Slot, + publication: cdcConfig.Publication, + relationMessageMapping: cdcConfig.RelationMessageMapping, + typeMap: pgtype.NewMap(), }, nil } // PullRecords pulls records from the cdc stream -func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (p *PostgresCDCSource) PullRecords(req *model.PullRecordsRequest) ( + *model.RecordsWithTableSchemaDelta, error) { // setup options pluginArguments := []string{ "proto_version '1'", @@ -110,16 +113,23 @@ func (p *PostgresCDCSource) consumeStream( conn *pgconn.PgConn, req *model.PullRecordsRequest, clientXLogPos pglogrepl.LSN, -) (*model.RecordBatch, error) { +) (*model.RecordsWithTableSchemaDelta, error) { // TODO (kaushik): take into consideration the MaxBatchSize // parameters in the original request. - result := &model.RecordBatch{ + records := &model.RecordBatch{ Records: make([]model.Record, 0), TablePKeyLastSeen: make(map[model.TableWithPkey]int), } + result := &model.RecordsWithTableSchemaDelta{ + RecordBatch: records, + TableSchemaDelta: nil, + RelationMessageMapping: p.relationMessageMapping, + } standbyMessageTimeout := req.IdleTimeout nextStandbyMessageDeadline := time.Now().Add(standbyMessageTimeout) + earlyReturn := false + defer func() { err := conn.Close(p.ctx) if err != nil { @@ -193,7 +203,7 @@ func (p *PostgresCDCSource) consumeStream( log.Debugf("XLogData => WALStart %s ServerWALEnd %s ServerTime %s\n", xld.WALStart, xld.ServerWALEnd, xld.ServerTime) - rec, err := p.processMessage(result, xld) + rec, err := p.processMessage(records, xld) if err != nil { return nil, fmt.Errorf("error processing message: %w", err) @@ -201,7 +211,7 @@ func (p *PostgresCDCSource) consumeStream( if !firstProcessed { firstProcessed = true - result.FirstCheckPointID = int64(xld.WALStart) + records.FirstCheckPointID = int64(xld.WALStart) } if rec != nil { tableName := rec.GetTableName() @@ -216,12 +226,12 @@ func (p *PostgresCDCSource) consumeStream( TableName: tableName, PkeyColVal: pkeyColVal, } - _, ok := result.TablePKeyLastSeen[tablePkeyVal] + _, ok := records.TablePKeyLastSeen[tablePkeyVal] if !ok { - result.Records = append(result.Records, rec) - result.TablePKeyLastSeen[tablePkeyVal] = len(result.Records) - 1 + records.Records = append(records.Records, rec) + records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 } else { - oldRec := result.Records[result.TablePKeyLastSeen[tablePkeyVal]] + oldRec := records.Records[records.TablePKeyLastSeen[tablePkeyVal]] // iterate through unchanged toast cols and set them for col, val := range oldRec.GetItems() { if _, ok := r.NewItems[col]; !ok { @@ -229,8 +239,8 @@ func (p *PostgresCDCSource) consumeStream( delete(r.UnchangedToastColumns, col) } } - result.Records = append(result.Records, rec) - result.TablePKeyLastSeen[tablePkeyVal] = len(result.Records) - 1 + records.Records = append(records.Records, rec) + records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 } case *model.InsertRecord: pkeyCol := req.TableNameSchemaMapping[tableName].PrimaryKeyColumn @@ -239,18 +249,27 @@ func (p *PostgresCDCSource) consumeStream( TableName: tableName, PkeyColVal: pkeyColVal, } - result.Records = append(result.Records, rec) + records.Records = append(records.Records, rec) // all columns will be set in insert record, so add it to the map - result.TablePKeyLastSeen[tablePkeyVal] = len(result.Records) - 1 + records.TablePKeyLastSeen[tablePkeyVal] = len(records.Records) - 1 case *model.DeleteRecord: - result.Records = append(result.Records, rec) + records.Records = append(records.Records, rec) + case *model.RelationRecord: + tableSchemaDelta := rec.(*model.RelationRecord).TableSchemaDelta + if len(tableSchemaDelta.AddedColumns) > 0 || len(tableSchemaDelta.DroppedColumns) > 0 { + result.TableSchemaDelta = tableSchemaDelta + log.Infof("Detected schema change for table %s, returning currently accumulated records", + result.TableSchemaDelta.SrcTableName) + earlyReturn = true + } } } currentPos := xld.WALStart + pglogrepl.LSN(len(xld.WALData)) - result.LastCheckPointID = int64(currentPos) + records.LastCheckPointID = int64(currentPos) - if result.Records != nil && len(result.Records) == int(req.MaxBatchSize) { + if records.Records != nil && + ((len(records.Records) == int(req.MaxBatchSize)) || earlyReturn) { return result, nil } } @@ -281,7 +300,12 @@ func (p *PostgresCDCSource) processMessage(batch *model.RecordBatch, xld pglogre // the state of the relation message somewhere log.Infof("RelationMessage => RelationID: %d, Namespace: %s, RelationName: %s, Columns: %v", msg.RelationID, msg.Namespace, msg.RelationName, msg.Columns) - p.relations[msg.RelationID] = msg + if p.relationMessageMapping[msg.RelationID] == nil { + p.relationMessageMapping[msg.RelationID] = convertRelationMessageToProto(msg) + } else { + return p.processRelationMessage(xld.WALStart, convertRelationMessageToProto(msg)) + } + case *pglogrepl.TruncateMessage: log.Warnf("TruncateMessage not supported") default: @@ -304,7 +328,7 @@ func (p *PostgresCDCSource) processInsertMessage( // log lsn and relation id for debugging log.Debugf("InsertMessage => LSN: %d, RelationID: %d, Relation Name: %s", lsn, msg.RelationID, tableName) - rel, ok := p.relations[msg.RelationID] + rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) } @@ -337,7 +361,7 @@ func (p *PostgresCDCSource) processUpdateMessage( // log lsn and relation id for debugging log.Debugf("UpdateMessage => LSN: %d, RelationID: %d, Relation Name: %s", lsn, msg.RelationID, tableName) - rel, ok := p.relations[msg.RelationID] + rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) } @@ -376,7 +400,7 @@ func (p *PostgresCDCSource) processDeleteMessage( // log lsn and relation id for debugging log.Debugf("DeleteMessage => LSN: %d, RelationID: %d, Relation Name: %s", lsn, msg.RelationID, tableName) - rel, ok := p.relations[msg.RelationID] + rel, ok := p.relationMessageMapping[msg.RelationID] if !ok { return nil, fmt.Errorf("unknown relation id: %d", msg.RelationID) } @@ -405,7 +429,7 @@ It takes a tuple and a relation message as input and returns */ func (p *PostgresCDCSource) convertTupleToMap( tuple *pglogrepl.TupleData, - rel *pglogrepl.RelationMessage, + rel *protos.RelationMessage, ) (model.RecordItems, map[string]bool, error) { // if the tuple is nil, return an empty map if tuple == nil { @@ -470,3 +494,67 @@ func (p *PostgresCDCSource) decodeColumnData(data []byte, dataType uint32, forma } return &qvalue.QValue{Kind: qvalue.QValueKindString, Value: string(data)}, nil } + +func convertRelationMessageToProto(msg *pglogrepl.RelationMessage) *protos.RelationMessage { + protoColArray := make([]*protos.RelationMessageColumn, 0) + for _, column := range msg.Columns { + protoColArray = append(protoColArray, &protos.RelationMessageColumn{ + Name: column.Name, + Flags: uint32(column.Flags), + DataType: column.DataType, + }) + } + return &protos.RelationMessage{ + RelationId: msg.RelationID, + RelationName: msg.RelationName, + Columns: protoColArray, + } +} + +// processRelationMessage processes a delete message and returns a TableSchemaDelta +func (p *PostgresCDCSource) processRelationMessage( + lsn pglogrepl.LSN, + currRel *protos.RelationMessage, +) (model.Record, error) { + // retrieve initial RelationMessage for table changed. + prevRel := p.relationMessageMapping[currRel.RelationId] + // creating maps for lookup later + prevRelMap := make(map[string]bool) + currRelMap := make(map[string]bool) + for _, column := range prevRel.Columns { + prevRelMap[column.Name] = true + } + for _, column := range currRel.Columns { + currRelMap[column.Name] = true + } + + 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]], + AddedColumns: make([]*protos.DeltaAddedColumn, 0), + DroppedColumns: make([]string, 0), + } + for _, column := range currRel.Columns { + // not present in previous relation message, but in current one, so added. + if !prevRelMap[column.Name] { + schemaDelta.AddedColumns = append(schemaDelta.AddedColumns, &protos.DeltaAddedColumn{ + ColumnName: column.Name, + ColumnType: string(postgresOIDToQValueKind(column.DataType)), + }) + } + } + for _, column := range prevRel.Columns { + // present in previous relation message, but not in current one, so dropped. + if !currRelMap[column.Name] { + schemaDelta.DroppedColumns = append(schemaDelta.DroppedColumns, column.Name) + } + } + + p.relationMessageMapping[currRel.RelationId] = currRel + return &model.RelationRecord{ + TableSchemaDelta: schemaDelta, + CheckPointID: int64(lsn), + }, nil +} diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 00b82705b7..078a912ce5 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -173,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.RecordBatch, error) { +func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { // Slotname would be the job name prefixed with "peerflow_slot_" slotName := fmt.Sprintf("peerflow_slot_%s", req.FlowJobName) if req.OverrideReplicationSlotName != "" { @@ -211,27 +211,28 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R }).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, + AppContext: c.ctx, + Connection: c.replPool, + SrcTableIDNameMapping: req.SrcTableIDNameMapping, + Slot: slotName, + Publication: publicationName, + TableNameMapping: req.TableNameMapping, + RelationMessageMapping: req.RelationMessageMapping, }) if err != nil { return nil, fmt.Errorf("failed to create cdc source: %w", err) } - recordBatch, err := cdc.PullRecords(req) + recordsWithSchemaDelta, err := cdc.PullRecords(req) if err != nil { return nil, err } - if len(recordBatch.Records) > 0 { + if len(recordsWithSchemaDelta.RecordBatch.Records) > 0 { totalRecordsAtSource, err := c.getApproxTableCounts(maps.Keys(req.TableNameMapping)) if err != nil { return nil, err } - metrics.LogPullMetrics(c.ctx, req.FlowJobName, recordBatch, totalRecordsAtSource) + metrics.LogPullMetrics(c.ctx, req.FlowJobName, recordsWithSchemaDelta.RecordBatch, totalRecordsAtSource) cdcMirrorMonitor, ok := c.ctx.Value(shared.CDCMirrorMonitorKey).(*monitoring.CatalogMirrorMonitor) if ok { latestLSN, err := c.getCurrentLSN() @@ -245,7 +246,7 @@ func (c *PostgresConnector) PullRecords(req *model.PullRecordsRequest) (*model.R } } - return recordBatch, nil + return recordsWithSchemaDelta, nil } // SyncRecords pushes records to the destination. @@ -654,6 +655,64 @@ func (c *PostgresConnector) InitializeTableSchema(req map[string]*protos.TableSc return nil } +// ReplayTableSchemaDelta changes a destination table to match the schema at source +// This could involve adding or dropping multiple columns. +func (c *PostgresConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + if (schemaDelta == nil) || (len(schemaDelta.AddedColumns) == 0 && len(schemaDelta.DroppedColumns) == 0) { + return nil + } + + // Postgres is cool and supports transactional DDL. So we use a transaction. + tableSchemaModifyTx, err := c.pool.Begin(c.ctx) + if err != nil { + return fmt.Errorf("error starting transaction for schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + defer func() { + deferErr := tableSchemaModifyTx.Rollback(c.ctx) + if deferErr != pgx.ErrTxClosed && deferErr != nil { + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Errorf("unexpected error rolling back transaction for table schema modification: %v", err) + } + }() + + for _, droppedColumn := range schemaDelta.DroppedColumns { + _, err = tableSchemaModifyTx.Exec(c.ctx, fmt.Sprintf("ALTER TABLE %s DROP COLUMN %s", schemaDelta.DstTableName, + droppedColumn)) + if err != nil { + return fmt.Errorf("failed to drop column %s for table %s: %w", droppedColumn, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] dropped column %s", droppedColumn) + } + for _, addedColumn := range schemaDelta.AddedColumns { + _, err = tableSchemaModifyTx.Exec(c.ctx, fmt.Sprintf("ALTER TABLE %s ADD COLUMN %s %s", + schemaDelta.DstTableName, addedColumn.ColumnName, + qValueKindToPostgresType(addedColumn.ColumnType))) + if err != nil { + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] added column %s with data type %s", + addedColumn.ColumnName, addedColumn.ColumnType) + } + + err = tableSchemaModifyTx.Commit(c.ctx) + if err != nil { + return fmt.Errorf("failed to commit transaction for table schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + + return nil +} + // EnsurePullability ensures that a table is pullable, implementing the Connector interface. func (c *PostgresConnector) EnsurePullability(req *protos.EnsurePullabilityBatchInput, ) (*protos.EnsurePullabilityBatchOutput, error) { diff --git a/flow/connectors/postgres/postgres_cdc_test.go b/flow/connectors/postgres/postgres_cdc_test.go index ecaf46d39c..727ffa133a 100644 --- a/flow/connectors/postgres/postgres_cdc_test.go +++ b/flow/connectors/postgres/postgres_cdc_test.go @@ -344,6 +344,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { tableNameMapping := map[string]string{ nonExistentFlowSrcTableName: nonExistentFlowDstTableName, } + relationMessageMapping := make(model.RelationMessageMapping) getTblSchemaInput := &protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{nonExistentFlowSrcTableName}, @@ -389,7 +390,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { }) suite.failTestError(err) suite.dropTable(nonExistentFlowSrcTableName) - records, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: nonExistentFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -397,8 +398,10 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) - suite.Equal(0, len(records.Records)) + suite.Equal(0, len(recordsWithSchemaDelta.RecordBatch.Records)) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) suite.Nil(err) err = suite.connector.PullFlowCleanup(nonExistentFlowName) @@ -429,6 +432,8 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { tableNameMapping := map[string]string{ simpleHappyFlowSrcTableName: simpleHappyFlowDstTableName, } + relationMessageMapping := make(model.RelationMessageMapping) + err = suite.connector.SetupReplication(nil, &protos.SetupReplicationInput{ FlowJobName: simpleHappyFlowName, TableNameMapping: tableNameMapping, @@ -458,8 +463,8 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { tableNameSchemaMapping[simpleHappyFlowDstTableName] = tableNameSchema.TableNameSchemaMapping[simpleHappyFlowSrcTableName] - // pulling with no records. - records, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + // pulling with no recordsWithSchemaDelta. + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -467,15 +472,18 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Equal(0, len(records.Records)) - suite.Equal(int64(0), records.FirstCheckPointID) - suite.Equal(int64(0), records.LastCheckPointID) + 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 // pulling after inserting records. suite.insertSimpleRecords(simpleHappyFlowSrcTableName) - records, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: nil, IdleTimeout: 5 * time.Second, @@ -483,19 +491,24 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateInsertedSimpleRecords(records.Records, simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) - suite.Greater(records.FirstCheckPointID, int64(0)) - suite.GreaterOrEqual(records.LastCheckPointID, records.FirstCheckPointID) - currentCheckPointID := records.LastCheckPointID + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.validateInsertedSimpleRecords(recordsWithSchemaDelta.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 // pulling after mutating records. suite.mutateSimpleRecords(simpleHappyFlowSrcTableName) - records, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: simpleHappyFlowName, LastSyncState: &protos.LastSyncState{ - Checkpoint: records.LastCheckPointID, + Checkpoint: recordsWithSchemaDelta.RecordBatch.LastCheckPointID, LastSyncedAt: nil, }, IdleTimeout: 5 * time.Second, @@ -503,11 +516,15 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateSimpleMutatedRecords(records.Records, simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) - suite.GreaterOrEqual(records.FirstCheckPointID, currentCheckPointID) - suite.GreaterOrEqual(records.LastCheckPointID, records.FirstCheckPointID) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.validateSimpleMutatedRecords(recordsWithSchemaDelta.RecordBatch.Records, + simpleHappyFlowSrcTableName, simpleHappyFlowDstTableName) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, currentCheckPointID) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) err = suite.connector.PullFlowCleanup(simpleHappyFlowName) suite.failTestError(err) @@ -538,6 +555,7 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { suite.failTestError(err) tableRelID := ensurePullabilityOutput.TableIdentifierMapping[allTypesHappyFlowSrcTableName]. GetPostgresTableIdentifier().RelId + relationMessageMapping := make(model.RelationMessageMapping) relIDTableNameMapping := map[uint32]string{ tableRelID: allTypesHappyFlowSrcTableName, @@ -627,10 +645,11 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.Equal(1, len(records.Records)) - suite.Equal(35, len(records.Records[0].GetItems())) + suite.Equal(1, len(records.RecordBatch.Records)) + suite.Equal(35, len(records.RecordBatch.Records[0].GetItems())) err = suite.connector.PullFlowCleanup(allTypesHappyFlowName) suite.failTestError(err) @@ -663,6 +682,8 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { tableNameMapping := map[string]string{ toastHappyFlowSrcTableName: toastHappyFlowDstTableName, } + relationMessageMapping := make(model.RelationMessageMapping) + err = suite.connector.SetupReplication(nil, &protos.SetupReplicationInput{ FlowJobName: toastHappyFlowName, TableNameMapping: tableNameMapping, @@ -695,7 +716,7 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { tableNameSchema.TableNameSchemaMapping[toastHappyFlowSrcTableName] suite.insertToastRecords(toastHappyFlowSrcTableName) - records, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: nil, IdleTimeout: 10 * time.Second, @@ -703,17 +724,22 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateInsertedToastRecords(records.Records, toastHappyFlowSrcTableName, toastHappyFlowDstTableName) - suite.Greater(records.FirstCheckPointID, int64(0)) - suite.GreaterOrEqual(records.LastCheckPointID, records.FirstCheckPointID) + suite.Nil(recordsWithSchemaDelta.TableSchemaDelta) + suite.validateInsertedToastRecords(recordsWithSchemaDelta.RecordBatch.Records, + toastHappyFlowSrcTableName, toastHappyFlowDstTableName) + suite.Greater(recordsWithSchemaDelta.RecordBatch.FirstCheckPointID, int64(0)) + suite.GreaterOrEqual(recordsWithSchemaDelta.RecordBatch.LastCheckPointID, + recordsWithSchemaDelta.RecordBatch.FirstCheckPointID) + relationMessageMapping = recordsWithSchemaDelta.RelationMessageMapping suite.mutateToastRecords(toastHappyFlowSrcTableName) - records, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: &protos.LastSyncState{ - Checkpoint: records.LastCheckPointID, + Checkpoint: recordsWithSchemaDelta.RecordBatch.LastCheckPointID, LastSyncedAt: nil, }, IdleTimeout: 10 * time.Second, @@ -721,9 +747,11 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { SrcTableIDNameMapping: relIDTableNameMapping, TableNameMapping: tableNameMapping, TableNameSchemaMapping: tableNameSchemaMapping, + RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - suite.validateMutatedToastRecords(records.Records, toastHappyFlowSrcTableName, toastHappyFlowDstTableName) + suite.validateMutatedToastRecords(recordsWithSchemaDelta.RecordBatch.Records, toastHappyFlowSrcTableName, + toastHappyFlowDstTableName) err = suite.connector.PullFlowCleanup(toastHappyFlowName) suite.failTestError(err) diff --git a/flow/connectors/postgres/postgres_repl_test.go b/flow/connectors/postgres/postgres_repl_test.go index 933273174c..11ef7e2be8 100644 --- a/flow/connectors/postgres/postgres_repl_test.go +++ b/flow/connectors/postgres/postgres_repl_test.go @@ -72,7 +72,7 @@ func (suite *PostgresReplicationSnapshotTestSuite) TearDownSuite() { // Fetch all the publications rows, err := teardownTx.Query(context.Background(), - "SELECT pubname FROM pg_publication WHERE pubname LIKE 'peerflow_pub%'") + "SELECT pubname FROM pg_publication WHERE pubname LIKE $1", fmt.Sprintf("%%%s", "test_simple_slot_creation")) require.NoError(suite.T(), err) // Iterate over the publications and drop them @@ -87,7 +87,8 @@ func (suite *PostgresReplicationSnapshotTestSuite) TearDownSuite() { } _, err = teardownTx.Exec(context.Background(), - "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots") + "SELECT pg_drop_replication_slot(slot_name) FROM pg_replication_slots WHERE slot_name LIKE $1", + fmt.Sprintf("%%%s", "test_simple_slot_creation")) require.NoError(suite.T(), err) err = teardownTx.Commit(context.Background()) diff --git a/flow/connectors/s3/s3.go b/flow/connectors/s3/s3.go index 742bd28981..39bf5297ab 100644 --- a/flow/connectors/s3/s3.go +++ b/flow/connectors/s3/s3.go @@ -82,7 +82,12 @@ func (c *S3Connector) InitializeTableSchema(req map[string]*protos.TableSchema) return fmt.Errorf("cdc based replication is not currently supported for S3 target") } -func (c *S3Connector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *S3Connector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + log.Warnf("ReplayTableSchemaDelta is a no-op for S3 flow connector") + return nil +} + +func (c *S3Connector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { log.Errorf("panicking at call to PullRecords for S3 flow connector") panic("PullRecords is not implemented for the S3 flow connector") } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 7852eedc9c..b2aa542e75 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -364,7 +364,64 @@ func (c *SnowflakeConnector) InitializeTableSchema(req map[string]*protos.TableS return nil } -func (c *SnowflakeConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +// ReplayTableSchemaDelta changes a destination table to match the schema at source +// This could involve adding or dropping multiple columns. +func (c *SnowflakeConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + if (schemaDelta == nil) || (len(schemaDelta.AddedColumns) == 0 && len(schemaDelta.DroppedColumns) == 0) { + return nil + } + + // Postgres is cool and supports transactional DDL. So we use a transaction. + tableSchemaModifyTx, err := c.database.Begin() + if err != nil { + return fmt.Errorf("error starting transaction for schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + defer func() { + deferErr := tableSchemaModifyTx.Rollback() + if deferErr != sql.ErrTxDone && deferErr != nil { + log.WithFields(log.Fields{ + "flowName": flowJobName, + }).Errorf("unexpected error rolling back transaction for table schema modification: %v", err) + } + }() + + for _, droppedColumn := range schemaDelta.DroppedColumns { + _, err = tableSchemaModifyTx.Exec(fmt.Sprintf("ALTER TABLE %s DROP COLUMN %s", schemaDelta.DstTableName, + droppedColumn)) + if err != nil { + return fmt.Errorf("failed to drop column %s for table %s: %w", droppedColumn, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] dropped column %s", droppedColumn) + } + for _, addedColumn := range schemaDelta.AddedColumns { + _, err = tableSchemaModifyTx.Exec(fmt.Sprintf("ALTER TABLE %s ADD COLUMN %s %s", schemaDelta.DstTableName, + addedColumn.ColumnName, qValueKindToSnowflakeType(qvalue.QValueKind(addedColumn.ColumnType)))) + if err != nil { + return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, + schemaDelta.SrcTableName, err) + } + log.WithFields(log.Fields{ + "flowName": flowJobName, + "tableName": schemaDelta.SrcTableName, + }).Infof("[schema delta replay] added column %s with data type %s", addedColumn.ColumnName, + addedColumn.ColumnType) + } + + err = tableSchemaModifyTx.Commit() + if err != nil { + return fmt.Errorf("failed to commit transaction for table schema modification for table %s: %w", + schemaDelta.SrcTableName, err) + } + + return nil +} + +func (c *SnowflakeConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { log.Errorf("panicking at call to PullRecords for Snowflake flow connector") panic("PullRecords is not implemented for the Snowflake flow connector") } diff --git a/flow/connectors/sqlserver/sqlserver.go b/flow/connectors/sqlserver/sqlserver.go index a6f1d70b52..feb15e8a47 100644 --- a/flow/connectors/sqlserver/sqlserver.go +++ b/flow/connectors/sqlserver/sqlserver.go @@ -105,7 +105,12 @@ func (c *SQLServerConnector) InitializeTableSchema(req map[string]*protos.TableS return fmt.Errorf("cdc based replication is not currently supported for SQLServer target") } -func (c *SQLServerConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordBatch, error) { +func (c *SQLServerConnector) ReplayTableSchemaDelta(flowJobName string, schemaDelta *protos.TableSchemaDelta) error { + log.Warnf("ReplayTableSchemaDelta is a no-op for SQLServer flow connector") + return nil +} + +func (c *SQLServerConnector) PullRecords(req *model.PullRecordsRequest) (*model.RecordsWithTableSchemaDelta, error) { log.Errorf("panicking at call to PullRecords for SQLServer flow connector") panic("PullRecords is not implemented for the SQLServer flow connector") } diff --git a/flow/e2e/bigquery/bigquery_helper.go b/flow/e2e/bigquery/bigquery_helper.go index 72e8b3edea..d17fe7a668 100644 --- a/flow/e2e/bigquery/bigquery_helper.go +++ b/flow/e2e/bigquery/bigquery_helper.go @@ -169,8 +169,8 @@ func (b *BigQueryTestHelper) RunCommand(command string) error { return nil } -// CountRows(tableName) returns the number of rows in the given table. -func (b *BigQueryTestHelper) CountRows(tableName string) (int, error) { +// countRows(tableName) returns the number of rows in the given table. +func (b *BigQueryTestHelper) countRows(tableName string) (int, error) { command := fmt.Sprintf("SELECT COUNT(*) FROM `%s.%s`", b.Config.DatasetId, tableName) it, err := b.client.Query(command).Read(context.Background()) if err != nil { diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 655a317854..72acb12230 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -261,7 +261,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Complete_Simple_Flow_BQ() { s.Error(err) s.Contains(err.Error(), "continue as new") - count, err := s.bqHelper.CountRows(dstTableName) + count, err := s.bqHelper.countRows(dstTableName) s.NoError(err) s.Equal(10, count) @@ -854,7 +854,7 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Flow_BQ_Avro_CDC() { s.Error(err) s.Contains(err.Error(), "continue as new") - count, err := s.bqHelper.CountRows(dstTableName) + count, err := s.bqHelper.countRows(dstTableName) s.NoError(err) s.Equal(10, count) @@ -913,9 +913,9 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { require.True(s.T(), env.IsWorkflowCompleted()) err = env.GetWorkflowError() - count1, err := s.bqHelper.CountRows(dstTable1Name) + count1, err := s.bqHelper.countRows(dstTable1Name) s.NoError(err) - count2, err := s.bqHelper.CountRows(dstTable2Name) + count2, err := s.bqHelper.countRows(dstTable2Name) s.NoError(err) s.Equal(1, count1) @@ -923,3 +923,107 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Multi_Table_BQ() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") + dstTableName := "test_simple_schema_changes" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 BIGINT + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_schema_changes"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 10, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and mutate schema repeatedly. + go func() { + // insert first row. + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) + s.NoError(err) + fmt.Println("Inserted initial row in the source table") + + // verify we got our first row. + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1") + + // alter source table, add column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, added column c2") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) + s.NoError(err) + fmt.Println("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 4) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1,c2") + + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c2 and added column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) + s.NoError(err) + fmt.Println("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 6) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1,c3") + + // alter source table, drop column c3 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c3`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) + s.NoError(err) + fmt.Println("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 8) + s.compareTableSchemasBQ("test_simple_schema_changes") + s.compareTableContentsBQ("test_simple_schema_changes", "id,c1") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/bigquery/qrep_flow_bq_test.go b/flow/e2e/bigquery/qrep_flow_bq_test.go index 5e6374cc1a..fb0fc3b876 100644 --- a/flow/e2e/bigquery/qrep_flow_bq_test.go +++ b/flow/e2e/bigquery/qrep_flow_bq_test.go @@ -3,6 +3,8 @@ package e2e_bigquery import ( "context" "fmt" + "sort" + "strings" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/e2e" @@ -27,6 +29,32 @@ func (s *PeerFlowE2ETestSuiteBQ) setupBQDestinationTable(dstTable string) { fmt.Printf("created table on bigquery: %s.%s. %v\n", s.bqHelper.Config.DatasetId, dstTable, err) } +func (s *PeerFlowE2ETestSuiteBQ) compareTableSchemasBQ(tableName string) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT * FROM e2e_test_%s.%s ORDER BY id", bigquerySuffix, tableName), + ) + s.NoError(err) + sort.Slice(pgRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(pgRows.Schema.Fields[i].Name, pgRows.Schema.Fields[j].Name) == -1 + }) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("`%s.%s`", s.bqHelper.Config.DatasetId, tableName) + bqRows, err := s.bqHelper.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT * FROM %s ORDER BY id", qualifiedTableName), + ) + s.NoError(err) + sort.Slice(bqRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(bqRows.Schema.Fields[i].Name, bqRows.Schema.Fields[j].Name) == -1 + }) + + s.True(pgRows.Schema.EqualNames(bqRows.Schema), "schemas from source and destination tables are not equal") +} + func (s *PeerFlowE2ETestSuiteBQ) compareTableContentsBQ(tableName string, colsString string) { // read rows from source table pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go new file mode 100644 index 0000000000..ec120a1e8c --- /dev/null +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -0,0 +1,184 @@ +package e2e_postgres + +import ( + "context" + "fmt" + + "github.com/PeerDB-io/peer-flow/e2e" + peerflow "github.com/PeerDB-io/peer-flow/workflows" +) + +func (s *PeerFlowE2ETestSuitePG) attachSchemaSuffix(tableName string) string { + return fmt.Sprintf("e2e_test_%s.%s", postgresSuffix, tableName) +} + +func (s *PeerFlowE2ETestSuitePG) attachSuffix(input string) string { + return fmt.Sprintf("%s_%s", input, postgresSuffix) +} + +func (s *PeerFlowE2ETestSuitePG) Test_Simple_Flow_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_flow") + dstTableName := s.attachSchemaSuffix("test_simple_flow_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id SERIAL PRIMARY KEY, + key TEXT NOT NULL, + value TEXT NOT NULL + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert 10 rows into the source table + go func() { + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testKey := fmt.Sprintf("test_key_%d", i) + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(key, value) VALUES ($1, $2) + `, srcTableName), testKey, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") + dstTableName := s.attachSchemaSuffix("test_simple_schema_changes_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 BIGINT + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_schema_changes"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 10, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and mutate schema repeatedly. + go func() { + // insert first row. + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) + s.NoError(err) + fmt.Println("Inserted initial row in the source table") + + // verify we got our first row. + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + // alter source table, add column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, added column c2") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) + s.NoError(err) + fmt.Println("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 4) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c2 and added column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) + s.NoError(err) + fmt.Println("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 6) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + + // alter source table, drop column c3 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c3`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) + s.NoError(err) + fmt.Println("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 8) + err = s.comparePGTables(srcTableName, dstTableName) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/postgres/postgres_helper.go b/flow/e2e/postgres/postgres_helper.go new file mode 100644 index 0000000000..6f9b471cc3 --- /dev/null +++ b/flow/e2e/postgres/postgres_helper.go @@ -0,0 +1,15 @@ +package e2e_postgres + +import "github.com/PeerDB-io/peer-flow/generated/protos" + +func generatePGPeer(postgresConfig *protos.PostgresConfig) *protos.Peer { + ret := &protos.Peer{} + ret.Name = "test_pg_peer" + ret.Type = protos.DBType_POSTGRES + + ret.Config = &protos.Peer_PostgresConfig{ + PostgresConfig: postgresConfig, + } + + return ret +} diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 241659c229..1f084409a2 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -3,6 +3,7 @@ package e2e_postgres import ( "context" "fmt" + "strings" "testing" "github.com/PeerDB-io/peer-flow/e2e" @@ -21,6 +22,7 @@ type PeerFlowE2ETestSuitePG struct { testsuite.WorkflowTestSuite pool *pgxpool.Pool + peer *protos.Peer } func TestPeerFlowE2ETestSuitePG(t *testing.T) { @@ -43,6 +45,7 @@ func (s *PeerFlowE2ETestSuitePG) SetupSuite() { s.Fail("failed to setup postgres", err) } s.pool = pool + s.peer = generatePGPeer(e2e.GetTestPostgresConf()) } // Implement TearDownAllSuite interface to tear down the test suite @@ -62,14 +65,28 @@ func (s *PeerFlowE2ETestSuitePG) setupSourceTable(tableName string, rowCount int func (s *PeerFlowE2ETestSuitePG) comparePGTables(srcSchemaQualified, dstSchemaQualified string) error { // Execute the two EXCEPT queries - err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) - if err != nil { - return err + for { + err := s.compareQuery(srcSchemaQualified, dstSchemaQualified) + // while testing, the prepared plan might break due to schema changes + // solution is to retry, prepared statement should be evicted upon the first error + if err != nil && !strings.Contains(err.Error(), "cached plan must not change result type") { + return err + } + if err == nil { + break + } } - err = s.compareQuery(dstSchemaQualified, srcSchemaQualified) - if err != nil { - return err + for { + err := s.compareQuery(dstSchemaQualified, srcSchemaQualified) + // while testing, the prepared plan might break due to schema changes + // solution is to retry, prepared statement should be evicted upon the first error + if err != nil && !strings.Contains(err.Error(), "cached plan must not change result type") { + return err + } + if err == nil { + break + } } // If no error is returned, then the contents of the two tables are the same @@ -79,9 +96,11 @@ func (s *PeerFlowE2ETestSuitePG) comparePGTables(srcSchemaQualified, dstSchemaQu func (s *PeerFlowE2ETestSuitePG) compareQuery(schema1, schema2 string) error { query := fmt.Sprintf("SELECT * FROM %s EXCEPT SELECT * FROM %s", schema1, schema2) rows, _ := s.pool.Query(context.Background(), query) + rowsPresent := false defer rows.Close() for rows.Next() { + rowsPresent = true values, err := rows.Values() if err != nil { return err @@ -95,7 +114,13 @@ func (s *PeerFlowE2ETestSuitePG) compareQuery(schema1, schema2 string) error { fmt.Println("---") } - return rows.Err() + if rows.Err() != nil { + return rows.Err() + } + if rowsPresent { + return fmt.Errorf("comparison failed: rows are not equal") + } + return nil } func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index db55046961..b6090a9df0 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -798,3 +798,107 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Multi_Table_SF() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_schema_changes") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_schema_changes") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + c1 BIGINT + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_simple_schema_changes"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.PeerFlowLimits{ + TotalSyncFlows: 10, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert and mutate schema repeatedly. + go func() { + // insert first row. + e2e.SetupPeerFlowStatusQuery(env, connectionGen) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 1) + s.NoError(err) + fmt.Println("Inserted initial row in the source table") + + // verify we got our first row. + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1", false) + + // alter source table, add column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s ADD COLUMN c2 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, added column c2") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c2) VALUES ($1,$2)`, srcTableName), 2, 2) + s.NoError(err) + fmt.Println("Inserted row with added c2 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 4) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1,c2", false) + + // alter source table, add column c3, drop column c2 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c2, ADD COLUMN c3 BIGINT`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c2 and added column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1,c3) VALUES ($1,$2)`, srcTableName), 3, 3) + s.NoError(err) + fmt.Println("Inserted row with added c3 in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 6) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1,c3", false) + + // alter source table, drop column c3 and insert another row. + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + ALTER TABLE %s DROP COLUMN c3`, srcTableName)) + s.NoError(err) + fmt.Println("Altered source table, dropped column c3") + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c1) VALUES ($1)`, srcTableName), 4) + s.NoError(err) + fmt.Println("Inserted row after dropping all columns in the source table") + + // verify we got our two rows, if schema did not match up it will error. + e2e.NormalizeFlowCountQuery(env, connectionGen, 8) + s.compareTableSchemasSF("test_simple_schema_changes") + s.compareTableContentsSF("test_simple_schema_changes", "id,c1", false) + }() + + env.ExecuteWorkflow(peerflow.PeerFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index 4d42adbcc3..4595a198e9 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -3,6 +3,8 @@ package e2e_snowflake import ( "context" "fmt" + "sort" + "strings" connpostgres "github.com/PeerDB-io/peer-flow/connectors/postgres" "github.com/PeerDB-io/peer-flow/e2e" @@ -30,6 +32,33 @@ func (s *PeerFlowE2ETestSuiteSF) setupSFDestinationTable(dstTable string) { fmt.Printf("created table on snowflake: %s.%s. %v\n", s.sfHelper.testSchemaName, dstTable, err) } +func (s *PeerFlowE2ETestSuiteSF) compareTableSchemasSF(tableName string) { + // read rows from source table + pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") + pgQueryExecutor.SetTestEnv(true) + pgRows, err := pgQueryExecutor.ExecuteAndProcessQuery( + fmt.Sprintf("SELECT * FROM e2e_test_%s.%s LIMIT 0", snowflakeSuffix, tableName), + ) + require.NoError(s.T(), err) + sort.Slice(pgRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(pgRows.Schema.Fields[i].Name, pgRows.Schema.Fields[j].Name) == -1 + }) + + // read rows from destination table + qualifiedTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, tableName) + // excluding soft-delete column during schema conversion + sfSelQuery := fmt.Sprintf(`SELECT * EXCLUDE _PEERDB_IS_DELETED FROM %s LIMIT 0`, qualifiedTableName) + fmt.Printf("running query on snowflake: %s\n", sfSelQuery) + + sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) + require.NoError(s.T(), err) + sort.Slice(sfRows.Schema.Fields, func(i int, j int) bool { + return strings.Compare(sfRows.Schema.Fields[i].Name, sfRows.Schema.Fields[j].Name) == -1 + }) + + s.True(pgRows.Schema.EqualNames(sfRows.Schema), "schemas from source and destination tables are not equal") +} + func (s *PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, selector string, caseSensitive bool) { // read rows from source table pgQueryExecutor := connpostgres.NewQRepQueryExecutor(s.pool, context.Background(), "testflow", "testpart") @@ -49,9 +78,6 @@ func (s *PeerFlowE2ETestSuiteSF) compareTableContentsSF(tableName string, select } fmt.Printf("running query on snowflake: %s\n", sfSelQuery) - // sleep for 1 min for debugging - // time.Sleep(1 * time.Minute) - sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) require.NoError(s.T(), err) diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 761e0cbf51..2d4b005dff 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -85,6 +85,36 @@ func SetupPeerFlowStatusQuery(env *testsuite.TestWorkflowEnvironment, } } +func NormalizeFlowCountQuery(env *testsuite.TestWorkflowEnvironment, + connectionGen FlowConnectionGenerationConfig, + minCount int) { + // wait for PeerFlowStatusQuery to finish setup + // sleep for 5 second to allow the workflow to start + time.Sleep(5 * time.Second) + for { + response, err := env.QueryWorkflow( + peerflow.PeerFlowStatusQuery, + connectionGen.FlowJobName, + ) + if err == nil { + var state peerflow.PeerFlowState + err = response.Get(&state) + if err != nil { + log.Errorln(err) + } + + if len(state.NormalizeFlowStatuses) >= minCount { + fmt.Println("query indicates setup is complete") + break + } + } else { + // log the error for informational purposes + log.Errorln(err) + } + time.Sleep(1 * time.Second) + } +} + func CreateSourceTableQRep(pool *pgxpool.Pool, suffix string, tableName string) error { tblFields := []string{ "id UUID NOT NULL PRIMARY KEY", diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 13fca6b7b5..acdc22f25f 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -169,6 +169,132 @@ func (x *TableNameMapping) GetDestinationTableName() string { return "" } +type RelationMessageColumn struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Flags uint32 `protobuf:"varint,1,opt,name=flags,proto3" json:"flags,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + DataType uint32 `protobuf:"varint,3,opt,name=data_type,json=dataType,proto3" json:"data_type,omitempty"` +} + +func (x *RelationMessageColumn) Reset() { + *x = RelationMessageColumn{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RelationMessageColumn) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RelationMessageColumn) ProtoMessage() {} + +func (x *RelationMessageColumn) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[1] + 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 RelationMessageColumn.ProtoReflect.Descriptor instead. +func (*RelationMessageColumn) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{1} +} + +func (x *RelationMessageColumn) GetFlags() uint32 { + if x != nil { + return x.Flags + } + return 0 +} + +func (x *RelationMessageColumn) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *RelationMessageColumn) GetDataType() uint32 { + if x != nil { + return x.DataType + } + return 0 +} + +type RelationMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + RelationId uint32 `protobuf:"varint,1,opt,name=relation_id,json=relationId,proto3" json:"relation_id,omitempty"` + RelationName string `protobuf:"bytes,2,opt,name=relation_name,json=relationName,proto3" json:"relation_name,omitempty"` + Columns []*RelationMessageColumn `protobuf:"bytes,3,rep,name=columns,proto3" json:"columns,omitempty"` +} + +func (x *RelationMessage) Reset() { + *x = RelationMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *RelationMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RelationMessage) ProtoMessage() {} + +func (x *RelationMessage) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[2] + 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 RelationMessage.ProtoReflect.Descriptor instead. +func (*RelationMessage) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{2} +} + +func (x *RelationMessage) GetRelationId() uint32 { + if x != nil { + return x.RelationId + } + return 0 +} + +func (x *RelationMessage) GetRelationName() string { + if x != nil { + return x.RelationName + } + return "" +} + +func (x *RelationMessage) GetColumns() []*RelationMessageColumn { + if x != nil { + return x.Columns + } + return nil +} + type FlowConnectionConfigs struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -206,7 +332,7 @@ type FlowConnectionConfigs struct { func (x *FlowConnectionConfigs) Reset() { *x = FlowConnectionConfigs{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[1] + mi := &file_flow_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -219,7 +345,7 @@ func (x *FlowConnectionConfigs) String() string { func (*FlowConnectionConfigs) ProtoMessage() {} func (x *FlowConnectionConfigs) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[1] + mi := &file_flow_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -232,7 +358,7 @@ func (x *FlowConnectionConfigs) ProtoReflect() protoreflect.Message { // Deprecated: Use FlowConnectionConfigs.ProtoReflect.Descriptor instead. func (*FlowConnectionConfigs) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{1} + return file_flow_proto_rawDescGZIP(), []int{3} } func (x *FlowConnectionConfigs) GetSource() *Peer { @@ -394,13 +520,14 @@ type SyncFlowOptions struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - BatchSize int32 `protobuf:"varint,1,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` + BatchSize int32 `protobuf:"varint,1,opt,name=batch_size,json=batchSize,proto3" json:"batch_size,omitempty"` + RelationMessageMapping map[uint32]*RelationMessage `protobuf:"bytes,2,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"` } func (x *SyncFlowOptions) Reset() { *x = SyncFlowOptions{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[2] + mi := &file_flow_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -413,7 +540,7 @@ func (x *SyncFlowOptions) String() string { func (*SyncFlowOptions) ProtoMessage() {} func (x *SyncFlowOptions) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[2] + mi := &file_flow_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -426,7 +553,7 @@ func (x *SyncFlowOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use SyncFlowOptions.ProtoReflect.Descriptor instead. func (*SyncFlowOptions) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{2} + return file_flow_proto_rawDescGZIP(), []int{4} } func (x *SyncFlowOptions) GetBatchSize() int32 { @@ -436,6 +563,13 @@ func (x *SyncFlowOptions) GetBatchSize() int32 { return 0 } +func (x *SyncFlowOptions) GetRelationMessageMapping() map[uint32]*RelationMessage { + if x != nil { + return x.RelationMessageMapping + } + return nil +} + type NormalizeFlowOptions struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -447,7 +581,7 @@ type NormalizeFlowOptions struct { func (x *NormalizeFlowOptions) Reset() { *x = NormalizeFlowOptions{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[3] + mi := &file_flow_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -460,7 +594,7 @@ func (x *NormalizeFlowOptions) String() string { func (*NormalizeFlowOptions) ProtoMessage() {} func (x *NormalizeFlowOptions) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[3] + mi := &file_flow_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -473,7 +607,7 @@ func (x *NormalizeFlowOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use NormalizeFlowOptions.ProtoReflect.Descriptor instead. func (*NormalizeFlowOptions) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{3} + return file_flow_proto_rawDescGZIP(), []int{5} } func (x *NormalizeFlowOptions) GetBatchSize() int32 { @@ -495,7 +629,7 @@ type LastSyncState struct { func (x *LastSyncState) Reset() { *x = LastSyncState{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[4] + mi := &file_flow_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -508,7 +642,7 @@ func (x *LastSyncState) String() string { func (*LastSyncState) ProtoMessage() {} func (x *LastSyncState) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[4] + mi := &file_flow_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -521,7 +655,7 @@ func (x *LastSyncState) ProtoReflect() protoreflect.Message { // Deprecated: Use LastSyncState.ProtoReflect.Descriptor instead. func (*LastSyncState) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{4} + return file_flow_proto_rawDescGZIP(), []int{6} } func (x *LastSyncState) GetCheckpoint() int64 { @@ -543,15 +677,16 @@ type StartFlowInput struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - LastSyncState *LastSyncState `protobuf:"bytes,1,opt,name=last_sync_state,json=lastSyncState,proto3" json:"last_sync_state,omitempty"` - FlowConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,2,opt,name=flow_connection_configs,json=flowConnectionConfigs,proto3" json:"flow_connection_configs,omitempty"` - SyncFlowOptions *SyncFlowOptions `protobuf:"bytes,3,opt,name=sync_flow_options,json=syncFlowOptions,proto3" json:"sync_flow_options,omitempty"` + LastSyncState *LastSyncState `protobuf:"bytes,1,opt,name=last_sync_state,json=lastSyncState,proto3" json:"last_sync_state,omitempty"` + FlowConnectionConfigs *FlowConnectionConfigs `protobuf:"bytes,2,opt,name=flow_connection_configs,json=flowConnectionConfigs,proto3" json:"flow_connection_configs,omitempty"` + SyncFlowOptions *SyncFlowOptions `protobuf:"bytes,3,opt,name=sync_flow_options,json=syncFlowOptions,proto3" json:"sync_flow_options,omitempty"` + RelationMessageMapping map[uint32]*RelationMessage `protobuf:"bytes,4,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"` } func (x *StartFlowInput) Reset() { *x = StartFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[5] + mi := &file_flow_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -564,7 +699,7 @@ func (x *StartFlowInput) String() string { func (*StartFlowInput) ProtoMessage() {} func (x *StartFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[5] + mi := &file_flow_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -577,7 +712,7 @@ func (x *StartFlowInput) ProtoReflect() protoreflect.Message { // Deprecated: Use StartFlowInput.ProtoReflect.Descriptor instead. func (*StartFlowInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{5} + return file_flow_proto_rawDescGZIP(), []int{7} } func (x *StartFlowInput) GetLastSyncState() *LastSyncState { @@ -601,6 +736,13 @@ func (x *StartFlowInput) GetSyncFlowOptions() *SyncFlowOptions { return nil } +func (x *StartFlowInput) GetRelationMessageMapping() map[uint32]*RelationMessage { + if x != nil { + return x.RelationMessageMapping + } + return nil +} + type StartNormalizeInput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -612,7 +754,7 @@ type StartNormalizeInput struct { func (x *StartNormalizeInput) Reset() { *x = StartNormalizeInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[6] + mi := &file_flow_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -625,7 +767,7 @@ func (x *StartNormalizeInput) String() string { func (*StartNormalizeInput) ProtoMessage() {} func (x *StartNormalizeInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[6] + mi := &file_flow_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -638,7 +780,7 @@ func (x *StartNormalizeInput) ProtoReflect() protoreflect.Message { // Deprecated: Use StartNormalizeInput.ProtoReflect.Descriptor instead. func (*StartNormalizeInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{6} + return file_flow_proto_rawDescGZIP(), []int{8} } func (x *StartNormalizeInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { @@ -660,7 +802,7 @@ type GetLastSyncedIDInput struct { func (x *GetLastSyncedIDInput) Reset() { *x = GetLastSyncedIDInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[7] + mi := &file_flow_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -673,7 +815,7 @@ func (x *GetLastSyncedIDInput) String() string { func (*GetLastSyncedIDInput) ProtoMessage() {} func (x *GetLastSyncedIDInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[7] + mi := &file_flow_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -686,7 +828,7 @@ func (x *GetLastSyncedIDInput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetLastSyncedIDInput.ProtoReflect.Descriptor instead. func (*GetLastSyncedIDInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{7} + return file_flow_proto_rawDescGZIP(), []int{9} } func (x *GetLastSyncedIDInput) GetPeerConnectionConfig() *Peer { @@ -716,7 +858,7 @@ type EnsurePullabilityInput struct { func (x *EnsurePullabilityInput) Reset() { *x = EnsurePullabilityInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[8] + mi := &file_flow_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -729,7 +871,7 @@ func (x *EnsurePullabilityInput) String() string { func (*EnsurePullabilityInput) ProtoMessage() {} func (x *EnsurePullabilityInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[8] + mi := &file_flow_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -742,7 +884,7 @@ func (x *EnsurePullabilityInput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityInput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{8} + return file_flow_proto_rawDescGZIP(), []int{10} } func (x *EnsurePullabilityInput) GetPeerConnectionConfig() *Peer { @@ -779,7 +921,7 @@ type EnsurePullabilityBatchInput struct { func (x *EnsurePullabilityBatchInput) Reset() { *x = EnsurePullabilityBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[9] + mi := &file_flow_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -792,7 +934,7 @@ func (x *EnsurePullabilityBatchInput) String() string { func (*EnsurePullabilityBatchInput) ProtoMessage() {} func (x *EnsurePullabilityBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[9] + mi := &file_flow_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -805,7 +947,7 @@ func (x *EnsurePullabilityBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityBatchInput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{9} + return file_flow_proto_rawDescGZIP(), []int{11} } func (x *EnsurePullabilityBatchInput) GetPeerConnectionConfig() *Peer { @@ -840,7 +982,7 @@ type PostgresTableIdentifier struct { func (x *PostgresTableIdentifier) Reset() { *x = PostgresTableIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[10] + mi := &file_flow_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -853,7 +995,7 @@ func (x *PostgresTableIdentifier) String() string { func (*PostgresTableIdentifier) ProtoMessage() {} func (x *PostgresTableIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[10] + mi := &file_flow_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -866,7 +1008,7 @@ func (x *PostgresTableIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use PostgresTableIdentifier.ProtoReflect.Descriptor instead. func (*PostgresTableIdentifier) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{10} + return file_flow_proto_rawDescGZIP(), []int{12} } func (x *PostgresTableIdentifier) GetRelId() uint32 { @@ -890,7 +1032,7 @@ type TableIdentifier struct { func (x *TableIdentifier) Reset() { *x = TableIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[11] + mi := &file_flow_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -903,7 +1045,7 @@ func (x *TableIdentifier) String() string { func (*TableIdentifier) ProtoMessage() {} func (x *TableIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[11] + mi := &file_flow_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -916,7 +1058,7 @@ func (x *TableIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use TableIdentifier.ProtoReflect.Descriptor instead. func (*TableIdentifier) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{11} + return file_flow_proto_rawDescGZIP(), []int{13} } func (m *TableIdentifier) GetTableIdentifier() isTableIdentifier_TableIdentifier { @@ -954,7 +1096,7 @@ type EnsurePullabilityOutput struct { func (x *EnsurePullabilityOutput) Reset() { *x = EnsurePullabilityOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[12] + mi := &file_flow_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -967,7 +1109,7 @@ func (x *EnsurePullabilityOutput) String() string { func (*EnsurePullabilityOutput) ProtoMessage() {} func (x *EnsurePullabilityOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[12] + mi := &file_flow_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -980,7 +1122,7 @@ func (x *EnsurePullabilityOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityOutput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{12} + return file_flow_proto_rawDescGZIP(), []int{14} } func (x *EnsurePullabilityOutput) GetTableIdentifier() *TableIdentifier { @@ -1001,7 +1143,7 @@ type EnsurePullabilityBatchOutput struct { func (x *EnsurePullabilityBatchOutput) Reset() { *x = EnsurePullabilityBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[13] + mi := &file_flow_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1014,7 +1156,7 @@ func (x *EnsurePullabilityBatchOutput) String() string { func (*EnsurePullabilityBatchOutput) ProtoMessage() {} func (x *EnsurePullabilityBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[13] + mi := &file_flow_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1027,7 +1169,7 @@ func (x *EnsurePullabilityBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use EnsurePullabilityBatchOutput.ProtoReflect.Descriptor instead. func (*EnsurePullabilityBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{13} + return file_flow_proto_rawDescGZIP(), []int{15} } func (x *EnsurePullabilityBatchOutput) GetTableIdentifierMapping() map[string]*TableIdentifier { @@ -1055,7 +1197,7 @@ type SetupReplicationInput struct { func (x *SetupReplicationInput) Reset() { *x = SetupReplicationInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[14] + mi := &file_flow_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1068,7 +1210,7 @@ func (x *SetupReplicationInput) String() string { func (*SetupReplicationInput) ProtoMessage() {} func (x *SetupReplicationInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[14] + mi := &file_flow_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1081,7 +1223,7 @@ func (x *SetupReplicationInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupReplicationInput.ProtoReflect.Descriptor instead. func (*SetupReplicationInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{14} + return file_flow_proto_rawDescGZIP(), []int{16} } func (x *SetupReplicationInput) GetPeerConnectionConfig() *Peer { @@ -1145,7 +1287,7 @@ type SetupReplicationOutput struct { func (x *SetupReplicationOutput) Reset() { *x = SetupReplicationOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[15] + mi := &file_flow_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1158,7 +1300,7 @@ func (x *SetupReplicationOutput) String() string { func (*SetupReplicationOutput) ProtoMessage() {} func (x *SetupReplicationOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[15] + mi := &file_flow_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1171,7 +1313,7 @@ func (x *SetupReplicationOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupReplicationOutput.ProtoReflect.Descriptor instead. func (*SetupReplicationOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{15} + return file_flow_proto_rawDescGZIP(), []int{17} } func (x *SetupReplicationOutput) GetSlotName() string { @@ -1202,7 +1344,7 @@ type CreateRawTableInput struct { func (x *CreateRawTableInput) Reset() { *x = CreateRawTableInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[16] + mi := &file_flow_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1215,7 +1357,7 @@ func (x *CreateRawTableInput) String() string { func (*CreateRawTableInput) ProtoMessage() {} func (x *CreateRawTableInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[16] + mi := &file_flow_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1228,7 +1370,7 @@ func (x *CreateRawTableInput) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateRawTableInput.ProtoReflect.Descriptor instead. func (*CreateRawTableInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{16} + return file_flow_proto_rawDescGZIP(), []int{18} } func (x *CreateRawTableInput) GetPeerConnectionConfig() *Peer { @@ -1270,7 +1412,7 @@ type CreateRawTableOutput struct { func (x *CreateRawTableOutput) Reset() { *x = CreateRawTableOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[17] + mi := &file_flow_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1283,7 +1425,7 @@ func (x *CreateRawTableOutput) String() string { func (*CreateRawTableOutput) ProtoMessage() {} func (x *CreateRawTableOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[17] + mi := &file_flow_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1296,7 +1438,7 @@ func (x *CreateRawTableOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateRawTableOutput.ProtoReflect.Descriptor instead. func (*CreateRawTableOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{17} + return file_flow_proto_rawDescGZIP(), []int{19} } func (x *CreateRawTableOutput) GetTableIdentifier() string { @@ -1321,7 +1463,7 @@ type TableSchema struct { func (x *TableSchema) Reset() { *x = TableSchema{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[18] + mi := &file_flow_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1334,7 +1476,7 @@ func (x *TableSchema) String() string { func (*TableSchema) ProtoMessage() {} func (x *TableSchema) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[18] + mi := &file_flow_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1347,7 +1489,7 @@ func (x *TableSchema) ProtoReflect() protoreflect.Message { // Deprecated: Use TableSchema.ProtoReflect.Descriptor instead. func (*TableSchema) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{18} + return file_flow_proto_rawDescGZIP(), []int{20} } func (x *TableSchema) GetTableIdentifier() string { @@ -1383,7 +1525,7 @@ type GetTableSchemaBatchInput struct { func (x *GetTableSchemaBatchInput) Reset() { *x = GetTableSchemaBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[19] + mi := &file_flow_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1396,7 +1538,7 @@ func (x *GetTableSchemaBatchInput) String() string { func (*GetTableSchemaBatchInput) ProtoMessage() {} func (x *GetTableSchemaBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[19] + mi := &file_flow_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1409,7 +1551,7 @@ func (x *GetTableSchemaBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTableSchemaBatchInput.ProtoReflect.Descriptor instead. func (*GetTableSchemaBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{19} + return file_flow_proto_rawDescGZIP(), []int{21} } func (x *GetTableSchemaBatchInput) GetPeerConnectionConfig() *Peer { @@ -1437,7 +1579,7 @@ type GetTableSchemaBatchOutput struct { func (x *GetTableSchemaBatchOutput) Reset() { *x = GetTableSchemaBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[20] + mi := &file_flow_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1450,7 +1592,7 @@ func (x *GetTableSchemaBatchOutput) String() string { func (*GetTableSchemaBatchOutput) ProtoMessage() {} func (x *GetTableSchemaBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[20] + mi := &file_flow_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1463,7 +1605,7 @@ func (x *GetTableSchemaBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use GetTableSchemaBatchOutput.ProtoReflect.Descriptor instead. func (*GetTableSchemaBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{20} + return file_flow_proto_rawDescGZIP(), []int{22} } func (x *GetTableSchemaBatchOutput) GetTableNameSchemaMapping() map[string]*TableSchema { @@ -1486,7 +1628,7 @@ type SetupNormalizedTableInput struct { func (x *SetupNormalizedTableInput) Reset() { *x = SetupNormalizedTableInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[21] + mi := &file_flow_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1499,7 +1641,7 @@ func (x *SetupNormalizedTableInput) String() string { func (*SetupNormalizedTableInput) ProtoMessage() {} func (x *SetupNormalizedTableInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[21] + mi := &file_flow_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1512,7 +1654,7 @@ func (x *SetupNormalizedTableInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableInput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{21} + return file_flow_proto_rawDescGZIP(), []int{23} } func (x *SetupNormalizedTableInput) GetPeerConnectionConfig() *Peer { @@ -1548,7 +1690,7 @@ type SetupNormalizedTableBatchInput struct { func (x *SetupNormalizedTableBatchInput) Reset() { *x = SetupNormalizedTableBatchInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[22] + mi := &file_flow_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1561,7 +1703,7 @@ func (x *SetupNormalizedTableBatchInput) String() string { func (*SetupNormalizedTableBatchInput) ProtoMessage() {} func (x *SetupNormalizedTableBatchInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[22] + mi := &file_flow_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1574,7 +1716,7 @@ func (x *SetupNormalizedTableBatchInput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableBatchInput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableBatchInput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{22} + return file_flow_proto_rawDescGZIP(), []int{24} } func (x *SetupNormalizedTableBatchInput) GetPeerConnectionConfig() *Peer { @@ -1603,7 +1745,7 @@ type SetupNormalizedTableOutput struct { func (x *SetupNormalizedTableOutput) Reset() { *x = SetupNormalizedTableOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[23] + mi := &file_flow_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1616,7 +1758,7 @@ func (x *SetupNormalizedTableOutput) String() string { func (*SetupNormalizedTableOutput) ProtoMessage() {} func (x *SetupNormalizedTableOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[23] + mi := &file_flow_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1629,7 +1771,7 @@ func (x *SetupNormalizedTableOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableOutput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{23} + return file_flow_proto_rawDescGZIP(), []int{25} } func (x *SetupNormalizedTableOutput) GetTableIdentifier() string { @@ -1657,7 +1799,7 @@ type SetupNormalizedTableBatchOutput struct { func (x *SetupNormalizedTableBatchOutput) Reset() { *x = SetupNormalizedTableBatchOutput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[24] + mi := &file_flow_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1670,7 +1812,7 @@ func (x *SetupNormalizedTableBatchOutput) String() string { func (*SetupNormalizedTableBatchOutput) ProtoMessage() {} func (x *SetupNormalizedTableBatchOutput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[24] + mi := &file_flow_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1683,7 +1825,7 @@ func (x *SetupNormalizedTableBatchOutput) ProtoReflect() protoreflect.Message { // Deprecated: Use SetupNormalizedTableBatchOutput.ProtoReflect.Descriptor instead. func (*SetupNormalizedTableBatchOutput) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{24} + return file_flow_proto_rawDescGZIP(), []int{26} } func (x *SetupNormalizedTableBatchOutput) GetTableExistsMapping() map[string]bool { @@ -1706,7 +1848,7 @@ type IntPartitionRange struct { func (x *IntPartitionRange) Reset() { *x = IntPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[25] + mi := &file_flow_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1719,7 +1861,7 @@ func (x *IntPartitionRange) String() string { func (*IntPartitionRange) ProtoMessage() {} func (x *IntPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[25] + mi := &file_flow_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1732,7 +1874,7 @@ func (x *IntPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use IntPartitionRange.ProtoReflect.Descriptor instead. func (*IntPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{25} + return file_flow_proto_rawDescGZIP(), []int{27} } func (x *IntPartitionRange) GetStart() int64 { @@ -1761,7 +1903,7 @@ type TimestampPartitionRange struct { func (x *TimestampPartitionRange) Reset() { *x = TimestampPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[26] + mi := &file_flow_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1774,7 +1916,7 @@ func (x *TimestampPartitionRange) String() string { func (*TimestampPartitionRange) ProtoMessage() {} func (x *TimestampPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[26] + mi := &file_flow_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1787,7 +1929,7 @@ func (x *TimestampPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use TimestampPartitionRange.ProtoReflect.Descriptor instead. func (*TimestampPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{26} + return file_flow_proto_rawDescGZIP(), []int{28} } func (x *TimestampPartitionRange) GetStart() *timestamppb.Timestamp { @@ -1816,7 +1958,7 @@ type TID struct { func (x *TID) Reset() { *x = TID{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[27] + mi := &file_flow_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1829,7 +1971,7 @@ func (x *TID) String() string { func (*TID) ProtoMessage() {} func (x *TID) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[27] + mi := &file_flow_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1842,7 +1984,7 @@ func (x *TID) ProtoReflect() protoreflect.Message { // Deprecated: Use TID.ProtoReflect.Descriptor instead. func (*TID) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{27} + return file_flow_proto_rawDescGZIP(), []int{29} } func (x *TID) GetBlockNumber() uint32 { @@ -1871,7 +2013,7 @@ type TIDPartitionRange struct { func (x *TIDPartitionRange) Reset() { *x = TIDPartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[28] + mi := &file_flow_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1884,7 +2026,7 @@ func (x *TIDPartitionRange) String() string { func (*TIDPartitionRange) ProtoMessage() {} func (x *TIDPartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[28] + mi := &file_flow_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1897,7 +2039,7 @@ func (x *TIDPartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use TIDPartitionRange.ProtoReflect.Descriptor instead. func (*TIDPartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{28} + return file_flow_proto_rawDescGZIP(), []int{30} } func (x *TIDPartitionRange) GetStart() *TID { @@ -1932,7 +2074,7 @@ type PartitionRange struct { func (x *PartitionRange) Reset() { *x = PartitionRange{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[29] + mi := &file_flow_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1945,7 +2087,7 @@ func (x *PartitionRange) String() string { func (*PartitionRange) ProtoMessage() {} func (x *PartitionRange) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[29] + mi := &file_flow_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1958,7 +2100,7 @@ func (x *PartitionRange) ProtoReflect() protoreflect.Message { // Deprecated: Use PartitionRange.ProtoReflect.Descriptor instead. func (*PartitionRange) Descriptor() ([]byte, []int) { - return file_flow_proto_rawDescGZIP(), []int{29} + return file_flow_proto_rawDescGZIP(), []int{31} } func (m *PartitionRange) GetRange() isPartitionRange_Range { @@ -2023,7 +2165,7 @@ type QRepWriteMode struct { func (x *QRepWriteMode) Reset() { *x = QRepWriteMode{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[30] + mi := &file_flow_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2036,7 +2178,7 @@ func (x *QRepWriteMode) String() string { func (*QRepWriteMode) ProtoMessage() {} func (x *QRepWriteMode) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[30] + mi := &file_flow_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2049,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{30} + return file_flow_proto_rawDescGZIP(), []int{32} } func (x *QRepWriteMode) GetWriteType() QRepWriteType { @@ -2102,7 +2244,7 @@ type QRepConfig struct { func (x *QRepConfig) Reset() { *x = QRepConfig{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[31] + mi := &file_flow_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2115,7 +2257,7 @@ func (x *QRepConfig) String() string { func (*QRepConfig) ProtoMessage() {} func (x *QRepConfig) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[31] + mi := &file_flow_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2128,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{31} + return file_flow_proto_rawDescGZIP(), []int{33} } func (x *QRepConfig) GetFlowJobName() string { @@ -2256,7 +2398,7 @@ type QRepPartition struct { func (x *QRepPartition) Reset() { *x = QRepPartition{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[32] + mi := &file_flow_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2269,7 +2411,7 @@ func (x *QRepPartition) String() string { func (*QRepPartition) ProtoMessage() {} func (x *QRepPartition) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[32] + mi := &file_flow_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2282,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{32} + return file_flow_proto_rawDescGZIP(), []int{34} } func (x *QRepPartition) GetPartitionId() string { @@ -2318,7 +2460,7 @@ type QRepPartitionBatch struct { func (x *QRepPartitionBatch) Reset() { *x = QRepPartitionBatch{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2331,7 +2473,7 @@ func (x *QRepPartitionBatch) String() string { func (*QRepPartitionBatch) ProtoMessage() {} func (x *QRepPartitionBatch) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[33] + mi := &file_flow_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2344,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{33} + return file_flow_proto_rawDescGZIP(), []int{35} } func (x *QRepPartitionBatch) GetBatchId() int32 { @@ -2372,7 +2514,7 @@ type QRepParitionResult struct { func (x *QRepParitionResult) Reset() { *x = QRepParitionResult{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2385,7 +2527,7 @@ func (x *QRepParitionResult) String() string { func (*QRepParitionResult) ProtoMessage() {} func (x *QRepParitionResult) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[34] + mi := &file_flow_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2398,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{34} + return file_flow_proto_rawDescGZIP(), []int{36} } func (x *QRepParitionResult) GetPartitions() []*QRepPartition { @@ -2419,7 +2561,7 @@ type DropFlowInput struct { func (x *DropFlowInput) Reset() { *x = DropFlowInput{} if protoimpl.UnsafeEnabled { - mi := &file_flow_proto_msgTypes[35] + mi := &file_flow_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2432,7 +2574,7 @@ func (x *DropFlowInput) String() string { func (*DropFlowInput) ProtoMessage() {} func (x *DropFlowInput) ProtoReflect() protoreflect.Message { - mi := &file_flow_proto_msgTypes[35] + mi := &file_flow_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2445,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{35} + return file_flow_proto_rawDescGZIP(), []int{37} } func (x *DropFlowInput) GetFlowName() string { @@ -2455,6 +2597,187 @@ func (x *DropFlowInput) GetFlowName() string { return "" } +type DeltaAddedColumn struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ColumnName string `protobuf:"bytes,1,opt,name=column_name,json=columnName,proto3" json:"column_name,omitempty"` + ColumnType string `protobuf:"bytes,2,opt,name=column_type,json=columnType,proto3" json:"column_type,omitempty"` +} + +func (x *DeltaAddedColumn) Reset() { + *x = DeltaAddedColumn{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DeltaAddedColumn) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DeltaAddedColumn) ProtoMessage() {} + +func (x *DeltaAddedColumn) ProtoReflect() protoreflect.Message { + mi := &file_flow_proto_msgTypes[38] + 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 DeltaAddedColumn.ProtoReflect.Descriptor instead. +func (*DeltaAddedColumn) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{38} +} + +func (x *DeltaAddedColumn) GetColumnName() string { + if x != nil { + return x.ColumnName + } + return "" +} + +func (x *DeltaAddedColumn) GetColumnType() string { + if x != nil { + return x.ColumnType + } + return "" +} + +type TableSchemaDelta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + SrcTableName string `protobuf:"bytes,1,opt,name=src_table_name,json=srcTableName,proto3" json:"src_table_name,omitempty"` + DstTableName string `protobuf:"bytes,2,opt,name=dst_table_name,json=dstTableName,proto3" json:"dst_table_name,omitempty"` + AddedColumns []*DeltaAddedColumn `protobuf:"bytes,3,rep,name=added_columns,json=addedColumns,proto3" json:"added_columns,omitempty"` + DroppedColumns []string `protobuf:"bytes,4,rep,name=dropped_columns,json=droppedColumns,proto3" json:"dropped_columns,omitempty"` +} + +func (x *TableSchemaDelta) Reset() { + *x = TableSchemaDelta{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *TableSchemaDelta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TableSchemaDelta) ProtoMessage() {} + +func (x *TableSchemaDelta) 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 TableSchemaDelta.ProtoReflect.Descriptor instead. +func (*TableSchemaDelta) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{39} +} + +func (x *TableSchemaDelta) GetSrcTableName() string { + if x != nil { + return x.SrcTableName + } + return "" +} + +func (x *TableSchemaDelta) GetDstTableName() string { + if x != nil { + return x.DstTableName + } + return "" +} + +func (x *TableSchemaDelta) GetAddedColumns() []*DeltaAddedColumn { + if x != nil { + return x.AddedColumns + } + return nil +} + +func (x *TableSchemaDelta) GetDroppedColumns() []string { + if x != nil { + return x.DroppedColumns + } + return nil +} + +type ReplayTableSchemaDeltaInput 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"` + TableSchemaDelta *TableSchemaDelta `protobuf:"bytes,2,opt,name=table_schema_delta,json=tableSchemaDelta,proto3" json:"table_schema_delta,omitempty"` +} + +func (x *ReplayTableSchemaDeltaInput) Reset() { + *x = ReplayTableSchemaDeltaInput{} + if protoimpl.UnsafeEnabled { + mi := &file_flow_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplayTableSchemaDeltaInput) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplayTableSchemaDeltaInput) ProtoMessage() {} + +func (x *ReplayTableSchemaDeltaInput) 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 ReplayTableSchemaDeltaInput.ProtoReflect.Descriptor instead. +func (*ReplayTableSchemaDeltaInput) Descriptor() ([]byte, []int) { + return file_flow_proto_rawDescGZIP(), []int{40} +} + +func (x *ReplayTableSchemaDeltaInput) GetFlowConnectionConfigs() *FlowConnectionConfigs { + if x != nil { + return x.FlowConnectionConfigs + } + return nil +} + +func (x *ReplayTableSchemaDeltaInput) GetTableSchemaDelta() *TableSchemaDelta { + if x != nil { + return x.TableSchemaDelta + } + return nil +} + var File_flow_proto protoreflect.FileDescriptor var file_flow_proto_rawDesc = []byte{ @@ -2469,511 +2792,585 @@ var file_flow_proto_rawDesc = []byte{ 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, 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, + 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, 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, 0x30, 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, 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, 0xfa, 0x01, 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, + 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, 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, 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, + 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, 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, + 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, 0xe3, 0x01, 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, 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, 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, 0xe3, - 0x01, 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, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, - 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, - 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x48, 0x0a, - 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xd4, 0x02, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x75, - 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, + 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, + 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, + 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, - 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6e, - 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, 0x65, 0x61, - 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0xe0, - 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, - 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, - 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, - 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, - 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, - 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, - 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, - 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, - 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, - 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, - 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, - 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, - 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, - 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, - 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, - 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, - 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, - 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, - 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, - 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, - 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, - 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, - 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, - 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, - 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, - 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, - 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, - 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, - 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, - 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, - 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, - 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, - 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, - 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, - 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, - 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, - 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, - 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, - 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, - 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, - 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, - 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, - 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, + 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xd4, 0x02, 0x0a, 0x1e, 0x53, 0x65, + 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, + 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, + 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, + 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, + 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, + 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, + 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, + 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, + 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, + 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, + 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, + 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, 0x0e, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, + 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, 0x0a, + 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x0e, + 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x3d, + 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, + 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, + 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x22, 0x96, 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, + 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, + 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, + 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, + 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, + 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, + 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, + 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, + 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, + 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, + 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, + 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, + 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, + 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, + 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, + 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, + 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, + 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, + 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0x50, - 0x0a, 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, + 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, + 0x22, 0x54, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, + 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0xcb, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x24, 0x0a, 0x0e, 0x73, + 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x73, 0x74, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x61, 0x64, 0x64, 0x65, 0x64, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, 0x65, 0x6c, + 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x0c, 0x61, + 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x64, + 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xc6, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x79, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, + 0x12, 0x4b, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 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, 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, 0x47, 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, 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, + 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, + 0x47, 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, 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 ( @@ -2989,118 +3386,133 @@ func file_flow_proto_rawDescGZIP() []byte { } var file_flow_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 46) +var file_flow_proto_msgTypes = make([]protoimpl.MessageInfo, 53) var file_flow_proto_goTypes = []interface{}{ (QRepSyncMode)(0), // 0: peerdb_flow.QRepSyncMode (QRepWriteType)(0), // 1: peerdb_flow.QRepWriteType (*TableNameMapping)(nil), // 2: peerdb_flow.TableNameMapping - (*FlowConnectionConfigs)(nil), // 3: peerdb_flow.FlowConnectionConfigs - (*SyncFlowOptions)(nil), // 4: peerdb_flow.SyncFlowOptions - (*NormalizeFlowOptions)(nil), // 5: peerdb_flow.NormalizeFlowOptions - (*LastSyncState)(nil), // 6: peerdb_flow.LastSyncState - (*StartFlowInput)(nil), // 7: peerdb_flow.StartFlowInput - (*StartNormalizeInput)(nil), // 8: peerdb_flow.StartNormalizeInput - (*GetLastSyncedIDInput)(nil), // 9: peerdb_flow.GetLastSyncedIDInput - (*EnsurePullabilityInput)(nil), // 10: peerdb_flow.EnsurePullabilityInput - (*EnsurePullabilityBatchInput)(nil), // 11: peerdb_flow.EnsurePullabilityBatchInput - (*PostgresTableIdentifier)(nil), // 12: peerdb_flow.PostgresTableIdentifier - (*TableIdentifier)(nil), // 13: peerdb_flow.TableIdentifier - (*EnsurePullabilityOutput)(nil), // 14: peerdb_flow.EnsurePullabilityOutput - (*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 - (*PartitionRange)(nil), // 31: peerdb_flow.PartitionRange - (*QRepWriteMode)(nil), // 32: peerdb_flow.QRepWriteMode - (*QRepConfig)(nil), // 33: peerdb_flow.QRepConfig - (*QRepPartition)(nil), // 34: peerdb_flow.QRepPartition - (*QRepPartitionBatch)(nil), // 35: peerdb_flow.QRepPartitionBatch - (*QRepParitionResult)(nil), // 36: peerdb_flow.QRepParitionResult - (*DropFlowInput)(nil), // 37: peerdb_flow.DropFlowInput - nil, // 38: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - nil, // 39: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - nil, // 40: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - nil, // 41: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - nil, // 42: peerdb_flow.SetupReplicationInput.TableNameMappingEntry - nil, // 43: peerdb_flow.CreateRawTableInput.TableNameMappingEntry - nil, // 44: peerdb_flow.TableSchema.ColumnsEntry - nil, // 45: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - nil, // 46: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - nil, // 47: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - (*Peer)(nil), // 48: peerdb_peers.Peer - (*timestamppb.Timestamp)(nil), // 49: google.protobuf.Timestamp + (*RelationMessageColumn)(nil), // 3: peerdb_flow.RelationMessageColumn + (*RelationMessage)(nil), // 4: peerdb_flow.RelationMessage + (*FlowConnectionConfigs)(nil), // 5: peerdb_flow.FlowConnectionConfigs + (*SyncFlowOptions)(nil), // 6: peerdb_flow.SyncFlowOptions + (*NormalizeFlowOptions)(nil), // 7: peerdb_flow.NormalizeFlowOptions + (*LastSyncState)(nil), // 8: peerdb_flow.LastSyncState + (*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 + (*PartitionRange)(nil), // 33: peerdb_flow.PartitionRange + (*QRepWriteMode)(nil), // 34: peerdb_flow.QRepWriteMode + (*QRepConfig)(nil), // 35: peerdb_flow.QRepConfig + (*QRepPartition)(nil), // 36: peerdb_flow.QRepPartition + (*QRepPartitionBatch)(nil), // 37: peerdb_flow.QRepPartitionBatch + (*QRepParitionResult)(nil), // 38: peerdb_flow.QRepParitionResult + (*DropFlowInput)(nil), // 39: peerdb_flow.DropFlowInput + (*DeltaAddedColumn)(nil), // 40: peerdb_flow.DeltaAddedColumn + (*TableSchemaDelta)(nil), // 41: peerdb_flow.TableSchemaDelta + (*ReplayTableSchemaDeltaInput)(nil), // 42: peerdb_flow.ReplayTableSchemaDeltaInput + nil, // 43: peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + nil, // 44: peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + nil, // 45: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + nil, // 46: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + nil, // 47: peerdb_flow.StartFlowInput.RelationMessageMappingEntry + nil, // 48: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + nil, // 49: peerdb_flow.SetupReplicationInput.TableNameMappingEntry + nil, // 50: peerdb_flow.CreateRawTableInput.TableNameMappingEntry + nil, // 51: peerdb_flow.TableSchema.ColumnsEntry + nil, // 52: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + nil, // 53: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + nil, // 54: peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + (*Peer)(nil), // 55: peerdb_peers.Peer + (*timestamppb.Timestamp)(nil), // 56: google.protobuf.Timestamp } var file_flow_proto_depIdxs = []int32{ - 48, // 0: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer - 48, // 1: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer - 20, // 2: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema - 38, // 3: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry - 39, // 4: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry - 40, // 5: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry - 48, // 6: peerdb_flow.FlowConnectionConfigs.metadata_peer:type_name -> peerdb_peers.Peer - 0, // 7: peerdb_flow.FlowConnectionConfigs.snapshot_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 0, // 8: peerdb_flow.FlowConnectionConfigs.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 49, // 9: peerdb_flow.LastSyncState.last_synced_at:type_name -> google.protobuf.Timestamp - 6, // 10: peerdb_flow.StartFlowInput.last_sync_state:type_name -> peerdb_flow.LastSyncState - 3, // 11: peerdb_flow.StartFlowInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 4, // 12: peerdb_flow.StartFlowInput.sync_flow_options:type_name -> peerdb_flow.SyncFlowOptions - 3, // 13: peerdb_flow.StartNormalizeInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs - 48, // 14: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer - 48, // 15: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer - 48, // 16: peerdb_flow.EnsurePullabilityBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 12, // 17: peerdb_flow.TableIdentifier.postgres_table_identifier:type_name -> peerdb_flow.PostgresTableIdentifier - 13, // 18: peerdb_flow.EnsurePullabilityOutput.table_identifier:type_name -> peerdb_flow.TableIdentifier - 41, // 19: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry - 48, // 20: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer - 42, // 21: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry - 48, // 22: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer - 48, // 23: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 43, // 24: peerdb_flow.CreateRawTableInput.table_name_mapping:type_name -> peerdb_flow.CreateRawTableInput.TableNameMappingEntry - 0, // 25: peerdb_flow.CreateRawTableInput.cdc_sync_mode:type_name -> peerdb_flow.QRepSyncMode - 44, // 26: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry - 48, // 27: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 45, // 28: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry - 48, // 29: peerdb_flow.SetupNormalizedTableInput.peer_connection_config:type_name -> peerdb_peers.Peer - 20, // 30: peerdb_flow.SetupNormalizedTableInput.source_table_schema:type_name -> peerdb_flow.TableSchema - 48, // 31: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer - 46, // 32: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry - 47, // 33: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry - 49, // 34: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp - 49, // 35: peerdb_flow.TimestampPartitionRange.end:type_name -> google.protobuf.Timestamp - 29, // 36: peerdb_flow.TIDPartitionRange.start:type_name -> peerdb_flow.TID - 29, // 37: peerdb_flow.TIDPartitionRange.end:type_name -> peerdb_flow.TID - 27, // 38: peerdb_flow.PartitionRange.int_range:type_name -> peerdb_flow.IntPartitionRange - 28, // 39: peerdb_flow.PartitionRange.timestamp_range:type_name -> peerdb_flow.TimestampPartitionRange - 30, // 40: peerdb_flow.PartitionRange.tid_range:type_name -> peerdb_flow.TIDPartitionRange - 1, // 41: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType - 48, // 42: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer - 48, // 43: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer - 0, // 44: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode - 32, // 45: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode - 31, // 46: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange - 34, // 47: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition - 34, // 48: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition - 20, // 49: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 13, // 50: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier - 20, // 51: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 20, // 52: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema - 53, // [53:53] is the sub-list for method output_type - 53, // [53:53] is the sub-list for method input_type - 53, // [53:53] is the sub-list for extension type_name - 53, // [53:53] is the sub-list for extension extendee - 0, // [0:53] is the sub-list for field type_name + 3, // 0: peerdb_flow.RelationMessage.columns:type_name -> peerdb_flow.RelationMessageColumn + 55, // 1: peerdb_flow.FlowConnectionConfigs.source:type_name -> peerdb_peers.Peer + 55, // 2: peerdb_flow.FlowConnectionConfigs.destination:type_name -> peerdb_peers.Peer + 22, // 3: peerdb_flow.FlowConnectionConfigs.table_schema:type_name -> peerdb_flow.TableSchema + 43, // 4: peerdb_flow.FlowConnectionConfigs.table_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameMappingEntry + 44, // 5: peerdb_flow.FlowConnectionConfigs.src_table_id_name_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.SrcTableIdNameMappingEntry + 45, // 6: peerdb_flow.FlowConnectionConfigs.table_name_schema_mapping:type_name -> peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry + 55, // 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 + 46, // 10: peerdb_flow.SyncFlowOptions.relation_message_mapping:type_name -> peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry + 56, // 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 + 47, // 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 + 55, // 17: peerdb_flow.GetLastSyncedIDInput.peer_connection_config:type_name -> peerdb_peers.Peer + 55, // 18: peerdb_flow.EnsurePullabilityInput.peer_connection_config:type_name -> peerdb_peers.Peer + 55, // 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 + 48, // 22: peerdb_flow.EnsurePullabilityBatchOutput.table_identifier_mapping:type_name -> peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry + 55, // 23: peerdb_flow.SetupReplicationInput.peer_connection_config:type_name -> peerdb_peers.Peer + 49, // 24: peerdb_flow.SetupReplicationInput.table_name_mapping:type_name -> peerdb_flow.SetupReplicationInput.TableNameMappingEntry + 55, // 25: peerdb_flow.SetupReplicationInput.destination_peer:type_name -> peerdb_peers.Peer + 55, // 26: peerdb_flow.CreateRawTableInput.peer_connection_config:type_name -> peerdb_peers.Peer + 50, // 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 + 51, // 29: peerdb_flow.TableSchema.columns:type_name -> peerdb_flow.TableSchema.ColumnsEntry + 55, // 30: peerdb_flow.GetTableSchemaBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 52, // 31: peerdb_flow.GetTableSchemaBatchOutput.table_name_schema_mapping:type_name -> peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry + 55, // 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 + 55, // 34: peerdb_flow.SetupNormalizedTableBatchInput.peer_connection_config:type_name -> peerdb_peers.Peer + 53, // 35: peerdb_flow.SetupNormalizedTableBatchInput.table_name_schema_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry + 54, // 36: peerdb_flow.SetupNormalizedTableBatchOutput.table_exists_mapping:type_name -> peerdb_flow.SetupNormalizedTableBatchOutput.TableExistsMappingEntry + 56, // 37: peerdb_flow.TimestampPartitionRange.start:type_name -> google.protobuf.Timestamp + 56, // 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 + 1, // 44: peerdb_flow.QRepWriteMode.write_type:type_name -> peerdb_flow.QRepWriteType + 55, // 45: peerdb_flow.QRepConfig.source_peer:type_name -> peerdb_peers.Peer + 55, // 46: peerdb_flow.QRepConfig.destination_peer:type_name -> peerdb_peers.Peer + 0, // 47: peerdb_flow.QRepConfig.sync_mode:type_name -> peerdb_flow.QRepSyncMode + 34, // 48: peerdb_flow.QRepConfig.write_mode:type_name -> peerdb_flow.QRepWriteMode + 33, // 49: peerdb_flow.QRepPartition.range:type_name -> peerdb_flow.PartitionRange + 36, // 50: peerdb_flow.QRepPartitionBatch.partitions:type_name -> peerdb_flow.QRepPartition + 36, // 51: peerdb_flow.QRepParitionResult.partitions:type_name -> peerdb_flow.QRepPartition + 40, // 52: peerdb_flow.TableSchemaDelta.added_columns:type_name -> peerdb_flow.DeltaAddedColumn + 5, // 53: peerdb_flow.ReplayTableSchemaDeltaInput.flow_connection_configs:type_name -> peerdb_flow.FlowConnectionConfigs + 41, // 54: peerdb_flow.ReplayTableSchemaDeltaInput.table_schema_delta:type_name -> peerdb_flow.TableSchemaDelta + 22, // 55: peerdb_flow.FlowConnectionConfigs.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 4, // 56: peerdb_flow.SyncFlowOptions.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 4, // 57: peerdb_flow.StartFlowInput.RelationMessageMappingEntry.value:type_name -> peerdb_flow.RelationMessage + 15, // 58: peerdb_flow.EnsurePullabilityBatchOutput.TableIdentifierMappingEntry.value:type_name -> peerdb_flow.TableIdentifier + 22, // 59: peerdb_flow.GetTableSchemaBatchOutput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 22, // 60: peerdb_flow.SetupNormalizedTableBatchInput.TableNameSchemaMappingEntry.value:type_name -> peerdb_flow.TableSchema + 61, // [61:61] is the sub-list for method output_type + 61, // [61:61] is the sub-list for method input_type + 61, // [61:61] is the sub-list for extension type_name + 61, // [61:61] is the sub-list for extension extendee + 0, // [0:61] is the sub-list for field type_name } func init() { file_flow_proto_init() } @@ -3123,7 +3535,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FlowConnectionConfigs); i { + switch v := v.(*RelationMessageColumn); i { case 0: return &v.state case 1: @@ -3135,7 +3547,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SyncFlowOptions); i { + switch v := v.(*RelationMessage); i { case 0: return &v.state case 1: @@ -3147,7 +3559,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NormalizeFlowOptions); i { + switch v := v.(*FlowConnectionConfigs); i { case 0: return &v.state case 1: @@ -3159,7 +3571,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LastSyncState); i { + switch v := v.(*SyncFlowOptions); i { case 0: return &v.state case 1: @@ -3171,7 +3583,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartFlowInput); i { + switch v := v.(*NormalizeFlowOptions); i { case 0: return &v.state case 1: @@ -3183,7 +3595,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*StartNormalizeInput); i { + switch v := v.(*LastSyncState); i { case 0: return &v.state case 1: @@ -3195,7 +3607,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetLastSyncedIDInput); i { + switch v := v.(*StartFlowInput); i { case 0: return &v.state case 1: @@ -3207,7 +3619,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityInput); i { + switch v := v.(*StartNormalizeInput); i { case 0: return &v.state case 1: @@ -3219,7 +3631,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityBatchInput); i { + switch v := v.(*GetLastSyncedIDInput); i { case 0: return &v.state case 1: @@ -3231,7 +3643,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PostgresTableIdentifier); i { + switch v := v.(*EnsurePullabilityInput); i { case 0: return &v.state case 1: @@ -3243,7 +3655,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableIdentifier); i { + switch v := v.(*EnsurePullabilityBatchInput); i { case 0: return &v.state case 1: @@ -3255,7 +3667,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityOutput); i { + switch v := v.(*PostgresTableIdentifier); i { case 0: return &v.state case 1: @@ -3267,7 +3679,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EnsurePullabilityBatchOutput); i { + switch v := v.(*TableIdentifier); i { case 0: return &v.state case 1: @@ -3279,7 +3691,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupReplicationInput); i { + switch v := v.(*EnsurePullabilityOutput); i { case 0: return &v.state case 1: @@ -3291,7 +3703,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupReplicationOutput); i { + switch v := v.(*EnsurePullabilityBatchOutput); i { case 0: return &v.state case 1: @@ -3303,7 +3715,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateRawTableInput); i { + switch v := v.(*SetupReplicationInput); i { case 0: return &v.state case 1: @@ -3315,7 +3727,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateRawTableOutput); i { + switch v := v.(*SetupReplicationOutput); i { case 0: return &v.state case 1: @@ -3327,7 +3739,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TableSchema); i { + switch v := v.(*CreateRawTableInput); i { case 0: return &v.state case 1: @@ -3339,7 +3751,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetTableSchemaBatchInput); i { + switch v := v.(*CreateRawTableOutput); i { case 0: return &v.state case 1: @@ -3351,7 +3763,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetTableSchemaBatchOutput); i { + switch v := v.(*TableSchema); i { case 0: return &v.state case 1: @@ -3363,7 +3775,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableInput); i { + switch v := v.(*GetTableSchemaBatchInput); i { case 0: return &v.state case 1: @@ -3375,7 +3787,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableBatchInput); i { + switch v := v.(*GetTableSchemaBatchOutput); i { case 0: return &v.state case 1: @@ -3387,7 +3799,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableOutput); i { + switch v := v.(*SetupNormalizedTableInput); i { case 0: return &v.state case 1: @@ -3399,7 +3811,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SetupNormalizedTableBatchOutput); i { + switch v := v.(*SetupNormalizedTableBatchInput); i { case 0: return &v.state case 1: @@ -3411,7 +3823,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IntPartitionRange); i { + switch v := v.(*SetupNormalizedTableOutput); i { case 0: return &v.state case 1: @@ -3423,7 +3835,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TimestampPartitionRange); i { + switch v := v.(*SetupNormalizedTableBatchOutput); i { case 0: return &v.state case 1: @@ -3435,7 +3847,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TID); i { + switch v := v.(*IntPartitionRange); i { case 0: return &v.state case 1: @@ -3447,7 +3859,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TIDPartitionRange); i { + switch v := v.(*TimestampPartitionRange); i { case 0: return &v.state case 1: @@ -3459,7 +3871,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartitionRange); i { + switch v := v.(*TID); i { case 0: return &v.state case 1: @@ -3471,7 +3883,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepWriteMode); i { + switch v := v.(*TIDPartitionRange); i { case 0: return &v.state case 1: @@ -3483,7 +3895,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepConfig); i { + switch v := v.(*PartitionRange); i { case 0: return &v.state case 1: @@ -3495,7 +3907,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepPartition); i { + switch v := v.(*QRepWriteMode); i { case 0: return &v.state case 1: @@ -3507,7 +3919,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepPartitionBatch); i { + switch v := v.(*QRepConfig); i { case 0: return &v.state case 1: @@ -3519,7 +3931,7 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*QRepParitionResult); i { + switch v := v.(*QRepPartition); i { case 0: return &v.state case 1: @@ -3531,6 +3943,30 @@ func file_flow_proto_init() { } } file_flow_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepPartitionBatch); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QRepParitionResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DropFlowInput); i { case 0: return &v.state @@ -3542,11 +3978,47 @@ func file_flow_proto_init() { return nil } } + file_flow_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeltaAddedColumn); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*TableSchemaDelta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_flow_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplayTableSchemaDeltaInput); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } - file_flow_proto_msgTypes[11].OneofWrappers = []interface{}{ + file_flow_proto_msgTypes[13].OneofWrappers = []interface{}{ (*TableIdentifier_PostgresTableIdentifier)(nil), } - file_flow_proto_msgTypes[29].OneofWrappers = []interface{}{ + file_flow_proto_msgTypes[31].OneofWrappers = []interface{}{ (*PartitionRange_IntRange)(nil), (*PartitionRange_TimestampRange)(nil), (*PartitionRange_TidRange)(nil), @@ -3557,7 +4029,7 @@ func file_flow_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_flow_proto_rawDesc, NumEnums: 2, - NumMessages: 46, + NumMessages: 53, NumExtensions: 0, NumServices: 0, }, diff --git a/flow/model/model.go b/flow/model/model.go index ad2f22ef7b..9bc5877cba 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -28,6 +28,8 @@ type PullRecordsRequest struct { OverridePublicationName string // override replication slot name OverrideReplicationSlotName string + // for supporting schema changes + RelationMessageMapping RelationMessageMapping } type Record interface { @@ -201,6 +203,10 @@ type SyncResponse struct { 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 { @@ -209,3 +215,31 @@ type NormalizeResponse struct { StartBatchID int64 EndBatchID int64 } + +// sync all the records normally, then apply the schema delta after NormalizeFlow. +type RecordsWithTableSchemaDelta struct { + RecordBatch *RecordBatch + TableSchemaDelta *protos.TableSchemaDelta + RelationMessageMapping RelationMessageMapping +} + +// being clever and passing the delta back as a regular record instead of heavy CDC refactoring. +type RelationRecord struct { + CheckPointID int64 + TableSchemaDelta *protos.TableSchemaDelta +} + +// Implement Record interface for RelationRecord. +func (r *RelationRecord) GetCheckPointID() int64 { + return r.CheckPointID +} + +func (r *RelationRecord) GetTableName() string { + return r.TableSchemaDelta.SrcTableName +} + +func (r *RelationRecord) GetItems() RecordItems { + return nil +} + +type RelationMessageMapping map[uint32]*protos.RelationMessage diff --git a/flow/workflows/peer_flow.go b/flow/workflows/peer_flow.go index e6cde4378c..e509e59e52 100644 --- a/flow/workflows/peer_flow.go +++ b/flow/workflows/peer_flow.go @@ -67,6 +67,9 @@ type PeerFlowState struct { SyncFlowErrors error // Errors encountered during child sync flow executions. NormalizeFlowErrors error + // Global mapping of relation IDs to RelationMessages sent as a part of logical replication. + // Needed to support schema changes. + RelationMessageMapping model.RelationMessageMapping } // returns a new empty PeerFlowState @@ -79,6 +82,13 @@ func NewStartedPeerFlowState() *PeerFlowState { SetupComplete: false, SyncFlowErrors: nil, NormalizeFlowErrors: nil, + // WORKAROUND: empty maps are protobufed into nil maps for reasons beyond me + RelationMessageMapping: model.RelationMessageMapping{ + 0: &protos.RelationMessage{ + RelationId: 0, + RelationName: "protobuf_workaround", + }, + }, } } @@ -325,6 +335,7 @@ func PeerFlowWorkflowWithConfig( }, } ctx = workflow.WithChildOptions(ctx, childSyncFlowOpts) + syncFlowOptions.RelationMessageMapping = state.RelationMessageMapping childSyncFlowFuture := workflow.ExecuteChildWorkflow( ctx, SyncFlowWorkflow, @@ -332,16 +343,16 @@ func PeerFlowWorkflowWithConfig( syncFlowOptions, ) - selector.AddFuture(childSyncFlowFuture, func(f workflow.Future) { - var childSyncFlowRes *model.SyncResponse - if err := f.Get(ctx, &childSyncFlowRes); err != nil { - w.logger.Error("failed to execute sync flow: ", err) - state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) - } else { - state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) + var childSyncFlowRes *model.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) + } else { + state.SyncFlowStatuses = append(state.SyncFlowStatuses, childSyncFlowRes) + if childSyncFlowRes != nil { + state.RelationMessageMapping = childSyncFlowRes.RelationMessageMapping } - }) - selector.Select(ctx) + } normalizeFlowID, err := GetChildWorkflowID(ctx, "normalize-flow", cfg.FlowJobName) if err != nil { @@ -356,10 +367,17 @@ func PeerFlowWorkflowWithConfig( }, } ctx = workflow.WithChildOptions(ctx, childNormalizeFlowOpts) + + var tableSchemaDelta *protos.TableSchemaDelta = nil + if childSyncFlowRes != nil { + tableSchemaDelta = childSyncFlowRes.TableSchemaDelta + } + childNormalizeFlowFuture := workflow.ExecuteChildWorkflow( ctx, NormalizeFlowWorkflow, cfg, + tableSchemaDelta, ) selector.AddFuture(childNormalizeFlowFuture, func(f workflow.Future) { @@ -372,6 +390,27 @@ func PeerFlowWorkflowWithConfig( } }) selector.Select(ctx) + + // slightly hacky: table schema mapping is cached, so we need to manually update it if schema changes. + if tableSchemaDelta != nil { + getModifiedSchemaCtx := workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 5 * time.Minute, + }) + getModifiedSchemaFuture := workflow.ExecuteActivity(getModifiedSchemaCtx, flowable.GetTableSchema, + &protos.GetTableSchemaBatchInput{ + PeerConnectionConfig: cfg.Source, + TableIdentifiers: []string{tableSchemaDelta.SrcTableName}, + }) + + var getModifiedSchemaRes *protos.GetTableSchemaBatchOutput + if err := getModifiedSchemaFuture.Get(ctx, &getModifiedSchemaRes); err != nil { + w.logger.Error("failed to execute schema update at source: ", err) + state.SyncFlowErrors = multierror.Append(state.SyncFlowErrors, err) + } else { + cfg.TableNameSchemaMapping[tableSchemaDelta.DstTableName] = + getModifiedSchemaRes.TableNameSchemaMapping[tableSchemaDelta.SrcTableName] + } + } } state.TruncateProgress() diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index 284bbf4f36..a2be62af9e 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -54,6 +54,7 @@ func (s *SyncFlowExecution) executeSyncFlow( ctx workflow.Context, config *protos.FlowConnectionConfigs, opts *protos.SyncFlowOptions, + relationMessageMapping model.RelationMessageMapping, ) (*model.SyncResponse, error) { s.logger.Info("executing sync flow - ", s.PeerFlowName) @@ -89,9 +90,10 @@ func (s *SyncFlowExecution) executeSyncFlow( // execute StartFlow on the peers to start the flow startFlowInput := &protos.StartFlowInput{ - FlowConnectionConfigs: config, - LastSyncState: dstSyncState, - SyncFlowOptions: opts, + FlowConnectionConfigs: config, + LastSyncState: dstSyncState, + SyncFlowOptions: opts, + RelationMessageMapping: relationMessageMapping, } fStartFlow := workflow.ExecuteActivity(startFlowCtx, flowable.StartFlow, startFlowInput) @@ -115,23 +117,25 @@ func SyncFlowWorkflow(ctx workflow.Context, Progress: []string{}, }) - return s.executeSyncFlow(ctx, config, options) + return s.executeSyncFlow(ctx, config, options, options.RelationMessageMapping) } func NormalizeFlowWorkflow(ctx workflow.Context, config *protos.FlowConnectionConfigs, + tableSchemaDelta *protos.TableSchemaDelta, ) (*model.NormalizeResponse, error) { s := NewNormalizeFlowExecution(ctx, &NormalizeFlowState{ PeerFlowName: config.FlowJobName, Progress: []string{}, }) - return s.executeNormalizeFlow(ctx, config) + return s.executeNormalizeFlow(ctx, config, tableSchemaDelta) } func (s *NormalizeFlowExecution) executeNormalizeFlow( ctx workflow.Context, config *protos.FlowConnectionConfigs, + tableSchemaDelta *protos.TableSchemaDelta, ) (*model.NormalizeResponse, error) { s.logger.Info("executing normalize flow - ", s.PeerFlowName) @@ -151,5 +155,16 @@ func (s *NormalizeFlowExecution) executeNormalizeFlow( return nil, fmt.Errorf("failed to flow: %w", err) } + replayTableSchemaInput := &protos.ReplayTableSchemaDeltaInput{ + FlowConnectionConfigs: config, + TableSchemaDelta: tableSchemaDelta, + } + + fReplayTableSchemaDelta := workflow.ExecuteActivity(normalizeFlowCtx, flowable.ReplayTableSchemaDelta, + replayTableSchemaInput) + if err := fReplayTableSchemaDelta.Get(normalizeFlowCtx, nil); err != nil { + return nil, fmt.Errorf("failed to replay schema delta: %w", err) + } + return normalizeResponse, nil } diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 8612d7632c..5a55396853 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -9,6 +9,26 @@ pub struct TableNameMapping { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct RelationMessageColumn { + #[prost(uint32, tag="1")] + pub flags: u32, + #[prost(string, tag="2")] + pub name: ::prost::alloc::string::String, + #[prost(uint32, tag="3")] + pub data_type: u32, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RelationMessage { + #[prost(uint32, tag="1")] + pub relation_id: u32, + #[prost(string, tag="2")] + pub relation_name: ::prost::alloc::string::String, + #[prost(message, repeated, tag="3")] + pub columns: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct FlowConnectionConfigs { #[prost(message, optional, tag="1")] pub source: ::core::option::Option, @@ -65,6 +85,8 @@ pub struct FlowConnectionConfigs { pub struct SyncFlowOptions { #[prost(int32, tag="1")] pub batch_size: i32, + #[prost(map="uint32, message", tag="2")] + pub relation_message_mapping: ::std::collections::HashMap, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -89,6 +111,8 @@ pub struct StartFlowInput { pub flow_connection_configs: ::core::option::Option, #[prost(message, optional, tag="3")] pub sync_flow_options: ::core::option::Option, + #[prost(map="uint32, message", tag="4")] + pub relation_message_mapping: ::std::collections::HashMap, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -398,6 +422,34 @@ pub struct DropFlowInput { #[prost(string, tag="1")] pub flow_name: ::prost::alloc::string::String, } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DeltaAddedColumn { + #[prost(string, tag="1")] + pub column_name: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub column_type: ::prost::alloc::string::String, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TableSchemaDelta { + #[prost(string, tag="1")] + pub src_table_name: ::prost::alloc::string::String, + #[prost(string, tag="2")] + pub dst_table_name: ::prost::alloc::string::String, + #[prost(message, repeated, tag="3")] + pub added_columns: ::prost::alloc::vec::Vec, + #[prost(string, repeated, tag="4")] + pub dropped_columns: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReplayTableSchemaDeltaInput { + #[prost(message, optional, tag="1")] + pub flow_connection_configs: ::core::option::Option, + #[prost(message, optional, tag="2")] + pub table_schema_delta: ::core::option::Option, +} /// 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 3470c31dd9..8d6aabe5b0 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -249,6 +249,120 @@ impl<'de> serde::Deserialize<'de> for CreateRawTableOutput { deserializer.deserialize_struct("peerdb_flow.CreateRawTableOutput", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for DeltaAddedColumn { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.column_name.is_empty() { + len += 1; + } + if !self.column_type.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.DeltaAddedColumn", len)?; + if !self.column_name.is_empty() { + struct_ser.serialize_field("columnName", &self.column_name)?; + } + if !self.column_type.is_empty() { + struct_ser.serialize_field("columnType", &self.column_type)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for DeltaAddedColumn { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "column_name", + "columnName", + "column_type", + "columnType", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + ColumnName, + ColumnType, + __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 { + "columnName" | "column_name" => Ok(GeneratedField::ColumnName), + "columnType" | "column_type" => Ok(GeneratedField::ColumnType), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = DeltaAddedColumn; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.DeltaAddedColumn") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut column_name__ = None; + let mut column_type__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::ColumnName => { + if column_name__.is_some() { + return Err(serde::de::Error::duplicate_field("columnName")); + } + column_name__ = Some(map.next_value()?); + } + GeneratedField::ColumnType => { + if column_type__.is_some() { + return Err(serde::de::Error::duplicate_field("columnType")); + } + column_type__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(DeltaAddedColumn { + column_name: column_name__.unwrap_or_default(), + column_type: column_type__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.DeltaAddedColumn", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for DropFlowInput { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -3160,6 +3274,387 @@ impl<'de> serde::Deserialize<'de> for QRepWriteType { deserializer.deserialize_any(GeneratedVisitor) } } +impl serde::Serialize for RelationMessage { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.relation_id != 0 { + len += 1; + } + if !self.relation_name.is_empty() { + len += 1; + } + if !self.columns.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.RelationMessage", len)?; + if self.relation_id != 0 { + struct_ser.serialize_field("relationId", &self.relation_id)?; + } + if !self.relation_name.is_empty() { + struct_ser.serialize_field("relationName", &self.relation_name)?; + } + if !self.columns.is_empty() { + struct_ser.serialize_field("columns", &self.columns)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for RelationMessage { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "relation_id", + "relationId", + "relation_name", + "relationName", + "columns", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + RelationId, + RelationName, + Columns, + __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 { + "relationId" | "relation_id" => Ok(GeneratedField::RelationId), + "relationName" | "relation_name" => Ok(GeneratedField::RelationName), + "columns" => Ok(GeneratedField::Columns), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = RelationMessage; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.RelationMessage") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut relation_id__ = None; + let mut relation_name__ = None; + let mut columns__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::RelationId => { + if relation_id__.is_some() { + return Err(serde::de::Error::duplicate_field("relationId")); + } + relation_id__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::RelationName => { + if relation_name__.is_some() { + return Err(serde::de::Error::duplicate_field("relationName")); + } + relation_name__ = Some(map.next_value()?); + } + GeneratedField::Columns => { + if columns__.is_some() { + return Err(serde::de::Error::duplicate_field("columns")); + } + columns__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(RelationMessage { + relation_id: relation_id__.unwrap_or_default(), + relation_name: relation_name__.unwrap_or_default(), + columns: columns__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.RelationMessage", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for RelationMessageColumn { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.flags != 0 { + len += 1; + } + if !self.name.is_empty() { + len += 1; + } + if self.data_type != 0 { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.RelationMessageColumn", len)?; + if self.flags != 0 { + struct_ser.serialize_field("flags", &self.flags)?; + } + if !self.name.is_empty() { + struct_ser.serialize_field("name", &self.name)?; + } + if self.data_type != 0 { + struct_ser.serialize_field("dataType", &self.data_type)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for RelationMessageColumn { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "flags", + "name", + "data_type", + "dataType", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Flags, + Name, + DataType, + __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 { + "flags" => Ok(GeneratedField::Flags), + "name" => Ok(GeneratedField::Name), + "dataType" | "data_type" => Ok(GeneratedField::DataType), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = RelationMessageColumn; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.RelationMessageColumn") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flags__ = None; + let mut name__ = None; + let mut data_type__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Flags => { + if flags__.is_some() { + return Err(serde::de::Error::duplicate_field("flags")); + } + flags__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::Name => { + if name__.is_some() { + return Err(serde::de::Error::duplicate_field("name")); + } + name__ = Some(map.next_value()?); + } + GeneratedField::DataType => { + if data_type__.is_some() { + return Err(serde::de::Error::duplicate_field("dataType")); + } + data_type__ = + Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + ; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(RelationMessageColumn { + flags: flags__.unwrap_or_default(), + name: name__.unwrap_or_default(), + data_type: data_type__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.RelationMessageColumn", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for ReplayTableSchemaDeltaInput { + #[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.table_schema_delta.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.ReplayTableSchemaDeltaInput", len)?; + if let Some(v) = self.flow_connection_configs.as_ref() { + struct_ser.serialize_field("flowConnectionConfigs", v)?; + } + if let Some(v) = self.table_schema_delta.as_ref() { + struct_ser.serialize_field("tableSchemaDelta", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ReplayTableSchemaDeltaInput { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "flow_connection_configs", + "flowConnectionConfigs", + "table_schema_delta", + "tableSchemaDelta", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + FlowConnectionConfigs, + TableSchemaDelta, + __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), + "tableSchemaDelta" | "table_schema_delta" => Ok(GeneratedField::TableSchemaDelta), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ReplayTableSchemaDeltaInput; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.ReplayTableSchemaDeltaInput") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut flow_connection_configs__ = None; + let mut table_schema_delta__ = 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::TableSchemaDelta => { + if table_schema_delta__.is_some() { + return Err(serde::de::Error::duplicate_field("tableSchemaDelta")); + } + table_schema_delta__ = map.next_value()?; + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(ReplayTableSchemaDeltaInput { + flow_connection_configs: flow_connection_configs__, + table_schema_delta: table_schema_delta__, + }) + } + } + deserializer.deserialize_struct("peerdb_flow.ReplayTableSchemaDeltaInput", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SetupNormalizedTableBatchInput { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -3957,6 +4452,9 @@ impl serde::Serialize for StartFlowInput { if self.sync_flow_options.is_some() { len += 1; } + if !self.relation_message_mapping.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.StartFlowInput", len)?; if let Some(v) = self.last_sync_state.as_ref() { struct_ser.serialize_field("lastSyncState", v)?; @@ -3967,6 +4465,9 @@ impl serde::Serialize for StartFlowInput { if let Some(v) = self.sync_flow_options.as_ref() { struct_ser.serialize_field("syncFlowOptions", v)?; } + if !self.relation_message_mapping.is_empty() { + struct_ser.serialize_field("relationMessageMapping", &self.relation_message_mapping)?; + } struct_ser.end() } } @@ -3983,6 +4484,8 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { "flowConnectionConfigs", "sync_flow_options", "syncFlowOptions", + "relation_message_mapping", + "relationMessageMapping", ]; #[allow(clippy::enum_variant_names)] @@ -3990,6 +4493,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { LastSyncState, FlowConnectionConfigs, SyncFlowOptions, + RelationMessageMapping, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4015,6 +4519,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { "lastSyncState" | "last_sync_state" => Ok(GeneratedField::LastSyncState), "flowConnectionConfigs" | "flow_connection_configs" => Ok(GeneratedField::FlowConnectionConfigs), "syncFlowOptions" | "sync_flow_options" => Ok(GeneratedField::SyncFlowOptions), + "relationMessageMapping" | "relation_message_mapping" => Ok(GeneratedField::RelationMessageMapping), _ => Ok(GeneratedField::__SkipField__), } } @@ -4037,6 +4542,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { let mut last_sync_state__ = None; let mut flow_connection_configs__ = None; let mut sync_flow_options__ = None; + let mut relation_message_mapping__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::LastSyncState => { @@ -4057,6 +4563,15 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { } sync_flow_options__ = map.next_value()?; } + 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::()?; } @@ -4066,6 +4581,7 @@ impl<'de> serde::Deserialize<'de> for StartFlowInput { last_sync_state: last_sync_state__, flow_connection_configs: flow_connection_configs__, sync_flow_options: sync_flow_options__, + relation_message_mapping: relation_message_mapping__.unwrap_or_default(), }) } } @@ -4179,10 +4695,16 @@ impl serde::Serialize for SyncFlowOptions { 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() } } @@ -4195,11 +4717,14 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { const FIELDS: &[&str] = &[ "batch_size", "batchSize", + "relation_message_mapping", + "relationMessageMapping", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { BatchSize, + RelationMessageMapping, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -4223,6 +4748,7 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { { match value { "batchSize" | "batch_size" => Ok(GeneratedField::BatchSize), + "relationMessageMapping" | "relation_message_mapping" => Ok(GeneratedField::RelationMessageMapping), _ => Ok(GeneratedField::__SkipField__), } } @@ -4243,6 +4769,7 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { V: serde::de::MapAccess<'de>, { let mut batch_size__ = None; + let mut relation_message_mapping__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::BatchSize => { @@ -4253,6 +4780,15 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { 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::()?; } @@ -4260,6 +4796,7 @@ impl<'de> serde::Deserialize<'de> for SyncFlowOptions { } Ok(SyncFlowOptions { batch_size: batch_size__.unwrap_or_default(), + relation_message_mapping: relation_message_mapping__.unwrap_or_default(), }) } } @@ -4844,6 +5381,156 @@ impl<'de> serde::Deserialize<'de> for TableSchema { deserializer.deserialize_struct("peerdb_flow.TableSchema", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for TableSchemaDelta { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.src_table_name.is_empty() { + len += 1; + } + if !self.dst_table_name.is_empty() { + len += 1; + } + if !self.added_columns.is_empty() { + len += 1; + } + if !self.dropped_columns.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("peerdb_flow.TableSchemaDelta", len)?; + if !self.src_table_name.is_empty() { + struct_ser.serialize_field("srcTableName", &self.src_table_name)?; + } + if !self.dst_table_name.is_empty() { + struct_ser.serialize_field("dstTableName", &self.dst_table_name)?; + } + if !self.added_columns.is_empty() { + struct_ser.serialize_field("addedColumns", &self.added_columns)?; + } + if !self.dropped_columns.is_empty() { + struct_ser.serialize_field("droppedColumns", &self.dropped_columns)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for TableSchemaDelta { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "src_table_name", + "srcTableName", + "dst_table_name", + "dstTableName", + "added_columns", + "addedColumns", + "dropped_columns", + "droppedColumns", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + SrcTableName, + DstTableName, + AddedColumns, + DroppedColumns, + __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 { + "srcTableName" | "src_table_name" => Ok(GeneratedField::SrcTableName), + "dstTableName" | "dst_table_name" => Ok(GeneratedField::DstTableName), + "addedColumns" | "added_columns" => Ok(GeneratedField::AddedColumns), + "droppedColumns" | "dropped_columns" => Ok(GeneratedField::DroppedColumns), + _ => Ok(GeneratedField::__SkipField__), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = TableSchemaDelta; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct peerdb_flow.TableSchemaDelta") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut src_table_name__ = None; + let mut dst_table_name__ = None; + let mut added_columns__ = None; + let mut dropped_columns__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::SrcTableName => { + if src_table_name__.is_some() { + return Err(serde::de::Error::duplicate_field("srcTableName")); + } + src_table_name__ = Some(map.next_value()?); + } + GeneratedField::DstTableName => { + if dst_table_name__.is_some() { + return Err(serde::de::Error::duplicate_field("dstTableName")); + } + dst_table_name__ = Some(map.next_value()?); + } + GeneratedField::AddedColumns => { + if added_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("addedColumns")); + } + added_columns__ = Some(map.next_value()?); + } + GeneratedField::DroppedColumns => { + if dropped_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("droppedColumns")); + } + dropped_columns__ = Some(map.next_value()?); + } + GeneratedField::__SkipField__ => { + let _ = map.next_value::()?; + } + } + } + Ok(TableSchemaDelta { + src_table_name: src_table_name__.unwrap_or_default(), + dst_table_name: dst_table_name__.unwrap_or_default(), + added_columns: added_columns__.unwrap_or_default(), + dropped_columns: dropped_columns__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("peerdb_flow.TableSchemaDelta", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for TimestampPartitionRange { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/protos/flow.proto b/protos/flow.proto index 7ddd873975..07d5af491a 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -10,6 +10,18 @@ message TableNameMapping { string destination_table_name = 2; } +message RelationMessageColumn { + uint32 flags = 1; + string name = 2; + uint32 data_type = 3; +} + +message RelationMessage { + uint32 relation_id = 1; + string relation_name = 2; + repeated RelationMessageColumn columns = 3; +} + message FlowConnectionConfigs { peerdb_peers.Peer source = 1; peerdb_peers.Peer destination = 2; @@ -46,7 +58,10 @@ message FlowConnectionConfigs { int64 push_parallelism = 22; } -message SyncFlowOptions { int32 batch_size = 1; } +message SyncFlowOptions { + int32 batch_size = 1; + map relation_message_mapping = 2; +} message NormalizeFlowOptions { int32 batch_size = 1; @@ -61,6 +76,7 @@ message StartFlowInput { LastSyncState last_sync_state = 1; FlowConnectionConfigs flow_connection_configs = 2; SyncFlowOptions sync_flow_options = 3; + map relation_message_mapping = 4; } message StartNormalizeInput { @@ -268,3 +284,20 @@ message QRepParitionResult { message DropFlowInput { string flow_name = 1; } + +message DeltaAddedColumn { + string column_name = 1; + string column_type = 2; +} + +message TableSchemaDelta { + string src_table_name = 1; + string dst_table_name = 2; + repeated DeltaAddedColumn added_columns = 3; + repeated string dropped_columns = 4; +} + +message ReplayTableSchemaDeltaInput { + FlowConnectionConfigs flow_connection_configs = 1; + TableSchemaDelta table_schema_delta = 2; +} diff --git a/stacks/flow-api.Dockerfile b/stacks/flow-api.Dockerfile index fc95cd4a7d..f84c5da352 100644 --- a/stacks/flow-api.Dockerfile +++ b/stacks/flow-api.Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1.2 # Start from the latest Golang base image -FROM golang:1.20-alpine AS builder +FROM golang:1.21-alpine AS builder WORKDIR /root/flow # first copy only go.mod and go.sum to cache dependencies diff --git a/stacks/flow-snapshot-worker.Dockerfile b/stacks/flow-snapshot-worker.Dockerfile index a58e800d50..cf3a788818 100644 --- a/stacks/flow-snapshot-worker.Dockerfile +++ b/stacks/flow-snapshot-worker.Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1.2 # Start from the latest Golang base image -FROM golang:1.20-alpine AS builder +FROM golang:1.21-alpine AS builder WORKDIR /root/ diff --git a/stacks/flow-worker.Dockerfile b/stacks/flow-worker.Dockerfile index b5226ff9ed..cd6725b9fb 100644 --- a/stacks/flow-worker.Dockerfile +++ b/stacks/flow-worker.Dockerfile @@ -1,7 +1,7 @@ # syntax=docker/dockerfile:1.2 # Start from the latest Golang base image -FROM golang:1.20-alpine AS builder +FROM golang:1.21-alpine AS builder WORKDIR /root/