diff --git a/flow/cmd/peer_data.go b/flow/cmd/peer_data.go index f9383d8c5e..f6e687435d 100644 --- a/flow/cmd/peer_data.go +++ b/flow/cmd/peer_data.go @@ -178,7 +178,8 @@ func (h *FlowRequestHandler) GetColumns( cols.column_name = pk.column_name WHERE cols.table_schema = $3 - AND cols.table_name = $4; + AND cols.table_name = $4 + ORDER BY cols.ordinal_position; `, req.SchemaName, req.TableName, req.SchemaName, req.TableName) if err != nil { return &protos.TableColumnsResponse{Columns: nil}, err diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index cc8ebfb344..2d278580f2 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -794,16 +794,14 @@ func (c *BigQueryConnector) SetupNormalizedTables( } // convert the column names and types to bigquery types - columns := make([]*bigquery.FieldSchema, len(tableSchema.Columns), len(tableSchema.Columns)+2) - idx := 0 - for colName, genericColType := range tableSchema.Columns { - columns[idx] = &bigquery.FieldSchema{ + columns := make([]*bigquery.FieldSchema, 0, len(tableSchema.Columns)+2) + utils.IterColumns(tableSchema, func(colName, genericColType string) { + columns = append(columns, &bigquery.FieldSchema{ Name: colName, Type: qValueKindToBigQueryType(genericColType), Repeated: qvalue.QValueKind(genericColType).IsArray(), - } - idx++ - } + }) + }) if req.SoftDeleteColName != "" { columns = append(columns, &bigquery.FieldSchema{ diff --git a/flow/connectors/bigquery/merge_statement_generator.go b/flow/connectors/bigquery/merge_statement_generator.go index 569ff611e9..2648b3ea6a 100644 --- a/flow/connectors/bigquery/merge_statement_generator.go +++ b/flow/connectors/bigquery/merge_statement_generator.go @@ -33,8 +33,8 @@ type mergeStmtGenerator struct { func (m *mergeStmtGenerator) generateFlattenedCTE() string { // for each column in the normalized table, generate CAST + JSON_EXTRACT_SCALAR // statement. - flattenedProjs := make([]string, 0) - for colName, colType := range m.normalizedTableSchema.Columns { + flattenedProjs := make([]string, 0, utils.TableSchemaColumns(m.normalizedTableSchema)+3) + utils.IterColumns(m.normalizedTableSchema, func(colName, colType string) { bqType := qValueKindToBigQueryType(colType) // CAST doesn't work for FLOAT, so rewrite it to FLOAT64. if bqType == bigquery.FloatFieldType { @@ -76,7 +76,7 @@ func (m *mergeStmtGenerator) generateFlattenedCTE() string { colName, bqType, colName) } flattenedProjs = append(flattenedProjs, castStmt) - } + }) flattenedProjs = append( flattenedProjs, "_peerdb_timestamp", @@ -111,12 +111,13 @@ func (m *mergeStmtGenerator) generateDeDupedCTE() string { // generateMergeStmt generates a merge statement. func (m *mergeStmtGenerator) generateMergeStmt() string { // comma separated list of column names - backtickColNames := make([]string, 0, len(m.normalizedTableSchema.Columns)) - pureColNames := make([]string, 0, len(m.normalizedTableSchema.Columns)) - for colName := range m.normalizedTableSchema.Columns { + columnCount := utils.TableSchemaColumns(m.normalizedTableSchema) + backtickColNames := make([]string, 0, columnCount) + pureColNames := make([]string, 0, columnCount) + utils.IterColumns(m.normalizedTableSchema, func(colName, _ string) { backtickColNames = append(backtickColNames, fmt.Sprintf("`%s`", colName)) pureColNames = append(pureColNames, colName) - } + }) csep := strings.Join(backtickColNames, ", ") insertColumnsSQL := csep + fmt.Sprintf(", `%s`", m.peerdbCols.SyncedAtColName) insertValuesSQL := csep + ",CURRENT_TIMESTAMP" diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index f24802aff3..7e79a78d1d 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -391,11 +391,11 @@ func generateCreateTableSQLForNormalizedTable( softDeleteColName string, syncedAtColName string, ) string { - createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)+2) - for columnName, genericColumnType := range sourceTableSchema.Columns { + createTableSQLArray := make([]string, 0, utils.TableSchemaColumns(sourceTableSchema)+2) + utils.IterColumns(sourceTableSchema, func(columnName, genericColumnType string) { createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("\"%s\" %s,", columnName, qValueKindToPostgresType(genericColumnType))) - } + }) if softDeleteColName != "" { createTableSQLArray = append(createTableSQLArray, @@ -591,10 +591,11 @@ func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifie rawTableIdentifier string, peerdbCols *protos.PeerDBColumns, ) []string { normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] - columnNames := make([]string, 0, len(normalizedTableSchema.Columns)) - flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) + columnCount := utils.TableSchemaColumns(normalizedTableSchema) + columnNames := make([]string, 0, columnCount) + flattenedCastsSQLArray := make([]string, 0, columnCount) primaryKeyColumnCasts := make(map[string]string) - for columnName, genericColumnType := range normalizedTableSchema.Columns { + utils.IterColumns(normalizedTableSchema, func(columnName, genericColumnType string) { columnNames = append(columnNames, fmt.Sprintf("\"%s\"", columnName)) pgType := qValueKindToPostgresType(genericColumnType) if qvalue.QValueKind(genericColumnType).IsArray() { @@ -608,15 +609,15 @@ func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifie if slices.Contains(normalizedTableSchema.PrimaryKeyColumns, columnName) { primaryKeyColumnCasts[columnName] = fmt.Sprintf("(_peerdb_data->>'%s')::%s", columnName, pgType) } - } + }) flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ","), ",") parsedDstTable, _ := utils.ParseSchemaTable(destinationTableIdentifier) insertColumnsSQL := strings.TrimSuffix(strings.Join(columnNames, ","), ",") - updateColumnsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) - for columnName := range normalizedTableSchema.Columns { + updateColumnsSQLArray := make([]string, 0, utils.TableSchemaColumns(normalizedTableSchema)) + utils.IterColumns(normalizedTableSchema, func(columnName, _ string) { updateColumnsSQLArray = append(updateColumnsSQLArray, fmt.Sprintf(`"%s"=EXCLUDED."%s"`, columnName, columnName)) - } + }) updateColumnsSQL := strings.TrimSuffix(strings.Join(updateColumnsSQLArray, ","), ",") deleteWhereClauseArray := make([]string, 0, len(normalizedTableSchema.PrimaryKeyColumns)) for columnName, columnCast := range primaryKeyColumnCasts { @@ -655,17 +656,17 @@ func (c *PostgresConnector) generateMergeStatement( peerdbCols *protos.PeerDBColumns, ) string { normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] - columnNames := maps.Keys(normalizedTableSchema.Columns) + columnNames := utils.TableSchemaColumnNames(normalizedTableSchema) for i, columnName := range columnNames { columnNames[i] = fmt.Sprintf("\"%s\"", columnName) } - flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) + flattenedCastsSQLArray := make([]string, 0, utils.TableSchemaColumns(normalizedTableSchema)) parsedDstTable, _ := utils.ParseSchemaTable(destinationTableIdentifier) primaryKeyColumnCasts := make(map[string]string) primaryKeySelectSQLArray := make([]string, 0, len(normalizedTableSchema.PrimaryKeyColumns)) - for columnName, genericColumnType := range normalizedTableSchema.Columns { + utils.IterColumns(normalizedTableSchema, func(columnName, genericColumnType string) { pgType := qValueKindToPostgresType(genericColumnType) if qvalue.QValueKind(genericColumnType).IsArray() { flattenedCastsSQLArray = append(flattenedCastsSQLArray, @@ -680,7 +681,7 @@ func (c *PostgresConnector) generateMergeStatement( primaryKeySelectSQLArray = append(primaryKeySelectSQLArray, fmt.Sprintf("src.%s=dst.%s", columnName, columnName)) } - } + }) flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ","), ",") insertValuesSQLArray := make([]string, 0, len(columnNames)) for _, columnName := range columnNames { diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index 4f8064b0d0..7e44bb7b8a 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -586,14 +586,10 @@ func (c *PostgresConnector) getTableSchemaForTable( } defer rows.Close() - res := &protos.TableSchema{ - TableIdentifier: tableName, - Columns: make(map[string]string), - PrimaryKeyColumns: pKeyCols, - IsReplicaIdentityFull: replicaIdentityType == ReplicaIdentityFull, - } - - for _, fieldDescription := range rows.FieldDescriptions() { + fields := rows.FieldDescriptions() + columnNames := make([]string, 0, len(fields)) + columnTypes := make([]string, 0, len(fields)) + for _, fieldDescription := range fields { genericColType := postgresOIDToQValueKind(fieldDescription.DataTypeOID) if genericColType == qvalue.QValueKindInvalid { typeName, ok := c.customTypesMapping[fieldDescription.DataTypeOID] @@ -604,14 +600,22 @@ func (c *PostgresConnector) getTableSchemaForTable( } } - res.Columns[fieldDescription.Name] = string(genericColType) + columnNames = append(columnNames, fieldDescription.Name) + columnTypes = append(columnTypes, string(genericColType)) } if err = rows.Err(); err != nil { return nil, fmt.Errorf("error iterating over table schema: %w", err) } - return res, nil + return &protos.TableSchema{ + TableIdentifier: tableName, + Columns: nil, + PrimaryKeyColumns: pKeyCols, + IsReplicaIdentityFull: replicaIdentityType == ReplicaIdentityFull, + ColumnNames: columnNames, + ColumnTypes: columnTypes, + }, nil } // SetupNormalizedTable sets up a normalized table, implementing the Connector interface. diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 8a919eb214..3ef85cf210 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -5,6 +5,7 @@ import ( "fmt" "testing" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" "github.com/jackc/pgx/v5" @@ -94,11 +95,9 @@ func (suite *PostgresSchemaDeltaTestSuite) TestSimpleAddColumn() { }) suite.failTestError(err) suite.Equal(&protos.TableSchema{ - TableIdentifier: tableName, - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt32), - "hi": string(qvalue.QValueKindInt64), - }, + TableIdentifier: tableName, + ColumnNames: []string{"id", "hi"}, + ColumnTypes: []string{string(qvalue.QValueKindInt32), string(qvalue.QValueKindInt64)}, PrimaryKeyColumns: []string{"id"}, }, output.TableNameSchemaMapping[tableName]) } @@ -112,36 +111,40 @@ func (suite *PostgresSchemaDeltaTestSuite) TestAddAllColumnTypes() { expectedTableSchema := &protos.TableSchema{ TableIdentifier: tableName, // goal is to test all types we're currently mapping to, not all QValue types - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt32), - "c1": string(qvalue.QValueKindBit), - "c2": string(qvalue.QValueKindBoolean), - "c3": string(qvalue.QValueKindBytes), - "c4": string(qvalue.QValueKindDate), - "c5": string(qvalue.QValueKindFloat32), - "c6": string(qvalue.QValueKindFloat64), - "c7": string(qvalue.QValueKindInt16), - "c8": string(qvalue.QValueKindInt32), - "c9": string(qvalue.QValueKindInt64), - "c10": string(qvalue.QValueKindJSON), - "c11": string(qvalue.QValueKindNumeric), - "c12": string(qvalue.QValueKindString), - "c13": string(qvalue.QValueKindTime), - "c14": string(qvalue.QValueKindTimestamp), - "c15": string(qvalue.QValueKindTimestampTZ), - "c16": string(qvalue.QValueKindUUID), + ColumnNames: []string{ + "id", "c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8", "c9", + "c10", "c11", "c12", "c13", "c14", "c15", "c16", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindInt32), + string(qvalue.QValueKindBit), + string(qvalue.QValueKindBoolean), + string(qvalue.QValueKindBytes), + string(qvalue.QValueKindDate), + string(qvalue.QValueKindFloat32), + string(qvalue.QValueKindFloat64), + string(qvalue.QValueKindInt16), + string(qvalue.QValueKindInt32), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindJSON), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindString), + string(qvalue.QValueKindTime), + string(qvalue.QValueKindTimestamp), + string(qvalue.QValueKindTimestampTZ), + string(qvalue.QValueKindUUID), }, PrimaryKeyColumns: []string{"id"}, } addedColumns := make([]*protos.DeltaAddedColumn, 0) - for columnName, columnType := range expectedTableSchema.Columns { + utils.IterColumns(expectedTableSchema, func(columnName, columnType string) { if columnName != "id" { addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ ColumnName: columnName, ColumnType: columnType, }) } - } + }) err = suite.connector.ReplayTableSchemaDeltas("schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, @@ -165,29 +168,33 @@ func (suite *PostgresSchemaDeltaTestSuite) TestAddTrickyColumnNames() { expectedTableSchema := &protos.TableSchema{ TableIdentifier: tableName, - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt32), - "c1": string(qvalue.QValueKindString), - "C1": string(qvalue.QValueKindString), - "C 1": string(qvalue.QValueKindString), - "right": string(qvalue.QValueKindString), - "select": string(qvalue.QValueKindString), - "XMIN": string(qvalue.QValueKindString), - "Cariño": string(qvalue.QValueKindString), - "±ªþ³§": string(qvalue.QValueKindString), - "カラム": string(qvalue.QValueKindString), + ColumnNames: []string{ + "id", "c1", "C1", "C 1", "right", + "select", "XMIN", "Cariño", "±ªþ³§", "カラム", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindInt32), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), }, PrimaryKeyColumns: []string{"id"}, } addedColumns := make([]*protos.DeltaAddedColumn, 0) - for columnName, columnType := range expectedTableSchema.Columns { + utils.IterColumns(expectedTableSchema, func(columnName, columnType string) { if columnName != "id" { addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ ColumnName: columnName, ColumnType: columnType, }) } - } + }) err = suite.connector.ReplayTableSchemaDeltas("schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, @@ -211,23 +218,24 @@ func (suite *PostgresSchemaDeltaTestSuite) TestAddDropWhitespaceColumnNames() { expectedTableSchema := &protos.TableSchema{ TableIdentifier: tableName, - Columns: map[string]string{ - " ": string(qvalue.QValueKindInt32), - " ": string(qvalue.QValueKindString), - " ": string(qvalue.QValueKindInt64), - " ": string(qvalue.QValueKindDate), + ColumnNames: []string{" ", " ", " ", "\t"}, + ColumnTypes: []string{ + string(qvalue.QValueKindInt32), + string(qvalue.QValueKindString), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindDate), }, PrimaryKeyColumns: []string{" "}, } addedColumns := make([]*protos.DeltaAddedColumn, 0) - for columnName, columnType := range expectedTableSchema.Columns { + utils.IterColumns(expectedTableSchema, func(columnName, columnType string) { if columnName != " " { addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ ColumnName: columnName, ColumnType: columnType, }) } - } + }) err = suite.connector.ReplayTableSchemaDeltas("schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, diff --git a/flow/connectors/snowflake/qrep.go b/flow/connectors/snowflake/qrep.go index ab460f01ea..0c43ab6a00 100644 --- a/flow/connectors/snowflake/qrep.go +++ b/flow/connectors/snowflake/qrep.go @@ -15,7 +15,6 @@ import ( "github.com/aws/aws-sdk-go/service/s3" "github.com/aws/aws-sdk-go/service/s3/s3manager" "github.com/jackc/pgx/v5/pgtype" - "golang.org/x/exp/maps" "google.golang.org/protobuf/encoding/protojson" ) @@ -259,14 +258,13 @@ func (c *SnowflakeConnector) ConsolidateQRepPartitions(config *protos.QRepConfig destTable := config.DestinationTableIdentifier stageName := c.getStageNameForJob(config.FlowJobName) - colInfo, err := c.getColsFromTable(destTable) + colNames, _, err := c.getColsFromTable(destTable) if err != nil { c.logger.Error(fmt.Sprintf("failed to get columns from table %s", destTable), slog.Any("error", err)) return fmt.Errorf("failed to get columns from table %s: %w", destTable, err) } - allCols := colInfo.Columns - err = CopyStageToDestination(c, config, destTable, stageName, allCols) + err = CopyStageToDestination(c, config, destTable, stageName, colNames) if err != nil { c.logger.Error("failed to copy stage to destination", slog.Any("error", err)) return fmt.Errorf("failed to copy stage to destination: %w", err) @@ -281,11 +279,11 @@ func (c *SnowflakeConnector) CleanupQRepFlow(config *protos.QRepConfig) error { return c.dropStage(config.StagingPath, config.FlowJobName) } -func (c *SnowflakeConnector) getColsFromTable(tableName string) (*model.ColumnInformation, error) { +func (c *SnowflakeConnector) getColsFromTable(tableName string) ([]string, []string, error) { // parse the table name to get the schema and table name schemaTable, err := utils.ParseSchemaTable(tableName) if err != nil { - return nil, fmt.Errorf("failed to parse table name: %w", err) + return nil, nil, fmt.Errorf("failed to parse table name: %w", err) } //nolint:gosec @@ -293,28 +291,27 @@ func (c *SnowflakeConnector) getColsFromTable(tableName string) (*model.ColumnIn SELECT column_name, data_type FROM information_schema.columns WHERE UPPER(table_name) = '%s' AND UPPER(table_schema) = '%s' + ORDER BY ordinal_position `, strings.ToUpper(schemaTable.Table), strings.ToUpper(schemaTable.Schema)) rows, err := c.database.QueryContext(c.ctx, queryString) if err != nil { - return nil, fmt.Errorf("failed to execute query: %w", err) + return nil, nil, fmt.Errorf("failed to execute query: %w", err) } defer rows.Close() - var colName pgtype.Text - var colType pgtype.Text - columnMap := map[string]string{} + var colName, colType pgtype.Text + colNames := make([]string, 0, 8) + colTypes := make([]string, 0, 8) for rows.Next() { if err := rows.Scan(&colName, &colType); err != nil { - return nil, fmt.Errorf("failed to scan row: %w", err) + return nil, nil, fmt.Errorf("failed to scan row: %w", err) } - columnMap[colName.String] = colType.String + colNames = append(colNames, colName.String) + colTypes = append(colTypes, colType.String) } - return &model.ColumnInformation{ - ColumnMap: columnMap, - Columns: maps.Keys(columnMap), - }, nil + return colNames, colTypes, nil } // dropStage drops the stage for the given job. diff --git a/flow/connectors/snowflake/qrep_avro_sync.go b/flow/connectors/snowflake/qrep_avro_sync.go index 86ceed4b5d..b9f7e270c4 100644 --- a/flow/connectors/snowflake/qrep_avro_sync.go +++ b/flow/connectors/snowflake/qrep_avro_sync.go @@ -73,19 +73,18 @@ func (s *SnowflakeAvroSyncMethod) SyncRecords( } s.connector.logger.Info(fmt.Sprintf("Created stage %s", stage)) - colInfo, err := s.connector.getColsFromTable(s.config.DestinationTableIdentifier) + colNames, _, err := s.connector.getColsFromTable(s.config.DestinationTableIdentifier) if err != nil { return 0, err } - allCols := colInfo.Columns err = s.putFileToStage(avroFile, stage) if err != nil { return 0, err } s.connector.logger.Info("pushed avro file to stage", tableLog) - err = CopyStageToDestination(s.connector, s.config, s.config.DestinationTableIdentifier, stage, allCols) + err = CopyStageToDestination(s.connector, s.config, s.config.DestinationTableIdentifier, stage, colNames) if err != nil { return 0, err } @@ -300,14 +299,15 @@ func (c *SnowflakeConnector) GetCopyTransformation( dstTableName string, syncedAtCol string, ) (*CopyInfo, error) { - colInfo, colsErr := c.getColsFromTable(dstTableName) + colNames, colTypes, colsErr := c.getColsFromTable(dstTableName) if colsErr != nil { return nil, fmt.Errorf("failed to get columns from destination table: %w", colsErr) } - transformations := make([]string, 0, len(colInfo.ColumnMap)) - columnOrder := make([]string, 0, len(colInfo.ColumnMap)) - for avroColName, colType := range colInfo.ColumnMap { + transformations := make([]string, 0, len(colNames)) + columnOrder := make([]string, 0, len(colNames)) + for idx, avroColName := range colNames { + colType := colTypes[idx] normalizedColName := SnowflakeIdentifierNormalize(avroColName) columnOrder = append(columnOrder, normalizedColName) if avroColName == syncedAtCol { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index 57f7f956d7..6ce31b07f2 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -21,7 +21,6 @@ import ( "github.com/jackc/pgx/v5/pgtype" "github.com/snowflakedb/gosnowflake" "go.temporal.io/sdk/activity" - "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" ) @@ -60,7 +59,7 @@ const ( _PEERDB_BATCH_ID > %d AND _PEERDB_BATCH_ID <= %d AND _PEERDB_RECORD_TYPE != 2 GROUP BY _PEERDB_DESTINATION_TABLE_NAME` getTableSchemaSQL = `SELECT COLUMN_NAME, DATA_TYPE FROM INFORMATION_SCHEMA.COLUMNS - WHERE TABLE_SCHEMA=? AND TABLE_NAME=?` + WHERE TABLE_SCHEMA=? AND TABLE_NAME=? ORDER BY ORDINAL_POSITION` insertJobMetadataSQL = "INSERT INTO %s.%s VALUES (?,?,?,?)" @@ -258,12 +257,9 @@ func (c *SnowflakeConnector) getTableSchemaForTable(tableName string) (*protos.T } }() - res := &protos.TableSchema{ - TableIdentifier: tableName, - Columns: make(map[string]string), - } - var columnName, columnType pgtype.Text + columnNames := make([]string, 0, 8) + columnTypes := make([]string, 0, 8) for rows.Next() { err = rows.Scan(&columnName, &columnType) if err != nil { @@ -275,10 +271,15 @@ func (c *SnowflakeConnector) getTableSchemaForTable(tableName string) (*protos.T genericColType = qvalue.QValueKindString } - res.Columns[columnName.String] = string(genericColType) + columnNames = append(columnNames, columnName.String) + columnTypes = append(columnTypes, string(genericColType)) } - return res, nil + return &protos.TableSchema{ + TableIdentifier: tableName, + ColumnNames: columnNames, + ColumnTypes: columnTypes, + }, nil } func (c *SnowflakeConnector) GetLastOffset(jobName string) (int64, error) { @@ -765,17 +766,17 @@ func generateCreateTableSQLForNormalizedTable( softDeleteColName string, syncedAtColName string, ) string { - createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)+2) - for columnName, genericColumnType := range sourceTableSchema.Columns { + createTableSQLArray := make([]string, 0, utils.TableSchemaColumns(sourceTableSchema)+2) + utils.IterColumns(sourceTableSchema, func(columnName, genericColumnType string) { normalizedColName := SnowflakeIdentifierNormalize(columnName) sfColType, err := qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)) if err != nil { slog.Warn(fmt.Sprintf("failed to convert column type %s to snowflake type", genericColumnType), slog.Any("error", err)) - continue + return } createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`%s %s,`, normalizedColName, sfColType)) - } + }) // add a _peerdb_is_deleted column to the normalized table // this is boolean default false, and is used to mark records as deleted @@ -826,15 +827,14 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( if err != nil { return 0, fmt.Errorf("unable to parse destination table '%s'", parsedDstTable) } - columnNames := maps.Keys(normalizedTableSchema.Columns) + columnNames := utils.TableSchemaColumnNames(normalizedTableSchema) - flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) - for columnName, genericColumnType := range normalizedTableSchema.Columns { + flattenedCastsSQLArray := make([]string, 0, utils.TableSchemaColumns(normalizedTableSchema)) + err = utils.IterColumnsError(normalizedTableSchema, func(columnName, genericColumnType string) error { 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) + return fmt.Errorf("failed to convert column type %s to snowflake type: %w", genericColumnType, err) } targetColumnName := SnowflakeIdentifierNormalize(columnName) @@ -865,6 +865,10 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( toVariantColumnName, columnName, sfType, targetColumnName)) } } + return nil + }) + if err != nil { + return 0, err } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ""), ",") @@ -1133,7 +1137,7 @@ func (c *SnowflakeConnector) RenameTables(req *protos.RenameTablesInput) (*proto for _, renameRequest := range req.RenameTableOptions { src := renameRequest.CurrentName dst := renameRequest.NewName - allCols := strings.Join(maps.Keys(renameRequest.TableSchema.Columns), ",") + allCols := strings.Join(utils.TableSchemaColumnNames(renameRequest.TableSchema), ",") pkeyCols := strings.Join(renameRequest.TableSchema.PrimaryKeyColumns, ",") c.logger.Info(fmt.Sprintf("handling soft-deletes for table '%s'...", dst)) diff --git a/flow/connectors/utils/columns.go b/flow/connectors/utils/columns.go new file mode 100644 index 0000000000..4a10ad444b --- /dev/null +++ b/flow/connectors/utils/columns.go @@ -0,0 +1,55 @@ +package utils + +import ( + "slices" + + "github.com/PeerDB-io/peer-flow/generated/protos" + "golang.org/x/exp/maps" +) + +func TableSchemaColumns(schema *protos.TableSchema) int { + if schema.Columns != nil { + return len(schema.Columns) + } else { + return len(schema.ColumnNames) + } +} + +func TableSchemaColumnNames(schema *protos.TableSchema) []string { + if schema.Columns != nil { + return maps.Keys(schema.Columns) + } else { + return slices.Clone(schema.ColumnNames) + } +} + +func IterColumns(schema *protos.TableSchema, iter func(k, v string)) { + if schema.Columns != nil { + for k, v := range schema.Columns { + iter(k, v) + } + } else { + for i, name := range schema.ColumnNames { + iter(name, schema.ColumnTypes[i]) + } + } +} + +func IterColumnsError(schema *protos.TableSchema, iter func(k, v string) error) error { + if schema.Columns != nil { + for k, v := range schema.Columns { + err := iter(k, v) + if err != nil { + return err + } + } + } else { + for i, name := range schema.ColumnNames { + err := iter(name, schema.ColumnTypes[i]) + if err != nil { + return err + } + } + } + return nil +} diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index 53658bf6ff..ecc912f655 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -146,11 +146,9 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { // verify we got our first row. e2e.NormalizeFlowCountQuery(env, connectionGen, 2) expectedTableSchema := &protos.TableSchema{ - TableIdentifier: dstTableName, - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt64), - "c1": string(qvalue.QValueKindInt64), - }, + TableIdentifier: dstTableName, + ColumnNames: []string{"id", "c1"}, + ColumnTypes: []string{string(qvalue.QValueKindInt64), string(qvalue.QValueKindInt64)}, PrimaryKeyColumns: []string{"id"}, } output, err := s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ @@ -175,10 +173,11 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { e2e.NormalizeFlowCountQuery(env, connectionGen, 4) expectedTableSchema = &protos.TableSchema{ TableIdentifier: dstTableName, - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt64), - "c1": string(qvalue.QValueKindInt64), - "c2": string(qvalue.QValueKindInt64), + ColumnNames: []string{"id", "c1", "c2"}, + ColumnTypes: []string{ + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), }, PrimaryKeyColumns: []string{"id"}, } @@ -204,11 +203,12 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { e2e.NormalizeFlowCountQuery(env, connectionGen, 6) expectedTableSchema = &protos.TableSchema{ TableIdentifier: dstTableName, - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt64), - "c1": string(qvalue.QValueKindInt64), - "c2": string(qvalue.QValueKindInt64), - "c3": string(qvalue.QValueKindInt64), + ColumnNames: []string{"id", "c1", "c2", "c3"}, + ColumnTypes: []string{ + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), }, PrimaryKeyColumns: []string{"id"}, } @@ -234,11 +234,12 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { e2e.NormalizeFlowCountQuery(env, connectionGen, 8) expectedTableSchema = &protos.TableSchema{ TableIdentifier: dstTableName, - Columns: map[string]string{ - "id": string(qvalue.QValueKindInt64), - "c1": string(qvalue.QValueKindInt64), - "c2": string(qvalue.QValueKindInt64), - "c3": string(qvalue.QValueKindInt64), + ColumnNames: []string{"id", "c1", "c2", "c3"}, + ColumnTypes: []string{ + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), + string(qvalue.QValueKindInt64), }, PrimaryKeyColumns: []string{"id"}, } diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 314c92d2b5..feae85af2d 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -858,11 +858,17 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { e2e.NormalizeFlowCountQuery(env, connectionGen, 2) expectedTableSchema := &protos.TableSchema{ TableIdentifier: strings.ToUpper(dstTableName), - Columns: map[string]string{ - "ID": string(qvalue.QValueKindNumeric), - "C1": string(qvalue.QValueKindNumeric), - "_PEERDB_IS_DELETED": string(qvalue.QValueKindBoolean), - "_PEERDB_SYNCED_AT": string(qvalue.QValueKindTimestamp), + ColumnNames: []string{ + "ID", + "C1", + "_PEERDB_IS_DELETED", + "_PEERDB_SYNCED_AT", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindBoolean), + string(qvalue.QValueKindTimestamp), }, } output, err := s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ @@ -886,12 +892,19 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { e2e.NormalizeFlowCountQuery(env, connectionGen, 4) expectedTableSchema = &protos.TableSchema{ TableIdentifier: strings.ToUpper(dstTableName), - Columns: map[string]string{ - "ID": string(qvalue.QValueKindNumeric), - "C1": string(qvalue.QValueKindNumeric), - "C2": string(qvalue.QValueKindNumeric), - "_PEERDB_IS_DELETED": string(qvalue.QValueKindBoolean), - "_PEERDB_SYNCED_AT": string(qvalue.QValueKindTimestamp), + ColumnNames: []string{ + "ID", + "C1", + "C2", + "_PEERDB_IS_DELETED", + "_PEERDB_SYNCED_AT", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindBoolean), + string(qvalue.QValueKindTimestamp), }, } output, err = s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ @@ -915,13 +928,21 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { e2e.NormalizeFlowCountQuery(env, connectionGen, 6) expectedTableSchema = &protos.TableSchema{ TableIdentifier: strings.ToUpper(dstTableName), - Columns: map[string]string{ - "ID": string(qvalue.QValueKindNumeric), - "C1": string(qvalue.QValueKindNumeric), - "C2": string(qvalue.QValueKindNumeric), - "C3": string(qvalue.QValueKindNumeric), - "_PEERDB_IS_DELETED": string(qvalue.QValueKindBoolean), - "_PEERDB_SYNCED_AT": string(qvalue.QValueKindTimestamp), + ColumnNames: []string{ + "ID", + "C1", + "C2", + "C3", + "_PEERDB_IS_DELETED", + "_PEERDB_SYNCED_AT", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindBoolean), + string(qvalue.QValueKindTimestamp), }, } output, err = s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ @@ -945,13 +966,21 @@ func (s PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { e2e.NormalizeFlowCountQuery(env, connectionGen, 8) expectedTableSchema = &protos.TableSchema{ TableIdentifier: strings.ToUpper(dstTableName), - Columns: map[string]string{ - "ID": string(qvalue.QValueKindNumeric), - "C1": string(qvalue.QValueKindNumeric), - "C2": string(qvalue.QValueKindNumeric), - "C3": string(qvalue.QValueKindNumeric), - "_PEERDB_IS_DELETED": string(qvalue.QValueKindBoolean), - "_PEERDB_SYNCED_AT": string(qvalue.QValueKindTimestamp), + ColumnNames: []string{ + "ID", + "C1", + "C2", + "C3", + "_PEERDB_IS_DELETED", + "_PEERDB_SYNCED_AT", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindBoolean), + string(qvalue.QValueKindTimestamp), }, } output, err = s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ @@ -1655,7 +1684,7 @@ func (s PeerFlowE2ETestSuiteSF) Test_Supported_Mixed_Case_Table_SF() { `, s.pgSuffix, "testMixedCase"), testKey, testValue) require.NoError(s.t, err) } - fmt.Println("Inserted 20 rows into the source table") + s.t.Log("Inserted 20 rows into the source table") }() env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) diff --git a/flow/e2e/snowflake/qrep_flow_sf_test.go b/flow/e2e/snowflake/qrep_flow_sf_test.go index f45a78d7c1..14e49d446a 100644 --- a/flow/e2e/snowflake/qrep_flow_sf_test.go +++ b/flow/e2e/snowflake/qrep_flow_sf_test.go @@ -43,7 +43,7 @@ func (s PeerFlowE2ETestSuiteSF) compareTableContentsWithDiffSelectorsSF(tableNam qualifiedTableName = fmt.Sprintf(`%s.%s.%s`, s.sfHelper.testDatabaseName, s.sfHelper.testSchemaName, tableName) } - sfSelQuery := fmt.Sprintf(`SELECT %s FROM %s ORDER BY id`, sfSelector, qualifiedTableName) + sfSelQuery := fmt.Sprintf(`SELECT %s FROM %s ORDER BY id`, sfSelector, qualifiedTableName) s.t.Logf("running query on snowflake: %s\n", sfSelQuery) sfRows, err := s.sfHelper.ExecuteAndProcessQuery(sfSelQuery) require.NoError(s.t, err) diff --git a/flow/e2e/snowflake/snowflake_schema_delta_test.go b/flow/e2e/snowflake/snowflake_schema_delta_test.go index 693062e39d..d61cbf1bb4 100644 --- a/flow/e2e/snowflake/snowflake_schema_delta_test.go +++ b/flow/e2e/snowflake/snowflake_schema_delta_test.go @@ -7,6 +7,7 @@ import ( "testing" connsnowflake "github.com/PeerDB-io/peer-flow/connectors/snowflake" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/e2eshared" "github.com/PeerDB-io/peer-flow/generated/protos" "github.com/PeerDB-io/peer-flow/model/qvalue" @@ -80,10 +81,8 @@ func (suite SnowflakeSchemaDeltaTestSuite) TestSimpleAddColumn() { suite.failTestError(err) suite.Equal(&protos.TableSchema{ TableIdentifier: tableName, - Columns: map[string]string{ - "ID": string(qvalue.QValueKindString), - "HI": string(qvalue.QValueKindJSON), - }, + ColumnNames: []string{"ID", "HI"}, + ColumnTypes: []string{string(qvalue.QValueKindString), string(qvalue.QValueKindJSON)}, }, output.TableNameSchemaMapping[tableName]) } @@ -95,29 +94,30 @@ func (suite SnowflakeSchemaDeltaTestSuite) TestAddAllColumnTypes() { expectedTableSchema := &protos.TableSchema{ TableIdentifier: tableName, // goal is to test all types we're currently mapping to, not all QValue types - Columns: map[string]string{ - "ID": string(qvalue.QValueKindString), - "C1": string(qvalue.QValueKindBoolean), - "C2": string(qvalue.QValueKindBytes), - "C3": string(qvalue.QValueKindDate), - "C4": string(qvalue.QValueKindFloat64), - "C5": string(qvalue.QValueKindJSON), - "C6": string(qvalue.QValueKindNumeric), - "C7": string(qvalue.QValueKindString), - "C8": string(qvalue.QValueKindTime), - "C9": string(qvalue.QValueKindTimestamp), - "C10": string(qvalue.QValueKindTimestampTZ), + ColumnNames: []string{"ID", "C1", "C2", "C3", "C4", "C5", "C6", "C7", "C8", "C9", "C10"}, + ColumnTypes: []string{ + string(qvalue.QValueKindString), + string(qvalue.QValueKindBoolean), + string(qvalue.QValueKindBytes), + string(qvalue.QValueKindDate), + string(qvalue.QValueKindFloat64), + string(qvalue.QValueKindJSON), + string(qvalue.QValueKindNumeric), + string(qvalue.QValueKindString), + string(qvalue.QValueKindTime), + string(qvalue.QValueKindTimestamp), + string(qvalue.QValueKindTimestampTZ), }, } addedColumns := make([]*protos.DeltaAddedColumn, 0) - for columnName, columnType := range expectedTableSchema.Columns { + utils.IterColumns(expectedTableSchema, func(columnName, columnType string) { if columnName != "ID" { addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ ColumnName: columnName, ColumnType: columnType, }) } - } + }) err = suite.connector.ReplayTableSchemaDeltas("schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, @@ -141,27 +141,38 @@ func (suite SnowflakeSchemaDeltaTestSuite) TestAddTrickyColumnNames() { expectedTableSchema := &protos.TableSchema{ TableIdentifier: tableName, // strings.ToUpper also does Unicode uppercasing :) - Columns: map[string]string{ - "ID": string(qvalue.QValueKindString), - "C1": string(qvalue.QValueKindString), - "C 1": string(qvalue.QValueKindString), - "RIGHT": string(qvalue.QValueKindString), - "SELECT": string(qvalue.QValueKindString), - "XMIN": string(qvalue.QValueKindString), - "CARIÑO": string(qvalue.QValueKindString), - "±ªÞ³§": string(qvalue.QValueKindString), - "カラム": string(qvalue.QValueKindString), + ColumnNames: []string{ + "ID", + "C1", + "C 1", + "RIGHT", + "SELECT", + "XMIN", + "CARIÑO", + "±ªÞ³§", + "カラム", + }, + ColumnTypes: []string{ + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), }, } addedColumns := make([]*protos.DeltaAddedColumn, 0) - for columnName, columnType := range expectedTableSchema.Columns { + utils.IterColumns(expectedTableSchema, func(columnName, columnType string) { if columnName != "ID" { addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ ColumnName: columnName, ColumnType: columnType, }) } - } + }) err = suite.connector.ReplayTableSchemaDeltas("schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, @@ -184,22 +195,23 @@ func (suite SnowflakeSchemaDeltaTestSuite) TestAddWhitespaceColumnNames() { expectedTableSchema := &protos.TableSchema{ TableIdentifier: tableName, - Columns: map[string]string{ - " ": string(qvalue.QValueKindString), - " ": string(qvalue.QValueKindString), - " ": string(qvalue.QValueKindTime), - " ": string(qvalue.QValueKindDate), + ColumnNames: []string{" ", " ", " ", "\t"}, + ColumnTypes: []string{ + string(qvalue.QValueKindString), + string(qvalue.QValueKindString), + string(qvalue.QValueKindTime), + string(qvalue.QValueKindDate), }, } addedColumns := make([]*protos.DeltaAddedColumn, 0) - for columnName, columnType := range expectedTableSchema.Columns { + utils.IterColumns(expectedTableSchema, func(columnName, columnType string) { if columnName != " " { addedColumns = append(addedColumns, &protos.DeltaAddedColumn{ ColumnName: columnName, ColumnType: columnType, }) } - } + }) err = suite.connector.ReplayTableSchemaDeltas("schema_delta_flow", []*protos.TableSchemaDelta{{ SrcTableName: tableName, diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index d38e3978c0..deffa53986 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -1917,10 +1917,13 @@ type TableSchema struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - TableIdentifier string `protobuf:"bytes,1,opt,name=table_identifier,json=tableIdentifier,proto3" json:"table_identifier,omitempty"` + TableIdentifier string `protobuf:"bytes,1,opt,name=table_identifier,json=tableIdentifier,proto3" json:"table_identifier,omitempty"` + // DEPRECATED: eliminate when breaking changes are allowed. Columns map[string]string `protobuf:"bytes,2,rep,name=columns,proto3" json:"columns,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` PrimaryKeyColumns []string `protobuf:"bytes,3,rep,name=primary_key_columns,json=primaryKeyColumns,proto3" json:"primary_key_columns,omitempty"` IsReplicaIdentityFull bool `protobuf:"varint,4,opt,name=is_replica_identity_full,json=isReplicaIdentityFull,proto3" json:"is_replica_identity_full,omitempty"` + ColumnNames []string `protobuf:"bytes,5,rep,name=column_names,json=columnNames,proto3" json:"column_names,omitempty"` + ColumnTypes []string `protobuf:"bytes,6,rep,name=column_types,json=columnTypes,proto3" json:"column_types,omitempty"` } func (x *TableSchema) Reset() { @@ -1983,6 +1986,20 @@ func (x *TableSchema) GetIsReplicaIdentityFull() bool { return false } +func (x *TableSchema) GetColumnNames() []string { + if x != nil { + return x.ColumnNames + } + return nil +} + +func (x *TableSchema) GetColumnTypes() []string { + if x != nil { + return x.ColumnTypes + } + return nil +} + type GetTableSchemaBatchInput struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3915,7 +3932,7 @@ var file_flow_proto_rawDesc = []byte{ 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, + 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0xe4, 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, @@ -3930,318 +3947,322 @@ var file_flow_proto_rawDesc = []byte{ 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, 0xec, 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, - 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, - 0x1b, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x70, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x72, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x17, 0x73, 0x6b, 0x69, 0x70, 0x50, 0x6b, 0x65, 0x79, 0x41, 0x6e, 0x64, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x22, 0xff, 0x01, 0x0a, 0x19, - 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, - 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, - 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, - 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, - 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, - 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, - 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x12, 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xcf, 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, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, - 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x54, 0x79, 0x70, 0x65, 0x73, 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, 0xec, 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, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x1b, 0x73, 0x6b, 0x69, 0x70, 0x5f, 0x70, 0x6b, 0x65, 0x79, + 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x73, 0x6b, 0x69, 0x70, 0x50, 0x6b, + 0x65, 0x79, 0x41, 0x6e, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x43, 0x68, 0x65, 0x63, + 0x6b, 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, 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, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, + 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, + 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, + 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, + 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x22, 0xcf, 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, 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, + 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, + 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 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, 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, + 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, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, - 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, - 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, - 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, - 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, - 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, - 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, - 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, - 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, - 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, - 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, - 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, - 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, - 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, - 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, - 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, - 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, - 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, - 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, - 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, - 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, - 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, - 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, - 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, - 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, - 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, 0x24, 0x73, 0x65, 0x74, 0x75, 0x70, 0x5f, 0x77, - 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, - 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x20, 0x73, 0x65, 0x74, 0x75, 0x70, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x6e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x31, 0x0a, 0x15, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x72, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x12, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, - 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x2b, 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, - 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x13, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, - 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x73, 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, - 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, - 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, - 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, - 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, - 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, - 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, - 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, 0x74, - 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x54, 0x0a, - 0x10, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x54, - 0x79, 0x70, 0x65, 0x22, 0xa2, 0x01, 0x0a, 0x10, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x24, 0x0a, 0x0e, 0x73, 0x72, 0x63, 0x5f, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0c, 0x73, 0x72, 0x63, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, - 0x0a, 0x0e, 0x64, 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x64, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x61, 0x64, 0x64, 0x65, 0x64, 0x5f, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x41, - 0x64, 0x64, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x0c, 0x61, 0x64, 0x64, 0x65, - 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xc8, 0x01, 0x0a, 0x1b, 0x52, 0x65, 0x70, - 0x6c, 0x61, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, - 0x6c, 0x74, 0x61, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x5a, 0x0a, 0x17, 0x66, 0x6c, 0x6f, 0x77, - 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x73, 0x52, 0x15, 0x66, - 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x73, 0x12, 0x4d, 0x0a, 0x13, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, 0x74, 0x61, - 0x52, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x44, 0x65, 0x6c, - 0x74, 0x61, 0x73, 0x22, 0xe9, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x46, 0x6c, 0x6f, 0x77, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x41, 0x0a, 0x0e, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, + 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, + 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x1a, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, + 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, + 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, + 0x5f, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, + 0x79, 0x12, 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, + 0x08, 0x73, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, + 0x34, 0x0a, 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x14, 0x62, 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, + 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, + 0x62, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, + 0x61, 0x69, 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, + 0x73, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, + 0x65, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, + 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, + 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x61, 0x74, 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, + 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, 0x24, 0x73, + 0x65, 0x74, 0x75, 0x70, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6f, 0x6e, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x08, 0x52, 0x20, 0x73, 0x65, 0x74, 0x75, 0x70, + 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x6e, + 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x31, 0x0a, 0x15, 0x64, + 0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x72, 0x65, + 0x73, 0x79, 0x6e, 0x63, 0x18, 0x12, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x73, 0x74, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x2b, + 0x0a, 0x12, 0x73, 0x79, 0x6e, 0x63, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x79, 0x6e, 0x63, + 0x65, 0x64, 0x41, 0x74, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2f, 0x0a, 0x14, 0x73, + 0x6f, 0x66, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x73, 0x6f, 0x66, 0x74, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x97, 0x01, 0x0a, + 0x0d, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, + 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x12, 0x31, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, + 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, + 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x22, 0x50, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 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, + 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/model/column.go b/flow/model/column.go deleted file mode 100644 index 5cbf25dc2e..0000000000 --- a/flow/model/column.go +++ /dev/null @@ -1,8 +0,0 @@ -package model - -type ColumnInformation struct { - // This is a mapping from column name to column type - // Example: "name" -> "VARCHAR" - ColumnMap map[string]string - Columns []string // List of column names -} diff --git a/flow/workflows/setup_flow.go b/flow/workflows/setup_flow.go index cf924e33d3..3fe76f90e9 100644 --- a/flow/workflows/setup_flow.go +++ b/flow/workflows/setup_flow.go @@ -2,10 +2,12 @@ package peerflow import ( "fmt" + "slices" "sort" "time" "github.com/PeerDB-io/peer-flow/activities" + "github.com/PeerDB-io/peer-flow/connectors/utils" "github.com/PeerDB-io/peer-flow/generated/protos" "golang.org/x/exp/maps" @@ -211,14 +213,22 @@ func (s *SetupFlowExecution) fetchTableSchemaAndSetupNormalizedTables( for _, mapping := range flowConnectionConfigs.TableMappings { if mapping.SourceTableIdentifier == srcTableName { if len(mapping.Exclude) != 0 { + columnCount := utils.TableSchemaColumns(tableSchema) + columnNames := make([]string, 0, columnCount) + columnTypes := make([]string, 0, columnCount) + utils.IterColumns(tableSchema, func(columnName, columnType string) { + if !slices.Contains(mapping.Exclude, columnName) { + columnNames = append(columnNames, columnName) + columnTypes = append(columnTypes, columnType) + } + }) tableSchema = &protos.TableSchema{ TableIdentifier: tableSchema.TableIdentifier, - Columns: maps.Clone(tableSchema.Columns), + Columns: nil, PrimaryKeyColumns: tableSchema.PrimaryKeyColumns, IsReplicaIdentityFull: tableSchema.IsReplicaIdentityFull, - } - for _, exclude := range mapping.Exclude { - delete(tableSchema.Columns, exclude) + ColumnNames: columnNames, + ColumnTypes: columnTypes, } } break diff --git a/flow/workflows/snapshot_flow.go b/flow/workflows/snapshot_flow.go index 1dc89fc58e..87b339d466 100644 --- a/flow/workflows/snapshot_flow.go +++ b/flow/workflows/snapshot_flow.go @@ -16,7 +16,6 @@ import ( "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" - "golang.org/x/exp/maps" ) type SnapshotFlowExecution struct { @@ -141,11 +140,11 @@ func (s *SnapshotFlowExecution) cloneTable( if len(mapping.Exclude) != 0 { for _, v := range s.config.TableNameSchemaMapping { if v.TableIdentifier == srcName { - cols := maps.Keys(v.Columns) - for i, col := range cols { - cols[i] = fmt.Sprintf(`"%s"`, col) + colNames := utils.TableSchemaColumnNames(v) + for i, colName := range colNames { + colNames[i] = fmt.Sprintf(`"%s"`, colName) } - from = strings.Join(cols, ",") + from = strings.Join(colNames, ",") break } } diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 15fd49f852..6a6c65c138 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -308,12 +308,17 @@ pub struct CreateRawTableOutput { pub struct TableSchema { #[prost(string, tag="1")] pub table_identifier: ::prost::alloc::string::String, + /// DEPRECATED: eliminate when breaking changes are allowed. #[prost(map="string, string", tag="2")] pub columns: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, #[prost(string, repeated, tag="3")] pub primary_key_columns: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, #[prost(bool, tag="4")] pub is_replica_identity_full: bool, + #[prost(string, repeated, tag="5")] + pub column_names: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(string, repeated, tag="6")] + pub column_types: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index 2f534f0d75..c047041ae0 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -6788,6 +6788,12 @@ impl serde::Serialize for TableSchema { if self.is_replica_identity_full { len += 1; } + if !self.column_names.is_empty() { + len += 1; + } + if !self.column_types.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("peerdb_flow.TableSchema", len)?; if !self.table_identifier.is_empty() { struct_ser.serialize_field("tableIdentifier", &self.table_identifier)?; @@ -6801,6 +6807,12 @@ impl serde::Serialize for TableSchema { if self.is_replica_identity_full { struct_ser.serialize_field("isReplicaIdentityFull", &self.is_replica_identity_full)?; } + if !self.column_names.is_empty() { + struct_ser.serialize_field("columnNames", &self.column_names)?; + } + if !self.column_types.is_empty() { + struct_ser.serialize_field("columnTypes", &self.column_types)?; + } struct_ser.end() } } @@ -6818,6 +6830,10 @@ impl<'de> serde::Deserialize<'de> for TableSchema { "primaryKeyColumns", "is_replica_identity_full", "isReplicaIdentityFull", + "column_names", + "columnNames", + "column_types", + "columnTypes", ]; #[allow(clippy::enum_variant_names)] @@ -6826,6 +6842,8 @@ impl<'de> serde::Deserialize<'de> for TableSchema { Columns, PrimaryKeyColumns, IsReplicaIdentityFull, + ColumnNames, + ColumnTypes, __SkipField__, } impl<'de> serde::Deserialize<'de> for GeneratedField { @@ -6852,6 +6870,8 @@ impl<'de> serde::Deserialize<'de> for TableSchema { "columns" => Ok(GeneratedField::Columns), "primaryKeyColumns" | "primary_key_columns" => Ok(GeneratedField::PrimaryKeyColumns), "isReplicaIdentityFull" | "is_replica_identity_full" => Ok(GeneratedField::IsReplicaIdentityFull), + "columnNames" | "column_names" => Ok(GeneratedField::ColumnNames), + "columnTypes" | "column_types" => Ok(GeneratedField::ColumnTypes), _ => Ok(GeneratedField::__SkipField__), } } @@ -6875,6 +6895,8 @@ impl<'de> serde::Deserialize<'de> for TableSchema { let mut columns__ = None; let mut primary_key_columns__ = None; let mut is_replica_identity_full__ = None; + let mut column_names__ = None; + let mut column_types__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::TableIdentifier => { @@ -6903,6 +6925,18 @@ impl<'de> serde::Deserialize<'de> for TableSchema { } is_replica_identity_full__ = Some(map.next_value()?); } + GeneratedField::ColumnNames => { + if column_names__.is_some() { + return Err(serde::de::Error::duplicate_field("columnNames")); + } + column_names__ = Some(map.next_value()?); + } + GeneratedField::ColumnTypes => { + if column_types__.is_some() { + return Err(serde::de::Error::duplicate_field("columnTypes")); + } + column_types__ = Some(map.next_value()?); + } GeneratedField::__SkipField__ => { let _ = map.next_value::()?; } @@ -6913,6 +6947,8 @@ impl<'de> serde::Deserialize<'de> for TableSchema { columns: columns__.unwrap_or_default(), primary_key_columns: primary_key_columns__.unwrap_or_default(), is_replica_identity_full: is_replica_identity_full__.unwrap_or_default(), + column_names: column_names__.unwrap_or_default(), + column_types: column_types__.unwrap_or_default(), }) } } diff --git a/protos/flow.proto b/protos/flow.proto index 6a4a100a17..1aa0f08acb 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -195,9 +195,12 @@ message CreateRawTableOutput { string table_identifier = 1; } message TableSchema { string table_identifier = 1; + // DEPRECATED: eliminate when breaking changes are allowed. map columns = 2; repeated string primary_key_columns = 3; bool is_replica_identity_full = 4; + repeated string column_names = 5; + repeated string column_types = 6; } message GetTableSchemaBatchInput { diff --git a/ui/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index d160981644..0e8a709f8c 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -305,9 +305,12 @@ export interface CreateRawTableOutput { export interface TableSchema { tableIdentifier: string; + /** DEPRECATED: eliminate when breaking changes are allowed. */ columns: { [key: string]: string }; primaryKeyColumns: string[]; isReplicaIdentityFull: boolean; + columnNames: string[]; + columnTypes: string[]; } export interface TableSchema_ColumnsEntry { @@ -3941,7 +3944,14 @@ export const CreateRawTableOutput = { }; function createBaseTableSchema(): TableSchema { - return { tableIdentifier: "", columns: {}, primaryKeyColumns: [], isReplicaIdentityFull: false }; + return { + tableIdentifier: "", + columns: {}, + primaryKeyColumns: [], + isReplicaIdentityFull: false, + columnNames: [], + columnTypes: [], + }; } export const TableSchema = { @@ -3958,6 +3968,12 @@ export const TableSchema = { if (message.isReplicaIdentityFull === true) { writer.uint32(32).bool(message.isReplicaIdentityFull); } + for (const v of message.columnNames) { + writer.uint32(42).string(v!); + } + for (const v of message.columnTypes) { + writer.uint32(50).string(v!); + } return writer; }, @@ -3999,6 +4015,20 @@ export const TableSchema = { message.isReplicaIdentityFull = reader.bool(); continue; + case 5: + if (tag !== 42) { + break; + } + + message.columnNames.push(reader.string()); + continue; + case 6: + if (tag !== 50) { + break; + } + + message.columnTypes.push(reader.string()); + continue; } if ((tag & 7) === 4 || tag === 0) { break; @@ -4021,6 +4051,8 @@ export const TableSchema = { ? object.primaryKeyColumns.map((e: any) => String(e)) : [], isReplicaIdentityFull: isSet(object.isReplicaIdentityFull) ? Boolean(object.isReplicaIdentityFull) : false, + columnNames: Array.isArray(object?.columnNames) ? object.columnNames.map((e: any) => String(e)) : [], + columnTypes: Array.isArray(object?.columnTypes) ? object.columnTypes.map((e: any) => String(e)) : [], }; }, @@ -4044,6 +4076,12 @@ export const TableSchema = { if (message.isReplicaIdentityFull === true) { obj.isReplicaIdentityFull = message.isReplicaIdentityFull; } + if (message.columnNames?.length) { + obj.columnNames = message.columnNames; + } + if (message.columnTypes?.length) { + obj.columnTypes = message.columnTypes; + } return obj; }, @@ -4061,6 +4099,8 @@ export const TableSchema = { }, {}); message.primaryKeyColumns = object.primaryKeyColumns?.map((e) => e) || []; message.isReplicaIdentityFull = object.isReplicaIdentityFull ?? false; + message.columnNames = object.columnNames?.map((e) => e) || []; + message.columnTypes = object.columnTypes?.map((e) => e) || []; return message; }, };