diff --git a/.github/workflows/flow.yml b/.github/workflows/flow.yml index 96b4e3aad3..dffa4e0840 100644 --- a/.github/workflows/flow.yml +++ b/.github/workflows/flow.yml @@ -95,7 +95,7 @@ jobs: - name: run tests run: | - gotestsum --format testname -- -p 8 ./... -timeout 1500s + gotestsum --format testname -- -p 8 ./... -timeout 1200s working-directory: ./flow env: AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }} diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index 269b219068..a52eac6e85 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -73,6 +73,7 @@ func (a *FlowableActivity) SetupMetadataTables(ctx context.Context, config *prot defer connectors.CloseConnector(dstConn) if err := dstConn.SetupMetadataTables(); err != nil { + a.Alerter.LogFlowError(ctx, config.Name, err) return fmt.Errorf("failed to setup metadata tables: %w", err) } @@ -111,6 +112,7 @@ func (a *FlowableActivity) EnsurePullability( output, err := srcConn.EnsurePullability(config) if err != nil { + a.Alerter.LogFlowError(ctx, config.PeerConnectionConfig.Name, err) return nil, fmt.Errorf("failed to ensure pullability: %w", err) } @@ -165,84 +167,13 @@ func (a *FlowableActivity) CreateNormalizedTable( } defer connectors.CloseConnector(conn) - return conn.SetupNormalizedTables(config) -} - -func (a *FlowableActivity) handleSlotInfo( - ctx context.Context, - srcConn connectors.CDCPullConnector, - slotName string, - peerName string, -) error { - slotInfo, err := srcConn.GetSlotInfo(slotName) - if err != nil { - slog.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) - return err - } - - deploymentUIDPrefix := "" - if peerdbenv.GetPeerDBDeploymentUID() != "" { - deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.GetPeerDBDeploymentUID()) - } - - slotLagInMBThreshold := peerdbenv.GetPeerDBSlotLagMBAlertThreshold() - if (slotLagInMBThreshold > 0) && (slotInfo[0].LagInMb >= float32(slotLagInMBThreshold)) { - a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-slot-lag-threshold-exceeded", peerName), - fmt.Sprintf(`%sSlot `+"`%s`"+` on peer `+"`%s`"+` has exceeded threshold size of %dMB, currently at %.2fMB! -cc: `, - deploymentUIDPrefix, slotName, peerName, slotLagInMBThreshold, slotInfo[0].LagInMb)) - } - - // Also handles alerts for PeerDB user connections exceeding a given limit here - maxOpenConnectionsThreshold := peerdbenv.GetPeerDBOpenConnectionsAlertThreshold() - res, err := srcConn.GetOpenConnectionsForUser() + setupNormalizedTablesOutput, err := conn.SetupNormalizedTables(config) if err != nil { - slog.WarnContext(ctx, "warning: failed to get current open connections", slog.Any("error", err)) - return err - } - if (maxOpenConnectionsThreshold > 0) && (res.CurrentOpenConnections >= int64(maxOpenConnectionsThreshold)) { - a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-max-open-connections-threshold-exceeded", peerName), - fmt.Sprintf(`%sOpen connections from PeerDB user `+"`%s`"+` on peer `+"`%s`"+ - ` has exceeded threshold size of %d connections, currently at %d connections! -cc: `, - deploymentUIDPrefix, res.UserName, peerName, maxOpenConnectionsThreshold, res.CurrentOpenConnections)) + a.Alerter.LogFlowError(ctx, config.PeerConnectionConfig.Name, err) + return nil, fmt.Errorf("failed to setup normalized tables: %w", err) } - if len(slotInfo) != 0 { - return monitoring.AppendSlotSizeInfo(ctx, a.CatalogPool, peerName, slotInfo[0]) - } - return nil -} - -func (a *FlowableActivity) recordSlotSizePeriodically( - ctx context.Context, - srcConn connectors.CDCPullConnector, - slotName string, - peerName string, -) { - // ensures slot info is logged at least once per SyncFlow - err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) - if err != nil { - return - } - - timeout := 5 * time.Minute - ticker := time.NewTicker(timeout) - - defer ticker.Stop() - for { - select { - case <-ticker.C: - err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) - if err != nil { - return - } - case <-ctx.Done(): - return - } - ticker.Stop() - ticker = time.NewTicker(timeout) - } + return setupNormalizedTablesOutput, nil } func (a *FlowableActivity) StartFlow(ctx context.Context, @@ -255,6 +186,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, return nil, fmt.Errorf("failed to get destination connector: %w", err) } defer connectors.CloseConnector(dstConn) + slog.InfoContext(ctx, "initializing table schema...") err = dstConn.InitializeTableSchema(input.FlowConnectionConfigs.TableNameSchemaMapping) if err != nil { @@ -267,10 +199,6 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, tblNameMapping[v.SourceTableIdentifier] = model.NewNameAndExclude(v.DestinationTableIdentifier, v.Exclude) } - recordBatch := model.NewCDCRecordStream() - - startTime := time.Now() - errGroup, errCtx := errgroup.WithContext(ctx) srcConn, err := connectors.GetCDCPullConnector(errCtx, conn.Source) if err != nil { @@ -286,21 +214,24 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, go a.recordSlotSizePeriodically(errCtx, srcConn, slotNameForMetrics, input.FlowConnectionConfigs.Source.Name) // start a goroutine to pull records from the source + recordBatch := model.NewCDCRecordStream() + startTime := time.Now() + flowName := input.FlowConnectionConfigs.FlowJobName errGroup.Go(func() error { return srcConn.PullRecords(a.CatalogPool, &model.PullRecordsRequest{ - FlowJobName: input.FlowConnectionConfigs.FlowJobName, + FlowJobName: flowName, SrcTableIDNameMapping: input.FlowConnectionConfigs.SrcTableIdNameMapping, TableNameMapping: tblNameMapping, LastOffset: input.LastSyncState.Checkpoint, MaxBatchSize: uint32(input.SyncFlowOptions.BatchSize), - IdleTimeout: peerdbenv.GetPeerDBCDCIdleTimeoutSeconds(), + IdleTimeout: peerdbenv.PeerDBCDCIdleTimeoutSeconds(), TableNameSchemaMapping: input.FlowConnectionConfigs.TableNameSchemaMapping, OverridePublicationName: input.FlowConnectionConfigs.PublicationName, OverrideReplicationSlotName: input.FlowConnectionConfigs.ReplicationSlotName, RelationMessageMapping: input.RelationMessageMapping, RecordStream: recordBatch, SetLastOffset: func(lastOffset int64) error { - return dstConn.SetLastOffset(input.FlowConnectionConfigs.FlowJobName, lastOffset) + return dstConn.SetLastOffset(flowName, lastOffset) }, }) }) @@ -308,12 +239,12 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, hasRecords := !recordBatch.WaitAndCheckEmpty() slog.InfoContext(ctx, fmt.Sprintf("the current sync flow has records: %v", hasRecords)) if a.CatalogPool != nil && hasRecords { - syncBatchID, err := dstConn.GetLastSyncBatchID(input.FlowConnectionConfigs.FlowJobName) + syncBatchID, err := dstConn.GetLastSyncBatchID(flowName) if err != nil && conn.Destination.Type != protos.DBType_EVENTHUB { return nil, err } - err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, input.FlowConnectionConfigs.FlowJobName, + err = monitoring.AddCDCBatchForFlow(ctx, a.CatalogPool, flowName, monitoring.CDCBatchInfo{ BatchID: syncBatchID + 1, RowsInBatch: 0, @@ -321,6 +252,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, StartTime: startTime, }) if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, err } } @@ -329,6 +261,7 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, // wait for the pull goroutine to finish err = errGroup.Wait() if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to pull records: %w", err) } slog.InfoContext(ctx, "no records to push") @@ -357,11 +290,13 @@ func (a *FlowableActivity) StartFlow(ctx context.Context, }) if err != nil { slog.Warn("failed to push records", slog.Any("error", err)) + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to push records: %w", err) } err = errGroup.Wait() if err != nil { + a.Alerter.LogFlowError(ctx, flowName, err) return nil, fmt.Errorf("failed to pull records: %w", err) } @@ -461,6 +396,7 @@ func (a *FlowableActivity) StartNormalize( SyncBatchID: syncBatchID, }) if err != nil { + a.Alerter.LogFlowError(ctx, input.FlowConnectionConfigs.FlowJobName, err) return nil, fmt.Errorf("failed to normalized records: %w", err) } @@ -498,7 +434,13 @@ func (a *FlowableActivity) ReplayTableSchemaDeltas( } defer connectors.CloseConnector(dest) - return dest.ReplayTableSchemaDeltas(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDeltas) + err = dest.ReplayTableSchemaDeltas(input.FlowConnectionConfigs.FlowJobName, input.TableSchemaDeltas) + if err != nil { + a.Alerter.LogFlowError(ctx, input.FlowConnectionConfigs.FlowJobName, err) + return fmt.Errorf("failed to replay table schema deltas: %w", err) + } + + return nil } // SetupQRepMetadataTables sets up the metadata tables for QReplication. @@ -509,7 +451,13 @@ func (a *FlowableActivity) SetupQRepMetadataTables(ctx context.Context, config * } defer connectors.CloseConnector(conn) - return conn.SetupQRepMetadataTables(config) + err = conn.SetupQRepMetadataTables(config) + if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) + return fmt.Errorf("failed to setup metadata tables: %w", err) + } + + return nil } // GetQRepPartitions returns the partitions for a given QRepConfig. @@ -534,6 +482,7 @@ func (a *FlowableActivity) GetQRepPartitions(ctx context.Context, partitions, err := srcConn.GetQRepPartitions(config, last) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return nil, fmt.Errorf("failed to get partitions from source: %w", err) } if len(partitions) > 0 { @@ -574,6 +523,7 @@ func (a *FlowableActivity) ReplicateQRepPartitions(ctx context.Context, slog.InfoContext(ctx, fmt.Sprintf("batch-%d - replicating partition - %s\n", partitions.BatchId, p.PartitionId)) err := a.replicateQRepPartition(ctx, config, i+1, numPartitions, p, runUUID) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return err } } @@ -713,6 +663,7 @@ func (a *FlowableActivity) ConsolidateQRepPartitions(ctx context.Context, config err = dstConn.ConsolidateQRepPartitions(config) if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return err } @@ -787,6 +738,11 @@ func (a *FlowableActivity) getPostgresPeerConfigs(ctx context.Context) ([]*proto } func (a *FlowableActivity) SendWALHeartbeat(ctx context.Context) error { + if !peerdbenv.PeerDBEnableWALHeartbeat() { + slog.InfoContext(ctx, "wal heartbeat is disabled") + return nil + } + sendTimeout := 10 * time.Minute ticker := time.NewTicker(sendTimeout) defer ticker.Stop() @@ -1013,6 +969,7 @@ func (a *FlowableActivity) ReplicateXminPartition(ctx context.Context, } else { err := errGroup.Wait() if err != nil { + a.Alerter.LogFlowError(ctx, config.FlowJobName, err) return 0, err } diff --git a/flow/activities/slot.go b/flow/activities/slot.go new file mode 100644 index 0000000000..cf1375b4e4 --- /dev/null +++ b/flow/activities/slot.go @@ -0,0 +1,89 @@ +package activities + +import ( + "context" + "fmt" + "log/slog" + "time" + + "github.com/PeerDB-io/peer-flow/connectors" + "github.com/PeerDB-io/peer-flow/connectors/utils/monitoring" + "github.com/PeerDB-io/peer-flow/peerdbenv" +) + +func (a *FlowableActivity) handleSlotInfo( + ctx context.Context, + srcConn connectors.CDCPullConnector, + slotName string, + peerName string, +) error { + slotInfo, err := srcConn.GetSlotInfo(slotName) + if err != nil { + slog.WarnContext(ctx, "warning: failed to get slot info", slog.Any("error", err)) + return err + } + + deploymentUIDPrefix := "" + if peerdbenv.PeerDBDeploymentUID() != "" { + deploymentUIDPrefix = fmt.Sprintf("[%s] ", peerdbenv.PeerDBDeploymentUID()) + } + + slotLagInMBThreshold := peerdbenv.PeerDBSlotLagMBAlertThreshold() + if (slotLagInMBThreshold > 0) && (slotInfo[0].LagInMb >= float32(slotLagInMBThreshold)) { + a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-slot-lag-threshold-exceeded", peerName), + fmt.Sprintf(`%sSlot `+"`%s`"+` on peer `+"`%s`"+` has exceeded threshold size of %dMB, currently at %.2fMB! +cc: `, + deploymentUIDPrefix, slotName, peerName, slotLagInMBThreshold, slotInfo[0].LagInMb)) + } + + // Also handles alerts for PeerDB user connections exceeding a given limit here + maxOpenConnectionsThreshold := peerdbenv.PeerDBOpenConnectionsAlertThreshold() + res, err := srcConn.GetOpenConnectionsForUser() + if err != nil { + slog.WarnContext(ctx, "warning: failed to get current open connections", slog.Any("error", err)) + return err + } + if (maxOpenConnectionsThreshold > 0) && (res.CurrentOpenConnections >= int64(maxOpenConnectionsThreshold)) { + a.Alerter.AlertIf(ctx, fmt.Sprintf("%s-max-open-connections-threshold-exceeded", peerName), + fmt.Sprintf(`%sOpen connections from PeerDB user `+"`%s`"+` on peer `+"`%s`"+ + ` has exceeded threshold size of %d connections, currently at %d connections! +cc: `, + deploymentUIDPrefix, res.UserName, peerName, maxOpenConnectionsThreshold, res.CurrentOpenConnections)) + } + + if len(slotInfo) != 0 { + return monitoring.AppendSlotSizeInfo(ctx, a.CatalogPool, peerName, slotInfo[0]) + } + return nil +} + +func (a *FlowableActivity) recordSlotSizePeriodically( + ctx context.Context, + srcConn connectors.CDCPullConnector, + slotName string, + peerName string, +) { + // ensures slot info is logged at least once per SyncFlow + err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) + if err != nil { + return + } + + timeout := 5 * time.Minute + ticker := time.NewTicker(timeout) + + defer ticker.Stop() + for { + select { + case <-ticker.C: + err := a.handleSlotInfo(ctx, srcConn, slotName, peerName) + if err != nil { + return + } + case <-ctx.Done(): + return + } + ticker.Stop() + ticker = time.NewTicker(timeout) + } +} diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index 454dceb20e..34f31219ed 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -209,7 +209,7 @@ func (h *FlowRequestHandler) GetSlotInfo( return &protos.PeerSlotResponse{SlotData: nil}, err } - pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) + pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig, false) if err != nil { slog.Error("Failed to create postgres connector", slog.Any("error", err)) return &protos.PeerSlotResponse{SlotData: nil}, err @@ -236,7 +236,7 @@ func (h *FlowRequestHandler) GetStatInfo( return &protos.PeerStatResponse{StatData: nil}, err } - pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig) + pgConnector, err := connpostgres.NewPostgresConnector(ctx, pgConfig, false) if err != nil { slog.Error("Failed to create postgres connector", slog.Any("error", err)) return &protos.PeerStatResponse{StatData: nil}, err diff --git a/flow/cmd/version.go b/flow/cmd/version.go index 94210ea9cb..3338a20e6a 100644 --- a/flow/cmd/version.go +++ b/flow/cmd/version.go @@ -11,6 +11,6 @@ func (h *FlowRequestHandler) GetVersion( ctx context.Context, req *protos.PeerDBVersionRequest, ) (*protos.PeerDBVersionResponse, error) { - version := peerdbenv.GetPeerDBVersionShaShort() + version := peerdbenv.PeerDBVersionShaShort() return &protos.PeerDBVersionResponse{Version: version}, nil } diff --git a/flow/cmd/worker.go b/flow/cmd/worker.go index a42ac76b47..eea0e9184f 100644 --- a/flow/cmd/worker.go +++ b/flow/cmd/worker.go @@ -132,9 +132,15 @@ func WorkerMain(opts *WorkerOptions) error { w.RegisterWorkflow(peerflow.XminFlowWorkflow) w.RegisterWorkflow(peerflow.DropFlowWorkflow) w.RegisterWorkflow(peerflow.HeartbeatFlowWorkflow) + + alerter, err := alerting.NewAlerter(conn) + if err != nil { + return fmt.Errorf("unable to create alerter: %w", err) + } + w.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, - Alerter: alerting.NewAlerter(conn), + Alerter: alerter, }) err = w.Run(worker.InterruptCh()) diff --git a/flow/connectors/core.go b/flow/connectors/core.go index e5efec63ca..477b7cf46b 100644 --- a/flow/connectors/core.go +++ b/flow/connectors/core.go @@ -43,9 +43,6 @@ type CDCPullConnector interface { // PullFlowCleanup drops both the Postgres publication and replication slot, as a part of DROP MIRROR PullFlowCleanup(jobName string) error - // SendWALHeartbeat allows for activity to progress restart_lsn on postgres. - SendWALHeartbeat() error - // GetSlotInfo returns the WAL (or equivalent) info of a slot for the connector. GetSlotInfo(slotName string) ([]*protos.SlotInfo, error) @@ -140,7 +137,7 @@ func GetCDCPullConnector(ctx context.Context, config *protos.Peer) (CDCPullConne inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), true) default: return nil, ErrUnsupportedFunctionality } @@ -150,7 +147,7 @@ func GetCDCSyncConnector(ctx context.Context, config *protos.Peer) (CDCSyncConne inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_BigqueryConfig: return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: @@ -172,7 +169,7 @@ func GetCDCNormalizeConnector(ctx context.Context, inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_BigqueryConfig: return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: @@ -186,7 +183,7 @@ func GetQRepPullConnector(ctx context.Context, config *protos.Peer) (QRepPullCon inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_SqlserverConfig: return connsqlserver.NewSQLServerConnector(ctx, config.GetSqlserverConfig()) default: @@ -198,7 +195,7 @@ func GetQRepSyncConnector(ctx context.Context, config *protos.Peer) (QRepSyncCon inner := config.Config switch inner.(type) { case *protos.Peer_PostgresConfig: - return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig()) + return connpostgres.NewPostgresConnector(ctx, config.GetPostgresConfig(), false) case *protos.Peer_BigqueryConfig: return connbigquery.NewBigQueryConnector(ctx, config.GetBigqueryConfig()) case *protos.Peer_SnowflakeConfig: @@ -219,7 +216,10 @@ func GetConnector(ctx context.Context, peer *protos.Peer) (Connector, error) { if pgConfig == nil { return nil, fmt.Errorf("missing postgres config for %s peer %s", peer.Type.String(), peer.Name) } - return connpostgres.NewPostgresConnector(ctx, pgConfig) + // we can't decide if a PG peer should have replication permissions on it because we don't know + // what the user wants to do with it, so defaulting to being permissive. + // can be revisited in the future or we can use some UI wizardry. + return connpostgres.NewPostgresConnector(ctx, pgConfig, false) case protos.DBType_BIGQUERY: bqConfig := peer.GetBigqueryConfig() if bqConfig == nil { diff --git a/flow/connectors/eventhub/eventhub.go b/flow/connectors/eventhub/eventhub.go index 027d3027fa..05347a4263 100644 --- a/flow/connectors/eventhub/eventhub.go +++ b/flow/connectors/eventhub/eventhub.go @@ -129,7 +129,7 @@ func (c *EventHubConnector) processBatch( batchPerTopic := NewHubBatches(c.hubManager) toJSONOpts := model.NewToJSONOptions(c.config.UnnestColumns) - eventHubFlushTimeout := peerdbenv.GetPeerDBEventhubFlushTimeoutSeconds() + eventHubFlushTimeout := peerdbenv.PeerDBEventhubFlushTimeoutSeconds() ticker := time.NewTicker(eventHubFlushTimeout) defer ticker.Stop() diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 211b29ee12..5fca1edf29 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -34,7 +34,7 @@ type PostgresConnector struct { } // NewPostgresConnector creates a new instance of PostgresConnector. -func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) (*PostgresConnector, error) { +func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig, initializeReplPool bool) (*PostgresConnector, error) { connectionString := utils.GetPGConnectionString(pgConfig) // create a separate connection pool for non-replication queries as replication connections cannot @@ -62,21 +62,23 @@ func NewPostgresConnector(ctx context.Context, pgConfig *protos.PostgresConfig) return nil, fmt.Errorf("failed to get custom type map: %w", err) } - // ensure that replication is set to database - replConnConfig, err := pgxpool.ParseConfig(connectionString) - if err != nil { - return nil, fmt.Errorf("failed to parse connection string: %w", err) - } + // only initialize for CDCPullConnector to reduce number of idle connections + var replPool *SSHWrappedPostgresPool + if initializeReplPool { + // ensure that replication is set to database + replConnConfig, err := pgxpool.ParseConfig(connectionString) + if err != nil { + return nil, fmt.Errorf("failed to parse connection string: %w", err) + } - replConnConfig.ConnConfig.RuntimeParams["replication"] = "database" - replConnConfig.ConnConfig.RuntimeParams["bytea_output"] = "hex" - replConnConfig.MaxConns = 1 + replConnConfig.ConnConfig.RuntimeParams["replication"] = "database" + replConnConfig.ConnConfig.RuntimeParams["bytea_output"] = "hex" + replConnConfig.MaxConns = 1 - // TODO: replPool not initializing might be intentional, if we only want to use QRep mirrors - // and the user doesn't have the REPLICATION permission - replPool, err := NewSSHWrappedPostgresPool(ctx, replConnConfig, pgConfig.SshConfig) - if err != nil { - return nil, fmt.Errorf("failed to create connection pool: %w", err) + replPool, err = NewSSHWrappedPostgresPool(ctx, replConnConfig, pgConfig.SshConfig) + if err != nil { + return nil, fmt.Errorf("failed to create replication connection pool: %w", err) + } } metadataSchema := "_peerdb_internal" @@ -878,22 +880,6 @@ func (c *PostgresConnector) SyncFlowCleanup(jobName string) error { return nil } -func (c *PostgresConnector) SendWALHeartbeat() error { - command := ` - BEGIN; - DROP aggregate IF EXISTS PEERDB_EPHEMERAL_HEARTBEAT(float4); - CREATE AGGREGATE PEERDB_EPHEMERAL_HEARTBEAT(float4) (SFUNC = float4pl, STYPE = float4); - DROP aggregate PEERDB_EPHEMERAL_HEARTBEAT(float4); - END; - ` - _, err := c.pool.Exec(c.ctx, command) - if err != nil { - return fmt.Errorf("error bumping wal position: %w", err) - } - - return nil -} - // GetLastOffset returns the last synced offset for a job. func (c *PostgresConnector) GetOpenConnectionsForUser() (*protos.GetOpenConnectionsForUserResult, error) { row := c.pool. diff --git a/flow/connectors/postgres/postgres_repl_test.go b/flow/connectors/postgres/postgres_repl_test.go index b50a1f89fc..df3a7de13f 100644 --- a/flow/connectors/postgres/postgres_repl_test.go +++ b/flow/connectors/postgres/postgres_repl_test.go @@ -28,7 +28,7 @@ func (suite *PostgresReplicationSnapshotTestSuite) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, true) require.NoError(suite.T(), err) setupTx, err := suite.connector.pool.Begin(context.Background()) diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 98a6a47b99..8a919eb214 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -32,7 +32,7 @@ func (suite *PostgresSchemaDeltaTestSuite) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, false) suite.failTestError(err) setupTx, err := suite.connector.pool.Begin(context.Background()) diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index 98d20b63ff..def870c183 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -84,7 +84,7 @@ func (c *SnowflakeConnector) getTableSchema(tableName string) ([]*sql.ColumnType LIMIT 0 `, tableName) - rows, err := c.database.Query(queryString) + rows, err := c.database.QueryContext(c.ctx, queryString) if err != nil { return nil, fmt.Errorf("failed to execute query: %w", err) } @@ -294,7 +294,7 @@ func (c *SnowflakeConnector) getColsFromTable(tableName string) (*model.ColumnIn WHERE UPPER(table_name) = '%s' AND UPPER(table_schema) = '%s' `, components.tableIdentifier, components.schemaIdentifier) - rows, err := c.database.Query(queryString) + rows, err := c.database.QueryContext(c.ctx, queryString) if err != nil { return nil, fmt.Errorf("failed to execute query: %w", err) } diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 7184898ae3..30834c2554 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -112,7 +112,6 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( s.connector.logger.Info("sync function called and schema acquired", partitionLog) err = s.addMissingColumns( - config.FlowJobName, schema, dstTableSchema, dstTableName, @@ -152,7 +151,6 @@ func (s *SnowflakeAvroSyncMethod) SyncQRepRecords( } func (s *SnowflakeAvroSyncMethod) addMissingColumns( - flowJobName string, schema *model.QRecordSchema, dstTableSchema []*sql.ColumnType, dstTableName string, @@ -197,7 +195,7 @@ func (s *SnowflakeAvroSyncMethod) addMissingColumns( s.connector.logger.Info(fmt.Sprintf("altering destination table %s with command `%s`", dstTableName, alterTableCmd), partitionLog) - if _, err := tx.Exec(alterTableCmd); err != nil { + if _, err := tx.ExecContext(s.connector.ctx, alterTableCmd); err != nil { return fmt.Errorf("failed to alter destination table: %w", err) } } @@ -290,7 +288,7 @@ func (s *SnowflakeAvroSyncMethod) putFileToStage(avroFile *avro.AvroFile, stage shutdown <- struct{}{} }() - if _, err := s.connector.database.Exec(putCmd); err != nil { + if _, err := s.connector.database.ExecContext(s.connector.ctx, putCmd); err != nil { return fmt.Errorf("failed to put file to stage: %w", err) } @@ -395,7 +393,7 @@ func (s *SnowflakeAvroSyncMethod) insertMetadata( return fmt.Errorf("failed to create metadata insert statement: %v", err) } - if _, err := s.connector.database.Exec(insertMetadataStmt); err != nil { + if _, err := s.connector.database.ExecContext(s.connector.ctx, insertMetadataStmt); err != nil { s.connector.logger.Error("failed to execute metadata insert statement "+insertMetadataStmt, slog.Any("error", err), partitionLog) return fmt.Errorf("failed to execute metadata insert statement: %v", err) @@ -434,7 +432,7 @@ func (s *SnowflakeAvroWriteHandler) HandleAppendMode( copyCmd := fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", s.dstTableName, copyInfo.columnsSQL, copyInfo.transformationSQL, s.stage, strings.Join(s.copyOpts, ",")) s.connector.logger.Info("running copy command: " + copyCmd) - _, err := s.connector.database.Exec(copyCmd) + _, err := s.connector.database.ExecContext(s.connector.ctx, copyCmd) if err != nil { return fmt.Errorf("failed to run COPY INTO command: %w", err) } @@ -518,7 +516,7 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( //nolint:gosec createTempTableCmd := fmt.Sprintf("CREATE TEMPORARY TABLE %s AS SELECT * FROM %s LIMIT 0", tempTableName, s.dstTableName) - if _, err := s.connector.database.Exec(createTempTableCmd); err != nil { + if _, err := s.connector.database.ExecContext(s.connector.ctx, createTempTableCmd); err != nil { return fmt.Errorf("failed to create temp table: %w", err) } s.connector.logger.Info("created temp table " + tempTableName) @@ -526,7 +524,7 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( //nolint:gosec copyCmd := fmt.Sprintf("COPY INTO %s(%s) FROM (SELECT %s FROM @%s) %s", tempTableName, copyInfo.columnsSQL, copyInfo.transformationSQL, s.stage, strings.Join(s.copyOpts, ",")) - _, err = s.connector.database.Exec(copyCmd) + _, err = s.connector.database.ExecContext(s.connector.ctx, copyCmd) if err != nil { return fmt.Errorf("failed to run COPY INTO command: %w", err) } @@ -538,7 +536,7 @@ func (s *SnowflakeAvroWriteHandler) HandleUpsertMode( } startTime := time.Now() - rows, err := s.connector.database.Exec(mergeCmd) + rows, err := s.connector.database.ExecContext(s.connector.ctx, mergeCmd) if err != nil { return fmt.Errorf("failed to merge data into destination table '%s': %w", mergeCmd, err) } diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index b247626ab0..6831156ee6 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -480,8 +480,9 @@ func (c *SnowflakeConnector) ReplayTableSchemaDeltas(flowJobName string, return fmt.Errorf("failed to convert column type %s to snowflake type: %w", addedColumn.ColumnType, err) } - _, err = tableSchemaModifyTx.Exec(fmt.Sprintf("ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", - schemaDelta.DstTableName, strings.ToUpper(addedColumn.ColumnName), sfColtype)) + _, err = tableSchemaModifyTx.ExecContext(c.ctx, + fmt.Sprintf("ALTER TABLE %s ADD COLUMN IF NOT EXISTS \"%s\" %s", + schemaDelta.DstTableName, strings.ToUpper(addedColumn.ColumnName), sfColtype)) if err != nil { return fmt.Errorf("failed to add column %s for table %s: %w", addedColumn.ColumnName, schemaDelta.DstTableName, err) @@ -821,7 +822,8 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) for columnName, genericColumnType := range normalizedTableSchema.Columns { - sfType, err := qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)) + qvKind := qvalue.QValueKind(genericColumnType) + sfType, err := qValueKindToSnowflakeType(qvKind) if err != nil { return 0, fmt.Errorf("failed to convert column type %s to snowflake type: %w", genericColumnType, err) @@ -846,8 +848,14 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( // "Microseconds*1000) "+ // "AS %s,", toVariantColumnName, columnName, columnName)) default: - flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("CAST(%s:\"%s\" AS %s) AS %s,", - toVariantColumnName, columnName, sfType, targetColumnName)) + if qvKind == qvalue.QValueKindNumeric { + flattenedCastsSQLArray = append(flattenedCastsSQLArray, + fmt.Sprintf("TRY_CAST((%s:\"%s\")::text AS %s) AS %s,", + toVariantColumnName, columnName, sfType, targetColumnName)) + } else { + flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("CAST(%s:\"%s\" AS %s) AS %s,", + toVariantColumnName, columnName, sfType, targetColumnName)) + } } } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ""), ",") diff --git a/flow/connectors/utils/catalog/env.go b/flow/connectors/utils/catalog/env.go index cdd85535b9..f5c8e0507d 100644 --- a/flow/connectors/utils/catalog/env.go +++ b/flow/connectors/utils/catalog/env.go @@ -39,10 +39,10 @@ func GetCatalogConnectionPoolFromEnv() (*pgxpool.Pool, error) { func genCatalogConnectionString() string { return utils.GetPGConnectionString(&protos.PostgresConfig{ - Host: peerdbenv.GetPeerDBCatalogHost(), - Port: peerdbenv.GetPeerDBCatalogPort(), - User: peerdbenv.GetPeerDBCatalogUser(), - Password: peerdbenv.GetPeerDBCatalogPassword(), - Database: peerdbenv.GetPeerDBCatalogDatabase(), + Host: peerdbenv.PeerDBCatalogHost(), + Port: peerdbenv.PeerDBCatalogPort(), + User: peerdbenv.PeerDBCatalogUser(), + Password: peerdbenv.PeerDBCatalogPassword(), + Database: peerdbenv.PeerDBCatalogDatabase(), }) } diff --git a/flow/connectors/utils/cdc_records/cdc_records_storage.go b/flow/connectors/utils/cdc_records/cdc_records_storage.go index 3147045a76..66722b69ed 100644 --- a/flow/connectors/utils/cdc_records/cdc_records_storage.go +++ b/flow/connectors/utils/cdc_records/cdc_records_storage.go @@ -43,7 +43,7 @@ func NewCDCRecordsStore(flowJobName string) *cdcRecordsStore { numRecords: 0, flowJobName: flowJobName, dbFolderName: fmt.Sprintf("%s/%s_%s", os.TempDir(), flowJobName, shared.RandomString(8)), - numRecordsSwitchThreshold: peerdbenv.GetPeerDBCDCDiskSpillThreshold(), + numRecordsSwitchThreshold: peerdbenv.PeerDBCDCDiskSpillThreshold(), } } diff --git a/flow/connectors/utils/monitoring/monitoring.go b/flow/connectors/utils/monitoring/monitoring.go index 7815372277..7e9263cf26 100644 --- a/flow/connectors/utils/monitoring/monitoring.go +++ b/flow/connectors/utils/monitoring/monitoring.go @@ -204,7 +204,7 @@ func AppendSlotSizeInfo( _, err := pool.Exec(ctx, "INSERT INTO peerdb_stats.peer_slot_size"+ "(peer_name, slot_name, restart_lsn, redo_lsn, confirmed_flush_lsn, slot_size, wal_status) "+ - "VALUES($1,$2,$3,$4,$5,$6) ON CONFLICT DO NOTHING;", + "VALUES($1,$2,$3,$4,$5,$6,$7) ON CONFLICT DO NOTHING;", peerName, slotInfo.SlotName, slotInfo.RestartLSN, diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index 1c86c973b9..0bb886f9a3 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -55,7 +55,7 @@ func (s *PeerFlowE2ETestSuitePG) SetupSuite() { User: "postgres", Password: "postgres", Database: "postgres", - }) + }, false) s.NoError(err) } diff --git a/flow/e2e/test_utils.go b/flow/e2e/test_utils.go index 13ca8044e5..8bea8cf984 100644 --- a/flow/e2e/test_utils.go +++ b/flow/e2e/test_utils.go @@ -60,9 +60,15 @@ func RegisterWorkflowsAndActivities(env *testsuite.TestWorkflowEnvironment, t *t env.RegisterWorkflow(peerflow.QRepFlowWorkflow) env.RegisterWorkflow(peerflow.XminFlowWorkflow) env.RegisterWorkflow(peerflow.QRepPartitionWorkflow) + + alerter, err := alerting.NewAlerter(conn) + if err != nil { + t.Fatalf("unable to create alerter: %v", err) + } + env.RegisterActivity(&activities.FlowableActivity{ CatalogPool: conn, - Alerter: alerting.NewAlerter(conn), + Alerter: alerter, }) env.RegisterActivity(&activities.SnapshotActivity{}) } diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 27d2bb5c04..22b8ad06c4 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -408,6 +408,7 @@ type FlowConnectionConfigs struct { Resync bool `protobuf:"varint,23,opt,name=resync,proto3" json:"resync,omitempty"` SoftDeleteColName string `protobuf:"bytes,24,opt,name=soft_delete_col_name,json=softDeleteColName,proto3" json:"soft_delete_col_name,omitempty"` SyncedAtColName string `protobuf:"bytes,25,opt,name=synced_at_col_name,json=syncedAtColName,proto3" json:"synced_at_col_name,omitempty"` + InitialCopyOnly bool `protobuf:"varint,26,opt,name=initial_copy_only,json=initialCopyOnly,proto3" json:"initial_copy_only,omitempty"` } func (x *FlowConnectionConfigs) Reset() { @@ -617,6 +618,13 @@ func (x *FlowConnectionConfigs) GetSyncedAtColName() string { return "" } +func (x *FlowConnectionConfigs) GetInitialCopyOnly() bool { + if x != nil { + return x.InitialCopyOnly + } + return false +} + type RenameTableOption struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3463,7 +3471,7 @@ var file_flow_proto_rawDesc = []byte{ 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x22, - 0xc4, 0x0c, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0xf0, 0x0c, 0x0a, 0x15, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x2a, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x06, 0x73, @@ -3552,249 +3560,153 @@ var file_flow_proto_rawDesc = []byte{ 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x19, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, - 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, - 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8e, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, 0x6d, - 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, - 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0b, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6e, 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xc9, 0x02, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, - 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, - 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x26, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, - 0x65, 0x65, 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x50, 0x0a, 0x14, 0x72, 0x65, 0x6e, - 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, 0x0a, 0x14, 0x73, - 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, 0x73, 0x6f, 0x66, - 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, - 0x01, 0x12, 0x30, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, - 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, - 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, - 0x88, 0x01, 0x01, 0x42, 0x17, 0x0a, 0x15, 0x5f, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x15, 0x0a, 0x13, - 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x22, 0x38, 0x0a, 0x12, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, + 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, + 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x1a, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, + 0x6e, 0x6c, 0x79, 0x1a, 0x48, 0x0a, 0x1a, 0x53, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, + 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0x8e, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x72, 0x72, + 0x65, 0x6e, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6e, + 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6e, + 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3b, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x22, 0xc9, 0x02, 0x0a, 0x11, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x73, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc7, 0x02, - 0x0a, 0x1d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, - 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, - 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x8b, 0x01, 0x0a, 0x1d, - 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, - 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, - 0x4e, 0x65, 0x77, 0x54, 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x19, - 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x4c, 0x0a, 0x1e, 0x4e, 0x65, 0x77, - 0x54, 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x44, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x8d, 0x02, - 0x0a, 0x0f, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, - 0x12, 0x72, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, - 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, - 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x35, 0x0a, - 0x14, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x53, 0x69, 0x7a, 0x65, 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, - 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, - 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, - 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, - 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x22, 0xd6, 0x03, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, - 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, - 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, - 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, - 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, - 0x6e, 0x63, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x71, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x16, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x71, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, - 0x7a, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, - 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x73, 0x22, 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, - 0x79, 0x6e, 0x63, 0x65, 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, - 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, - 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, - 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, - 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, - 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, - 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, - 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, - 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, + 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, + 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, - 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, + 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x50, 0x0a, 0x14, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, + 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x30, 0x0a, + 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0f, 0x73, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, + 0x17, 0x0a, 0x15, 0x5f, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, + 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x73, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, + 0x38, 0x0a, 0x12, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, + 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, + 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc7, 0x02, 0x0a, 0x1d, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x26, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x8b, 0x01, 0x0a, 0x1d, 0x6e, 0x65, 0x77, 0x5f, + 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x49, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x4e, 0x65, 0x77, 0x54, + 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x19, 0x6e, 0x65, 0x77, 0x54, + 0x6f, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x4c, 0x0a, 0x1e, 0x4e, 0x65, 0x77, 0x54, 0x6f, 0x45, 0x78, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x44, 0x0a, 0x1e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x73, 0x46, 0x72, 0x6f, 0x6d, 0x45, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, - 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x73, 0x22, 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, - 0x0a, 0x06, 0x72, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, - 0x72, 0x65, 0x6c, 0x49, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, - 0x74, 0x67, 0x72, 0x65, 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, - 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x48, 0x00, 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, - 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, - 0x72, 0x22, 0x62, 0x0a, 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, - 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x66, 0x69, 0x65, 0x72, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, - 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, - 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x9a, 0x04, 0x0a, 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, - 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, - 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, - 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, - 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, - 0x26, 0x0a, 0x0f, 0x64, 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, - 0x70, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, - 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, 0x69, 0x73, - 0x74, 0x69, 0x6e, 0x67, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, - 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, 0x78, 0x69, - 0x73, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x53, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, - 0x16, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, - 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, - 0x70, 0x73, 0x68, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x8d, 0x02, 0x0a, 0x0f, 0x53, 0x79, + 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x72, 0x0a, 0x18, + 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x79, 0x6e, + 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x52, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, + 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x52, + 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x35, 0x0a, 0x14, 0x4e, 0x6f, 0x72, + 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, + 0x22, 0x71, 0x0a, 0x0d, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, + 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, + 0x5f, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x41, 0x74, 0x22, 0xd6, 0x03, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, + 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x73, + 0x79, 0x6e, 0x63, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x4c, 0x61, + 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x6c, 0x61, 0x73, + 0x74, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, + 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, + 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x48, 0x0a, 0x11, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, + 0x53, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x46, 0x6c, 0x6f, 0x77, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x12, 0x71, 0x0a, 0x18, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, + 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x72, 0x65, 0x6c, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x71, 0x0a, 0x13, + 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x22, + 0x84, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4c, 0x61, 0x73, 0x74, 0x53, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x49, 0x44, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, + 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, + 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, + 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xbe, 0x01, 0x0a, 0x16, 0x45, 0x6e, 0x73, 0x75, 0x72, + 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, @@ -3802,343 +3714,442 @@ var file_flow_proto_rawDesc = []byte{ 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x64, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, - 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, - 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, - 0x4d, 0x6f, 0x64, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x36, 0x0a, 0x17, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xc5, 0x01, 0x0a, 0x1b, 0x45, 0x6e, 0x73, 0x75, + 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, + 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x16, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, + 0x30, 0x0a, 0x17, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x15, 0x0a, 0x06, 0x72, 0x65, + 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x72, 0x65, 0x6c, 0x49, + 0x64, 0x22, 0x89, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x62, 0x0a, 0x19, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, + 0x73, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x48, 0x00, + 0x52, 0x17, 0x70, 0x6f, 0x73, 0x74, 0x67, 0x72, 0x65, 0x73, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x42, 0x12, 0x0a, 0x10, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x62, 0x0a, + 0x17, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, 0x6c, 0x69, + 0x74, 0x79, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x47, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, + 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x22, 0x88, 0x02, 0x0a, 0x1c, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, + 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, + 0x75, 0x74, 0x12, 0x7f, 0x0a, 0x18, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x50, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x69, + 0x6c, 0x69, 0x74, 0x79, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x1a, 0x67, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x03, 0x6b, 0x65, 0x79, 0x12, 0x32, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9a, 0x04, 0x0a, + 0x15, 0x53, 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x66, 0x0a, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x38, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, + 0x65, 0x74, 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x3d, 0x0a, 0x10, + 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x26, 0x0a, 0x0f, 0x64, + 0x6f, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x6f, 0x49, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, + 0x6f, 0x70, 0x79, 0x12, 0x3a, 0x0a, 0x19, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, + 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, + 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x43, 0x0a, 0x1e, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1b, 0x65, 0x78, 0x69, 0x73, 0x74, 0x69, 0x6e, + 0x67, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6c, 0x6f, 0x74, + 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x9e, 0x02, 0x0a, - 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x69, 0x6d, - 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, - 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x72, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, - 0x66, 0x75, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x73, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x46, 0x75, 0x6c, - 0x6c, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, - 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, - 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, - 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, - 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, - 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, - 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x16, 0x53, 0x65, 0x74, + 0x75, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x6c, 0x6f, 0x74, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, + 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xed, 0x02, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x52, 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, + 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x64, 0x0a, 0x12, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x61, 0x77, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x12, 0x3d, 0x0a, 0x0d, 0x63, 0x64, 0x63, 0x5f, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, + 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, + 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x63, 0x64, 0x63, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, + 0x1a, 0x43, 0x0a, 0x15, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, + 0x61, 0x77, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, + 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x9e, 0x02, 0x0a, 0x0b, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x12, 0x3f, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, + 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x11, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, 0x66, 0x75, 0x6c, 0x6c, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x73, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x46, 0x75, 0x6c, 0x6c, 0x1a, 0x3a, 0x0a, + 0x0c, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, - 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, - 0x22, 0xb2, 0x03, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, - 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, - 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, - 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, - 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, - 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, - 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, - 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, - 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, - 0x0a, 0x0e, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, - 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, - 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, - 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, - 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, - 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, - 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, - 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, - 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, - 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, - 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, - 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, - 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, - 0x22, 0xe8, 0x01, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x12, 0x4f, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, - 0x65, 0x48, 0x00, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x12, 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, - 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, - 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, - 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, - 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, - 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xf7, 0x07, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, - 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, - 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, + 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x2b, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, + 0x0a, 0x19, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, + 0xda, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, + 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, - 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, - 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, - 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, - 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, - 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, - 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, - 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, - 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, - 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, - 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, - 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, - 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, - 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, - 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, - 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, - 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, - 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, - 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, - 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, - 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, - 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, - 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, - 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, - 0x0a, 0x24, 0x73, 0x65, 0x74, 0x75, 0x70, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x73, 0x65, - 0x74, 0x75, 0x70, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4f, 0x6e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x31, - 0x0a, 0x15, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6c, 0x6c, - 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, - 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x79, 0x6e, - 0x63, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, - 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, - 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2f, + 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x12, 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xb2, 0x03, 0x0a, + 0x1e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, + 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, + 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, + 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, + 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, - 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, - 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, - 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, - 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, - 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, - 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, - 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, - 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, - 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, - 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, - 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, - 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0xa2, 0x01, 0x0a, - 0x10, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, - 0x61, 0x12, 0x24, 0x0a, 0x0e, 0x73, 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x72, 0x63, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x73, 0x74, 0x5f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0c, 0x64, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, - 0x0d, 0x61, 0x64, 0x64, 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, - 0x6f, 0x77, 0x2e, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x52, 0x0c, 0x61, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x22, 0xc8, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x79, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x4d, 0x0a, - 0x13, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x64, 0x65, - 0x6c, 0x74, 0x61, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x22, 0xe9, 0x01, 0x0a, - 0x0d, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x41, - 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x38, 0x0a, 0x18, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x16, 0x6e, 0x75, 0x6d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6e, - 0x65, 0x65, 0x64, 0x73, 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0b, 0x6e, 0x65, 0x65, 0x64, 0x73, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x38, - 0x0a, 0x19, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x66, - 0x6f, 0x72, 0x5f, 0x6e, 0x65, 0x77, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x15, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, - 0x72, 0x4e, 0x65, 0x77, 0x52, 0x6f, 0x77, 0x73, 0x22, 0x8e, 0x01, 0x0a, 0x0d, 0x50, 0x65, 0x65, - 0x72, 0x44, 0x42, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, + 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, + 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x63, 0x0a, 0x1b, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, + 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, + 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, + 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, + 0x73, 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, + 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, + 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, + 0x17, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, + 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, + 0x64, 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, + 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, + 0x54, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, + 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x12, 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, + 0x44, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, + 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xe8, 0x01, 0x0a, + 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x3d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x4f, + 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, + 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x3d, 0x0a, 0x09, 0x74, 0x69, 0x64, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x54, 0x49, 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x08, 0x74, 0x69, 0x64, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, + 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, + 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, + 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, + 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, + 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x10, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x22, 0xf7, 0x07, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x12, 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, + 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, + 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, + 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, + 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, + 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, + 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, + 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, + 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, + 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, + 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, + 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, + 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, + 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, + 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, + 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, + 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, + 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, + 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, + 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, + 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, 0x24, 0x73, 0x65, + 0x74, 0x75, 0x70, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x73, 0x65, 0x74, 0x75, 0x70, 0x57, + 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x6e, 0x44, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x31, 0x0a, 0x15, 0x64, 0x73, + 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x72, 0x65, 0x73, + 0x79, 0x6e, 0x63, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x73, 0x74, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x2b, 0x0a, + 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, - 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, - 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, - 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, - 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x22, 0x78, 0x0a, 0x1f, 0x47, 0x65, 0x74, - 0x4f, 0x70, 0x65, 0x6e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x46, - 0x6f, 0x72, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1b, 0x0a, 0x09, - 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x75, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x63, 0x75, 0x72, - 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x70, 0x65, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x63, 0x75, 0x72, - 0x72, 0x65, 0x6e, 0x74, 0x4f, 0x70, 0x65, 0x6e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x2a, 0x50, 0x0a, 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, - 0x6f, 0x64, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, - 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, - 0x52, 0x54, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, - 0x43, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, - 0x56, 0x52, 0x4f, 0x10, 0x01, 0x2a, 0x66, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, - 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, - 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, - 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, - 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x12, 0x1d, - 0x0a, 0x19, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, - 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x02, 0x42, 0x76, 0x0a, - 0x0f, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x42, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, - 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, - 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, - 0x6f, 0x77, 0xca, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, - 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x0d, + 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, + 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, + 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, + 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, + 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x24, 0x0a, + 0x0e, 0x73, 0x72, 0x63, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x73, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x61, 0x64, 0x64, + 0x65, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, + 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, + 0x0c, 0x61, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xc8, 0x01, + 0x0a, 0x1b, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, + 0x17, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, + 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x73, 0x52, 0x15, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x12, 0x4d, 0x0a, 0x13, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, + 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x44, 0x65, 0x6c, 0x74, 0x61, 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x22, 0xe9, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, + 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x41, 0x0a, 0x0e, 0x6c, 0x61, + 0x73, 0x74, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, + 0x6c, 0x61, 0x73, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x38, 0x0a, + 0x18, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, + 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x16, 0x6e, 0x75, 0x6d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x50, 0x72, + 0x6f, 0x63, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x73, + 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x6e, + 0x65, 0x65, 0x64, 0x73, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x38, 0x0a, 0x19, 0x64, 0x69, + 0x73, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x6e, + 0x65, 0x77, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x64, + 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x4e, 0x65, 0x77, + 0x52, 0x6f, 0x77, 0x73, 0x22, 0x8e, 0x01, 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, 0x44, 0x42, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, + 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x73, 0x6f, 0x66, 0x74, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x22, 0x78, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x6e, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x46, 0x6f, 0x72, 0x55, 0x73, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x75, 0x73, 0x65, 0x72, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x75, 0x73, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, + 0x5f, 0x6f, 0x70, 0x65, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x16, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, + 0x4f, 0x70, 0x65, 0x6e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2a, + 0x50, 0x0a, 0x0c, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, + 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x4d, 0x55, 0x4c, 0x54, 0x49, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x00, + 0x12, 0x1f, 0x0a, 0x1b, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x5f, 0x4d, 0x4f, + 0x44, 0x45, 0x5f, 0x53, 0x54, 0x4f, 0x52, 0x41, 0x47, 0x45, 0x5f, 0x41, 0x56, 0x52, 0x4f, 0x10, + 0x01, 0x2a, 0x66, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, + 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x00, 0x12, 0x1a, + 0x0a, 0x16, 0x51, 0x52, 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, + 0x45, 0x5f, 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x51, 0x52, + 0x45, 0x50, 0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x56, + 0x45, 0x52, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x02, 0x42, 0x76, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x09, 0x46, 0x6c, + 0x6f, 0x77, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x10, 0x67, 0x65, 0x6e, 0x65, 0x72, + 0x61, 0x74, 0x65, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x73, 0xa2, 0x02, 0x03, 0x50, 0x58, + 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xca, 0x02, + 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0xe2, 0x02, 0x16, 0x50, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, 0x77, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x64, 0x62, 0x46, 0x6c, 0x6f, + 0x77, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/flow/go.mod b/flow/go.mod index dd59212b71..5cd582a51c 100644 --- a/flow/go.mod +++ b/flow/go.mod @@ -9,7 +9,7 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.4.0 github.com/Azure/azure-sdk-for-go/sdk/messaging/azeventhubs v1.0.2 github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/eventhub/armeventhub v1.2.0 - github.com/aws/aws-sdk-go v1.49.5 + github.com/aws/aws-sdk-go v1.49.8 github.com/cenkalti/backoff/v4 v4.2.1 github.com/cockroachdb/pebble v0.0.0-20231210175914-b4d301aeb46a github.com/google/uuid v1.5.0 @@ -39,7 +39,7 @@ require ( google.golang.org/api v0.154.0 google.golang.org/genproto/googleapis/api v0.0.0-20231212172506-995d672761c0 google.golang.org/grpc v1.60.1 - google.golang.org/protobuf v1.31.0 + google.golang.org/protobuf v1.32.0 ) require ( @@ -52,7 +52,7 @@ require ( github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/getsentry/sentry-go v0.25.0 // indirect - github.com/go-logr/logr v1.3.0 // indirect + github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/gorilla/websocket v1.5.1 // indirect github.com/kr/pretty v0.3.1 // indirect @@ -90,8 +90,8 @@ require ( github.com/apache/thrift v0.19.0 // indirect github.com/aws/aws-sdk-go-v2 v1.24.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 // indirect - github.com/aws/aws-sdk-go-v2/credentials v1.16.12 // indirect - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.8 // indirect + github.com/aws/aws-sdk-go-v2/credentials v1.16.13 // indirect + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.9 // indirect github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 // indirect github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.9 // indirect @@ -99,12 +99,12 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.9 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.9 // indirect - github.com/aws/aws-sdk-go-v2/service/s3 v1.47.6 // indirect + github.com/aws/aws-sdk-go-v2/service/s3 v1.47.7 // indirect github.com/aws/smithy-go v1.19.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.3 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.1 // indirect - github.com/dvsekhvalnov/jose2go v1.5.0 // indirect + github.com/dvsekhvalnov/jose2go v1.6.0 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/form3tech-oss/jwt-go v3.2.5+incompatible // indirect github.com/gabriel-vasile/mimetype v1.4.3 // indirect @@ -150,7 +150,7 @@ require ( github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect golang.org/x/crypto v0.17.0 - golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611 + golang.org/x/exp v0.0.0-20231219180239-dc181d75b848 golang.org/x/mod v0.14.0 // indirect golang.org/x/net v0.19.0 // indirect golang.org/x/oauth2 v0.15.0 // indirect diff --git a/flow/go.sum b/flow/go.sum index e9a2d2b5eb..714e5dfd3f 100644 --- a/flow/go.sum +++ b/flow/go.sum @@ -62,20 +62,20 @@ github.com/apache/arrow/go/v12 v12.0.1 h1:JsR2+hzYYjgSUkBSaahpqCetqZMr76djX80fF/ github.com/apache/arrow/go/v12 v12.0.1/go.mod h1:weuTY7JvTG/HDPtMQxEUp7pU73vkLWMLpY67QwZ/WWw= github.com/apache/thrift v0.19.0 h1:sOqkWPzMj7w6XaYbJQG7m4sGqVolaW/0D28Ln7yPzMk= github.com/apache/thrift v0.19.0/go.mod h1:SUALL216IiaOw2Oy+5Vs9lboJ/t9g40C+G07Dc0QC1I= -github.com/aws/aws-sdk-go v1.49.5 h1:y2yfBlwjPDi3/sBVKeznYEdDy6wIhjA2L5NCBMLUIYA= -github.com/aws/aws-sdk-go v1.49.5/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.49.8 h1:gKgEiyJ8CPnr4r6pS06WfNXvp6z34JER1pBIwuocvVA= +github.com/aws/aws-sdk-go v1.49.8/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/aws/aws-sdk-go-v2 v1.24.0 h1:890+mqQ+hTpNuw0gGP6/4akolQkSToDJgHfQE7AwGuk= github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4 h1:OCs21ST2LrepDfD3lwlQiOqIGp6JiEUqG84GzTDoyJs= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.4/go.mod h1:usURWEKSNNAcAZuzRn/9ZYPT8aZQkR7xcCtunK/LkJo= -github.com/aws/aws-sdk-go-v2/config v1.26.1 h1:z6DqMxclFGL3Zfo+4Q0rLnAZ6yVkzCRxhRMsiRQnD1o= -github.com/aws/aws-sdk-go-v2/config v1.26.1/go.mod h1:ZB+CuKHRbb5v5F0oJtGdhFTelmrxd4iWO1lf0rQwSAg= -github.com/aws/aws-sdk-go-v2/credentials v1.16.12 h1:v/WgB8NxprNvr5inKIiVVrXPuuTegM+K8nncFkr1usU= -github.com/aws/aws-sdk-go-v2/credentials v1.16.12/go.mod h1:X21k0FjEJe+/pauud82HYiQbEr9jRKY3kXEIQ4hXeTQ= +github.com/aws/aws-sdk-go-v2/config v1.26.2 h1:+RWLEIWQIGgrz2pBPAUoGgNGs1TOyF4Hml7hCnYj2jc= +github.com/aws/aws-sdk-go-v2/config v1.26.2/go.mod h1:l6xqvUxt0Oj7PI/SUXYLNyZ9T/yBPn3YTQcJLLOdtR8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.13 h1:WLABQ4Cp4vXtXfOWOS3MEZKr6AAYUpMczLhgKtAjQ/8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.13/go.mod h1:Qg6x82FXwW0sJHzYruxGiuApNo31UEtJvXVSZAXeWiw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.8 h1:7wCngExMTAW2Bjf0Y92uWap6ZUcenLLWI5T3VJiQneU= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.8/go.mod h1:XVrAWYYM4ZRwOCOuLoUiao5hbLqNutEdqwCR3ZvkXgc= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.9 h1:5zA8qVCXMPGt6YneFnll5B157SfdK2SewU85PH9/yM0= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.15.9/go.mod h1:t4gy210hPxkbtYM8xOzrWdxVq1PyekR76OOKXy3s0Vs= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 h1:v+HbZaCGmOwnTTVS86Fleq0vPzOd7tnJGbFhP0stNLs= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9/go.mod h1:Xjqy+Nyj7VDLBtCMkQYOw1QYfAEZCVLrfI0ezve8wd4= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 h1:N94sVhRACtXyVcjXxrwK1SKFIJrA9pOJ5yu2eSHnmls= @@ -92,14 +92,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 h1:Nf2sHxjMJ github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9/go.mod h1:idky4TER38YIjr2cADF1/ugFMKvZV7p//pVeV5LZbF0= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.9 h1:iEAeF6YC3l4FzlJPP9H3Ko1TXpdjdqWffxXjp8SY6uk= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.9/go.mod h1:kjsXoK23q9Z/tLBrckZLLyvjhZoS+AGrzqzUfEClvMM= -github.com/aws/aws-sdk-go-v2/service/s3 v1.47.6 h1:bkmlzokzTJyrFNA0J+EPlsF8x4/wp+9D45HTHO/ZUiY= -github.com/aws/aws-sdk-go-v2/service/s3 v1.47.6/go.mod h1:vADO6Jn+Rq4nDtfwNjhgR84qkZwiC6FqCaXdw/kYwjA= +github.com/aws/aws-sdk-go-v2/service/s3 v1.47.7 h1:o0ASbVwUAIrfp/WcCac+6jioZt4Hd8k/1X8u7GJ/QeM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.47.7/go.mod h1:vADO6Jn+Rq4nDtfwNjhgR84qkZwiC6FqCaXdw/kYwjA= github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 h1:ldSFWz9tEHAwHNmjx2Cvy1MjP5/L9kNoR0skc6wyOOM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.5/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 h1:2k9KmFawS63euAkY4/ixVNsYYwrwnd5fIvgEKkfZFNM= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.5 h1:5UYvv8JUvllZsRnfrcMQ+hJ9jNICmcgKPAO1CER25Wg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.5/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 h1:HJeiuZ2fldpd0WqngyMR6KW7ofkXNLyOaHwEIGm39Cs= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.6/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -138,8 +138,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= -github.com/dvsekhvalnov/jose2go v1.5.0 h1:3j8ya4Z4kMCwT5nXIKFSV84YS+HdqSSO0VsTQxaLAeM= -github.com/dvsekhvalnov/jose2go v1.5.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= +github.com/dvsekhvalnov/jose2go v1.6.0 h1:Y9gnSnP4qEI0+/uQkHvFXeD2PLPJeXEL+ySMEA2EjTY= +github.com/dvsekhvalnov/jose2go v1.6.0/go.mod h1:QsHjhyTlD/lAVqn/NSbVZmSCGeDehTB/mPZadG+mhXU= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -164,8 +164,8 @@ github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3Bop github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY= -github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= +github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= @@ -434,8 +434,8 @@ golang.org/x/crypto v0.0.0-20220829220503-c86fa9a7ed90/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611 h1:qCEDpW1G+vcj3Y7Fy52pEM1AWm3abj8WimGYejI3SC4= -golang.org/x/exp v0.0.0-20231214170342-aacd6d4b4611/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= +golang.org/x/exp v0.0.0-20231219180239-dc181d75b848 h1:+iq7lrkxmFNBM7xx+Rae2W6uyPfhPeDWD+n+JgppptE= +golang.org/x/exp v0.0.0-20231219180239-dc181d75b848/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -569,8 +569,8 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/flow/model/model.go b/flow/model/model.go index 6bbd32e65d..5ffeb37f61 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -329,7 +329,7 @@ type CDCRecordStream struct { } func NewCDCRecordStream() *CDCRecordStream { - channelBuffer := peerdbenv.GetPeerDBCDCChannelBufferSize() + channelBuffer := peerdbenv.PeerDBCDCChannelBufferSize() return &CDCRecordStream{ records: make(chan Record, channelBuffer), // TODO (kaushik): more than 1024 schema deltas can cause problems! diff --git a/flow/peerdbenv/config.go b/flow/peerdbenv/config.go index 970be3455d..cdefa6a376 100644 --- a/flow/peerdbenv/config.go +++ b/flow/peerdbenv/config.go @@ -9,74 +9,79 @@ import ( // throughout the codebase. // PEERDB_VERSION_SHA_SHORT -func GetPeerDBVersionShaShort() string { +func PeerDBVersionShaShort() string { return getEnvString("PEERDB_VERSION_SHA_SHORT", "unknown") } // PEERDB_DEPLOYMENT_UID -func GetPeerDBDeploymentUID() string { +func PeerDBDeploymentUID() string { return getEnvString("PEERDB_DEPLOYMENT_UID", "") } // PEERDB_CDC_CHANNEL_BUFFER_SIZE -func GetPeerDBCDCChannelBufferSize() int { +func PeerDBCDCChannelBufferSize() int { return getEnvInt("PEERDB_CDC_CHANNEL_BUFFER_SIZE", 1<<18) } // PEERDB_EVENTHUB_FLUSH_TIMEOUT_SECONDS -func GetPeerDBEventhubFlushTimeoutSeconds() time.Duration { +func PeerDBEventhubFlushTimeoutSeconds() time.Duration { x := getEnvInt("PEERDB_EVENTHUB_FLUSH_TIMEOUT_SECONDS", 10) return time.Duration(x) * time.Second } // PEERDB_CDC_IDLE_TIMEOUT_SECONDS -func GetPeerDBCDCIdleTimeoutSeconds() time.Duration { +func PeerDBCDCIdleTimeoutSeconds() time.Duration { x := getEnvInt("PEERDB_CDC_IDLE_TIMEOUT_SECONDS", 60) return time.Duration(x) * time.Second } // PEERDB_CDC_DISK_SPILL_THRESHOLD -func GetPeerDBCDCDiskSpillThreshold() int { +func PeerDBCDCDiskSpillThreshold() int { return getEnvInt("PEERDB_CDC_DISK_SPILL_THRESHOLD", 1_000_000) } // PEERDB_CATALOG_HOST -func GetPeerDBCatalogHost() string { +func PeerDBCatalogHost() string { return getEnvString("PEERDB_CATALOG_HOST", "") } // PEERDB_CATALOG_PORT -func GetPeerDBCatalogPort() uint32 { +func PeerDBCatalogPort() uint32 { return getEnvUint32("PEERDB_CATALOG_PORT", 5432) } // PEERDB_CATALOG_USER -func GetPeerDBCatalogUser() string { +func PeerDBCatalogUser() string { return getEnvString("PEERDB_CATALOG_USER", "") } // PEERDB_CATALOG_PASSWORD -func GetPeerDBCatalogPassword() string { +func PeerDBCatalogPassword() string { return getEnvString("PEERDB_CATALOG_PASSWORD", "") } // PEERDB_CATALOG_DATABASE -func GetPeerDBCatalogDatabase() string { +func PeerDBCatalogDatabase() string { return getEnvString("PEERDB_CATALOG_DATABASE", "") } // PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD, 0 disables slot lag alerting entirely -func GetPeerDBSlotLagMBAlertThreshold() uint32 { +func PeerDBSlotLagMBAlertThreshold() uint32 { return getEnvUint32("PEERDB_SLOT_LAG_MB_ALERT_THRESHOLD", 5000) } // PEERDB_ALERTING_GAP_MINUTES, 0 disables all alerting entirely -func GetPeerDBAlertingGapMinutesAsDuration() time.Duration { +func PeerDBAlertingGapMinutesAsDuration() time.Duration { why := time.Duration(getEnvUint32("PEERDB_ALERTING_GAP_MINUTES", 15)) return why * time.Minute } // PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD, 0 disables open connections alerting entirely -func GetPeerDBOpenConnectionsAlertThreshold() uint32 { +func PeerDBOpenConnectionsAlertThreshold() uint32 { return getEnvUint32("PEERDB_PGPEER_OPEN_CONNECTIONS_ALERT_THRESHOLD", 5) } + +// PEERDB_ENABLE_WAL_HEARTBEAT +func PeerDBEnableWALHeartbeat() bool { + return getEnvBool("PEERDB_ENABLE_WAL_HEARTBEAT", false) +} diff --git a/flow/peerdbenv/env.go b/flow/peerdbenv/env.go index 11e363d1eb..3bba77c46d 100644 --- a/flow/peerdbenv/env.go +++ b/flow/peerdbenv/env.go @@ -46,6 +46,23 @@ func getEnvUint32(name string, defaultValue uint32) uint32 { return uint32(i) } +// getEnvBool returns the value of the environment variable with the given name +// or defaultValue if the environment variable is not set or is not a valid +// boolean value. +func getEnvBool(name string, defaultValue bool) bool { + val, ok := getEnv(name) + if !ok { + return defaultValue + } + + b, err := strconv.ParseBool(val) + if err != nil { + return defaultValue + } + + return b +} + // GetEnvString returns the value of the environment variable with the given name // or defaultValue if the environment variable is not set. func getEnvString(name string, defaultValue string) string { diff --git a/flow/shared/alerting/alerting.go b/flow/shared/alerting/alerting.go index 7dc3cb489f..50608087d2 100644 --- a/flow/shared/alerting/alerting.go +++ b/flow/shared/alerting/alerting.go @@ -18,8 +18,8 @@ type Alerter struct { logger *slog.Logger } -func registerSendersFromPool(catalogPool *pgxpool.Pool) ([]*slackAlertSender, error) { - rows, err := catalogPool.Query(context.Background(), +func registerSendersFromPool(ctx context.Context, catalogPool *pgxpool.Pool) ([]*slackAlertSender, error) { + rows, err := catalogPool.Query(ctx, "SELECT service_type,service_config FROM peerdb_stats.alerting_config") if err != nil { return nil, fmt.Errorf("failed to read alerter config from catalog: %w", err) @@ -47,55 +47,79 @@ func registerSendersFromPool(catalogPool *pgxpool.Pool) ([]*slackAlertSender, er } // doesn't take care of closing pool, needs to be done externally. -func NewAlerter(catalogPool *pgxpool.Pool) *Alerter { +func NewAlerter(catalogPool *pgxpool.Pool) (*Alerter, error) { + logger := slog.Default() + if catalogPool == nil { + logger.Error("catalog pool is nil for Alerter") + return nil, fmt.Errorf("catalog pool is nil for Alerter") + } + return &Alerter{ catalogPool: catalogPool, - logger: slog.Default(), - } + logger: logger, + }, nil } // Only raises an alert if another alert with the same key hasn't been raised // in the past X minutes, where X is configurable and defaults to 15 minutes func (a *Alerter) AlertIf(ctx context.Context, alertKey string, alertMessage string) { - if peerdbenv.GetPeerDBAlertingGapMinutesAsDuration() == 0 { + if peerdbenv.PeerDBAlertingGapMinutesAsDuration() == 0 { a.logger.WarnContext(ctx, "Alerting disabled via environment variable, returning") return } - if a.catalogPool != nil { - slackAlertSenders, err := registerSendersFromPool(a.catalogPool) + var err error + row := a.catalogPool.QueryRow(ctx, + `SELECT created_timestamp FROM peerdb_stats.alerts_v1 WHERE alert_key=$1 + ORDER BY created_timestamp DESC LIMIT 1`, + alertKey) + var createdTimestamp time.Time + err = row.Scan(&createdTimestamp) + if err != nil && err != pgx.ErrNoRows { + a.logger.Warn("failed to send alert: %v", err) + return + } + + if time.Since(createdTimestamp) >= peerdbenv.PeerDBAlertingGapMinutesAsDuration() { + a.AddAlertToCatalog(ctx, alertKey, alertMessage) + a.AlertToSlack(ctx, alertKey, alertMessage) + } +} + +func (a *Alerter) AlertToSlack(ctx context.Context, alertKey string, alertMessage string) { + slackAlertSenders, err := registerSendersFromPool(ctx, a.catalogPool) + if err != nil { + a.logger.WarnContext(ctx, "failed to set Slack senders", slog.Any("error", err)) + return + } + + for _, slackAlertSender := range slackAlertSenders { + err = slackAlertSender.sendAlert(ctx, + fmt.Sprintf(":rotating_light:Alert:rotating_light:: %s", alertKey), alertMessage) if err != nil { - a.logger.WarnContext(ctx, "failed to set Slack senders", slog.Any("error", err)) - return - } - if len(slackAlertSenders) == 0 { - a.logger.WarnContext(ctx, "no Slack senders configured, returning") + a.logger.WarnContext(ctx, "failed to send alert", slog.Any("error", err)) return } + } +} - row := a.catalogPool.QueryRow(context.Background(), - `SELECT created_timestamp FROM peerdb_stats.alerts_v1 WHERE alert_key=$1 - ORDER BY created_timestamp DESC LIMIT 1`, - alertKey) - var createdTimestamp time.Time - err = row.Scan(&createdTimestamp) - if err != nil && err != pgx.ErrNoRows { - a.logger.Warn("failed to send alert: %v", err) - return - } +func (a *Alerter) AddAlertToCatalog(ctx context.Context, alertKey string, alertMessage string) { + _, err := a.catalogPool.Exec(ctx, + "INSERT INTO peerdb_stats.alerts_v1(alert_key,alert_message) VALUES($1,$2)", + alertKey, alertMessage) + if err != nil { + a.logger.WarnContext(ctx, "failed to insert alert", slog.Any("error", err)) + return + } +} - if time.Since(createdTimestamp) >= peerdbenv.GetPeerDBAlertingGapMinutesAsDuration() { - for _, slackAlertSender := range slackAlertSenders { - err = slackAlertSender.sendAlert(context.Background(), - fmt.Sprintf(":rotating_light:Alert:rotating_light:: %s", alertKey), alertMessage) - if err != nil { - a.logger.WarnContext(ctx, "failed to send alert", slog.Any("error", err)) - return - } - _, _ = a.catalogPool.Exec(context.Background(), - "INSERT INTO peerdb_stats.alerts_v1(alert_key,alert_message) VALUES($1,$2)", - alertKey, alertMessage) - } - } +func (a *Alerter) LogFlowError(ctx context.Context, flowName string, err error) { + errorWithStack := fmt.Sprintf("%+v", err) + _, err = a.catalogPool.Exec(ctx, + "INSERT INTO peerdb_stats.flow_errors(flow_name,error_message,error_type) VALUES($1,$2,$3)", + flowName, errorWithStack, "error") + if err != nil { + a.logger.WarnContext(ctx, "failed to insert flow error", slog.Any("error", err)) + return } } diff --git a/flow/shared/constants.go b/flow/shared/constants.go index 8589b55487..aa447520b5 100644 --- a/flow/shared/constants.go +++ b/flow/shared/constants.go @@ -51,7 +51,7 @@ func GetPeerFlowTaskQueueName(taskQueueID TaskQueueID) (string, error) { } func prependUIDToTaskQueueName(taskQueueName string) string { - deploymentUID := peerdbenv.GetPeerDBDeploymentUID() + deploymentUID := peerdbenv.PeerDBDeploymentUID() if deploymentUID == "" { return taskQueueName } diff --git a/flow/workflows/cdc_flow.go b/flow/workflows/cdc_flow.go index 51ec801921..62a26ae42c 100644 --- a/flow/workflows/cdc_flow.go +++ b/flow/workflows/cdc_flow.go @@ -275,6 +275,10 @@ func CDCFlowWorkflowWithConfig( state.SnapshotComplete = true state.Progress = append(state.Progress, "executed setup flow and snapshot flow") + // if initial_copy_only is opted for, we end the flow here. + if cfg.InitialCopyOnly { + return nil, nil + } } syncFlowOptions := &protos.SyncFlowOptions{ diff --git a/flow/workflows/sync_flow.go b/flow/workflows/sync_flow.go index eb8778211e..e6e72a9b8e 100644 --- a/flow/workflows/sync_flow.go +++ b/flow/workflows/sync_flow.go @@ -35,7 +35,6 @@ func (s *SyncFlowExecution) executeSyncFlow( ctx workflow.Context, config *protos.FlowConnectionConfigs, opts *protos.SyncFlowOptions, - relationMessageMapping model.RelationMessageMapping, normFlowId string, normFlowRunId string, ) (*model.SyncResponse, error) { @@ -74,7 +73,7 @@ func (s *SyncFlowExecution) executeSyncFlow( FlowConnectionConfigs: config, LastSyncState: dstSyncState, SyncFlowOptions: opts, - RelationMessageMapping: relationMessageMapping, + RelationMessageMapping: opts.RelationMessageMapping, } fStartFlow := workflow.ExecuteActivity(startFlowCtx, flowable.StartFlow, startFlowInput) @@ -127,7 +126,6 @@ func SyncFlowWorkflow(ctx workflow.Context, ctx, config, options, - options.RelationMessageMapping, normFlowId, normFlowRunId, ) diff --git a/nexus/analyzer/src/lib.rs b/nexus/analyzer/src/lib.rs index f65fe4b2d0..8a2a088f1a 100644 --- a/nexus/analyzer/src/lib.rs +++ b/nexus/analyzer/src/lib.rs @@ -306,6 +306,11 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { _ => None, }; + let initial_copy_only = match raw_options.remove("initial_copy_only") { + Some(sqlparser::ast::Value::Boolean(b)) => *b, + _ => false, + }; + let flow_job = FlowJob { name: cdc.mirror_name.to_string().to_lowercase(), source_peer: cdc.source_peer.to_string().to_lowercase(), @@ -327,8 +332,13 @@ impl<'a> StatementAnalyzer for PeerDDLAnalyzer<'a> { resync, soft_delete_col_name, synced_at_col_name, + initial_copy_only, }; + if initial_copy_only && !do_initial_copy { + anyhow::bail!("initial_copy_only is set to true, but do_initial_copy is set to false"); + } + Ok(Some(PeerDDL::CreateMirrorForCDC { if_not_exists: *if_not_exists, flow_job, diff --git a/nexus/catalog/migrations/V17__mirror_errors.sql b/nexus/catalog/migrations/V17__mirror_errors.sql new file mode 100644 index 0000000000..06f2352ea2 --- /dev/null +++ b/nexus/catalog/migrations/V17__mirror_errors.sql @@ -0,0 +1,10 @@ +CREATE TABLE peerdb_stats.flow_errors ( + id BIGINT PRIMARY KEY GENERATED ALWAYS AS IDENTITY, + flow_name TEXT NOT NULL, + error_message TEXT NOT NULL, + error_type TEXT NOT NULL, + error_timestamp TIMESTAMP NOT NULL DEFAULT now(), + ack BOOLEAN NOT NULL DEFAULT FALSE +); + +CREATE INDEX idx_flow_errors_flow_name ON peerdb_stats.flow_errors (flow_name); diff --git a/nexus/flow-rs/src/grpc.rs b/nexus/flow-rs/src/grpc.rs index 7b11a6a2dd..7a051e3d23 100644 --- a/nexus/flow-rs/src/grpc.rs +++ b/nexus/flow-rs/src/grpc.rs @@ -195,6 +195,7 @@ impl FlowGrpcClient { resync: job.resync, soft_delete_col_name: job.soft_delete_col_name.clone().unwrap_or_default(), synced_at_col_name: job.synced_at_col_name.clone().unwrap_or_default(), + initial_copy_only: job.initial_copy_only, ..Default::default() }; diff --git a/nexus/pt/src/flow_model.rs b/nexus/pt/src/flow_model.rs index 5c8b1e3857..9a39485e20 100644 --- a/nexus/pt/src/flow_model.rs +++ b/nexus/pt/src/flow_model.rs @@ -70,6 +70,7 @@ pub struct FlowJob { pub resync: bool, pub soft_delete_col_name: Option, pub synced_at_col_name: Option, + pub initial_copy_only: bool, } #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index a58fcbb95d..86ec33a2b0 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -100,6 +100,8 @@ pub struct FlowConnectionConfigs { pub soft_delete_col_name: ::prost::alloc::string::String, #[prost(string, tag="25")] pub synced_at_col_name: ::prost::alloc::string::String, + #[prost(bool, tag="26")] + pub initial_copy_only: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 0e079f82fa..a5a601c807 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -1229,6 +1229,9 @@ impl serde::Serialize for FlowConnectionConfigs { if !self.synced_at_col_name.is_empty() { len += 1; } + if self.initial_copy_only { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.FlowConnectionConfigs", len)?; if let Some(v) = self.source.as_ref() { struct_ser.serialize_field("source", v)?; @@ -1309,6 +1312,9 @@ impl serde::Serialize for FlowConnectionConfigs { if !self.synced_at_col_name.is_empty() { struct_ser.serialize_field("syncedAtColName", &self.synced_at_col_name)?; } + if self.initial_copy_only { + struct_ser.serialize_field("initialCopyOnly", &self.initial_copy_only)?; + } struct_ser.end() } } @@ -1366,6 +1372,8 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "softDeleteColName", "synced_at_col_name", "syncedAtColName", + "initial_copy_only", + "initialCopyOnly", ]; #[allow(clippy::enum_variant_names)] @@ -1395,6 +1403,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { Resync, SoftDeleteColName, SyncedAtColName, + InitialCopyOnly, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -1442,6 +1451,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { "resync" => Ok(GeneratedField::Resync), "softDeleteColName" | "soft_delete_col_name" => Ok(GeneratedField::SoftDeleteColName), "syncedAtColName" | "synced_at_col_name" => Ok(GeneratedField::SyncedAtColName), + "initialCopyOnly" | "initial_copy_only" => Ok(GeneratedField::InitialCopyOnly), _ => Ok(GeneratedField::__SkipField__), } } @@ -1486,6 +1496,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { let mut resync__ = None; let mut soft_delete_col_name__ = None; let mut synced_at_col_name__ = None; + let mut initial_copy_only__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::Source => { @@ -1655,6 +1666,12 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { } synced_at_col_name__ = Some(map.next_value()?); } + GeneratedField::InitialCopyOnly => { + if initial_copy_only__.is_some() { + return Err(serde::de::Error::duplicate_field("initialCopyOnly")); + } + initial_copy_only__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -1686,6 +1703,7 @@ impl<'de> serde::Deserialize<'de> for FlowConnectionConfigs { resync: resync__.unwrap_or_default(), soft_delete_col_name: soft_delete_col_name__.unwrap_or_default(), synced_at_col_name: synced_at_col_name__.unwrap_or_default(), + initial_copy_only: initial_copy_only__.unwrap_or_default(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index 430968bcba..cd09ddf8cc 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -71,6 +71,8 @@ message FlowConnectionConfigs { string soft_delete_col_name = 24; string synced_at_col_name = 25; + + bool initial_copy_only = 26; } message RenameTableOption { diff --git a/ui/app/api/mirrors/alerts/route.ts b/ui/app/api/mirrors/alerts/route.ts new file mode 100644 index 0000000000..ecb9891cbd --- /dev/null +++ b/ui/app/api/mirrors/alerts/route.ts @@ -0,0 +1,21 @@ +import prisma from '@/app/utils/prisma'; + +export const dynamic = 'force-dynamic'; + +export async function POST(request: Request) { + const { flowName } = await request.json(); + const errCount = await prisma.flow_errors.count({ + where: { + flow_name: flowName, + error_type: 'error', + ack: false, + }, + }); + let mirrorStatus: 'healthy' | 'failed'; + if (errCount > 0) { + mirrorStatus = 'failed'; + } else { + mirrorStatus = 'healthy'; + } + return new Response(JSON.stringify(mirrorStatus)); +} diff --git a/ui/app/dto/MirrorsDTO.ts b/ui/app/dto/MirrorsDTO.ts index 4a76200fd4..977f7f353c 100644 --- a/ui/app/dto/MirrorsDTO.ts +++ b/ui/app/dto/MirrorsDTO.ts @@ -28,3 +28,12 @@ export type SyncStatusRow = { endTime: Date | null; numRows: number; }; + +export type AlertErr = { + id: bigint; + flow_name: string; + error_message: string; + error_type: string; + error_timestamp: Date; + ack: boolean; +}; diff --git a/ui/app/mirrors/create/handlers.ts b/ui/app/mirrors/create/handlers.ts index 7c0b28ae07..039766171d 100644 --- a/ui/app/mirrors/create/handlers.ts +++ b/ui/app/mirrors/create/handlers.ts @@ -170,6 +170,15 @@ export const handleCreateCDC = async ( config.cdcSyncMode = QRepSyncMode.QREP_SYNC_MODE_STORAGE_AVRO; config.snapshotSyncMode = QRepSyncMode.QREP_SYNC_MODE_STORAGE_AVRO; } + + if (config.doInitialCopy == false && config.initialCopyOnly == true) { + setMsg({ + ok: false, + msg: 'Initial Copy Only cannot be true if Initial Copy is false.', + }); + return; + } + setLoading(true); const statusMessage: UCreateMirrorResponse = await fetch('/api/mirrors/cdc', { method: 'POST', diff --git a/ui/app/mirrors/create/helpers/cdc.ts b/ui/app/mirrors/create/helpers/cdc.ts index d24e9bd9f4..88d39a6dc8 100644 --- a/ui/app/mirrors/create/helpers/cdc.ts +++ b/ui/app/mirrors/create/helpers/cdc.ts @@ -103,4 +103,15 @@ export const cdcSettings: MirrorSetting[] = [ default: 'SQL', type: 'switch', }, + { + label: 'Initial Copy Only', + stateHandler: (value, setter) => + setter((curr: CDCConfig) => ({ + ...curr, + initialCopyOnly: (value as boolean) || false, + })), + tips: 'If set, PeerDB will only perform initial load and will not perform CDC sync.', + type: 'switch', + advanced: true, + }, ]; diff --git a/ui/app/mirrors/create/helpers/common.ts b/ui/app/mirrors/create/helpers/common.ts index 063588b2c1..15f0607920 100644 --- a/ui/app/mirrors/create/helpers/common.ts +++ b/ui/app/mirrors/create/helpers/common.ts @@ -45,6 +45,7 @@ export const blankCDCSetting: FlowConnectionConfigs = { resync: false, softDeleteColName: '', syncedAtColName: '', + initialCopyOnly: false, }; export const blankQRepSetting = { diff --git a/ui/app/mirrors/errors/[mirrorName]/page.tsx b/ui/app/mirrors/errors/[mirrorName]/page.tsx new file mode 100644 index 0000000000..75e8c91d2b --- /dev/null +++ b/ui/app/mirrors/errors/[mirrorName]/page.tsx @@ -0,0 +1,75 @@ +import { AlertErr } from '@/app/dto/MirrorsDTO'; +import prisma from '@/app/utils/prisma'; +import TimeLabel from '@/components/TimeComponent'; +import { Label } from '@/lib/Label'; +import { Table, TableCell, TableRow } from '@/lib/Table'; + +type MirrorErrorProps = { + params: { mirrorName: string }; +}; + +const MirrorError = async ({ params: { mirrorName } }: MirrorErrorProps) => { + const mirrorErrors: AlertErr[] = await prisma.flow_errors.findMany({ + where: { + flow_name: mirrorName, + error_type: 'error', + }, + distinct: ['error_message'], + }); + + return ( +
+ +
+
+ + +
+ + Type + Message + + + + + } + > + {mirrorErrors.map((mirrorError) => ( + + + {mirrorError.error_type.toUpperCase()} + + + {mirrorError.error_message} + + + + + + ))} +
+
+
+
+ ); +}; + +export default MirrorError; diff --git a/ui/app/mirrors/mirror-status.tsx b/ui/app/mirrors/mirror-status.tsx new file mode 100644 index 0000000000..2a68b7b3d2 --- /dev/null +++ b/ui/app/mirrors/mirror-status.tsx @@ -0,0 +1,83 @@ +'use client'; + +import { Button } from '@/lib/Button'; +import { Icon } from '@/lib/Icon'; +import { Label } from '@/lib/Label'; +import { ProgressCircle } from '@/lib/ProgressCircle'; +import Link from 'next/link'; +import { useRouter } from 'next/navigation'; +import { useEffect, useState } from 'react'; +export const ErrorModal = ({ flowName }: { flowName: string }) => { + const router = useRouter(); + return ( + + + + ); +}; + +export const MirrorError = ({ flowName }: { flowName: string }) => { + const [flowStatus, setFlowStatus] = useState(); + const [isLoading, setIsLoading] = useState(true); + const [error, setError] = useState(null); + + useEffect(() => { + const fetchData = async () => { + setIsLoading(true); + try { + const response = await fetch(`/api/mirrors/alerts`, { + method: 'POST', + headers: { + 'Content-Type': 'application/json', + }, + body: JSON.stringify({ flowName }), + }); + + if (!response.ok) { + throw new Error('Network response was not ok'); + } + + const flowStatus = await response.json(); + setFlowStatus(flowStatus); + } catch (err: any) { + setError(err.message); + } finally { + setIsLoading(false); + } + }; + + fetchData(); + }, [flowName]); + + if (isLoading) { + return ( +
+ +
+ ); + } + + if (error) { + return ( +
+ +
+ ); + } + + if (flowStatus == 'healthy') { + return ; + } + + return ; +}; diff --git a/ui/app/mirrors/tables.tsx b/ui/app/mirrors/tables.tsx index 743bbc28a4..6c1289befc 100644 --- a/ui/app/mirrors/tables.tsx +++ b/ui/app/mirrors/tables.tsx @@ -7,6 +7,7 @@ import { SearchField } from '@/lib/SearchField'; import { Table, TableCell, TableRow } from '@/lib/Table'; import Link from 'next/link'; import { useMemo, useState } from 'react'; +import { MirrorError } from './mirror-status'; export function CDCFlows({ cdcFlows }: { cdcFlows: any }) { const [searchQuery, setSearchQuery] = useState(''); @@ -43,15 +44,26 @@ export function CDCFlows({ cdcFlows }: { cdcFlows: any }) { }} header={ - {['Name', 'Source', 'Destination', 'Start Time', ''].map( - (heading, index) => ( - - - - ) - )} + {[ + 'Name', + 'Source', + 'Destination', + 'Start Time', + 'Status', + '', + ].map((heading, index) => ( + + + + ))} } > @@ -77,6 +89,9 @@ export function CDCFlows({ cdcFlows }: { cdcFlows: any }) { + + +