From 17549b2e1d9b81a6855f9f43f6ada9eab0c5ee33 Mon Sep 17 00:00:00 2001 From: Kevin K Biju <52661649+heavycrystal@users.noreply.github.com> Date: Tue, 17 Oct 2023 22:55:58 +0100 Subject: [PATCH] composite primary key support for SF, PG and BQ (#499) Fixes issues: #491 #492 #493 #494 #495 Supersedes: #129 and #164 --- flow/activities/flowable.go | 11 +- flow/connectors/bigquery/bigquery.go | 37 +- .../bigquery/merge_stmt_generator_test.go | 4 +- flow/connectors/bigquery/qrep_avro_sync.go | 1 - flow/connectors/postgres/cdc.go | 63 ++- flow/connectors/postgres/client.go | 114 ++-- flow/connectors/postgres/postgres.go | 40 +- flow/connectors/postgres/postgres_cdc_test.go | 20 +- .../postgres/postgres_schema_delta_test.go | 8 +- flow/connectors/snowflake/snowflake.go | 41 +- flow/connectors/utils/map.go | 6 +- flow/connectors/utils/stream.go | 6 +- flow/e2e/bigquery/peer_flow_bq_test.go | 229 ++++++++ flow/e2e/postgres/peer_flow_pg_test.go | 241 +++++++- flow/e2e/postgres/qrep_flow_pg_test.go | 6 +- flow/e2e/snowflake/peer_flow_sf_test.go | 230 ++++++++ flow/generated/protos/flow.pb.go | 532 +++++++++--------- flow/model/model.go | 8 +- nexus/pt/src/peerdb_flow.rs | 4 +- nexus/pt/src/peerdb_flow.serde.rs | 26 +- protos/flow.proto | 4 +- ui/grpc_generated/flow.ts | 20 +- 22 files changed, 1204 insertions(+), 447 deletions(-) diff --git a/flow/activities/flowable.go b/flow/activities/flowable.go index f66efa25d1..1d49ae795c 100644 --- a/flow/activities/flowable.go +++ b/flow/activities/flowable.go @@ -358,10 +358,13 @@ func (a *FlowableActivity) StartNormalize( return nil, fmt.Errorf("failed to normalized records: %w", err) } - err = a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, - res.EndBatchID) - if err != nil { - return nil, err + // normalize flow did not run due to no records, no need to update end time. + if res.Done { + err = a.CatalogMirrorMonitor.UpdateEndTimeForCDCBatch(ctx, input.FlowConnectionConfigs.FlowJobName, + res.EndBatchID) + if err != nil { + return nil, err + } } // log the number of batches normalized diff --git a/flow/connectors/bigquery/bigquery.go b/flow/connectors/bigquery/bigquery.go index 19c8bc596c..a00759ee09 100644 --- a/flow/connectors/bigquery/bigquery.go +++ b/flow/connectors/bigquery/bigquery.go @@ -512,7 +512,7 @@ func (c *BigQueryConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, matchData: "", batchID: syncBatchID, stagingBatchID: stagingBatchID, - unchangedToastColumns: utils.KeysToString(r.UnchangedToastColumns), + unchangedToastColumns: "", }) tableNameRowsMapping[r.DestinationTableName] += 1 case *model.UpdateRecord: @@ -571,7 +571,7 @@ func (c *BigQueryConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, matchData: itemsJSON, batchID: syncBatchID, stagingBatchID: stagingBatchID, - unchangedToastColumns: utils.KeysToString(r.UnchangedToastColumns), + unchangedToastColumns: "", }) tableNameRowsMapping[r.DestinationTableName] += 1 @@ -741,7 +741,7 @@ func (c *BigQueryConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, } entries[9] = qvalue.QValue{ Kind: qvalue.QValueKindString, - Value: utils.KeysToString(r.UnchangedToastColumns), + Value: "", } tableNameRowsMapping[r.DestinationTableName] += 1 @@ -802,7 +802,7 @@ func (c *BigQueryConnector) syncRecordsViaAvro(req *model.SyncRecordsRequest, } entries[9] = qvalue.QValue{ Kind: qvalue.QValueKindString, - Value: utils.KeysToString(r.UnchangedToastColumns), + Value: "", } tableNameRowsMapping[r.DestinationTableName] += 1 @@ -893,7 +893,7 @@ func (c *BigQueryConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) if !hasJob || normalizeBatchID == syncBatchID { log.Printf("waiting for sync to catch up for job %s, so finishing", req.FlowJobName) return &model.NormalizeResponse{ - Done: true, + Done: false, StartBatchID: normalizeBatchID, EndBatchID: syncBatchID, }, nil @@ -1035,7 +1035,7 @@ func (c *BigQueryConnector) getUpdateMetadataStmt(jobName string, lastSyncedChec // create the job in the metadata table jobStatement := fmt.Sprintf( - "INSERT INTO %s.%s (mirror_job_name, offset,sync_batch_id) VALUES ('%s',%d,%d);", + "INSERT INTO %s.%s (mirror_job_name,offset,sync_batch_id) VALUES ('%s',%d,%d);", c.datasetID, MirrorJobsTable, jobName, lastSyncedCheckpointID, batchID) if hasJob { jobStatement = fmt.Sprintf( @@ -1288,14 +1288,13 @@ func (m *MergeStmtGenerator) generateDeDupedCTE() string { ) _peerdb_ranked WHERE _peerdb_rank = 1 ) SELECT * FROM _peerdb_de_duplicated_data_res` - pkey := m.NormalizedTableSchema.PrimaryKeyColumn - return fmt.Sprintf(cte, pkey) + pkeyColsStr := fmt.Sprintf("(CONCAT(%s))", strings.Join(m.NormalizedTableSchema.PrimaryKeyColumns, + ", '_peerdb_concat_', ")) + return fmt.Sprintf(cte, pkeyColsStr) } // generateMergeStmt generates a merge statement. func (m *MergeStmtGenerator) generateMergeStmt(tempTable string) string { - pkey := m.NormalizedTableSchema.PrimaryKeyColumn - // comma separated list of column names backtickColNames := make([]string, 0) pureColNames := make([]string, 0) @@ -1305,18 +1304,26 @@ func (m *MergeStmtGenerator) generateMergeStmt(tempTable string) string { } csep := strings.Join(backtickColNames, ", ") - udateStatementsforToastCols := m.generateUpdateStatement(pureColNames, m.UnchangedToastColumns) - updateStringToastCols := strings.Join(udateStatementsforToastCols, " ") + updateStatementsforToastCols := m.generateUpdateStatements(pureColNames, m.UnchangedToastColumns) + updateStringToastCols := strings.Join(updateStatementsforToastCols, " ") + + pkeySelectSQLArray := make([]string, 0, len(m.NormalizedTableSchema.PrimaryKeyColumns)) + for _, pkeyColName := range m.NormalizedTableSchema.PrimaryKeyColumns { + pkeySelectSQLArray = append(pkeySelectSQLArray, fmt.Sprintf("_peerdb_target.%s = _peerdb_deduped.%s", + pkeyColName, pkeyColName)) + } + // _peerdb_target. = _peerdb_deduped. AND _peerdb_target. = _peerdb_deduped. ... + pkeySelectSQL := strings.Join(pkeySelectSQLArray, " AND ") return fmt.Sprintf(` MERGE %s.%s _peerdb_target USING %s _peerdb_deduped - ON _peerdb_target.%s = _peerdb_deduped.%s + ON %s WHEN NOT MATCHED and (_peerdb_deduped._peerdb_record_type != 2) THEN INSERT (%s) VALUES (%s) %s WHEN MATCHED AND (_peerdb_deduped._peerdb_record_type = 2) THEN DELETE; - `, m.Dataset, m.NormalizedTable, tempTable, pkey, pkey, csep, csep, updateStringToastCols) + `, m.Dataset, m.NormalizedTable, tempTable, pkeySelectSQL, csep, csep, updateStringToastCols) } /* @@ -1334,7 +1341,7 @@ and updating the other columns (not the unchanged toast columns) 6. Repeat steps 1-5 for each unique unchanged toast column group. 7. Return the list of generated update statements. */ -func (m *MergeStmtGenerator) generateUpdateStatement(allCols []string, unchangedToastCols []string) []string { +func (m *MergeStmtGenerator) generateUpdateStatements(allCols []string, unchangedToastCols []string) []string { updateStmts := make([]string, 0) for _, cols := range unchangedToastCols { diff --git a/flow/connectors/bigquery/merge_stmt_generator_test.go b/flow/connectors/bigquery/merge_stmt_generator_test.go index 7320639680..3d8892d4c5 100644 --- a/flow/connectors/bigquery/merge_stmt_generator_test.go +++ b/flow/connectors/bigquery/merge_stmt_generator_test.go @@ -30,7 +30,7 @@ func TestGenerateUpdateStatement_WithUnchangedToastCols(t *testing.T) { " `col2` = _peerdb_deduped.col2", } - result := m.generateUpdateStatement(allCols, unchangedToastCols) + result := m.generateUpdateStatements(allCols, unchangedToastCols) for i := range expected { expected[i] = removeSpacesTabsNewlines(expected[i]) @@ -56,7 +56,7 @@ func TestGenerateUpdateStatement_NoUnchangedToastCols(t *testing.T) { " `col3` = _peerdb_deduped.col3", } - result := m.generateUpdateStatement(allCols, unchangedToastCols) + result := m.generateUpdateStatements(allCols, unchangedToastCols) for i := range expected { expected[i] = removeSpacesTabsNewlines(expected[i]) diff --git a/flow/connectors/bigquery/qrep_avro_sync.go b/flow/connectors/bigquery/qrep_avro_sync.go index a4cd92a82c..db9c6ef6c0 100644 --- a/flow/connectors/bigquery/qrep_avro_sync.go +++ b/flow/connectors/bigquery/qrep_avro_sync.go @@ -387,7 +387,6 @@ func (s *QRepAvroSyncMethod) writeToStage( return 0, fmt.Errorf("failed to write record to OCF file: %w", err) } numRecords++ - } activity.RecordHeartbeat(s.connector.ctx, fmt.Sprintf( "Writing OCF contents to BigQuery for partition/batch ID %s", diff --git a/flow/connectors/postgres/cdc.go b/flow/connectors/postgres/cdc.go index c20f0f20d6..b48143150b 100644 --- a/flow/connectors/postgres/cdc.go +++ b/flow/connectors/postgres/cdc.go @@ -2,6 +2,7 @@ package connpostgres import ( "context" + "crypto/sha256" "fmt" "reflect" "time" @@ -244,15 +245,14 @@ func (p *PostgresCDCSource) consumeStream( if isFullReplica { records.Records = append(records.Records, rec) } else { - pkeyCol := req.TableNameSchemaMapping[tableName].PrimaryKeyColumn - pkeyColVal, err := rec.GetItems().GetValueByColName(pkeyCol) + compositePKeyString, err := p.compositePKeyToString(req, rec) if err != nil { - return nil, fmt.Errorf("error getting pkey column value: %w", err) + return nil, err } tablePkeyVal := model.TableWithPkey{ TableName: tableName, - PkeyColVal: *pkeyColVal, + PkeyColVal: compositePKeyString, } _, ok := records.TablePKeyLastSeen[tablePkeyVal] if !ok { @@ -274,14 +274,14 @@ func (p *PostgresCDCSource) consumeStream( if isFullReplica { records.Records = append(records.Records, rec) } else { - pkeyCol := req.TableNameSchemaMapping[tableName].PrimaryKeyColumn - pkeyColVal, err := rec.GetItems().GetValueByColName(pkeyCol) + compositePKeyString, err := p.compositePKeyToString(req, rec) if err != nil { - return nil, fmt.Errorf("error getting pkey column value: %w", err) + return nil, err } + tablePkeyVal := model.TableWithPkey{ TableName: tableName, - PkeyColVal: *pkeyColVal, + PkeyColVal: compositePKeyString, } records.Records = append(records.Records, rec) // all columns will be set in insert record, so add it to the map @@ -370,17 +370,16 @@ func (p *PostgresCDCSource) processInsertMessage( } // create empty map of string to interface{} - items, unchangedToastColumns, err := p.convertTupleToMap(msg.Tuple, rel) + items, _, err := p.convertTupleToMap(msg.Tuple, rel) if err != nil { return nil, fmt.Errorf("error converting tuple to map: %w", err) } return &model.InsertRecord{ - CheckPointID: int64(lsn), - Items: items, - DestinationTableName: p.TableNameMapping[tableName], - SourceTableName: tableName, - UnchangedToastColumns: unchangedToastColumns, + CheckPointID: int64(lsn), + Items: items, + DestinationTableName: p.TableNameMapping[tableName], + SourceTableName: tableName, }, nil } @@ -442,17 +441,16 @@ func (p *PostgresCDCSource) processDeleteMessage( } // create empty map of string to interface{} - items, unchangedToastColumns, err := p.convertTupleToMap(msg.OldTuple, rel) + items, _, err := p.convertTupleToMap(msg.OldTuple, rel) if err != nil { return nil, fmt.Errorf("error converting tuple to map: %w", err) } return &model.DeleteRecord{ - CheckPointID: int64(lsn), - Items: items, - DestinationTableName: p.TableNameMapping[tableName], - SourceTableName: tableName, - UnchangedToastColumns: unchangedToastColumns, + CheckPointID: int64(lsn), + Items: items, + DestinationTableName: p.TableNameMapping[tableName], + SourceTableName: tableName, }, nil } @@ -466,15 +464,15 @@ It takes a tuple and a relation message as input and returns func (p *PostgresCDCSource) convertTupleToMap( tuple *pglogrepl.TupleData, rel *protos.RelationMessage, -) (*model.RecordItems, map[string]bool, error) { +) (*model.RecordItems, map[string]struct{}, error) { // if the tuple is nil, return an empty map if tuple == nil { - return model.NewRecordItems(), make(map[string]bool), nil + return model.NewRecordItems(), make(map[string]struct{}), nil } // create empty map of string to interface{} items := model.NewRecordItems() - unchangedToastColumns := make(map[string]bool) + unchangedToastColumns := make(map[string]struct{}) for idx, col := range tuple.Columns { colName := rel.Columns[idx].Name @@ -496,7 +494,7 @@ func (p *PostgresCDCSource) convertTupleToMap( } items.AddColumn(colName, data) case 'u': // unchanged toast - unchangedToastColumns[colName] = true + unchangedToastColumns[colName] = struct{}{} default: return nil, nil, fmt.Errorf("unknown column data type: %s", string(col.DataType)) } @@ -617,3 +615,20 @@ func (p *PostgresCDCSource) processRelationMessage( CheckPointID: int64(lsn), }, nil } + +func (p *PostgresCDCSource) compositePKeyToString(req *model.PullRecordsRequest, rec model.Record) (string, error) { + tableName := rec.GetTableName() + pkeyColsMerged := make([]byte, 0) + + for _, pkeyCol := range req.TableNameSchemaMapping[tableName].PrimaryKeyColumns { + pkeyColVal, err := rec.GetItems().GetValueByColName(pkeyCol) + if err != nil { + return "", fmt.Errorf("error getting pkey column value: %w", err) + } + pkeyColsMerged = append(pkeyColsMerged, []byte(fmt.Sprintf("%v", pkeyColVal.Value))...) + } + + hasher := sha256.New() + hasher.Write(pkeyColsMerged) + return fmt.Sprintf("%x", hasher.Sum(nil)), nil +} diff --git a/flow/connectors/postgres/client.go b/flow/connectors/postgres/client.go index 550d798c02..1c89edf6cf 100644 --- a/flow/connectors/postgres/client.go +++ b/flow/connectors/postgres/client.go @@ -12,6 +12,7 @@ import ( "github.com/jackc/pgx/v5" log "github.com/sirupsen/logrus" "golang.org/x/exp/maps" + "golang.org/x/exp/slices" ) //nolint:stylecheck @@ -26,6 +27,8 @@ const ( _peerdb_timestamp BIGINT NOT NULL,_peerdb_destination_table_name TEXT NOT NULL,_peerdb_data JSONB NOT NULL, _peerdb_record_type INTEGER NOT NULL, _peerdb_match_data JSONB,_peerdb_batch_id INTEGER, _peerdb_unchanged_toast_columns TEXT)` + createRawTableBatchIDIndexSQL = "CREATE INDEX IF NOT EXISTS %s_batchid_idx ON %s.%s(_peerdb_batch_id)" + createRawTableDstTableIndexSQL = "CREATE INDEX IF NOT EXISTS %s_dst_table_idx ON %s.%s(_peerdb_destination_table_name)" getLastOffsetSQL = "SELECT lsn_offset FROM %s.%s WHERE mirror_job_name=$1" getLastSyncBatchID_SQL = "SELECT sync_batch_id FROM %s.%s WHERE mirror_job_name=$1" @@ -48,7 +51,7 @@ const ( ) MERGE INTO %s dst USING (SELECT %s,_peerdb_record_type,_peerdb_unchanged_toast_columns FROM src_rank WHERE _peerdb_rank=1) src - ON dst.%s=src.%s + ON %s WHEN NOT MATCHED AND src._peerdb_record_type!=2 THEN INSERT (%s) VALUES (%s) %s @@ -66,7 +69,7 @@ const ( RANK() OVER (PARTITION BY %s ORDER BY _peerdb_timestamp DESC) AS _peerdb_rank FROM %s.%s WHERE _peerdb_batch_id>$1 AND _peerdb_batch_id<=$2 AND _peerdb_destination_table_name=$3 ) - DELETE FROM %s USING src_rank WHERE %s.%s=%s AND src_rank._peerdb_rank=1 AND src_rank._peerdb_record_type=2` + DELETE FROM %s USING src_rank WHERE %s AND src_rank._peerdb_rank=1 AND src_rank._peerdb_record_type=2` dropTableIfExistsSQL = "DROP TABLE IF EXISTS %s.%s" deleteJobMetadataSQL = "DELETE FROM %s.%s WHERE MIRROR_JOB_NAME=$1" @@ -103,39 +106,42 @@ func (c *PostgresConnector) isTableFullReplica(schemaTable *SchemaTable) (bool, return string(replicaIdentity) == "f", nil } -// getPrimaryKeyColumn for table returns the primary key column for a given table +// getPrimaryKeyColumns for table returns the primary key column for a given table // errors if there is no primary key column or if there is more than one primary key column. -func (c *PostgresConnector) getPrimaryKeyColumn(schemaTable *SchemaTable) (string, error) { +func (c *PostgresConnector) getPrimaryKeyColumns(schemaTable *SchemaTable) ([]string, error) { relID, err := c.getRelIDForTable(schemaTable) if err != nil { - return "", fmt.Errorf("failed to get relation id for table %s: %w", schemaTable, err) + return nil, fmt.Errorf("failed to get relation id for table %s: %w", schemaTable, err) } // Get the primary key column name var pkCol string + pkCols := make([]string, 0) rows, err := c.pool.Query(c.ctx, `SELECT a.attname FROM pg_index i JOIN pg_attribute a ON a.attrelid = i.indrelid AND a.attnum = ANY(i.indkey) - WHERE i.indrelid = $1 AND i.indisprimary`, + WHERE i.indrelid = $1 AND i.indisprimary ORDER BY a.attname ASC`, relID) if err != nil { - return "", fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) + return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) } defer rows.Close() // 0 rows returned, table has no primary keys if !rows.Next() { - return "", fmt.Errorf("table %s has no primary keys", schemaTable) + return nil, fmt.Errorf("table %s has no primary keys", schemaTable) } - err = rows.Scan(&pkCol) - if err != nil { - return "", fmt.Errorf("error scanning primary key column for table %s: %w", schemaTable, err) - } - // more than 1 row returned, table has more than 1 primary key - if rows.Next() { - return "", fmt.Errorf("table %s has more than one primary key", schemaTable) + for { + err = rows.Scan(&pkCol) + if err != nil { + return nil, fmt.Errorf("error scanning primary key column for table %s: %w", schemaTable, err) + } + pkCols = append(pkCols, pkCol) + if !rows.Next() { + break + } } - return pkCol, nil + return pkCols, nil } func (c *PostgresConnector) tableExists(schemaTable *SchemaTable) (bool, error) { @@ -301,14 +307,19 @@ func generateCreateTableSQLForNormalizedTable(sourceTableIdentifier string, sourceTableSchema *protos.TableSchema) string { createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)) for columnName, genericColumnType := range sourceTableSchema.Columns { - if sourceTableSchema.PrimaryKeyColumn == strings.ToLower(columnName) { - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("\"%s\" %s PRIMARY KEY,", - columnName, qValueKindToPostgresType(genericColumnType))) - } else { - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("\"%s\" %s,", columnName, - qValueKindToPostgresType(genericColumnType))) - } + createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("\"%s\" %s,", columnName, + qValueKindToPostgresType(genericColumnType))) } + + // add composite primary key to the table + primaryKeyColsQuoted := make([]string, 0) + for _, primaryKeyCol := range sourceTableSchema.PrimaryKeyColumns { + primaryKeyColsQuoted = append(primaryKeyColsQuoted, + fmt.Sprintf(`"%s"`, primaryKeyCol)) + } + createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("PRIMARY KEY(%s),", + strings.TrimSuffix(strings.Join(primaryKeyColsQuoted, ","), ","))) + return fmt.Sprintf(createNormalizedTableSQL, sourceTableIdentifier, strings.TrimSuffix(strings.Join(createTableSQLArray, ""), ",")) } @@ -462,11 +473,10 @@ func (c *PostgresConnector) generateNormalizeStatements(destinationTableIdentifi unchangedToastColumns []string, rawTableIdentifier string, supportsMerge bool) []string { if supportsMerge { return []string{c.generateMergeStatement(destinationTableIdentifier, unchangedToastColumns, rawTableIdentifier)} - } else { - log.Warnf("Postgres version is not high enough to support MERGE, falling back to UPSERT + DELETE") - log.Warnf("TOAST columns will not be updated properly, use REPLICA IDENTITY FULL or upgrade Postgres") - return c.generateFallbackStatements(destinationTableIdentifier, rawTableIdentifier) } + log.Warnf("Postgres version is not high enough to support MERGE, falling back to UPSERT + DELETE") + log.Warnf("TOAST columns will not be updated properly, use REPLICA IDENTITY FULL or upgrade Postgres") + return c.generateFallbackStatements(destinationTableIdentifier, rawTableIdentifier) } func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifier string, @@ -474,14 +484,14 @@ func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifie normalizedTableSchema := c.tableSchemaMapping[destinationTableIdentifier] columnNames := make([]string, 0, len(normalizedTableSchema.Columns)) flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) - var primaryKeyColumnCast string + primaryKeyColumnCasts := make(map[string]string) for columnName, genericColumnType := range normalizedTableSchema.Columns { columnNames = append(columnNames, fmt.Sprintf("\"%s\"", columnName)) pgType := qValueKindToPostgresType(genericColumnType) flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>'%s')::%s AS \"%s\"", columnName, pgType, columnName)) - if normalizedTableSchema.PrimaryKeyColumn == columnName { - primaryKeyColumnCast = fmt.Sprintf("(_peerdb_data->>'%s')::%s", columnName, pgType) + if slices.Contains(normalizedTableSchema.PrimaryKeyColumns, columnName) { + primaryKeyColumnCasts[columnName] = fmt.Sprintf("(_peerdb_data->>'%s')::%s", columnName, pgType) } } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ","), ",") @@ -492,12 +502,20 @@ func (c *PostgresConnector) generateFallbackStatements(destinationTableIdentifie updateColumnsSQLArray = append(updateColumnsSQLArray, fmt.Sprintf("%s=EXCLUDED.%s", columnName, columnName)) } updateColumnsSQL := strings.TrimSuffix(strings.Join(updateColumnsSQLArray, ","), ",") - fallbackUpsertStatement := fmt.Sprintf(fallbackUpsertStatementSQL, primaryKeyColumnCast, internalSchema, + deleteWhereClauseArray := make([]string, 0, len(normalizedTableSchema.PrimaryKeyColumns)) + for columnName, columnCast := range primaryKeyColumnCasts { + deleteWhereClauseArray = append(deleteWhereClauseArray, fmt.Sprintf("%s.%s=%s AND ", + destinationTableIdentifier, columnName, columnCast)) + } + deleteWhereClauseSQL := strings.TrimSuffix(strings.Join(deleteWhereClauseArray, ""), "AND ") + + fallbackUpsertStatement := fmt.Sprintf(fallbackUpsertStatementSQL, + strings.TrimSuffix(strings.Join(maps.Values(primaryKeyColumnCasts), ","), ","), internalSchema, rawTableIdentifier, destinationTableIdentifier, insertColumnsSQL, flattenedCastsSQL, - normalizedTableSchema.PrimaryKeyColumn, updateColumnsSQL) - fallbackDeleteStatement := fmt.Sprintf(fallbackDeleteStatementSQL, primaryKeyColumnCast, internalSchema, - rawTableIdentifier, destinationTableIdentifier, destinationTableIdentifier, - normalizedTableSchema.PrimaryKeyColumn, primaryKeyColumnCast) + strings.Join(normalizedTableSchema.PrimaryKeyColumns, ","), updateColumnsSQL) + fallbackDeleteStatement := fmt.Sprintf(fallbackDeleteStatementSQL, + strings.Join(maps.Values(primaryKeyColumnCasts), ","), internalSchema, + rawTableIdentifier, destinationTableIdentifier, deleteWhereClauseSQL) return []string{fallbackUpsertStatement, fallbackDeleteStatement} } @@ -511,7 +529,8 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st } flattenedCastsSQLArray := make([]string, 0, len(normalizedTableSchema.Columns)) - var primaryKeyColumnCast string + primaryKeyColumnCasts := make(map[string]string) + primaryKeySelectSQLArray := make([]string, 0, len(normalizedTableSchema.PrimaryKeyColumns)) for columnName, genericColumnType := range normalizedTableSchema.Columns { pgType := qValueKindToPostgresType(genericColumnType) if strings.Contains(genericColumnType, "array") { @@ -522,8 +541,10 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st flattenedCastsSQLArray = append(flattenedCastsSQLArray, fmt.Sprintf("(_peerdb_data->>'%s')::%s AS \"%s\"", strings.Trim(columnName, "\""), pgType, columnName)) } - if normalizedTableSchema.PrimaryKeyColumn == columnName { - primaryKeyColumnCast = fmt.Sprintf("(_peerdb_data->>'%s')::%s", strings.Trim(columnName, "\""), pgType) + if slices.Contains(normalizedTableSchema.PrimaryKeyColumns, columnName) { + primaryKeyColumnCasts[columnName] = fmt.Sprintf("(_peerdb_data->>'%s')::%s", columnName, pgType) + primaryKeySelectSQLArray = append(primaryKeySelectSQLArray, fmt.Sprintf("src.%s=dst.%s", + columnName, columnName)) } } flattenedCastsSQL := strings.TrimSuffix(strings.Join(flattenedCastsSQLArray, ","), ",") @@ -536,18 +557,18 @@ func (c *PostgresConnector) generateMergeStatement(destinationTableIdentifier st insertValuesSQL := strings.TrimSuffix(strings.Join(insertValuesSQLArray, ","), ",") updateStatements := c.generateUpdateStatement(columnNames, unchangedToastColumns) - return fmt.Sprintf(mergeStatementSQL, primaryKeyColumnCast, internalSchema, rawTableIdentifier, - destinationTableIdentifier, flattenedCastsSQL, normalizedTableSchema.PrimaryKeyColumn, - normalizedTableSchema.PrimaryKeyColumn, insertColumnsSQL, insertValuesSQL, updateStatements) + return fmt.Sprintf(mergeStatementSQL, strings.Join(maps.Values(primaryKeyColumnCasts), ","), + internalSchema, rawTableIdentifier, destinationTableIdentifier, flattenedCastsSQL, + strings.Join(primaryKeySelectSQLArray, " AND "), insertColumnsSQL, insertValuesSQL, updateStatements) } func (c *PostgresConnector) generateUpdateStatement(allCols []string, unchangedToastColsLists []string) string { updateStmts := make([]string, 0) for _, cols := range unchangedToastColsLists { - unchangedColsArray := strings.Split(cols, ",") - for i, col := range unchangedColsArray { - unchangedColsArray[i] = fmt.Sprintf("\"%s\"", col) + unchangedColsArray := make([]string, 0) + for _, unchangedToastCol := range strings.Split(cols, ",") { + unchangedColsArray = append(unchangedColsArray, fmt.Sprintf(`"%s"`, unchangedToastCol)) } otherCols := utils.ArrayMinus(allCols, unchangedColsArray) tmpArray := make([]string, 0) @@ -555,10 +576,9 @@ func (c *PostgresConnector) generateUpdateStatement(allCols []string, unchangedT tmpArray = append(tmpArray, fmt.Sprintf("%s=src.%s", colName, colName)) } ssep := strings.Join(tmpArray, ",") - quotedCols := strings.Join(unchangedColsArray, ",") updateStmt := fmt.Sprintf(`WHEN MATCHED AND - src._peerdb_record_type=1 AND _peerdb_unchanged_toast_columns='%s' - THEN UPDATE SET %s `, quotedCols, ssep) + src._peerdb_record_type=1 AND _peerdb_unchanged_toast_columns='%s' + THEN UPDATE SET %s `, cols, ssep) updateStmts = append(updateStmts, updateStmt) } return strings.Join(updateStmts, "\n") diff --git a/flow/connectors/postgres/postgres.go b/flow/connectors/postgres/postgres.go index e7588a74df..199469bf56 100644 --- a/flow/connectors/postgres/postgres.go +++ b/flow/connectors/postgres/postgres.go @@ -293,7 +293,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S 0, "{}", syncBatchID, - utils.KeysToString(typedRecord.UnchangedToastColumns), + "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 case *model.UpdateRecord: @@ -331,7 +331,7 @@ func (c *PostgresConnector) SyncRecords(req *model.SyncRecordsRequest) (*model.S 2, itemsJSON, syncBatchID, - utils.KeysToString(typedRecord.UnchangedToastColumns), + "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 default: @@ -423,7 +423,7 @@ func (c *PostgresConnector) NormalizeRecords(req *model.NormalizeRecordsRequest) "flowName": req.FlowJobName, }).Printf("no records to normalize: syncBatchID %d, normalizeBatchID %d", syncBatchID, normalizeBatchID) return &model.NormalizeResponse{ - Done: true, + Done: false, StartBatchID: normalizeBatchID, EndBatchID: syncBatchID, }, nil @@ -532,6 +532,16 @@ func (c *PostgresConnector) CreateRawTable(req *protos.CreateRawTableInput) (*pr if err != nil { return nil, fmt.Errorf("error creating raw table: %w", err) } + _, err = createRawTableTx.Exec(c.ctx, fmt.Sprintf(createRawTableBatchIDIndexSQL, rawTableIdentifier, + internalSchema, rawTableIdentifier)) + if err != nil { + return nil, fmt.Errorf("error creating batch ID index on raw table: %w", err) + } + _, err = createRawTableTx.Exec(c.ctx, fmt.Sprintf(createRawTableDstTableIndexSQL, rawTableIdentifier, + internalSchema, rawTableIdentifier)) + if err != nil { + return nil, fmt.Errorf("error creating destion table index on raw table: %w", err) + } err = createRawTableTx.Commit(c.ctx) if err != nil { @@ -572,28 +582,26 @@ func (c *PostgresConnector) getTableSchemaForTable( return nil, fmt.Errorf("error getting replica identity for table %s: %w", schemaTable, replErr) } - pkey, err := c.getPrimaryKeyColumn(schemaTable) + // Get the column names and types + rows, err := c.pool.Query(c.ctx, + fmt.Sprintf(`SELECT * FROM %s LIMIT 0`, tableName), pgx.QueryExecModeSimpleProtocol) if err != nil { - if !isFullReplica { - return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) - } + return nil, fmt.Errorf("error getting table schema for table %s: %w", schemaTable, err) + } + defer rows.Close() + + pKeyCols, err := c.getPrimaryKeyColumns(schemaTable) + if err != nil { + return nil, fmt.Errorf("error getting primary key column for table %s: %w", schemaTable, err) } res := &protos.TableSchema{ TableIdentifier: tableName, Columns: make(map[string]string), - PrimaryKeyColumn: pkey, + PrimaryKeyColumns: pKeyCols, IsReplicaIdentityFull: isFullReplica, } - // Get the column names and types - rows, err := c.pool.Query(c.ctx, - fmt.Sprintf(`SELECT * FROM %s LIMIT 0`, tableName), pgx.QueryExecModeSimpleProtocol) - if err != nil { - return nil, fmt.Errorf("error getting table schema for table %s: %w", schemaTable, err) - } - defer rows.Close() - for _, fieldDescription := range rows.FieldDescriptions() { genericColType := postgresOIDToQValueKind(fieldDescription.DataTypeOID) if genericColType == qvalue.QValueKindInvalid { diff --git a/flow/connectors/postgres/postgres_cdc_test.go b/flow/connectors/postgres/postgres_cdc_test.go index c18b188801..2ef5609359 100644 --- a/flow/connectors/postgres/postgres_cdc_test.go +++ b/flow/connectors/postgres/postgres_cdc_test.go @@ -235,9 +235,9 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R suite.Equal(qvalue.QValueKindString, v.Kind) suite.Equal(65536, len(v.Value.(string))) suite.Equal(3, len(updateRecord.UnchangedToastColumns)) - suite.True(updateRecord.UnchangedToastColumns["lz4_t"]) - suite.True(updateRecord.UnchangedToastColumns["n_b"]) - suite.True(updateRecord.UnchangedToastColumns["lz4_b"]) + suite.Contains(updateRecord.UnchangedToastColumns, "lz4_t") + suite.Contains(updateRecord.UnchangedToastColumns, "n_b") + suite.Contains(updateRecord.UnchangedToastColumns, "lz4_b") suite.IsType(&model.UpdateRecord{}, records[1]) updateRecord = records[1].(*model.UpdateRecord) suite.Equal(srcTableName, updateRecord.SourceTableName) @@ -258,7 +258,7 @@ func (suite *PostgresCDCTestSuite) validateMutatedToastRecords(records []model.R suite.Equal(qvalue.QValueKindBytes, v.Kind) suite.Equal(65536, len(v.Value.([]byte))) suite.Equal(1, len(updateRecord.UnchangedToastColumns)) - suite.True(updateRecord.UnchangedToastColumns["n_t"]) + suite.Contains(updateRecord.UnchangedToastColumns, "n_t") // Test case for records[2] suite.IsType(&model.UpdateRecord{}, records[2]) updateRecord = records[2].(*model.UpdateRecord) @@ -408,7 +408,7 @@ func (suite *PostgresCDCTestSuite) TestErrorForTableNotExist() { "id": string(qvalue.QValueKindInt32), "name": string(qvalue.QValueKindString), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } err = suite.connector.PullFlowCleanup(nonExistentFlowName) @@ -505,7 +505,7 @@ func (suite *PostgresCDCTestSuite) TestSimpleHappyFlow() { "id": string(qvalue.QValueKindInt32), "name": string(qvalue.QValueKindString), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, }, }}, tableNameSchema) tableNameSchemaMapping[simpleHappyFlowDstTableName] = @@ -666,7 +666,7 @@ func (suite *PostgresCDCTestSuite) TestAllTypesHappyFlow() { "c40": string(qvalue.QValueKindUUID), "c41": string(qvalue.QValueKindString), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, }, }, }, tableNameSchema) @@ -765,14 +765,14 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { "n_b": string(qvalue.QValueKindBytes), "lz4_b": string(qvalue.QValueKindBytes), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, }, }}, tableNameSchema) tableNameSchemaMapping[toastHappyFlowDstTableName] = tableNameSchema.TableNameSchemaMapping[toastHappyFlowSrcTableName] suite.insertToastRecords(toastHappyFlowSrcTableName) - recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ + _, err = suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: nil, IdleTimeout: 10 * time.Second, @@ -783,7 +783,7 @@ func (suite *PostgresCDCTestSuite) TestToastHappyFlow() { RelationMessageMapping: relationMessageMapping, }) suite.failTestError(err) - recordsWithSchemaDelta, err = suite.connector.PullRecords(&model.PullRecordsRequest{ + recordsWithSchemaDelta, err := suite.connector.PullRecords(&model.PullRecordsRequest{ FlowJobName: toastHappyFlowName, LastSyncState: nil, IdleTimeout: 10 * time.Second, diff --git a/flow/connectors/postgres/postgres_schema_delta_test.go b/flow/connectors/postgres/postgres_schema_delta_test.go index 38400ac741..eec14c915a 100644 --- a/flow/connectors/postgres/postgres_schema_delta_test.go +++ b/flow/connectors/postgres/postgres_schema_delta_test.go @@ -99,7 +99,7 @@ func (suite *PostgresSchemaDeltaTestSuite) TestSimpleAddColumn() { "id": string(qvalue.QValueKindInt32), "hi": string(qvalue.QValueKindInt64), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, }, output.TableNameSchemaMapping[tableName]) } @@ -131,7 +131,7 @@ func (suite *PostgresSchemaDeltaTestSuite) TestAddAllColumnTypes() { "c15": string(qvalue.QValueKindTimestampTZ), "c16": string(qvalue.QValueKindUUID), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } addedColumns := make([]*protos.DeltaAddedColumn, 0) for columnName, columnType := range expectedTableSchema.Columns { @@ -177,7 +177,7 @@ func (suite *PostgresSchemaDeltaTestSuite) TestAddTrickyColumnNames() { "±ªþ³§": string(qvalue.QValueKindString), "カラム": string(qvalue.QValueKindString), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } addedColumns := make([]*protos.DeltaAddedColumn, 0) for columnName, columnType := range expectedTableSchema.Columns { @@ -217,7 +217,7 @@ func (suite *PostgresSchemaDeltaTestSuite) TestAddDropWhitespaceColumnNames() { " ": string(qvalue.QValueKindInt64), " ": string(qvalue.QValueKindDate), }, - PrimaryKeyColumn: " ", + PrimaryKeyColumns: []string{" "}, } addedColumns := make([]*protos.DeltaAddedColumn, 0) for columnName, columnType := range expectedTableSchema.Columns { diff --git a/flow/connectors/snowflake/snowflake.go b/flow/connectors/snowflake/snowflake.go index d691fef970..4e6a913d38 100644 --- a/flow/connectors/snowflake/snowflake.go +++ b/flow/connectors/snowflake/snowflake.go @@ -565,7 +565,7 @@ func (c *SnowflakeConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, ra recordType: 0, matchData: "", batchID: syncBatchID, - unchangedToastColumns: utils.KeysToString(typedRecord.UnchangedToastColumns), + unchangedToastColumns: "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 case *model.UpdateRecord: @@ -605,7 +605,7 @@ func (c *SnowflakeConnector) syncRecordsViaSQL(req *model.SyncRecordsRequest, ra recordType: 2, matchData: itemsJSON, batchID: syncBatchID, - unchangedToastColumns: utils.KeysToString(typedRecord.UnchangedToastColumns), + unchangedToastColumns: "", }) tableNameRowsMapping[typedRecord.DestinationTableName] += 1 default: @@ -700,7 +700,7 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest // normalize has caught up with sync, chill until more records are loaded. if syncBatchID == normalizeBatchID { return &model.NormalizeResponse{ - Done: true, + Done: false, StartBatchID: normalizeBatchID, EndBatchID: syncBatchID, }, nil @@ -713,7 +713,7 @@ func (c *SnowflakeConnector) NormalizeRecords(req *model.NormalizeRecordsRequest // sync hasn't created job metadata yet, chill. if !jobMetadataExists { return &model.NormalizeResponse{ - Done: true, + Done: false, }, nil } destinationTableNames, err := c.getDistinctTableNamesInBatch(req.FlowJobName, syncBatchID, normalizeBatchID) @@ -887,16 +887,10 @@ func generateCreateTableSQLForNormalizedTable( sourceTableSchema *protos.TableSchema, ) string { createTableSQLArray := make([]string, 0, len(sourceTableSchema.Columns)) - primaryColUpper := strings.ToUpper(sourceTableSchema.PrimaryKeyColumn) for columnName, genericColumnType := range sourceTableSchema.Columns { columnNameUpper := strings.ToUpper(columnName) - if primaryColUpper == columnNameUpper { - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`"%s" %s PRIMARY KEY,`, - columnNameUpper, qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)))) - } else { - createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`"%s" %s,`, columnNameUpper, - qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)))) - } + createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`"%s" %s,`, columnNameUpper, + qValueKindToSnowflakeType(qvalue.QValueKind(genericColumnType)))) } // add a _peerdb_is_deleted column to the normalized table @@ -904,6 +898,15 @@ func generateCreateTableSQLForNormalizedTable( createTableSQLArray = append(createTableSQLArray, fmt.Sprintf(`"%s" BOOLEAN DEFAULT FALSE,`, isDeletedColumnName)) + // add composite primary key to the table + primaryKeyColsUpperQuoted := make([]string, 0) + for _, primaryKeyCol := range sourceTableSchema.PrimaryKeyColumns { + primaryKeyColsUpperQuoted = append(primaryKeyColsUpperQuoted, + fmt.Sprintf(`"%s"`, strings.ToUpper(primaryKeyCol))) + } + createTableSQLArray = append(createTableSQLArray, fmt.Sprintf("PRIMARY KEY(%s),", + strings.TrimSuffix(strings.Join(primaryKeyColsUpperQuoted, ","), ","))) + return fmt.Sprintf(createNormalizedTableSQL, sourceTableIdentifier, strings.TrimSuffix(strings.Join(createTableSQLArray, ""), ",")) } @@ -989,9 +992,13 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( updateStatementsforToastCols := c.generateUpdateStatement(columnNames, unchangedToastColumns) updateStringToastCols := strings.Join(updateStatementsforToastCols, " ") - // TARGET. = SOURCE. - pkeyColStr := fmt.Sprintf("TARGET.%s = SOURCE.%s", - normalizedTableSchema.PrimaryKeyColumn, normalizedTableSchema.PrimaryKeyColumn) + pkeySelectSQLArray := make([]string, 0, len(normalizedTableSchema.PrimaryKeyColumns)) + for _, pkeyColName := range normalizedTableSchema.PrimaryKeyColumns { + pkeySelectSQLArray = append(pkeySelectSQLArray, fmt.Sprintf("TARGET.%s = SOURCE.%s", + pkeyColName, pkeyColName)) + } + // TARGET. = SOURCE. AND TARGET. = SOURCE. ... + pkeySelectSQL := strings.Join(pkeySelectSQLArray, " AND ") deletePart := "DELETE" if softDelete { @@ -1000,8 +1007,8 @@ func (c *SnowflakeConnector) generateAndExecuteMergeStatement( mergeStatement := fmt.Sprintf(mergeStatementSQL, destinationTableIdentifier, toVariantColumnName, rawTableIdentifier, normalizeBatchID, syncBatchID, flattenedCastsSQL, - normalizedTableSchema.PrimaryKeyColumn, pkeyColStr, insertColumnsSQL, insertValuesSQL, - updateStringToastCols, deletePart) + fmt.Sprintf("(%s)", strings.Join(normalizedTableSchema.PrimaryKeyColumns, ",")), + pkeySelectSQL, insertColumnsSQL, insertValuesSQL, updateStringToastCols, deletePart) result, err := normalizeRecordsTx.ExecContext(c.ctx, mergeStatement, destinationTableIdentifier) if err != nil { diff --git a/flow/connectors/utils/map.go b/flow/connectors/utils/map.go index 829da94bfb..769ef225a6 100644 --- a/flow/connectors/utils/map.go +++ b/flow/connectors/utils/map.go @@ -2,7 +2,11 @@ package utils import "strings" -func KeysToString(m map[string]bool) string { +func KeysToString(m map[string]struct{}) string { + if m == nil { + return "" + } + var keys []string for k := range m { keys = append(keys, k) diff --git a/flow/connectors/utils/stream.go b/flow/connectors/utils/stream.go index 422106ea5c..c16020578a 100644 --- a/flow/connectors/utils/stream.go +++ b/flow/connectors/utils/stream.go @@ -90,7 +90,7 @@ func RecordsToRawTableStream(req model.RecordsToStreamRequest) (*model.RecordsTo } entries[7] = qvalue.QValue{ Kind: qvalue.QValueKindString, - Value: KeysToString(typedRecord.UnchangedToastColumns), + Value: "", } req.TableMapping[typedRecord.DestinationTableName] += 1 case *model.UpdateRecord: @@ -149,11 +149,11 @@ func RecordsToRawTableStream(req model.RecordsToStreamRequest) (*model.RecordsTo } entries[7] = qvalue.QValue{ Kind: qvalue.QValueKindString, - Value: KeysToString(typedRecord.UnchangedToastColumns), + Value: "", } req.TableMapping[typedRecord.DestinationTableName] += 1 default: - return nil, fmt.Errorf("record type %T not supported in Snowflake flow connector", typedRecord) + return nil, fmt.Errorf("record type %T not supported", typedRecord) } if first { diff --git a/flow/e2e/bigquery/peer_flow_bq_test.go b/flow/e2e/bigquery/peer_flow_bq_test.go index 75e1d35153..400ecc3d01 100644 --- a/flow/e2e/bigquery/peer_flow_bq_test.go +++ b/flow/e2e/bigquery/peer_flow_bq_test.go @@ -1024,3 +1024,232 @@ func (s *PeerFlowE2ETestSuiteBQ) Test_Simple_Schema_Changes_BQ() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_cpkey") + dstTableName := "test_simple_cpkey" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + PRIMARY KEY(id,t) + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 4, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t) VALUES ($1,$2) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + // verify we got our 10 rows + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + s.compareTableContentsBQ(dstTableName, "id,c1,c2,t") + + _, err := s.pool.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + s.compareTableContentsBQ(dstTableName, "id,c1,c2,t") + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_1_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") + dstTableName := "test_cpkey_toast1" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + t2 TEXT, + PRIMARY KEY(id,t) + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_toast1_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + rowsTx, err := s.pool.Begin(context.Background()) + s.NoError(err) + + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + _, err = rowsTx.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + + err = rowsTx.Commit(context.Background()) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsBQ(dstTableName, "id,c1,c2,t,t2") + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteBQ) Test_Composite_PKey_Toast_2_BQ() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") + dstTableName := "test_cpkey_toast2" + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + t2 TEXT, + PRIMARY KEY(id,t) + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_toast2_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.bqHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 4, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + _, err = s.pool.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsBQ(dstTableName, "id,c1,c2,t,t2") + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/postgres/peer_flow_pg_test.go b/flow/e2e/postgres/peer_flow_pg_test.go index f8f0c243f2..45666f0369 100644 --- a/flow/e2e/postgres/peer_flow_pg_test.go +++ b/flow/e2e/postgres/peer_flow_pg_test.go @@ -129,7 +129,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { "id": string(qvalue.QValueKindInt64), "c1": string(qvalue.QValueKindInt64), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } output, err := s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, @@ -158,7 +158,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { "c1": string(qvalue.QValueKindInt64), "c2": string(qvalue.QValueKindInt64), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } output, err = s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, @@ -188,7 +188,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { "c2": string(qvalue.QValueKindInt64), "c3": string(qvalue.QValueKindInt64), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } output, err = s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, @@ -218,7 +218,7 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { "c2": string(qvalue.QValueKindInt64), "c3": string(qvalue.QValueKindInt64), }, - PrimaryKeyColumn: "id", + PrimaryKeyColumns: []string{"id"}, } output, err = s.connector.GetTableSchema(&protos.GetTableSchemaBatchInput{ TableIdentifiers: []string{dstTableName}, @@ -241,3 +241,236 @@ func (s *PeerFlowE2ETestSuitePG) Test_Simple_Schema_Changes_PG() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_cpkey") + dstTableName := s.attachSchemaSuffix("test_simple_cpkey_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + PRIMARY KEY(id,t) + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 4, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t) VALUES ($1,$2) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + // verify we got our 10 rows + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + err = s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") + s.NoError(err) + + _, err := s.pool.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + err = s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t") + s.NoError(err) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_1_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") + dstTableName := s.attachSchemaSuffix("test_cpkey_toast1_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + t2 TEXT, + PRIMARY KEY(id,t) + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_toast1_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + rowsTx, err := s.pool.Begin(context.Background()) + s.NoError(err) + + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + _, err = rowsTx.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + + err = rowsTx.Commit(context.Background()) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + err = s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") + s.NoError(err) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuitePG) Test_Composite_PKey_Toast_2_PG() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") + dstTableName := s.attachSchemaSuffix("test_cpkey_toast2_dst") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + t2 TEXT, + PRIMARY KEY(id,t) + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_toast2_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 4, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + _, err = s.pool.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + err = s.comparePGTables(srcTableName, dstTableName, "id,c1,c2,t,t2") + s.NoError(err) + + env.AssertExpectations(s.T()) +} diff --git a/flow/e2e/postgres/qrep_flow_pg_test.go b/flow/e2e/postgres/qrep_flow_pg_test.go index b4cad3dca7..df1653b992 100644 --- a/flow/e2e/postgres/qrep_flow_pg_test.go +++ b/flow/e2e/postgres/qrep_flow_pg_test.go @@ -142,11 +142,15 @@ func (s *PeerFlowE2ETestSuitePG) Test_Complete_QRep_Flow_Multi_Insert_PG() { numRows := 10 + //nolint:gosec srcTable := "test_qrep_flow_avro_pg_1" s.setupSourceTable(srcTable, numRows) + //nolint:gosec dstTable := "test_qrep_flow_avro_pg_2" - e2e.CreateSourceTableQRep(s.pool, postgresSuffix, dstTable) // the name is misleading, but this is the destination table + // the name is misleading, but this is the destination table + err := e2e.CreateSourceTableQRep(s.pool, postgresSuffix, dstTable) + s.NoError(err) srcSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, srcTable) dstSchemaQualified := fmt.Sprintf("%s_%s.%s", "e2e_test", postgresSuffix, dstTable) diff --git a/flow/e2e/snowflake/peer_flow_sf_test.go b/flow/e2e/snowflake/peer_flow_sf_test.go index 418bcfd63e..f335b3c78f 100644 --- a/flow/e2e/snowflake/peer_flow_sf_test.go +++ b/flow/e2e/snowflake/peer_flow_sf_test.go @@ -974,3 +974,233 @@ func (s *PeerFlowE2ETestSuiteSF) Test_Simple_Schema_Changes_SF() { env.AssertExpectations(s.T()) } + +func (s *PeerFlowE2ETestSuiteSF) Test_Composite_PKey_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_simple_cpkey") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_simple_cpkey") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + PRIMARY KEY(id,t) + ); + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 5, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t) VALUES ($1,$2) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + // verify we got our 10 rows + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + s.compareTableContentsSF("test_simple_cpkey", "id,c1,c2,t", false) + + _, err := s.pool.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsSF("test_simple_cpkey", "id,c1,c2,t", false) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_1_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_cpkey_toast1") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_cpkey_toast1") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + t2 TEXT, + PRIMARY KEY(id,t) + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_toast1_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 2, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + rowsTx, err := s.pool.Begin(context.Background()) + s.NoError(err) + + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + _, err = rowsTx.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = rowsTx.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + + err = rowsTx.Commit(context.Background()) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsSF("test_cpkey_toast1", "id,c1,c2,t,t2", false) + + env.AssertExpectations(s.T()) +} + +func (s *PeerFlowE2ETestSuiteSF) Test_Composite_PKey_Toast_2_SF() { + env := s.NewTestWorkflowEnvironment() + e2e.RegisterWorkflowsAndActivities(env) + + srcTableName := s.attachSchemaSuffix("test_cpkey_toast2") + dstTableName := fmt.Sprintf("%s.%s", s.sfHelper.testSchemaName, "test_cpkey_toast2") + + _, err := s.pool.Exec(context.Background(), fmt.Sprintf(` + CREATE TABLE IF NOT EXISTS %s ( + id INT GENERATED ALWAYS AS IDENTITY, + c1 INT GENERATED BY DEFAULT AS IDENTITY, + c2 INT, + t TEXT, + t2 TEXT, + PRIMARY KEY(id,t) + );CREATE OR REPLACE FUNCTION random_string( int ) RETURNS TEXT as $$ + SELECT string_agg(substring('0123456789bcdfghjkmnpqrstvwxyz', + round(random() * 30)::integer, 1), '') FROM generate_series(1, $1); + $$ language sql; + `, srcTableName)) + s.NoError(err) + + connectionGen := e2e.FlowConnectionGenerationConfig{ + FlowJobName: s.attachSuffix("test_cpkey_toast2_flow"), + TableNameMapping: map[string]string{srcTableName: dstTableName}, + PostgresPort: e2e.PostgresPort, + Destination: s.sfHelper.Peer, + } + + flowConnConfig, err := connectionGen.GenerateFlowConnectionConfigs() + s.NoError(err) + + limits := peerflow.CDCFlowLimits{ + TotalSyncFlows: 4, + MaxBatchSize: 100, + } + + // in a separate goroutine, wait for PeerFlowStatusQuery to finish setup + // and then insert, update and delete rows in the table. + go func() { + e2e.SetupCDCFlowStatusQuery(env, connectionGen) + + // insert 10 rows into the source table + for i := 0; i < 10; i++ { + testValue := fmt.Sprintf("test_value_%d", i) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(` + INSERT INTO %s(c2,t,t2) VALUES ($1,$2,random_string(9000)) + `, srcTableName), i, testValue) + s.NoError(err) + } + fmt.Println("Inserted 10 rows into the source table") + + e2e.NormalizeFlowCountQuery(env, connectionGen, 2) + _, err = s.pool.Exec(context.Background(), + fmt.Sprintf(`UPDATE %s SET c1=c1+1 WHERE MOD(c2,2)=$1`, srcTableName), 1) + s.NoError(err) + _, err = s.pool.Exec(context.Background(), fmt.Sprintf(`DELETE FROM %s WHERE MOD(c2,2)=$1`, srcTableName), 0) + s.NoError(err) + }() + + env.ExecuteWorkflow(peerflow.CDCFlowWorkflowWithConfig, flowConnConfig, &limits, nil) + + // Verify workflow completes without error + s.True(env.IsWorkflowCompleted()) + err = env.GetWorkflowError() + + // allow only continue as new error + s.Error(err) + s.Contains(err.Error(), "continue as new") + + // verify our updates and delete happened + s.compareTableContentsSF("test_cpkey_toast2", "id,c1,c2,t,t2", false) + + env.AssertExpectations(s.T()) +} diff --git a/flow/generated/protos/flow.pb.go b/flow/generated/protos/flow.pb.go index 1a1203014b..f08b021610 100644 --- a/flow/generated/protos/flow.pb.go +++ b/flow/generated/protos/flow.pb.go @@ -1524,7 +1524,7 @@ type TableSchema struct { // list of column names and types, types can be one of the following: // "string", "int", "float", "bool", "timestamp". 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"` - PrimaryKeyColumn string `protobuf:"bytes,3,opt,name=primary_key_column,json=primaryKeyColumn,proto3" json:"primary_key_column,omitempty"` + 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"` } @@ -1574,11 +1574,11 @@ func (x *TableSchema) GetColumns() map[string]string { return nil } -func (x *TableSchema) GetPrimaryKeyColumn() string { +func (x *TableSchema) GetPrimaryKeyColumns() []string { if x != nil { - return x.PrimaryKeyColumn + return x.PrimaryKeyColumns } - return "" + return nil } func (x *TableSchema) GetIsReplicaIdentityFull() bool { @@ -3247,7 +3247,7 @@ var file_flow_proto_rawDesc = []byte{ 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x9c, 0x02, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, + 0x69, 0x66, 0x69, 0x65, 0x72, 0x22, 0x9e, 0x02, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, @@ -3255,273 +3255,273 @@ var file_flow_proto_rawDesc = []byte{ 0x0b, 0x32, 0x25, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, - 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x70, - 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, - 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x69, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x15, 0x69, 0x73, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x74, 0x79, 0x46, 0x75, 0x6c, 0x6c, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, 0x11, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, 0x65, 0x74, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, - 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, - 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, - 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, 0x19, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, - 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, - 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, - 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x48, 0x0a, - 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xd4, 0x02, 0x0a, 0x1e, 0x53, 0x65, 0x74, 0x75, - 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, + 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, + 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, + 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x5f, 0x66, 0x75, 0x6c, 0x6c, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x73, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x74, 0x79, 0x46, 0x75, 0x6c, 0x6c, 0x1a, 0x3a, 0x0a, 0x0c, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x91, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, + 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, + 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x0a, + 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, + 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x19, 0x47, + 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x7d, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, + 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, + 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, + 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xda, 0x01, 0x0a, + 0x19, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, - 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, - 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, - 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x6e, - 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, 0x0a, 0x10, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, 0x65, 0x61, - 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x22, 0xe0, - 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, - 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, - 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, - 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x2e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, - 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, - 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, - 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x79, - 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, 0x03, 0x65, - 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, 0x49, 0x44, - 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, - 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x75, - 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, 0x66, 0x73, - 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, 0x44, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x26, 0x0a, - 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, - 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x3c, 0x0a, 0x12, 0x58, 0x4d, 0x49, - 0x4e, 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, 0x0d, 0x52, 0x05, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xaa, 0x02, 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, 0x12, 0x40, 0x0a, 0x0a, 0x78, 0x6d, 0x69, - 0x6e, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x58, 0x4d, 0x49, 0x4e, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, - 0x52, 0x09, 0x78, 0x6d, 0x69, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, - 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, - 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, - 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x75, 0x70, - 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x96, - 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x22, 0x0a, - 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, - 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, 0x65, 0x73, - 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, - 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x27, 0x0a, - 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, - 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x70, - 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x6e, - 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x36, 0x0a, - 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, - 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, 0x79, 0x6e, - 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x73, - 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, 0x16, 0x62, - 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x65, - 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, 0x61, 0x74, - 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, - 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, 0x74, 0x77, - 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, 0x74, 0x42, - 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, 0x65, 0x63, - 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6d, 0x6f, - 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, - 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, 0x74, 0x65, - 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, - 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, - 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x50, 0x61, - 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x70, - 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x05, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x12, - 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x66, - 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, - 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x50, - 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x29, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, + 0x48, 0x0a, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xd4, 0x02, 0x0a, 0x1e, 0x53, 0x65, + 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x48, 0x0a, 0x16, + 0x70, 0x65, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x52, 0x14, 0x70, 0x65, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x82, 0x01, 0x0a, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x6d, 0x61, 0x70, + 0x70, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, + 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x16, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x63, 0x0a, 0x1b, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x4d, 0x61, + 0x70, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2e, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x6e, 0x0a, 0x1a, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, + 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x29, + 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6c, 0x72, + 0x65, 0x61, 0x64, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0d, 0x61, 0x6c, 0x72, 0x65, 0x61, 0x64, 0x79, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, + 0x22, 0xe0, 0x01, 0x0a, 0x1f, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, + 0x69, 0x7a, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, + 0x74, 0x70, 0x75, 0x74, 0x12, 0x76, 0x0a, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78, + 0x69, 0x73, 0x74, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x53, 0x65, 0x74, 0x75, 0x70, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, + 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x12, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, + 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x1a, 0x45, 0x0a, 0x17, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x4d, 0x61, 0x70, 0x70, 0x69, + 0x6e, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, + 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65, 0x6e, 0x64, + 0x22, 0x79, 0x0a, 0x17, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2c, 0x0a, + 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x4d, 0x0a, 0x03, 0x54, + 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x5f, + 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x5f, 0x0a, 0x11, 0x54, 0x49, + 0x44, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, + 0x26, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, + 0x6f, 0x77, 0x2e, 0x54, 0x49, 0x44, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0x3c, 0x0a, 0x12, 0x58, + 0x4d, 0x49, 0x4e, 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, 0x0d, + 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x22, 0xaa, 0x02, 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, 0x12, 0x40, 0x0a, 0x0a, 0x78, + 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x58, 0x4d, + 0x49, 0x4e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x48, 0x00, 0x52, 0x09, 0x78, 0x6d, 0x69, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x42, 0x07, 0x0a, + 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x22, 0x78, 0x0a, 0x0d, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x70, 0x65, + 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x5f, 0x6b, 0x65, 0x79, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, + 0x75, 0x70, 0x73, 0x65, 0x72, 0x74, 0x4b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x22, 0x96, 0x06, 0x0a, 0x0a, 0x51, 0x52, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, + 0x22, 0x0a, 0x0d, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x33, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x65, + 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, + 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x50, 0x65, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x10, 0x64, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x70, 0x65, 0x65, 0x72, + 0x73, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x0f, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x65, 0x72, 0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x73, 0x74, 0x69, + 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, + 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, + 0x27, 0x0a, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, + 0x61, 0x72, 0x6b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x77, 0x61, 0x74, 0x65, + 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x63, + 0x6f, 0x70, 0x79, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, + 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x6f, 0x70, 0x79, 0x4f, 0x6e, 0x6c, 0x79, 0x12, + 0x36, 0x0a, 0x09, 0x73, 0x79, 0x6e, 0x63, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, + 0x2e, 0x51, 0x52, 0x65, 0x70, 0x53, 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x73, + 0x79, 0x6e, 0x63, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x62, 0x61, 0x74, 0x63, 0x68, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x0c, 0x62, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x49, 0x6e, 0x74, 0x12, 0x34, 0x0a, + 0x16, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x14, 0x62, + 0x61, 0x74, 0x63, 0x68, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x12, 0x30, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6c, + 0x6c, 0x65, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x12, 0x6d, 0x61, 0x78, 0x50, 0x61, 0x72, 0x61, 0x6c, 0x6c, 0x65, 0x6c, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x3f, 0x0a, 0x1c, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x62, 0x65, + 0x74, 0x77, 0x65, 0x65, 0x6e, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x5f, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x19, 0x77, 0x61, 0x69, + 0x74, 0x42, 0x65, 0x74, 0x77, 0x65, 0x65, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x53, + 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, + 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, + 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x57, 0x72, 0x69, + 0x74, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x09, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4d, 0x6f, 0x64, + 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x50, 0x61, 0x74, 0x68, 0x12, 0x33, 0x0a, 0x16, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x10, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x50, 0x65, 0x72, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x51, 0x52, + 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x31, + 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, + 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x05, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x12, 0x66, 0x75, 0x6c, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x6b, 0x0a, 0x12, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x62, 0x61, 0x74, + 0x63, 0x68, 0x49, 0x64, 0x12, 0x3a, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x64, 0x62, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x51, 0x52, 0x65, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x2c, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x46, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x70, 0x75, - 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77, 0x4e, 0x61, 0x6d, 0x65, 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, 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, + 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, 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/model.go b/flow/model/model.go index 610305d295..08aa786f16 100644 --- a/flow/model/model.go +++ b/flow/model/model.go @@ -208,8 +208,6 @@ type InsertRecord struct { CommitID int64 // Items is a map of column name to value. Items *RecordItems - // unchanged toast columns - UnchangedToastColumns map[string]bool } // Implement Record interface for InsertRecord. @@ -237,7 +235,7 @@ type UpdateRecord struct { // NewItems is a map of column name to value. NewItems *RecordItems // unchanged toast columns - UnchangedToastColumns map[string]bool + UnchangedToastColumns map[string]struct{} } // Implement Record interface for UpdateRecord. @@ -263,8 +261,6 @@ type DeleteRecord struct { CheckPointID int64 // Items is a map of column name to value. Items *RecordItems - // unchanged toast columns - UnchangedToastColumns map[string]bool } // Implement Record interface for DeleteRecord. @@ -282,7 +278,7 @@ func (r *DeleteRecord) GetItems() *RecordItems { type TableWithPkey struct { TableName string - PkeyColVal qvalue.QValue + PkeyColVal string } type RecordBatch struct { diff --git a/nexus/pt/src/peerdb_flow.rs b/nexus/pt/src/peerdb_flow.rs index 3a6303c287..05eaf43a79 100644 --- a/nexus/pt/src/peerdb_flow.rs +++ b/nexus/pt/src/peerdb_flow.rs @@ -245,8 +245,8 @@ pub struct TableSchema { /// "string", "int", "float", "bool", "timestamp". #[prost(map="string, string", tag="2")] pub columns: ::std::collections::HashMap<::prost::alloc::string::String, ::prost::alloc::string::String>, - #[prost(string, tag="3")] - pub primary_key_column: ::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, } diff --git a/nexus/pt/src/peerdb_flow.serde.rs b/nexus/pt/src/peerdb_flow.serde.rs index ca2445fc4f..c69bcdc15e 100644 --- a/nexus/pt/src/peerdb_flow.serde.rs +++ b/nexus/pt/src/peerdb_flow.serde.rs @@ -5409,7 +5409,7 @@ impl serde::Serialize for TableSchema { if !self.columns.is_empty() { len += 1; } - if !self.primary_key_column.is_empty() { + if !self.primary_key_columns.is_empty() { len += 1; } if self.is_replica_identity_full { @@ -5422,8 +5422,8 @@ impl serde::Serialize for TableSchema { if !self.columns.is_empty() { struct_ser.serialize_field("columns", &self.columns)?; } - if !self.primary_key_column.is_empty() { - struct_ser.serialize_field("primaryKeyColumn", &self.primary_key_column)?; + if !self.primary_key_columns.is_empty() { + struct_ser.serialize_field("primaryKeyColumns", &self.primary_key_columns)?; } if self.is_replica_identity_full { struct_ser.serialize_field("isReplicaIdentityFull", &self.is_replica_identity_full)?; @@ -5441,8 +5441,8 @@ impl<'de> serde::Deserialize<'de> for TableSchema { "table_identifier", "tableIdentifier", "columns", - "primary_key_column", - "primaryKeyColumn", + "primary_key_columns", + "primaryKeyColumns", "is_replica_identity_full", "isReplicaIdentityFull", ]; @@ -5451,7 +5451,7 @@ impl<'de> serde::Deserialize<'de> for TableSchema { enum GeneratedField { TableIdentifier, Columns, - PrimaryKeyColumn, + PrimaryKeyColumns, IsReplicaIdentityFull, __SkipField__, } @@ -5477,7 +5477,7 @@ impl<'de> serde::Deserialize<'de> for TableSchema { match value { "tableIdentifier" | "table_identifier" => Ok(GeneratedField::TableIdentifier), "columns" => Ok(GeneratedField::Columns), - "primaryKeyColumn" | "primary_key_column" => Ok(GeneratedField::PrimaryKeyColumn), + "primaryKeyColumns" | "primary_key_columns" => Ok(GeneratedField::PrimaryKeyColumns), "isReplicaIdentityFull" | "is_replica_identity_full" => Ok(GeneratedField::IsReplicaIdentityFull), _ => Ok(GeneratedField::__SkipField__), } @@ -5500,7 +5500,7 @@ impl<'de> serde::Deserialize<'de> for TableSchema { { let mut table_identifier__ = None; let mut columns__ = None; - let mut primary_key_column__ = None; + let mut primary_key_columns__ = None; let mut is_replica_identity_full__ = None; while let Some(k) = map.next_key()? { match k { @@ -5518,11 +5518,11 @@ impl<'de> serde::Deserialize<'de> for TableSchema { map.next_value::>()? ); } - GeneratedField::PrimaryKeyColumn => { - if primary_key_column__.is_some() { - return Err(serde::de::Error::duplicate_field("primaryKeyColumn")); + GeneratedField::PrimaryKeyColumns => { + if primary_key_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("primaryKeyColumns")); } - primary_key_column__ = Some(map.next_value()?); + primary_key_columns__ = Some(map.next_value()?); } GeneratedField::IsReplicaIdentityFull => { if is_replica_identity_full__.is_some() { @@ -5538,7 +5538,7 @@ impl<'de> serde::Deserialize<'de> for TableSchema { Ok(TableSchema { table_identifier: table_identifier__.unwrap_or_default(), columns: columns__.unwrap_or_default(), - primary_key_column: primary_key_column__.unwrap_or_default(), + primary_key_columns: primary_key_columns__.unwrap_or_default(), is_replica_identity_full: is_replica_identity_full__.unwrap_or_default(), }) } diff --git a/protos/flow.proto b/protos/flow.proto index 0e03b86fe6..86eb36d5a5 100644 --- a/protos/flow.proto +++ b/protos/flow.proto @@ -154,7 +154,7 @@ message TableSchema { // list of column names and types, types can be one of the following: // "string", "int", "float", "bool", "timestamp". map columns = 2; - string primary_key_column = 3; + repeated string primary_key_columns = 3; bool is_replica_identity_full = 4; } @@ -314,4 +314,4 @@ message TableSchemaDelta { message ReplayTableSchemaDeltaInput { FlowConnectionConfigs flow_connection_configs = 1; repeated TableSchemaDelta table_schema_deltas = 2; -} +} \ No newline at end of file diff --git a/ui/grpc_generated/flow.ts b/ui/grpc_generated/flow.ts index aae30e5b51..7faf3c636b 100644 --- a/ui/grpc_generated/flow.ts +++ b/ui/grpc_generated/flow.ts @@ -262,7 +262,7 @@ export interface TableSchema { * "string", "int", "float", "bool", "timestamp". */ columns: { [key: string]: string }; - primaryKeyColumn: string; + primaryKeyColumns: string[]; isReplicaIdentityFull: boolean; } @@ -3185,7 +3185,7 @@ export const CreateRawTableOutput = { }; function createBaseTableSchema(): TableSchema { - return { tableIdentifier: "", columns: {}, primaryKeyColumn: "", isReplicaIdentityFull: false }; + return { tableIdentifier: "", columns: {}, primaryKeyColumns: [], isReplicaIdentityFull: false }; } export const TableSchema = { @@ -3196,8 +3196,8 @@ export const TableSchema = { Object.entries(message.columns).forEach(([key, value]) => { TableSchema_ColumnsEntry.encode({ key: key as any, value }, writer.uint32(18).fork()).ldelim(); }); - if (message.primaryKeyColumn !== "") { - writer.uint32(26).string(message.primaryKeyColumn); + for (const v of message.primaryKeyColumns) { + writer.uint32(26).string(v!); } if (message.isReplicaIdentityFull === true) { writer.uint32(32).bool(message.isReplicaIdentityFull); @@ -3234,7 +3234,7 @@ export const TableSchema = { break; } - message.primaryKeyColumn = reader.string(); + message.primaryKeyColumns.push(reader.string()); continue; case 4: if (tag !== 32) { @@ -3261,7 +3261,9 @@ export const TableSchema = { return acc; }, {}) : {}, - primaryKeyColumn: isSet(object.primaryKeyColumn) ? String(object.primaryKeyColumn) : "", + primaryKeyColumns: Array.isArray(object?.primaryKeyColumns) + ? object.primaryKeyColumns.map((e: any) => String(e)) + : [], isReplicaIdentityFull: isSet(object.isReplicaIdentityFull) ? Boolean(object.isReplicaIdentityFull) : false, }; }, @@ -3280,8 +3282,8 @@ export const TableSchema = { }); } } - if (message.primaryKeyColumn !== "") { - obj.primaryKeyColumn = message.primaryKeyColumn; + if (message.primaryKeyColumns?.length) { + obj.primaryKeyColumns = message.primaryKeyColumns; } if (message.isReplicaIdentityFull === true) { obj.isReplicaIdentityFull = message.isReplicaIdentityFull; @@ -3301,7 +3303,7 @@ export const TableSchema = { } return acc; }, {}); - message.primaryKeyColumn = object.primaryKeyColumn ?? ""; + message.primaryKeyColumns = object.primaryKeyColumns?.map((e) => e) || []; message.isReplicaIdentityFull = object.isReplicaIdentityFull ?? false; return message; },